diff --git a/.gitignore b/.gitignore index d162fa9cca994..d54d21b802be8 100644 --- a/.gitignore +++ b/.gitignore @@ -63,6 +63,8 @@ ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml +R-unit-tests.log +R/unit-tests.out # For Hive metastore_db/ diff --git a/.rat-excludes b/.rat-excludes index 8c61e67a0c7d1..ac652ed29fc09 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -15,6 +15,7 @@ TAGS RELEASE control docs +docker.properties.template fairscheduler.xml.template spark-defaults.conf.template log4j.properties @@ -29,7 +30,13 @@ spark-env.sh.template log4j-defaults.properties bootstrap-tooltip.js jquery-1.11.1.min.js +d3.min.js +dagre-d3.min.js +graphlib-dot.min.js sorttable.js +vis.min.js +vis.min.css +vis.map .*avsc .*txt .*json @@ -67,3 +74,12 @@ logs .*scalastyle-output.xml .*dependency-reduced-pom.xml known_translations +json_expectation +local-1422981759269/* +local-1422981780767/* +local-1425081759269/* +local-1426533911241/* +local-1426633911242/* +local-1427397477963/* +DESCRIPTION +NAMESPACE diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index b6c6b050fa331..f10d7e277eea3 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,12 +1,16 @@ ## Contributing to Spark -Contributions via GitHub pull requests are gladly accepted from their original -author. Along with any pull requests, please state that the contribution is -your original work and that you license the work to the project under the -project's open source license. Whether or not you state this explicitly, by -submitting any copyrighted material via pull request, email, or other means -you agree to license the material under the project's open source license and -warrant that you have the legal authority to do so. +*Before opening a pull request*, review the +[Contributing to Spark wiki](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark). +It lists steps that are required before creating a PR. In particular, consider: + +- Is the change important and ready enough to ask the community to spend time reviewing? +- Have you searched for existing, related JIRAs and pull requests? +- Is this a new feature that can stand alone as a package on http://spark-packages.org ? +- Is the change being proposed clearly explained and motivated? -Please see the [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) -for more information. +When you contribute code, you affirm that the contribution is your original work and that you +license the work to the project under the project's open source license. Whether or not you +state this explicitly, by submitting any copyrighted material via pull request, email, or +other means you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. diff --git a/LICENSE b/LICENSE index 9b364a4d00079..21c42e9a20fa3 100644 --- a/LICENSE +++ b/LICENSE @@ -814,6 +814,7 @@ BSD-style licenses The following components are provided under a BSD-style license. See project link for details. (BSD 3 Clause) core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) + (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.1.15 - https://github.com/jpmml/jpmml-model) (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.3 - http://jblas.org/) (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) diff --git a/R/.gitignore b/R/.gitignore new file mode 100644 index 0000000000000..9a5889ba28b2a --- /dev/null +++ b/R/.gitignore @@ -0,0 +1,6 @@ +*.o +*.so +*.Rd +lib +pkg/man +pkg/html diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md new file mode 100644 index 0000000000000..931d01549b265 --- /dev/null +++ b/R/DOCUMENTATION.md @@ -0,0 +1,12 @@ +# SparkR Documentation + +SparkR documentation is generated using in-source comments annotated using using +`roxygen2`. After making changes to the documentation, to generate man pages, +you can run the following from an R console in the SparkR home directory + + library(devtools) + devtools::document(pkg="./pkg", roclets=c("rd")) + +You can verify if your changes are good by running + + R CMD check pkg/ diff --git a/R/README.md b/R/README.md new file mode 100644 index 0000000000000..a6970e39b55f3 --- /dev/null +++ b/R/README.md @@ -0,0 +1,67 @@ +# R on Spark + +SparkR is an R package that provides a light-weight frontend to use Spark from R. + +### SparkR development + +#### Build Spark + +Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run +``` + build/mvn -DskipTests -Psparkr package +``` + +#### Running sparkR + +You can start using SparkR by launching the SparkR shell with + + ./bin/sparkR + +The `sparkR` script automatically creates a SparkContext with Spark by default in +local mode. To specify the Spark master of a cluster for the automatically created +SparkContext, you can run + + ./bin/sparkR --master "local[2]" + +To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR` + +#### Using SparkR from RStudio + +If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example +``` +# Set this to where Spark is installed +Sys.setenv(SPARK_HOME="/Users/shivaram/spark") +# This line loads SparkR from the installed directory +.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) +library(SparkR) +sc <- sparkR.init(master="local") +``` + +#### Making changes to SparkR + +The [instructions](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for making contributions to Spark also apply to SparkR. +If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes. +Once you have made your changes, please include unit tests for them and run existing unit tests using the `run-tests.sh` script as described below. + +#### Generating documentation + +The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script. + +### Examples, Unit tests + +SparkR comes with several sample programs in the `examples/src/main/r` directory. +To run one of them, use `./bin/sparkR `. For example: + + ./bin/sparkR examples/src/main/r/pi.R local[2] + +You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first): + + R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' + ./R/run-tests.sh + +### Running on YARN +The `./bin/spark-submit` and `./bin/sparkR` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run +``` +export YARN_CONF_DIR=/etc/hadoop/conf +./bin/spark-submit --master yarn examples/src/main/r/pi.R 4 +``` diff --git a/R/WINDOWS.md b/R/WINDOWS.md new file mode 100644 index 0000000000000..3f889c0ca3d1e --- /dev/null +++ b/R/WINDOWS.md @@ -0,0 +1,13 @@ +## Building SparkR on Windows + +To build SparkR on Windows, the following steps are required + +1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to +include Rtools and R in `PATH`. +2. Install +[JDK7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) and set +`JAVA_HOME` in the system environment variables. +3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` +directory in Maven in `PATH`. +4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). +5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package` diff --git a/R/create-docs.sh b/R/create-docs.sh new file mode 100755 index 0000000000000..4194172a2e115 --- /dev/null +++ b/R/create-docs.sh @@ -0,0 +1,46 @@ +#!/bin/bash + +# +# 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. +# + +# Script to create API docs for SparkR +# This requires `devtools` and `knitr` to be installed on the machine. + +# After running this script the html docs can be found in +# $SPARK_HOME/R/pkg/html + +# Figure out where the script is +export FWDIR="$(cd "`dirname "$0"`"; pwd)" +pushd $FWDIR + +# Generate Rd file +Rscript -e 'library(devtools); devtools::document(pkg="./pkg", roclets=c("rd"))' + +# Install the package +./install-dev.sh + +# Now create HTML files + +# knit_rd puts html in current working directory +mkdir -p pkg/html +pushd pkg/html + +Rscript -e 'library(SparkR, lib.loc="../../lib"); library(knitr); knit_rd("SparkR")' + +popd + +popd diff --git a/R/install-dev.bat b/R/install-dev.bat new file mode 100644 index 0000000000000..008a5c668bc45 --- /dev/null +++ b/R/install-dev.bat @@ -0,0 +1,27 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Install development version of SparkR +rem + +set SPARK_HOME=%~dp0.. + +MKDIR %SPARK_HOME%\R\lib + +R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ diff --git a/R/install-dev.sh b/R/install-dev.sh new file mode 100755 index 0000000000000..55ed6f4be1a4a --- /dev/null +++ b/R/install-dev.sh @@ -0,0 +1,36 @@ +#!/bin/bash + +# +# 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. +# + +# This scripts packages the SparkR source files (R and C files) and +# creates a package that can be loaded in R. The package is by default installed to +# $FWDIR/lib and the package can be loaded by using the following command in R: +# +# library(SparkR, lib.loc="$FWDIR/lib") +# +# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory +# to load the SparkR package on the worker nodes. + + +FWDIR="$(cd `dirname $0`; pwd)" +LIB_DIR="$FWDIR/lib" + +mkdir -p $LIB_DIR + +# Install R +R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/ diff --git a/R/log4j.properties b/R/log4j.properties new file mode 100644 index 0000000000000..701adb2a3da1d --- /dev/null +++ b/R/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=R-unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION new file mode 100644 index 0000000000000..1c1779a763c7e --- /dev/null +++ b/R/pkg/DESCRIPTION @@ -0,0 +1,35 @@ +Package: SparkR +Type: Package +Title: R frontend for Spark +Version: 1.4.0 +Date: 2013-09-09 +Author: The Apache Software Foundation +Maintainer: Shivaram Venkataraman +Imports: + methods +Depends: + R (>= 3.0), + methods, +Suggests: + testthat +Description: R frontend for Spark +License: Apache License (== 2.0) +Collate: + 'generics.R' + 'jobj.R' + 'RDD.R' + 'pairRDD.R' + 'schema.R' + 'column.R' + 'group.R' + 'DataFrame.R' + 'SQLContext.R' + 'backend.R' + 'broadcast.R' + 'client.R' + 'context.R' + 'deserialize.R' + 'serialize.R' + 'sparkR.R' + 'utils.R' + 'zzz.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE new file mode 100644 index 0000000000000..e077eace74375 --- /dev/null +++ b/R/pkg/NAMESPACE @@ -0,0 +1,197 @@ +#exportPattern("^[[:alpha:]]+") +exportClasses("RDD") +exportClasses("Broadcast") +exportMethods( + "aggregateByKey", + "aggregateRDD", + "cache", + "cartesian", + "checkpoint", + "coalesce", + "cogroup", + "collect", + "collectAsMap", + "collectPartition", + "combineByKey", + "count", + "countByKey", + "countByValue", + "distinct", + "Filter", + "filterRDD", + "first", + "flatMap", + "flatMapValues", + "fold", + "foldByKey", + "foreach", + "foreachPartition", + "fullOuterJoin", + "glom", + "groupByKey", + "intersection", + "join", + "keyBy", + "keys", + "length", + "lapply", + "lapplyPartition", + "lapplyPartitionsWithIndex", + "leftOuterJoin", + "lookup", + "map", + "mapPartitions", + "mapPartitionsWithIndex", + "mapValues", + "maximum", + "minimum", + "numPartitions", + "partitionBy", + "persist", + "pipeRDD", + "reduce", + "reduceByKey", + "reduceByKeyLocally", + "repartition", + "rightOuterJoin", + "sampleByKey", + "sampleRDD", + "saveAsTextFile", + "saveAsObjectFile", + "sortBy", + "sortByKey", + "subtract", + "subtractByKey", + "sumRDD", + "take", + "takeOrdered", + "takeSample", + "top", + "unionRDD", + "unpersist", + "value", + "values", + "zipPartitions", + "zipRDD", + "zipWithIndex", + "zipWithUniqueId" + ) + +# S3 methods exported +export( + "textFile", + "objectFile", + "parallelize", + "hashCode", + "includePackage", + "broadcast", + "setBroadcastValue", + "setCheckpointDir" + ) +export("sparkR.init") +export("sparkR.stop") +export("print.jobj") +useDynLib(SparkR, stringHashCode) +importFrom(methods, setGeneric, setMethod, setOldClass) + +# SparkRSQL + +exportClasses("DataFrame") + +exportMethods("columns", + "distinct", + "dtypes", + "except", + "explain", + "filter", + "groupBy", + "head", + "insertInto", + "intersect", + "isLocal", + "limit", + "orderBy", + "names", + "printSchema", + "registerTempTable", + "repartition", + "sampleDF", + "saveAsParquetFile", + "saveAsTable", + "saveDF", + "schema", + "select", + "selectExpr", + "show", + "showDF", + "sortDF", + "toJSON", + "toRDD", + "unionAll", + "where", + "withColumn", + "withColumnRenamed") + +exportClasses("Column") + +exportMethods("abs", + "alias", + "approxCountDistinct", + "asc", + "avg", + "cast", + "contains", + "countDistinct", + "desc", + "endsWith", + "getField", + "getItem", + "isNotNull", + "isNull", + "last", + "like", + "lower", + "max", + "mean", + "min", + "rlike", + "sqrt", + "startsWith", + "substr", + "sum", + "sumDistinct", + "upper") + +exportClasses("GroupedData") +exportMethods("agg") + +export("sparkRSQL.init", + "sparkRHive.init") + +export("cacheTable", + "clearCache", + "createDataFrame", + "createExternalTable", + "dropTempTable", + "jsonFile", + "jsonRDD", + "loadDF", + "parquetFile", + "sql", + "table", + "tableNames", + "tables", + "toDF", + "uncacheTable") + +export("sparkRSQL.init", + "sparkRHive.init") + +export("structField", + "structField.jobj", + "structField.character", + "print.structField", + "structType", + "structType.jobj", + "structType.structField", + "print.structType") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R new file mode 100644 index 0000000000000..841e77e55e0d8 --- /dev/null +++ b/R/pkg/R/DataFrame.R @@ -0,0 +1,1278 @@ +# +# 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. +# + +# DataFrame.R - DataFrame class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame +#' @description DataFrames can be created using functions like +#' \code{jsonFile}, \code{table} etc. +#' @rdname DataFrame +#' @seealso jsonFile, table +#' +#' @param env An R environment that stores bookkeeping states of the DataFrame +#' @param sdf A Java object reference to the backing Scala DataFrame +#' @export +setClass("DataFrame", + slots = list(env = "environment", + sdf = "jobj")) + +setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { + .Object@env <- new.env() + .Object@env$isCached <- isCached + + .Object@sdf <- sdf + .Object +}) + +#' @rdname DataFrame +#' @export +dataFrame <- function(sdf, isCached = FALSE) { + new("DataFrame", sdf, isCached) +} + +############################ DataFrame Methods ############################################## + +#' Print Schema of a DataFrame +#' +#' Prints out the schema in tree format +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname printSchema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' printSchema(df) +#'} +setMethod("printSchema", + signature(x = "DataFrame"), + function(x) { + schemaString <- callJMethod(schema(x)$jobj, "treeString") + cat(schemaString) + }) + +#' Get schema object +#' +#' Returns the schema of this DataFrame as a structType object. +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname schema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dfSchema <- schema(df) +#'} +setMethod("schema", + signature(x = "DataFrame"), + function(x) { + structType(callJMethod(x@sdf, "schema")) + }) + +#' Explain +#' +#' Print the logical and physical Catalyst plans to the console for debugging. +#' +#' @param x A SparkSQL DataFrame +#' @param extended Logical. If extended is False, explain() only prints the physical plan. +#' @rdname explain +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' explain(df, TRUE) +#'} +setMethod("explain", + signature(x = "DataFrame"), + function(x, extended = FALSE) { + queryExec <- callJMethod(x@sdf, "queryExecution") + if (extended) { + cat(callJMethod(queryExec, "toString")) + } else { + execPlan <- callJMethod(queryExec, "executedPlan") + cat(callJMethod(execPlan, "toString")) + } + }) + +#' isLocal +#' +#' Returns True if the `collect` and `take` methods can be run locally +#' (without any Spark executors). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname isLocal +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' isLocal(df) +#'} +setMethod("isLocal", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "isLocal") + }) + +#' ShowDF +#' +#' Print the first numRows rows of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' @param numRows The number of rows to print. Defaults to 20. +#' +#' @rdname showDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' showDF(df) +#'} +setMethod("showDF", + signature(x = "DataFrame"), + function(x, numRows = 20) { + callJMethod(x@sdf, "showString", numToInt(numRows)) + }) + +#' show +#' +#' Print the DataFrame column names and types +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname show +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' show(df) +#'} +setMethod("show", "DataFrame", + function(object) { + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste("DataFrame[", s, "]\n", sep = "")) + }) + +#' DataTypes +#' +#' Return all column names and their data types as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname dtypes +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dtypes(df) +#'} +setMethod("dtypes", + signature(x = "DataFrame"), + function(x) { + lapply(schema(x)$fields(), function(f) { + c(f$name(), f$dataType.simpleString()) + }) + }) + +#' Column names +#' +#' Return all column names as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname columns +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' columns(df) +#'} +setMethod("columns", + signature(x = "DataFrame"), + function(x) { + sapply(schema(x)$fields(), function(f) { + f$name() + }) + }) + +#' @rdname columns +#' @export +setMethod("names", + signature(x = "DataFrame"), + function(x) { + columns(x) + }) + +#' Register Temporary Table +#' +#' Registers a DataFrame as a Temporary Table in the SQLContext +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @rdname registerTempTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "json_df") +#' new_df <- sql(sqlCtx, "SELECT * FROM json_df") +#'} +setMethod("registerTempTable", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName) { + callJMethod(x@sdf, "registerTempTable", tableName) + }) + +#' insertInto +#' +#' Insert the contents of a DataFrame into a table registered in the current SQL Context. +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' @param overwrite A logical argument indicating whether or not to overwrite +#' the existing rows in the table. +#' +#' @rdname insertInto +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' df2 <- loadDF(sqlCtx, path2, "parquet") +#' registerTempTable(df, "table1") +#' insertInto(df2, "table1", overwrite = TRUE) +#'} +setMethod("insertInto", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName, overwrite = FALSE) { + callJMethod(x@sdf, "insertInto", tableName, overwrite) + }) + +#' Cache +#' +#' Persist with the default storage level (MEMORY_ONLY). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname cache-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' cache(df) +#'} +setMethod("cache", + signature(x = "DataFrame"), + function(x) { + cached <- callJMethod(x@sdf, "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist +#' +#' Persist this DataFrame with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The DataFrame to persist +#' @rdname persist +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#'} +setMethod("persist", + signature(x = "DataFrame", newLevel = "character"), + function(x, newLevel) { + callJMethod(x@sdf, "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist +#' +#' Mark this DataFrame as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The DataFrame to unpersist +#' @param blocking Whether to block until all blocks are deleted +#' @rdname unpersist-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#' unpersist(df) +#'} +setMethod("unpersist", + signature(x = "DataFrame"), + function(x, blocking = TRUE) { + callJMethod(x@sdf, "unpersist", blocking) + x@env$isCached <- FALSE + x + }) + +#' Repartition +#' +#' Return a new DataFrame that has exactly numPartitions partitions. +#' +#' @param x A SparkSQL DataFrame +#' @param numPartitions The number of partitions to use. +#' @rdname repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- repartition(df, 2L) +#'} +setMethod("repartition", + signature(x = "DataFrame", numPartitions = "numeric"), + function(x, numPartitions) { + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + dataFrame(sdf) + }) + +#' toJSON +#' +#' Convert the rows of a DataFrame into JSON objects and return an RDD where +#' each element contains a JSON string. +#' +#' @param x A SparkSQL DataFrame +#' @return A StringRRDD of JSON objects +#' @rdname tojson +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newRDD <- toJSON(df) +#'} +setMethod("toJSON", + signature(x = "DataFrame"), + function(x) { + rdd <- callJMethod(x@sdf, "toJSON") + jrdd <- callJMethod(rdd, "toJavaRDD") + RDD(jrdd, serializedMode = "string") + }) + +#' saveAsParquetFile +#' +#' Save the contents of a DataFrame as a Parquet file, preserving the schema. Files written out +#' with this method can be read back in as a DataFrame using parquetFile(). +#' +#' @param x A SparkSQL DataFrame +#' @param path The directory where the file is saved +#' @rdname saveAsParquetFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsParquetFile(df, "/tmp/sparkr-tmp/") +#'} +setMethod("saveAsParquetFile", + signature(x = "DataFrame", path = "character"), + function(x, path) { + invisible(callJMethod(x@sdf, "saveAsParquetFile", path)) + }) + +#' Distinct +#' +#' Return a new DataFrame containing the distinct rows in this DataFrame. +#' +#' @param x A SparkSQL DataFrame +#' @rdname distinct +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' distinctDF <- distinct(df) +#'} +setMethod("distinct", + signature(x = "DataFrame"), + function(x) { + sdf <- callJMethod(x@sdf, "distinct") + dataFrame(sdf) + }) + +#' SampleDF +#' +#' Return a sampled subset of this DataFrame using a random seed. +#' +#' @param x A SparkSQL DataFrame +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @rdname sampleDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collect(sampleDF(df, FALSE, 0.5)) +#' collect(sampleDF(df, TRUE, 0.5)) +#'} +setMethod("sampleDF", + # TODO : Figure out how to send integer as java.lang.Long to JVM so + # we can send seed as an argument through callJMethod + signature(x = "DataFrame", withReplacement = "logical", + fraction = "numeric"), + function(x, withReplacement, fraction) { + if (fraction < 0.0) stop(cat("Negative fraction value:", fraction)) + sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction) + dataFrame(sdf) + }) + +#' Count +#' +#' Returns the number of rows in a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname count +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' count(df) +#' } +setMethod("count", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "count") + }) + +#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame. +#' +#' @param x A SparkSQL DataFrame +#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns +#' should be converted to factors. FALSE by default. + +#' @rdname collect-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collected <- collect(df) +#' firstName <- collected[[1]]$name +#' } +setMethod("collect", + signature(x = "DataFrame"), + function(x, stringsAsFactors = FALSE) { + # listCols is a list of raw vectors, one per column + listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) + cols <- lapply(listCols, function(col) { + objRaw <- rawConnection(col) + numRows <- readInt(objRaw) + col <- readCol(objRaw, numRows) + close(objRaw) + col + }) + names(cols) <- columns(x) + do.call(cbind.data.frame, list(cols, stringsAsFactors = stringsAsFactors)) + }) + +#' Limit +#' +#' Limit the resulting DataFrame to the number of rows specified. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return +#' @return A new DataFrame containing the number of rows specified. +#' +#' @rdname limit +#' @export +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' limitedDF <- limit(df, 10) +#' } +setMethod("limit", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + res <- callJMethod(x@sdf, "limit", as.integer(num)) + dataFrame(res) + }) + +# Take the first NUM rows of a DataFrame and return a the results as a data.frame + +#' @rdname take +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' take(df, 2) +#' } +setMethod("take", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + limited <- limit(x, num) + collect(limited) + }) + +#' Head +#' +#' Return the first NUM rows of a DataFrame as a data.frame. If NUM is NULL, +#' then head() returns the first 6 rows in keeping with the current data.frame +#' convention in R. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return. Default is 6. +#' @return A data.frame +#' +#' @rdname head +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' head(df) +#' } +setMethod("head", + signature(x = "DataFrame"), + function(x, num = 6L) { + # Default num is 6L in keeping with R's data.frame convention + take(x, num) + }) + +#' Return the first row of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' first(df) +#' } +setMethod("first", + signature(x = "DataFrame"), + function(x) { + take(x, 1) + }) + +#' toRDD() +#' +#' Converts a Spark DataFrame to an RDD while preserving column names. +#' +#' @param x A Spark DataFrame +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' rdd <- toRDD(df) +#' } +setMethod("toRDD", + signature(x = "DataFrame"), + function(x) { + jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf) + colNames <- callJMethod(x@sdf, "columns") + rdd <- RDD(jrdd, serializedMode = "row") + lapply(rdd, function(row) { + names(row) <- colNames + row + }) + }) + +#' GroupBy +#' +#' Groups the DataFrame using the specified columns, so we can run aggregation on them. +#' +#' @param x a DataFrame +#' @return a GroupedData +#' @seealso GroupedData +#' @rdname DataFrame +#' @export +#' @examples +#' \dontrun{ +#' # Compute the average for all numeric columns grouped by department. +#' avg(groupBy(df, "department")) +#' +#' # Compute the max age and average salary, grouped by department and gender. +#' agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max") +#' } +setMethod("groupBy", + signature(x = "DataFrame"), + function(x, ...) { + cols <- list(...) + if (length(cols) >= 1 && class(cols[[1]]) == "character") { + sgd <- callJMethod(x@sdf, "groupBy", cols[[1]], listToSeq(cols[-1])) + } else { + jcol <- lapply(cols, function(c) { c@jc }) + sgd <- callJMethod(x@sdf, "groupBy", listToSeq(jcol)) + } + groupedData(sgd) + }) + +#' Agg +#' +#' Compute aggregates by specifying a list of columns +#' +#' @rdname DataFrame +#' @export +setMethod("agg", + signature(x = "DataFrame"), + function(x, ...) { + agg(groupBy(x), ...) + }) + + +############################## RDD Map Functions ################################## +# All of the following functions mirror the existing RDD map functions, # +# but allow for use with DataFrames by first converting to an RRDD before calling # +# the requested map function. # +################################################################################### + +#' @rdname lapply +setMethod("lapply", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapply(rdd, FUN) + }) + +#' @rdname lapply +setMethod("map", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' @rdname flatMap +setMethod("flatMap", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + flatMap(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("lapplyPartition", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapplyPartition(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("mapPartitions", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' @rdname foreach +setMethod("foreach", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreach(rdd, func) + }) + +#' @rdname foreach +setMethod("foreachPartition", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreachPartition(rdd, func) + }) + + +############################## SELECT ################################## + +getColumn <- function(x, c) { + column(callJMethod(x@sdf, "col", c)) +} + +#' @rdname select +setMethod("$", signature(x = "DataFrame"), + function(x, name) { + getColumn(x, name) + }) + +setMethod("$<-", signature(x = "DataFrame"), + function(x, name, value) { + stopifnot(class(value) == "Column" || is.null(value)) + cols <- columns(x) + if (name %in% cols) { + if (is.null(value)) { + cols <- Filter(function(c) { c != name }, cols) + } + cols <- lapply(cols, function(c) { + if (c == name) { + alias(value, name) + } else { + col(c) + } + }) + nx <- select(x, cols) + } else { + if (is.null(value)) { + return(x) + } + nx <- withColumn(x, name, value) + } + x@sdf <- nx@sdf + x + }) + +#' @rdname select +setMethod("[[", signature(x = "DataFrame"), + function(x, i) { + if (is.numeric(i)) { + cols <- columns(x) + i <- cols[[i]] + } + getColumn(x, i) + }) + +#' @rdname select +setMethod("[", signature(x = "DataFrame", i = "missing"), + function(x, i, j, ...) { + if (is.numeric(j)) { + cols <- columns(x) + j <- cols[j] + } + if (length(j) > 1) { + j <- as.list(j) + } + select(x, j) + }) + +#' Select +#' +#' Selects a set of columns with names or Column expressions. +#' @param x A DataFrame +#' @param col A list of columns or single Column or name +#' @return A new DataFrame with selected columns +#' @export +#' @rdname select +#' @examples +#' \dontrun{ +#' select(df, "*") +#' select(df, "col1", "col2") +#' select(df, df$name, df$age + 1) +#' select(df, c("col1", "col2")) +#' select(df, list(df$name, df$age + 1)) +#' # Columns can also be selected using `[[` and `[` +#' df[[2]] == df[["age"]] +#' df[,2] == df[,"age"] +#' # Similar to R data frames columns can also be selected using `$` +#' df$age +#' } +setMethod("select", signature(x = "DataFrame", col = "character"), + function(x, col, ...) { + sdf <- callJMethod(x@sdf, "select", col, toSeq(...)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", signature(x = "DataFrame", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(jcols)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", + signature(x = "DataFrame", col = "list"), + function(x, col) { + cols <- lapply(col, function(c) { + if (class(c)== "Column") { + c@jc + } else { + col(c)@jc + } + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(cols)) + dataFrame(sdf) + }) + +#' SelectExpr +#' +#' Select from a DataFrame using a set of SQL expressions. +#' +#' @param x A DataFrame to be selected from. +#' @param expr A string containing a SQL expression +#' @param ... Additional expressions +#' @return A DataFrame +#' @rdname selectExpr +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' selectExpr(df, "col1", "(col2 * 5) as newCol") +#' } +setMethod("selectExpr", + signature(x = "DataFrame", expr = "character"), + function(x, expr, ...) { + exprList <- list(expr, ...) + sdf <- callJMethod(x@sdf, "selectExpr", listToSeq(exprList)) + dataFrame(sdf) + }) + +#' WithColumn +#' +#' Return a new DataFrame with the specified column added. +#' +#' @param x A DataFrame +#' @param colName A string containing the name of the new column. +#' @param col A Column expression. +#' @return A DataFrame with the new column added. +#' @rdname withColumn +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumn(df, "newCol", df$col1 * 5) +#' } +setMethod("withColumn", + signature(x = "DataFrame", colName = "character", col = "Column"), + function(x, colName, col) { + select(x, x$"*", alias(col, colName)) + }) + +#' WithColumnRenamed +#' +#' Rename an existing column in a DataFrame. +#' +#' @param x A DataFrame +#' @param existingCol The name of the column you want to change. +#' @param newCol The new column name. +#' @return A DataFrame with the column name changed. +#' @rdname withColumnRenamed +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumnRenamed(df, "col1", "newCol1") +#' } +setMethod("withColumnRenamed", + signature(x = "DataFrame", existingCol = "character", newCol = "character"), + function(x, existingCol, newCol) { + cols <- lapply(columns(x), function(c) { + if (c == existingCol) { + alias(col(c), newCol) + } else { + col(c) + } + }) + select(x, cols) + }) + +setClassUnion("characterOrColumn", c("character", "Column")) + +#' SortDF +#' +#' Sort a DataFrame by the specified column(s). +#' +#' @param x A DataFrame to be sorted. +#' @param col Either a Column object or character vector indicating the field to sort on +#' @param ... Additional sorting fields +#' @return A DataFrame where all elements are sorted. +#' @rdname sortDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' sortDF(df, df$col1) +#' sortDF(df, "col1") +#' sortDF(df, asc(df$col1), desc(abs(df$col2))) +#' } +setMethod("sortDF", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col, ...) { + if (class(col) == "character") { + sdf <- callJMethod(x@sdf, "sort", col, toSeq(...)) + } else if (class(col) == "Column") { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "sort", listToSeq(jcols)) + } + dataFrame(sdf) + }) + +#' @rdname sortDF +#' @export +setMethod("orderBy", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col) { + sortDF(x, col) + }) + +#' Filter +#' +#' Filter the rows of a DataFrame according to a given condition. +#' +#' @param x A DataFrame to be sorted. +#' @param condition The condition to sort on. This may either be a Column expression +#' or a string containing a SQL statement +#' @return A DataFrame containing only the rows that meet the condition. +#' @rdname filter +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' filter(df, "col1 > 0") +#' filter(df, df$col2 != "abcdefg") +#' } +setMethod("filter", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + if (class(condition) == "Column") { + condition <- condition@jc + } + sdf <- callJMethod(x@sdf, "filter", condition) + dataFrame(sdf) + }) + +#' @rdname filter +#' @export +setMethod("where", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + filter(x, condition) + }) + +#' Join +#' +#' Join two DataFrames based on the given join expression. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a +#' Column expression. If joinExpr is omitted, join() wil perform a Cartesian join +#' @param joinType The type of join to perform. The following join types are available: +#' 'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'. The default joinType is "inner". +#' @return A DataFrame containing the result of the join operation. +#' @rdname join +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' join(df1, df2) # Performs a Cartesian +#' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression +#' join(df1, df2, df1$col1 == df2$col2, "right_outer") +#' } +setMethod("join", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y, joinExpr = NULL, joinType = NULL) { + if (is.null(joinExpr)) { + sdf <- callJMethod(x@sdf, "join", y@sdf) + } else { + if (class(joinExpr) != "Column") stop("joinExpr must be a Column") + if (is.null(joinType)) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc) + } else { + if (joinType %in% c("inner", "outer", "left_outer", "right_outer", "semijoin")) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) + } else { + stop("joinType must be one of the following types: ", + "'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'") + } + } + } + dataFrame(sdf) + }) + +#' UnionAll +#' +#' Return a new DataFrame containing the union of rows in this DataFrame +#' and another DataFrame. This is equivalent to `UNION ALL` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the union. +#' @rdname unionAll +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' unioned <- unionAll(df, df2) +#' } +setMethod("unionAll", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + unioned <- callJMethod(x@sdf, "unionAll", y@sdf) + dataFrame(unioned) + }) + +#' Intersect +#' +#' Return a new DataFrame containing rows only in both this DataFrame +#' and another DataFrame. This is equivalent to `INTERSECT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the intersect. +#' @rdname intersect +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' intersectDF <- intersect(df, df2) +#' } +setMethod("intersect", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + intersected <- callJMethod(x@sdf, "intersect", y@sdf) + dataFrame(intersected) + }) + +#' except +#' +#' Return a new DataFrame containing rows in this DataFrame +#' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the except operation. +#' @rdname except +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' exceptDF <- except(df, df2) +#' } +#' @rdname except +#' @export +setMethod("except", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + excepted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(excepted) + }) + +#' Save the contents of the DataFrame to a data source +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param path A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveDF", + signature(df = "DataFrame", path = 'character', source = 'character', + mode = 'character'), + function(df, path = NULL, source = NULL, mode = "append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] = path + } + callJMethod(df@sdf, "save", source, jmode, options) + }) + + +#' saveAsTable +#' +#' Save the contents of the DataFrame to a data source as a table +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param tableName A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveAsTable", + signature(df = "DataFrame", tableName = 'character', source = 'character', + mode = 'character'), + function(df, tableName, source = NULL, mode="append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) + }) + diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R new file mode 100644 index 0000000000000..a3a0421a0746d --- /dev/null +++ b/R/pkg/R/RDD.R @@ -0,0 +1,1643 @@ +# +# 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. +# + +# RDD in R implemented in S4 OO system. + +setOldClass("jobj") + +#' @title S4 class that represents an RDD +#' @description RDD can be created using functions like +#' \code{parallelize}, \code{textFile} etc. +#' @rdname RDD +#' @seealso parallelize, textFile +#' +#' @slot env An R environment that stores bookkeeping states of the RDD +#' @slot jrdd Java object reference to the backing JavaRDD +#' to an RDD +#' @export +setClass("RDD", + slots = list(env = "environment", + jrdd = "jobj")) + +setClass("PipelinedRDD", + slots = list(prev = "RDD", + func = "function", + prev_jrdd = "jobj"), + contains = "RDD") + +setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, + isCached, isCheckpointed) { + # Check that RDD constructor is using the correct version of serializedMode + stopifnot(class(serializedMode) == "character") + stopifnot(serializedMode %in% c("byte", "string", "row")) + # RDD has three serialization types: + # byte: The RDD stores data serialized in R. + # string: The RDD stores data as strings. + # row: The RDD stores the serialized rows of a DataFrame. + + # We use an environment to store mutable states inside an RDD object. + # Note that R's call-by-value semantics makes modifying slots inside an + # object (passed as an argument into a function, such as cache()) difficult: + # i.e. one needs to make a copy of the RDD object and sets the new slot value + # there. + + # The slots are inheritable from superclass. Here, both `env' and `jrdd' are + # inherited from RDD, but only the former is used. + .Object@env <- new.env() + .Object@env$isCached <- isCached + .Object@env$isCheckpointed <- isCheckpointed + .Object@env$serializedMode <- serializedMode + + .Object@jrdd <- jrdd + .Object +}) + +setMethod("show", "RDD", + function(.Object) { + cat(paste(callJMethod(.Object@jrdd, "toString"), "\n", sep="")) + }) + +setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { + .Object@env <- new.env() + .Object@env$isCached <- FALSE + .Object@env$isCheckpointed <- FALSE + .Object@env$jrdd_val <- jrdd_val + if (!is.null(jrdd_val)) { + # This tracks the serialization mode for jrdd_val + .Object@env$serializedMode <- prev@env$serializedMode + } + + .Object@prev <- prev + + isPipelinable <- function(rdd) { + e <- rdd@env + !(e$isCached || e$isCheckpointed) + } + + if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { + # This transformation is the first in its stage: + .Object@func <- cleanClosure(func) + .Object@prev_jrdd <- getJRDD(prev) + .Object@env$prev_serializedMode <- prev@env$serializedMode + # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD + # prev_serializedMode is used during the delayed computation of JRDD in getJRDD + } else { + pipelinedFunc <- function(partIndex, part) { + func(partIndex, prev@func(partIndex, part)) + } + .Object@func <- cleanClosure(pipelinedFunc) + .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline + # Get the serialization mode of the parent RDD + .Object@env$prev_serializedMode <- prev@env$prev_serializedMode + } + + .Object +}) + +#' @rdname RDD +#' @export +#' +#' @param jrdd Java object reference to the backing JavaRDD +#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +#' stores strings, and "row" if the RDD stores the rows of a DataFrame +#' @param isCached TRUE if the RDD is cached +#' @param isCheckpointed TRUE if the RDD has been checkpointed +RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, + isCheckpointed = FALSE) { + new("RDD", jrdd, serializedMode, isCached, isCheckpointed) +} + +PipelinedRDD <- function(prev, func) { + new("PipelinedRDD", prev, func, NULL) +} + +# Return the serialization mode for an RDD. +setGeneric("getSerializedMode", function(rdd, ...) { standardGeneric("getSerializedMode") }) +# For normal RDDs we can directly read the serializedMode +setMethod("getSerializedMode", signature(rdd = "RDD"), function(rdd) rdd@env$serializedMode ) +# For pipelined RDDs if jrdd_val is set then serializedMode should exist +# if not we return the defaultSerialization mode of "byte" as we don't know the serialization +# mode at this point in time. +setMethod("getSerializedMode", signature(rdd = "PipelinedRDD"), + function(rdd) { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$serializedMode) + } else { + return("byte") + } + }) + +# The jrdd accessor function. +setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) +setMethod("getJRDD", signature(rdd = "PipelinedRDD"), + function(rdd, serializedMode = "byte") { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$jrdd_val) + } + + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + serializedFuncArr <- serialize(rdd@func, connection = NULL) + + prev_jrdd <- rdd@prev_jrdd + + if (serializedMode == "string") { + rddRef <- newJObject("org.apache.spark.api.r.StringRRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } else { + rddRef <- newJObject("org.apache.spark.api.r.RRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } + # Save the serialization flag after we create a RRDD + rdd@env$serializedMode <- serializedMode + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val + }) + +setValidity("RDD", + function(object) { + jrdd <- getJRDD(object) + cls <- callJMethod(jrdd, "getClass") + className <- callJMethod(cls, "getName") + if (grep("spark.api.java.*RDD*", className) == 1) { + TRUE + } else { + paste("Invalid RDD class ", className) + } + }) + + +############ Actions and Transformations ############ + +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} +#' @rdname cache-methods +#' @aliases cache,RDD-method +setMethod("cache", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persist(rdd, "MEMORY_AND_DISK") +#'} +#' @rdname persist +#' @aliases persist,RDD-method +setMethod("persist", + signature(x = "RDD", newLevel = "character"), + function(x, newLevel) { + callJMethod(getJRDD(x), "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The RDD to unpersist +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersist(rdd) # rdd@@env$isCached == FALSE +#'} +#' @rdname unpersist-methods +#' @aliases unpersist,RDD-method +setMethod("unpersist", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "unpersist") + x@env$isCached <- FALSE + x + }) + +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param x The RDD to checkpoint +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoint") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} +#' @rdname checkpoint-methods +#' @aliases checkpoint,RDD-method +setMethod("checkpoint", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + callJMethod(jrdd, "checkpoint") + x@env$isCheckpointed <- TRUE + x + }) + +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' numPartitions(rdd) # 2L +#'} +#' @rdname numPartitions +#' @aliases numPartitions,RDD-method +setMethod("numPartitions", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + partitions <- callJMethod(jrdd, "partitions") + callJMethod(partitions, "size") + }) + +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collect(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} +#' @rdname collect-methods +#' @aliases collect,RDD-method +setMethod("collect", + signature(x = "RDD"), + function(x, flatten = TRUE) { + # Assumes a pairwise RDD is backed by a JavaPairRDD. + collected <- callJMethod(getJRDD(x), "collect") + convertJListToRList(collected, flatten, + serializedMode = getSerializedMode(x)) + }) + + +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) +#' @rdname collect-methods +#' @aliases collectPartition,integer,RDD-method +setMethod("collectPartition", + signature(x = "RDD", partitionId = "integer"), + function(x, partitionId) { + jPartitionsList <- callJMethod(getJRDD(x), + "collectPartitions", + as.list(as.integer(partitionId))) + + jList <- jPartitionsList[[1]] + convertJListToRList(jList, flatten = TRUE, + serializedMode = getSerializedMode(x)) + }) + +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +#' @rdname collect-methods +#' @aliases collectAsMap,RDD-method +setMethod("collectAsMap", + signature(x = "RDD"), + function(x) { + pairList <- collect(x) + map <- new.env() + lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) + as.list(map) + }) + +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' count(rdd) # 10 +#' length(rdd) # Same as count +#'} +#' @rdname count +#' @aliases count,RDD-method +setMethod("count", + signature(x = "RDD"), + function(x) { + countPartition <- function(part) { + as.integer(length(part)) + } + valsRDD <- lapplyPartition(x, countPartition) + vals <- collect(valsRDD) + sum(as.integer(vals)) + }) + +#' Return the number of elements in the RDD +#' @export +#' @rdname count +setMethod("length", + signature(x = "RDD"), + function(x) { + count(x) + }) + +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} +#' @rdname countByValue +#' @aliases countByValue,RDD-method +setMethod("countByValue", + signature(x = "RDD"), + function(x) { + ones <- lapply(x, function(item) { list(item, 1L) }) + collect(reduceByKey(ones, `+`, numPartitions(x))) + }) + +#' Apply a function to all elements +#' +#' This function creates a new RDD by applying the given transformation to all +#' elements of the given RDD +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @rdname lapply +#' @aliases lapply +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +#' collect(multiplyByTwo) # 2,4,6... +#'} +setMethod("lapply", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(partIndex, part) { + lapply(part, FUN) + } + lapplyPartitionsWithIndex(X, func) + }) + +#' @rdname lapply +#' @aliases map,RDD,function-method +setMethod("map", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' Flatten results after apply a function to all elements +#' +#' This function return a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collect(multiplyByTwo) # 2,20,4,40,6,60... +#'} +#' @rdname flatMap +#' @aliases flatMap,RDD,function-method +setMethod("flatMap", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + partitionFunc <- function(part) { + unlist( + lapply(part, FUN), + recursive = F + ) + } + lapplyPartition(X, partitionFunc) + }) + +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collect(partitionSum) # 15, 40 +#'} +#' @rdname lapplyPartition +#' @aliases lapplyPartition,RDD,function-method +setMethod("lapplyPartition", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) + }) + +#' mapPartitions is the same as lapplyPartition. +#' +#' @rdname lapplyPartition +#' @aliases mapPartitions,RDD,function-method +setMethod("mapPartitions", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { +#' partIndex * Reduce("+", part) }) +#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} +#' @rdname lapplyPartitionsWithIndex +#' @aliases lapplyPartitionsWithIndex,RDD,function-method +setMethod("lapplyPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + PipelinedRDD(X, FUN) + }) + +#' @rdname lapplyPartitionsWithIndex +#' @aliases mapPartitionsWithIndex,RDD,function-method +setMethod("mapPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, FUN) + }) + +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} +#' @rdname filterRDD +#' @aliases filterRDD,RDD,function-method +setMethod("filterRDD", + signature(x = "RDD", f = "function"), + function(x, f) { + filter.func <- function(part) { + Filter(f, part) + } + lapplyPartition(x, filter.func) + }) + +#' @rdname filterRDD +#' @aliases Filter +setMethod("Filter", + signature(f = "function", x = "RDD"), + function(f, x) { + filterRDD(x, f) + }) + +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param x The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} +#' @rdname reduce +#' @aliases reduce,RDD,ANY-method +setMethod("reduce", + signature(x = "RDD", func = "ANY"), + function(x, func) { + + reducePartition <- function(part) { + Reduce(func, part) + } + + partitionList <- collect(lapplyPartition(x, reducePartition), + flatten = FALSE) + Reduce(func, partitionList) + }) + +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} +#' @rdname maximum +#' @aliases maximum,RDD +setMethod("maximum", + signature(x = "RDD"), + function(x) { + reduce(x, max) + }) + +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} +#' @rdname minimum +#' @aliases minimum,RDD +setMethod("minimum", + signature(x = "RDD"), + function(x) { + reduce(x, min) + }) + +#' Add up the elements in an RDD. +#' +#' @param x The RDD to add up the elements in +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' sumRDD(rdd) # 55 +#'} +#' @rdname sumRDD +#' @aliases sumRDD,RDD +setMethod("sumRDD", + signature(x = "RDD"), + function(x) { + reduce(x, "+") + }) + +#' Applies a function to all elements in an RDD, and force evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} +#' @rdname foreach +#' @aliases foreach,RDD,function-method +setMethod("foreach", + signature(x = "RDD", func = "function"), + function(x, func) { + partition.func <- function(x) { + lapply(x, func) + NULL + } + invisible(collect(mapPartitions(x, partition.func))) + }) + +#' Applies a function to each partition in an RDD, and force evaluation. +#' +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} +#' @rdname foreach +#' @aliases foreachPartition,RDD,function-method +setMethod("foreachPartition", + signature(x = "RDD", func = "function"), + function(x, func) { + invisible(collect(mapPartitions(x, func))) + }) + +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' take(rdd, 2L) # list(1, 2) +#'} +#' @rdname take +#' @aliases take,RDD,numeric-method +setMethod("take", + signature(x = "RDD", num = "numeric"), + function(x, num) { + resList <- list() + index <- -1 + jrdd <- getJRDD(x) + numPartitions <- numPartitions(x) + serializedModeRDD <- getSerializedMode(x) + + # TODO(shivaram): Collect more than one partition based on size + # estimates similar to the scala version of `take`. + while (TRUE) { + index <- index + 1 + + if (length(resList) >= num || index >= numPartitions) + break + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + size <- num - length(resList) + # elems is capped to have at most `size` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = size, + serializedMode = serializedModeRDD) + + resList <- append(resList, elems) + } + resList + }) + + +#' First +#' +#' Return the first element of an RDD +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' first(rdd) +#' } +setMethod("first", + signature(x = "RDD"), + function(x) { + take(x, 1)[[1]] + }) + +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#'} +#' @rdname distinct +#' @aliases distinct,RDD-method +setMethod("distinct", + signature(x = "RDD"), + function(x, numPartitions = SparkR::numPartitions(x)) { + identical.mapped <- lapply(x, function(x) { list(x, NULL) }) + reduced <- reduceByKey(identical.mapped, + function(x, y) { x }, + numPartitions) + resRDD <- lapply(reduced, function(x) { x[[1]] }) + resRDD + }) + +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} +#' @rdname sampleRDD +#' @aliases sampleRDD,RDD +setMethod("sampleRDD", + signature(x = "RDD", withReplacement = "logical", + fraction = "numeric", seed = "integer"), + function(x, withReplacement, fraction, seed) { + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(partIndex, part) { + set.seed(seed) + res <- vector("list", length(part)) + len <- 0 + + # Discards some random values to ensure each partition has a + # different random seed. + runif(partIndex) + + for (elem in part) { + if (withReplacement) { + count <- rpois(1, fraction) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < fraction) { + len <- len + 1 + res[[len]] <- elem + } + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. + if (len > 0) + res[1:len] + else + list() + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) + +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} +#' @rdname takeSample +#' @aliases takeSample,RDD +setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", + num = "integer", seed = "integer"), + function(x, withReplacement, num, seed) { + # This function is ported from RDD.scala. + fraction <- 0.0 + total <- 0 + multiplier <- 3.0 + initialCount <- count(x) + maxSelected <- 0 + MAXINT <- .Machine$integer.max + + if (num < 0) + stop(paste("Negative number of elements requested")) + + if (initialCount > MAXINT - 1) { + maxSelected <- MAXINT - 1 + } else { + maxSelected <- initialCount + } + + if (num > initialCount && !withReplacement) { + total <- maxSelected + fraction <- multiplier * (maxSelected + 1) / initialCount + } else { + total <- num + fraction <- multiplier * (num + 1) / initialCount + } + + set.seed(seed) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + # If the first sample didn't turn out large enough, keep trying to + # take samples; this shouldn't happen often because we use a big + # multiplier for thei initial size + while (length(samples) < total) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + + # TODO(zongheng): investigate if this call is an in-place shuffle? + sample(samples)[1:total] + }) + +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} +#' @rdname keyBy +#' @aliases keyBy,RDD +setMethod("keyBy", + signature(x = "RDD", func = "function"), + function(x, func) { + apply.func <- function(x) { + list(func(x), x) + } + lapply(x, apply.func) + }) + +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso coalesce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} +#' @rdname repartition +#' @aliases repartition,RDD +setMethod("repartition", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions) { + coalesce(x, numPartitions, TRUE) + }) + +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso repartition +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} +#' @rdname coalesce +#' @aliases coalesce,RDD +setMethod("coalesce", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions, shuffle = FALSE) { + numPartitions <- numToInt(numPartitions) + if (shuffle || numPartitions > SparkR::numPartitions(x)) { + func <- function(partIndex, part) { + set.seed(partIndex) # partIndex as seed + start <- as.integer(sample(numPartitions, 1) - 1) + lapply(seq_along(part), + function(i) { + pos <- (start + i) %% numPartitions + list(pos, part[[i]]) + }) + } + shuffled <- lapplyPartitionsWithIndex(x, func) + repartitioned <- partitionBy(shuffled, numPartitions) + values(repartitioned) + } else { + jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) + RDD(jrdd) + } + }) + +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved +#' @seealso objectFile +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsObjectFile +#' @aliases saveAsObjectFile,RDD +setMethod("saveAsObjectFile", + signature(x = "RDD", path = "character"), + function(x, path) { + # If serializedMode == "string" we need to serialize the data before saving it since + # objectFile() assumes serializedMode == "byte". + if (getSerializedMode(x) != "byte") { + x <- serializeToBytes(x) + } + # Return nothing + invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) + }) + +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the partitions of the text file are saved +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsTextFile +#' @aliases saveAsTextFile,RDD +setMethod("saveAsTextFile", + signature(x = "RDD", path = "character"), + function(x, path) { + func <- function(str) { + toString(str) + } + stringRdd <- lapply(x, func) + # Return nothing + invisible( + callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) + }) + +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +#' @rdname sortBy +#' @aliases sortBy,RDD,RDD-method +setMethod("sortBy", + signature(x = "RDD", func = "function"), + function(x, func, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + values(sortByKey(keyBy(x, func), ascending, numPartitions)) + }) + +# Helper function to get first N elements from an RDD in the specified order. +# Param: +# x An RDD. +# num Number of elements to return. +# ascending A flag to indicate whether the sorting is ascending or descending. +# Return: +# A list of the first N elements from the RDD in the specified order. +# +takeOrderedElem <- function(x, num, ascending = TRUE) { + if (num <= 0L) { + return(list()) + } + + partitionFunc <- function(part) { + if (num < length(part)) { + # R limitation: order works only on primitive types! + ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) + part[ord[1:num]] + } else { + part + } + } + + newRdd <- mapPartitions(x, partitionFunc) + + resList <- list() + index <- -1 + jrdd <- getJRDD(newRdd) + numPartitions <- numPartitions(newRdd) + serializedModeRDD <- getSerializedMode(newRdd) + + while (TRUE) { + index <- index + 1 + + if (index >= numPartitions) { + ord <- order(unlist(resList, recursive = FALSE), decreasing = !ascending) + resList <- resList[ord[1:num]] + break + } + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + # elems is capped to have at most `num` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = num, + serializedMode = serializedModeRDD) + + resList <- append(resList, elems) + } + resList +} + +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +#' @rdname takeOrdered +#' @aliases takeOrdered,RDD,RDD-method +setMethod("takeOrdered", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num) + }) + +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +#' @rdname top +#' @aliases top,RDD,RDD-method +setMethod("top", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num, FALSE) + }) + +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +#' @rdname fold +#' @aliases fold,RDD,RDD-method +setMethod("fold", + signature(x = "RDD", zeroValue = "ANY", op = "ANY"), + function(x, zeroValue, op) { + aggregateRDD(x, zeroValue, op, op) + }) + +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4)) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +#' @rdname aggregateRDD +#' @aliases aggregateRDD,RDD,RDD-method +setMethod("aggregateRDD", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), + function(x, zeroValue, seqOp, combOp) { + partitionFunc <- function(part) { + Reduce(seqOp, part, zeroValue) + } + + partitionList <- collect(lapplyPartition(x, partitionFunc), + flatten = FALSE) + Reduce(combOp, partitionList, zeroValue) + }) + +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. +#' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +#' @rdname pipeRDD +#' @aliases pipeRDD,RDD,character-method +setMethod("pipeRDD", + signature(x = "RDD", command = "character"), + function(x, command, env = list()) { + func <- function(part) { + trim.trailing.func <- function(x) { + sub("[\r\n]*$", "", toString(x)) + } + input <- unlist(lapply(part, trim.trailing.func)) + res <- system2(command, stdout = TRUE, input = input, env = env) + lapply(res, trim.trailing.func) + } + lapplyPartition(x, func) + }) + +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. +#' @rdname name +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} +#' @rdname name +#' @aliases name,RDD +setMethod("name", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "name") + }) + +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +#' @rdname setName +#' @aliases setName,RDD +setMethod("setName", + signature(x = "RDD", name = "character"), + function(x, name) { + callJMethod(getJRDD(x), "setName", name) + x + }) + +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithIndex +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +#' @rdname zipWithUniqueId +#' @aliases zipWithUniqueId,RDD +setMethod("zipWithUniqueId", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + + partitionFunc <- function(partIndex, part) { + mapply( + function(item, index) { + list(item, (index - 1) * n + partIndex) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithUniqueId +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +#' @rdname zipWithIndex +#' @aliases zipWithIndex,RDD +setMethod("zipWithIndex", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + if (n > 1) { + nums <- collect(lapplyPartition(x, + function(part) { + list(length(part)) + })) + startIndices <- Reduce("+", nums, accumulate = TRUE) + } + + partitionFunc <- function(partIndex, part) { + if (partIndex == 0) { + startIndex <- 0 + } else { + startIndex <- startIndices[[partIndex]] + } + + mapply( + function(item, index) { + list(item, index - 1 + startIndex) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Coalesce all elements within each partition of an RDD into a list. +#' +#' @param x An RDD. +#' @return An RDD created by coalescing all elements within +#' each partition into a list. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, as.list(1:4), 2L) +#' collect(glom(rdd)) +#' # list(list(1, 2), list(3, 4)) +#'} +#' @rdname glom +#' @aliases glom,RDD +setMethod("glom", + signature(x = "RDD"), + function(x) { + partitionFunc <- function(part) { + list(part) + } + + lapplyPartition(x, partitionFunc) + }) + +############ Binary Functions ############# + +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} +#' @rdname unionRDD +#' @aliases unionRDD,RDD,RDD-method +setMethod("unionRDD", + signature(x = "RDD", y = "RDD"), + function(x, y) { + if (getSerializedMode(x) == getSerializedMode(y)) { + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, getSerializedMode(x)) + } else { + # One of the RDDs is not serialized, we need to serialize it first. + if (getSerializedMode(x) != "byte") x <- serializeToBytes(x) + if (getSerializedMode(y) != "byte") y <- serializeToBytes(y) + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, "byte") + } + union.rdd + }) + +#' Zip an RDD with another RDD. +#' +#' Zips this RDD with another one, returning key-value pairs with the +#' first element in each RDD second element in each RDD, etc. Assumes +#' that the two RDDs have the same number of partitions and the same +#' number of elements in each partition (e.g. one was made through +#' a map on the other). +#' +#' @param x An RDD to be zipped. +#' @param other Another RDD to be zipped. +#' @return An RDD zipped from the two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 0:4) +#' rdd2 <- parallelize(sc, 1000:1004) +#' collect(zipRDD(rdd1, rdd2)) +#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#'} +#' @rdname zipRDD +#' @aliases zipRDD,RDD +setMethod("zipRDD", + signature(x = "RDD", other = "RDD"), + function(x, other) { + n1 <- numPartitions(x) + n2 <- numPartitions(other) + if (n1 != n2) { + stop("Can only zip RDDs which have the same number of partitions.") + } + + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "zip", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) + + mergePartitions(rdd, TRUE) + }) + +#' Cartesian product of this RDD and another one. +#' +#' Return the Cartesian product of this RDD and another one, +#' that is, the RDD of all pairs of elements (a, b) where a +#' is in this and b is in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @return A new RDD which is the Cartesian product of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2) +#' sortByKey(cartesian(rdd, rdd)) +#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#'} +#' @rdname cartesian +#' @aliases cartesian,RDD,RDD-method +setMethod("cartesian", + signature(x = "RDD", other = "RDD"), + function(x, other) { + rdds <- appendPartitionLengths(x, other) + jrdd <- callJMethod(getJRDD(rdds[[1]]), "cartesian", getJRDD(rdds[[2]])) + # The jrdd's elements are of scala Tuple2 type. The serialized + # flag here is used for the elements inside the tuples. + rdd <- RDD(jrdd, getSerializedMode(rdds[[1]])) + + mergePartitions(rdd, FALSE) + }) + +#' Subtract an RDD with another RDD. +#' +#' Return an RDD with the elements from this that are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the elements from this that are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +#' rdd2 <- parallelize(sc, list(2, 4)) +#' collect(subtract(rdd1, rdd2)) +#' # list(1, 1, 3) +#'} +#' @rdname subtract +#' @aliases subtract,RDD +setMethod("subtract", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + mapFunction <- function(e) { list(e, NA) } + rdd1 <- map(x, mapFunction) + rdd2 <- map(other, mapFunction) + keys(subtractByKey(rdd1, rdd2, numPartitions)) + }) + +#' Intersection of this RDD and another one. +#' +#' Return the intersection of this RDD and another one. +#' The output will not contain any duplicate elements, +#' even if the input RDDs did. Performs a hash partition +#' across the cluster. +#' Note that this method performs a shuffle internally. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions The number of partitions in the result RDD. +#' @return An RDD which is the intersection of these two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +#' collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' # list(1, 2, 3) +#'} +#' @rdname intersection +#' @aliases intersection,RDD +setMethod("intersection", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + rdd1 <- map(x, function(v) { list(v, NA) }) + rdd2 <- map(other, function(v) { list(v, NA) }) + + filterFunction <- function(elem) { + iters <- elem[[2]] + all(as.vector( + lapply(iters, function(iter) { length(iter) > 0 }), mode = "logical")) + } + + keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) + }) + +#' Zips an RDD's partitions with one (or more) RDD(s). +#' Same as zipPartitions in Spark. +#' +#' @param ... RDDs to be zipped. +#' @param func A function to transform zipped partitions. +#' @return A new RDD by applying a function to the zipped partitions. +#' Assumes that all the RDDs have the *same number of partitions*, but +#' does *not* require them to have the same number of elements in each partition. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 +#' rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 +#' rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 +#' collect(zipPartitions(rdd1, rdd2, rdd3, +#' func = function(x, y, z) { list(list(x, y, z))} )) +#' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) +#'} +#' @rdname zipRDD +#' @aliases zipPartitions,RDD +setMethod("zipPartitions", + "RDD", + function(..., func) { + rrdds <- list(...) + if (length(rrdds) == 1) { + return(rrdds[[1]]) + } + nPart <- sapply(rrdds, numPartitions) + if (length(unique(nPart)) != 1) { + stop("Can only zipPartitions RDDs which have the same number of partitions.") + } + + rrdds <- lapply(rrdds, function(rdd) { + mapPartitionsWithIndex(rdd, function(partIndex, part) { + print(length(part)) + list(list(partIndex, part)) + }) + }) + union.rdd <- Reduce(unionRDD, rrdds) + zipped.rdd <- values(groupByKey(union.rdd, numPartitions = nPart[1])) + res <- mapPartitions(zipped.rdd, function(plist) { + do.call(func, plist[[1]]) + }) + res + }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R new file mode 100644 index 0000000000000..4f05ba524a01a --- /dev/null +++ b/R/pkg/R/SQLContext.R @@ -0,0 +1,494 @@ +# +# 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. +# + +# SQLcontext.R: SQLContext-driven functions + +#' infer the SQL type +infer_type <- function(x) { + if (is.null(x)) { + stop("can not infer type from NULL") + } + + # class of POSIXlt is c("POSIXlt" "POSIXt") + type <- switch(class(x)[[1]], + integer = "integer", + character = "string", + logical = "boolean", + double = "double", + numeric = "double", + raw = "binary", + list = "array", + environment = "map", + Date = "date", + POSIXlt = "timestamp", + POSIXct = "timestamp", + stop(paste("Unsupported type for DataFrame:", class(x)))) + + if (type == "map") { + stopifnot(length(x) > 0) + key <- ls(x)[[1]] + list(type = "map", + keyType = "string", + valueType = infer_type(get(key, x)), + valueContainsNull = TRUE) + } else if (type == "array") { + stopifnot(length(x) > 0) + names <- names(x) + if (is.null(names)) { + list(type = "array", elementType = infer_type(x[[1]]), containsNull = TRUE) + } else { + # StructType + types <- lapply(x, infer_type) + fields <- lapply(1:length(x), function(i) { + structField(names[[i]], types[[i]], TRUE) + }) + do.call(structType, fields) + } + } else if (length(x) > 1) { + list(type = "array", elementType = type, containsNull = TRUE) + } else { + type + } +} + +#' Create a DataFrame from an RDD +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param sqlCtx A SQLContext +#' @param data An RDD or list or data.frame +#' @param schema a list of column names or named list (StructType), optional +#' @return an DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- createDataFrame(sqlCtx, rdd) +#' } + +# TODO(davies): support sampling and infer type from NA +createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { + if (is.data.frame(data)) { + # get the names of columns, they will be put into RDD + schema <- names(data) + n <- nrow(data) + m <- ncol(data) + # get rid of factor type + dropFactor <- function(x) { + if (is.factor(x)) { + as.character(x) + } else { + x + } + } + data <- lapply(1:n, function(i) { + lapply(1:m, function(j) { dropFactor(data[i,j]) }) + }) + } + if (is.list(data)) { + sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlCtx) + rdd <- parallelize(sc, data) + } else if (inherits(data, "RDD")) { + rdd <- data + } else { + stop(paste("unexpected type:", class(data))) + } + + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { + row <- first(rdd) + names <- if (is.null(schema)) { + names(row) + } else { + as.list(schema) + } + if (is.null(names)) { + names <- lapply(1:length(row), function(x) { + paste("_", as.character(x), sep = "") + }) + } + + # SPAKR-SQL does not support '.' in column name, so replace it with '_' + # TODO(davies): remove this once SPARK-2775 is fixed + names <- lapply(names, function(n) { + nn <- gsub("[.]", "_", n) + if (nn != n) { + warning(paste("Use", nn, "instead of", n, " as column name")) + } + nn + }) + + types <- lapply(row, infer_type) + fields <- lapply(1:length(row), function(i) { + structField(names[[i]], types[[i]], TRUE) + }) + schema <- do.call(structType, fields) + } + + stopifnot(class(schema) == "structType") + # schemaString <- tojson(schema) + + jrdd <- getJRDD(lapply(rdd, function(x) x), "row") + srdd <- callJMethod(jrdd, "rdd") + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", + srdd, schema$jobj, sqlCtx) + dataFrame(sdf) +} + +#' toDF +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param x An RDD +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- toDF(rdd) +#' } + +setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) + +setMethod("toDF", signature(x = "RDD"), + function(x, ...) { + sqlCtx <- if (exists(".sparkRHivesc", envir = .sparkREnv)) { + get(".sparkRHivesc", envir = .sparkREnv) + } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + get(".sparkRSQLsc", envir = .sparkREnv) + } else { + stop("no SQL context available") + } + createDataFrame(sqlCtx, x, ...) + }) + +#' Create a DataFrame from a JSON file. +#' +#' Loads a JSON file (one object per line), returning the result as a DataFrame +#' It goes through the entire dataset once to determine the schema. +#' +#' @param sqlCtx SQLContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' } + +jsonFile <- function(sqlCtx, path) { + # Allow the user to have a more flexible definiton of the text file path + path <- normalizePath(path) + # Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + sdf <- callJMethod(sqlCtx, "jsonFile", path) + dataFrame(sdf) +} + + +#' JSON RDD +#' +#' Loads an RDD storing one JSON object per string as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param rdd An RDD of JSON string +#' @param schema A StructType object to use as schema +#' @param samplingRatio The ratio of simpling used to infer the schema +#' @return A DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- texFile(sc, "path/to/json") +#' df <- jsonRDD(sqlCtx, rdd) +#' } + +# TODO: support schema +jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) { + rdd <- serializeToString(rdd) + if (is.null(schema)) { + sdf <- callJMethod(sqlCtx, "jsonRDD", callJMethod(getJRDD(rdd), "rdd"), samplingRatio) + dataFrame(sdf) + } else { + stop("not implemented") + } +} + + +#' Create a DataFrame from a Parquet file. +#' +#' Loads a Parquet file, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param ... Path(s) of parquet file(s) to read. +#' @return DataFrame +#' @export + +# TODO: Implement saveasParquetFile and write examples for both +parquetFile <- function(sqlCtx, ...) { + # Allow the user to have a more flexible definiton of the text file path + paths <- lapply(list(...), normalizePath) + sdf <- callJMethod(sqlCtx, "parquetFile", paths) + dataFrame(sdf) +} + +#' SQL Query +#' +#' Executes a SQL query using Spark, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param sqlQuery A character vector containing the SQL query +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- sql(sqlCtx, "SELECT * FROM table") +#' } + +sql <- function(sqlCtx, sqlQuery) { + sdf <- callJMethod(sqlCtx, "sql", sqlQuery) + dataFrame(sdf) +} + + +#' Create a DataFrame from a SparkSQL Table +#' +#' Returns the specified Table as a DataFrame. The Table must have already been registered +#' in the SQLContext. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The SparkSQL Table to convert to a DataFrame. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- table(sqlCtx, "table") +#' } + +table <- function(sqlCtx, tableName) { + sdf <- callJMethod(sqlCtx, "table", tableName) + dataFrame(sdf) +} + + +#' Tables +#' +#' Returns a DataFrame containing names of tables in the given database. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tables(sqlCtx, "hive") +#' } + +tables <- function(sqlCtx, databaseName = NULL) { + jdf <- if (is.null(databaseName)) { + callJMethod(sqlCtx, "tables") + } else { + callJMethod(sqlCtx, "tables", databaseName) + } + dataFrame(jdf) +} + + +#' Table Names +#' +#' Returns the names of tables in the given database as an array. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a list of table names +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tableNames(sqlCtx, "hive") +#' } + +tableNames <- function(sqlCtx, databaseName = NULL) { + if (is.null(databaseName)) { + callJMethod(sqlCtx, "tableNames") + } else { + callJMethod(sqlCtx, "tableNames", databaseName) + } +} + + +#' Cache Table +#' +#' Caches the specified table in-memory. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being cached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' cacheTable(sqlCtx, "table") +#' } + +cacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "cacheTable", tableName) +} + +#' Uncache Table +#' +#' Removes the specified table from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being uncached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' uncacheTable(sqlCtx, "table") +#' } + +uncacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "uncacheTable", tableName) +} + +#' Clear Cache +#' +#' Removes all cached tables from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @examples +#' \dontrun{ +#' clearCache(sqlCtx) +#' } + +clearCache <- function(sqlCtx) { + callJMethod(sqlCtx, "clearCache") +} + +#' Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the SparkSQL table to be dropped. +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' registerTempTable(df, "table") +#' dropTempTable(sqlCtx, "table") +#' } + +dropTempTable <- function(sqlCtx, tableName) { + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + callJMethod(sqlCtx, "dropTempTable", tableName) +} + +#' Load an DataFrame +#' +#' Returns the dataset in a data source as a DataFrame +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- load(sqlCtx, "path/to/file.json", source = "json") +#' } + +loadDF <- function(sqlCtx, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "load", source, options) + dataFrame(sdf) +} + +#' Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns the DataFrame associated with the external table. +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName A name of the table +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- sparkRSQL.createExternalTable(sqlCtx, "myjson", path="path/to/json", source="json") +#' } + +createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options) + dataFrame(sdf) +} diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R new file mode 100644 index 0000000000000..2fb6fae55f28c --- /dev/null +++ b/R/pkg/R/backend.R @@ -0,0 +1,115 @@ +# +# 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. +# + +# Methods to call into SparkRBackend. + + +# Returns TRUE if object is an instance of given class +isInstanceOf <- function(jobj, className) { + stopifnot(class(jobj) == "jobj") + cls <- callJStatic("java.lang.Class", "forName", className) + callJMethod(cls, "isInstance", jobj) +} + +# Call a Java method named methodName on the object +# specified by objId. objId should be a "jobj" returned +# from the SparkRBackend. +callJMethod <- function(objId, methodName, ...) { + stopifnot(class(objId) == "jobj") + if (!isValidJobj(objId)) { + stop("Invalid jobj ", objId$id, + ". If SparkR was restarted, Spark operations need to be re-executed.") + } + invokeJava(isStatic = FALSE, objId$id, methodName, ...) +} + +# Call a static method on a specified className +callJStatic <- function(className, methodName, ...) { + invokeJava(isStatic = TRUE, className, methodName, ...) +} + +# Create a new object of the specified class name +newJObject <- function(className, ...) { + invokeJava(isStatic = TRUE, className, methodName = "", ...) +} + +# Remove an object from the SparkR backend. This is done +# automatically when a jobj is garbage collected. +removeJObject <- function(objId) { + invokeJava(isStatic = TRUE, "SparkRHandler", "rm", objId) +} + +isRemoveMethod <- function(isStatic, objId, methodName) { + isStatic == TRUE && objId == "SparkRHandler" && methodName == "rm" +} + +# Invoke a Java method on the SparkR backend. Users +# should typically use one of the higher level methods like +# callJMethod, callJStatic etc. instead of using this. +# +# isStatic - TRUE if the method to be called is static +# objId - String that refers to the object on which method is invoked +# Should be a jobj id for non-static methods and the classname +# for static methods +# methodName - name of method to be invoked +invokeJava <- function(isStatic, objId, methodName, ...) { + if (!exists(".sparkRCon", .sparkREnv)) { + stop("No connection to backend found. Please re-run sparkR.init") + } + + # If this isn't a removeJObject call + if (!isRemoveMethod(isStatic, objId, methodName)) { + objsToRemove <- ls(.toRemoveJobjs) + if (length(objsToRemove) > 0) { + sapply(objsToRemove, + function(e) { + removeJObject(e) + }) + rm(list = objsToRemove, envir = .toRemoveJobjs) + } + } + + + rc <- rawConnection(raw(0), "r+") + + writeBoolean(rc, isStatic) + writeString(rc, objId) + writeString(rc, methodName) + + args <- list(...) + writeInt(rc, length(args)) + writeArgs(rc, args) + + # Construct the whole request message to send it once, + # avoiding write-write-read pattern in case of Nagle's algorithm. + # Refer to http://en.wikipedia.org/wiki/Nagle%27s_algorithm for the details. + bytesToSend <- rawConnectionValue(rc) + close(rc) + rc <- rawConnection(raw(0), "r+") + writeInt(rc, length(bytesToSend)) + writeBin(bytesToSend, rc) + requestMessage <- rawConnectionValue(rc) + close(rc) + + conn <- get(".sparkRCon", .sparkREnv) + writeBin(requestMessage, conn) + + # TODO: check the status code to output error information + returnStatus <- readInt(conn) + stopifnot(returnStatus == 0) + readObject(conn) +} diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R new file mode 100644 index 0000000000000..583fa2e7fdcfd --- /dev/null +++ b/R/pkg/R/broadcast.R @@ -0,0 +1,86 @@ +# +# 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. +# + +# S4 class representing Broadcast variables + +# Hidden environment that holds values for broadcast variables +# This will not be serialized / shipped by default +.broadcastNames <- new.env() +.broadcastValues <- new.env() +.broadcastIdToName <- new.env() + +#' @title S4 class that represents a Broadcast variable +#' @description Broadcast variables can be created using the broadcast +#' function from a \code{SparkContext}. +#' @rdname broadcast-class +#' @seealso broadcast +#' +#' @param id Id of the backing Spark broadcast variable +#' @export +setClass("Broadcast", slots = list(id = "character")) + +#' @rdname broadcast-class +#' @param value Value of the broadcast variable +#' @param jBroadcastRef reference to the backing Java broadcast object +#' @param objName name of broadcasted object +#' @export +Broadcast <- function(id, value, jBroadcastRef, objName) { + .broadcastValues[[id]] <- value + .broadcastNames[[as.character(objName)]] <- jBroadcastRef + .broadcastIdToName[[id]] <- as.character(objName) + new("Broadcast", id = id) +} + +#' @description +#' \code{value} can be used to get the value of a broadcast variable inside +#' a distributed function. +#' +#' @param bcast The broadcast variable to get +#' @rdname broadcast +#' @aliases value,Broadcast-method +setMethod("value", + signature(bcast = "Broadcast"), + function(bcast) { + if (exists(bcast@id, envir = .broadcastValues)) { + get(bcast@id, envir = .broadcastValues) + } else { + NULL + } + }) + +#' Internal function to set values of a broadcast variable. +#' +#' This function is used internally by Spark to set the value of a broadcast +#' variable on workers. Not intended for use outside the package. +#' +#' @rdname broadcast-internal +#' @seealso broadcast, value + +#' @param bcastId The id of broadcast variable to set +#' @param value The value to be set +#' @export +setBroadcastValue <- function(bcastId, value) { + bcastIdStr <- as.character(bcastId) + .broadcastValues[[bcastIdStr]] <- value +} + +#' Helper function to clear the list of broadcast variables we know about +#' Should be called when the SparkR JVM backend is shutdown +clearBroadcastVariables <- function() { + bcasts <- ls(.broadcastNames) + rm(list = bcasts, envir = .broadcastNames) +} diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R new file mode 100644 index 0000000000000..1281c41213e32 --- /dev/null +++ b/R/pkg/R/client.R @@ -0,0 +1,57 @@ +# +# 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. +# + +# Client code to connect to SparkRBackend + +# Creates a SparkR client connection object +# if one doesn't already exist +connectBackend <- function(hostname, port, timeout = 6000) { + if (exists(".sparkRcon", envir = .sparkREnv)) { + if (isOpen(.sparkREnv[[".sparkRCon"]])) { + cat("SparkRBackend client connection already exists\n") + return(get(".sparkRcon", envir = .sparkREnv)) + } + } + + con <- socketConnection(host = hostname, port = port, server = FALSE, + blocking = TRUE, open = "wb", timeout = timeout) + + assign(".sparkRCon", con, envir = .sparkREnv) + con +} + +launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts) { + if (.Platform$OS.type == "unix") { + sparkSubmitBinName = "spark-submit" + } else { + sparkSubmitBinName = "spark-submit.cmd" + } + + if (sparkHome != "") { + sparkSubmitBin <- file.path(sparkHome, "bin", sparkSubmitBinName) + } else { + sparkSubmitBin <- sparkSubmitBinName + } + + if (jars != "") { + jars <- paste("--jars", jars) + } + + combinedArgs <- paste(jars, sparkSubmitOpts, args, sep = " ") + cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n") + invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) +} diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R new file mode 100644 index 0000000000000..95fb9ff0887b6 --- /dev/null +++ b/R/pkg/R/column.R @@ -0,0 +1,199 @@ +# +# 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. +# + +# Column Class + +#' @include generics.R jobj.R schema.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame column +#' @description The column class supports unary, binary operations on DataFrame columns + +#' @rdname column +#' +#' @param jc reference to JVM DataFrame column +#' @export +setClass("Column", + slots = list(jc = "jobj")) + +setMethod("initialize", "Column", function(.Object, jc) { + .Object@jc <- jc + .Object +}) + +column <- function(jc) { + new("Column", jc) +} + +col <- function(x) { + column(callJStatic("org.apache.spark.sql.functions", "col", x)) +} + +#' @rdname show +setMethod("show", "Column", + function(object) { + cat("Column", callJMethod(object@jc, "toString"), "\n") + }) + +operators <- list( + "+" = "plus", "-" = "minus", "*" = "multiply", "/" = "divide", "%%" = "mod", + "==" = "equalTo", ">" = "gt", "<" = "lt", "!=" = "notEqual", "<=" = "leq", ">=" = "geq", + # we can not override `&&` and `||`, so use `&` and `|` instead + "&" = "and", "|" = "or" #, "!" = "unary_$bang" +) +column_functions1 <- c("asc", "desc", "isNull", "isNotNull") +column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains") +functions <- c("min", "max", "sum", "avg", "mean", "count", "abs", "sqrt", + "first", "last", "lower", "upper", "sumDistinct") + +createOperator <- function(op) { + setMethod(op, + signature(e1 = "Column"), + function(e1, e2) { + jc <- if (missing(e2)) { + if (op == "-") { + callJMethod(e1@jc, "unary_$minus") + } else { + callJMethod(e1@jc, operators[[op]]) + } + } else { + if (class(e2) == "Column") { + e2 <- e2@jc + } + callJMethod(e1@jc, operators[[op]], e2) + } + column(jc) + }) +} + +createColumnFunction1 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + column(callJMethod(x@jc, name)) + }) +} + +createColumnFunction2 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x, data) { + if (class(data) == "Column") { + data <- data@jc + } + jc <- callJMethod(x@jc, name, data) + column(jc) + }) +} + +createStaticFunction <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", name, x@jc) + column(jc) + }) +} + +createMethods <- function() { + for (op in names(operators)) { + createOperator(op) + } + for (name in column_functions1) { + createColumnFunction1(name) + } + for (name in column_functions2) { + createColumnFunction2(name) + } + for (x in functions) { + createStaticFunction(x) + } +} + +createMethods() + +#' alias +#' +#' Set a new name for a column +setMethod("alias", + signature(object = "Column"), + function(object, data) { + if (is.character(data)) { + column(callJMethod(object@jc, "as", data)) + } else { + stop("data should be character") + } + }) + +#' An expression that returns a substring. +#' +#' @param start starting position +#' @param stop ending position +setMethod("substr", signature(x = "Column"), + function(x, start, stop) { + jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1)) + column(jc) + }) + +#' Casts the column to a different data type. +#' @examples +#' \dontrun{ +#' cast(df$age, "string") +#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) +#' } +setMethod("cast", + signature(x = "Column"), + function(x, dataType) { + if (is.character(dataType)) { + column(callJMethod(x@jc, "cast", dataType)) + } else if (is.list(dataType)) { + json <- tojson(dataType) + jdataType <- callJStatic("org.apache.spark.sql.types.DataType", "fromJson", json) + column(callJMethod(x@jc, "cast", jdataType)) + } else { + stop("dataType should be character or list") + } + }) + +#' Approx Count Distinct +#' +#' Returns the approximate number of distinct items in a group. +#' +setMethod("approxCountDistinct", + signature(x = "Column"), + function(x, rsd = 0.95) { + jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd) + column(jc) + }) + +#' Count Distinct +#' +#' returns the number of distinct items in a group. +#' +setMethod("countDistinct", + signature(x = "Column"), + function(x, ...) { + jcol <- lapply(list(...), function (x) { + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "countDistinct", x@jc, + listToSeq(jcol)) + column(jc) + }) + diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R new file mode 100644 index 0000000000000..b4845b6948997 --- /dev/null +++ b/R/pkg/R/context.R @@ -0,0 +1,225 @@ +# +# 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. +# + +# context.R: SparkContext driven functions + +getMinPartitions <- function(sc, minPartitions) { + if (is.null(minPartitions)) { + defaultParallelism <- callJMethod(sc, "defaultParallelism") + minPartitions <- min(defaultParallelism, 2) + } + as.integer(minPartitions) +} + +#' Create an RDD from a text file. +#' +#' This function reads a text file from HDFS, a local file system (available on all +#' nodes), or any Hadoop-supported file system URI, and creates an +#' RDD of strings from it. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minPartitions Minimum number of partitions to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD where each item is of type \code{character} +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' lines <- textFile(sc, "myfile.txt") +#'} +textFile <- function(sc, path, minPartitions = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "textFile", path, getMinPartitions(sc, minPartitions)) + # jrdd is of type JavaRDD[String] + RDD(jrdd, "string") +} + +#' Load an RDD saved as a SequenceFile containing serialized objects. +#' +#' The file to be loaded should be one that was previously generated by calling +#' saveAsObjectFile() of the RDD class. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minPartitions Minimum number of partitions to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD containing serialized R objects. +#' @seealso saveAsObjectFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- objectFile(sc, "myfile") +#'} +objectFile <- function(sc, path, minPartitions = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "objectFile", path, getMinPartitions(sc, minPartitions)) + # Assume the RDD contains serialized R objects. + RDD(jrdd, "byte") +} + +#' Create an RDD from a homogeneous list or vector. +#' +#' This function creates an RDD from a local homogeneous list in R. The elements +#' in the list are split into \code{numSlices} slices and distributed to nodes +#' in the cluster. +#' +#' @param sc SparkContext to use +#' @param coll collection to parallelize +#' @param numSlices number of partitions to create in the RDD +#' @return an RDD created from this collection +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2) +#' # The RDD should contain 10 elements +#' length(rdd) +#'} +parallelize <- function(sc, coll, numSlices = 1) { + # TODO: bound/safeguard numSlices + # TODO: unit tests for if the split works for all primitives + # TODO: support matrix, data frame, etc + if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) { + if (is.data.frame(coll)) { + message(paste("context.R: A data frame is parallelized by columns.")) + } else { + if (is.matrix(coll)) { + message(paste("context.R: A matrix is parallelized by elements.")) + } else { + message(paste("context.R: parallelize() currently only supports lists and vectors.", + "Calling as.list() to coerce coll into a list.")) + } + } + coll <- as.list(coll) + } + + if (numSlices > length(coll)) + numSlices <- length(coll) + + sliceLen <- ceiling(length(coll) / numSlices) + slices <- split(coll, rep(1:(numSlices + 1), each = sliceLen)[1:length(coll)]) + + # Serialize each slice: obtain a list of raws, or a list of lists (slices) of + # 2-tuples of raws + serializedSlices <- lapply(slices, serialize, connection = NULL) + + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", + "createRDDFromArray", sc, serializedSlices) + + RDD(jrdd, "byte") +} + +#' Include this specified package on all workers +#' +#' This function can be used to include a package on all workers before the +#' user's code is executed. This is useful in scenarios where other R package +#' functions are used in a function passed to functions like \code{lapply}. +#' NOTE: The package is assumed to be installed on every node in the Spark +#' cluster. +#' +#' @param sc SparkContext to use +#' @param pkg Package name +#' +#' @export +#' @examples +#'\dontrun{ +#' library(Matrix) +#' +#' sc <- sparkR.init() +#' # Include the matrix library we will be using +#' includePackage(sc, Matrix) +#' +#' generateSparse <- function(x) { +#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +#' } +#' +#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +#' collect(rdd) +#'} +includePackage <- function(sc, pkg) { + pkg <- as.character(substitute(pkg)) + if (exists(".packages", .sparkREnv)) { + packages <- .sparkREnv$.packages + } else { + packages <- list() + } + packages <- c(packages, pkg) + .sparkREnv$.packages <- packages +} + +#' @title Broadcast a variable to all workers +#' +#' @description +#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +#' object for reading it in distributed functions. +#' +#' @param sc Spark Context to use +#' @param object Object to be broadcast +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2, 2L) +#' +#' # Large Matrix object that we want to broadcast +#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +#' randomMatBr <- broadcast(sc, randomMat) +#' +#' # Use the broadcast variable inside the function +#' useBroadcast <- function(x) { +#' sum(value(randomMatBr) * x) +#' } +#' sumRDD <- lapply(rdd, useBroadcast) +#'} +broadcast <- function(sc, object) { + objName <- as.character(substitute(object)) + serializedObj <- serialize(object, connection = NULL) + + jBroadcast <- callJMethod(sc, "broadcast", serializedObj) + id <- as.character(callJMethod(jBroadcast, "id")) + + Broadcast(id, object, jBroadcast, objName) +} + +#' @title Set the checkpoint directory +#' +#' Set the directory under which RDDs are going to be checkpointed. The +#' directory must be a HDFS path if running on a cluster. +#' +#' @param sc Spark Context to use +#' @param dirName Directory path +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "~/checkpoint") +#' rdd <- parallelize(sc, 1:2, 2L) +#' checkpoint(rdd) +#'} +setCheckpointDir <- function(sc, dirName) { + invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) +} diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R new file mode 100644 index 0000000000000..257b435607ce8 --- /dev/null +++ b/R/pkg/R/deserialize.R @@ -0,0 +1,184 @@ +# +# 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. +# + +# Utility functions to deserialize objects from Java. + +# Type mapping from Java to R +# +# void -> NULL +# Int -> integer +# String -> character +# Boolean -> logical +# Double -> double +# Long -> double +# Array[Byte] -> raw +# Date -> Date +# Time -> POSIXct +# +# Array[T] -> list() +# Object -> jobj + +readObject <- function(con) { + # Read type first + type <- readType(con) + readTypedObject(con, type) +} + +readTypedObject <- function(con, type) { + switch (type, + "i" = readInt(con), + "c" = readString(con), + "b" = readBoolean(con), + "d" = readDouble(con), + "r" = readRaw(con), + "D" = readDate(con), + "t" = readTime(con), + "l" = readList(con), + "n" = NULL, + "j" = getJobj(readString(con)), + stop(paste("Unsupported type for deserialization", type))) +} + +readString <- function(con) { + stringLen <- readInt(con) + string <- readBin(con, raw(), stringLen, endian = "big") + rawToChar(string) +} + +readInt <- function(con) { + readBin(con, integer(), n = 1, endian = "big") +} + +readDouble <- function(con) { + readBin(con, double(), n = 1, endian = "big") +} + +readBoolean <- function(con) { + as.logical(readInt(con)) +} + +readType <- function(con) { + rawToChar(readBin(con, "raw", n = 1L)) +} + +readDate <- function(con) { + as.Date(readString(con)) +} + +readTime <- function(con) { + t <- readDouble(con) + as.POSIXct(t, origin = "1970-01-01") +} + +# We only support lists where all elements are of same type +readList <- function(con) { + type <- readType(con) + len <- readInt(con) + if (len > 0) { + l <- vector("list", len) + for (i in 1:len) { + l[[i]] <- readTypedObject(con, type) + } + l + } else { + list() + } +} + +readRaw <- function(con) { + dataLen <- readInt(con) + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readRawLen <- function(con, dataLen) { + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readDeserialize <- function(con) { + # We have two cases that are possible - In one, the entire partition is + # encoded as a byte array, so we have only one value to read. If so just + # return firstData + dataLen <- readInt(con) + firstData <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + + # Else, read things into a list + dataLen <- readInt(con) + if (length(dataLen) > 0 && dataLen > 0) { + data <- list(firstData) + while (length(dataLen) > 0 && dataLen > 0) { + data[[length(data) + 1L]] <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + dataLen <- readInt(con) + } + unlist(data, recursive = FALSE) + } else { + firstData + } +} + +readDeserializeRows <- function(inputCon) { + # readDeserializeRows will deserialize a DataOutputStream composed of + # a list of lists. Since the DOS is one continuous stream and + # the number of rows varies, we put the readRow function in a while loop + # that termintates when the next row is empty. + data <- list() + while(TRUE) { + row <- readRow(inputCon) + if (length(row) == 0) { + break + } + data[[length(data) + 1L]] <- row + } + data # this is a list of named lists now +} + +readRowList <- function(obj) { + # readRowList is meant for use inside an lapply. As a result, it is + # necessary to open a standalone connection for the row and consume + # the numCols bytes inside the read function in order to correctly + # deserialize the row. + rawObj <- rawConnection(obj, "r+") + on.exit(close(rawObj)) + readRow(rawObj) +} + +readRow <- function(inputCon) { + numCols <- readInt(inputCon) + if (length(numCols) > 0 && numCols > 0) { + lapply(1:numCols, function(x) { + obj <- readObject(inputCon) + if (is.null(obj)) { + NA + } else { + obj + } + }) # each row is a list now + } else { + list() + } +} + +# Take a single column as Array[Byte] and deserialize it into an atomic vector +readCol <- function(inputCon, numRows) { + # sapply can not work with POSIXlt + do.call(c, lapply(1:numRows, function(x) { + value <- readObject(inputCon) + # Replace NULL with NA so we can coerce to vectors + if (is.null(value)) NA else value + })) +} diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R new file mode 100644 index 0000000000000..e88729387ef95 --- /dev/null +++ b/R/pkg/R/generics.R @@ -0,0 +1,578 @@ +# +# 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. +# + +############ RDD Actions and Transformations ############ + +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' @rdname cache-methods +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname coalesce +#' @seealso repartition +#' @export +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) + +#' @rdname checkpoint-methods +#' @export +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) + +#' @rdname collect-methods +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectPartition", + function(x, partitionId) { + standardGeneric("collectPartition") + }) + +#' @rdname count +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + +#' @rdname countByValue +#' @export +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) + +#' @rdname distinct +#' @export +setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") }) + +#' @rdname filterRDD +#' @export +setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) + +#' @rdname first +#' @export +setGeneric("first", function(x) { standardGeneric("first") }) + +#' @rdname flatMap +#' @export +setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) + +#' @rdname fold +#' @seealso reduce +#' @export +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) + +#' @rdname foreach +#' @export +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + +#' @rdname foreach +#' @export +setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) + +# The jrdd accessor function. +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) + +#' @rdname glom +#' @export +setGeneric("glom", function(x) { standardGeneric("glom") }) + +#' @rdname keyBy +#' @export +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("lapplyPartitionsWithIndex", + function(X, FUN) { + standardGeneric("lapplyPartitionsWithIndex") + }) + +#' @rdname lapply +#' @export +setGeneric("map", function(X, FUN) { standardGeneric("map") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("mapPartitionsWithIndex", + function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) + +#' @rdname maximum +#' @export +setGeneric("maximum", function(x) { standardGeneric("maximum") }) + +#' @rdname minimum +#' @export +setGeneric("minimum", function(x) { standardGeneric("minimum") }) + +#' @rdname sumRDD +#' @export +setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) + +#' @rdname name +#' @export +setGeneric("name", function(x) { standardGeneric("name") }) + +#' @rdname numPartitions +#' @export +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' @rdname pipeRDD +#' @export +setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) + +#' @rdname reduce +#' @export +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) + +#' @rdname repartition +#' @seealso coalesce +#' @export +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) + +#' @rdname sampleRDD +#' @export +setGeneric("sampleRDD", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleRDD") + }) + +#' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) + +#' @rdname saveAsTextFile +#' @export +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) + +#' @rdname setName +#' @export +setGeneric("setName", function(x, name) { standardGeneric("setName") }) + +#' @rdname sortBy +#' @export +setGeneric("sortBy", + function(x, func, ascending = TRUE, numPartitions = 1) { + standardGeneric("sortBy") + }) + +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' @rdname takeOrdered +#' @export +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) + +#' @rdname takeSample +#' @export +setGeneric("takeSample", + function(x, withReplacement, num, seed) { + standardGeneric("takeSample") + }) + +#' @rdname top +#' @export +setGeneric("top", function(x, num) { standardGeneric("top") }) + +#' @rdname unionRDD +#' @export +setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) + +#' @rdname unpersist-methods +#' @export +setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) + +#' @rdname zipRDD +#' @export +setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") }) + +#' @rdname zipRDD +#' @export +setGeneric("zipPartitions", function(..., func) { standardGeneric("zipPartitions") }, + signature = "...") + +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + + +############ Binary Functions ############# + +#' @rdname cartesian +#' @export +setGeneric("cartesian", function(x, other) { standardGeneric("cartesian") }) + +#' @rdname countByKey +#' @export +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) + +#' @rdname flatMapValues +#' @export +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + +#' @rdname intersection +#' @export +setGeneric("intersection", function(x, other, numPartitions = 1) { + standardGeneric("intersection") }) + +#' @rdname keys +#' @export +setGeneric("keys", function(x) { standardGeneric("keys") }) + +#' @rdname lookup +#' @export +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) + +#' @rdname mapValues +#' @export +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' @rdname sampleByKey +#' @export +setGeneric("sampleByKey", + function(x, withReplacement, fractions, seed) { + standardGeneric("sampleByKey") + }) + +#' @rdname values +#' @export +setGeneric("values", function(x) { standardGeneric("values") }) + + +############ Shuffle Functions ############ + +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +setGeneric("aggregateByKey", + function(x, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' @rdname cogroup +#' @export +setGeneric("cogroup", + function(..., numPartitions) { + standardGeneric("cogroup") + }, + signature = "...") + +#' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +setGeneric("combineByKey", + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +setGeneric("foldByKey", + function(x, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' @rdname join-methods +#' @export +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' @rdname groupByKey +#' @seealso reduceByKey +#' @export +setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") }) + +#' @rdname join-methods +#' @export +setGeneric("join", function(x, y, ...) { standardGeneric("join") }) + +#' @rdname join-methods +#' @export +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) + +#' @rdname reduceByKey +#' @seealso groupByKey +#' @export +setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")}) + +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +setGeneric("reduceByKeyLocally", + function(x, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' @rdname join-methods +#' @export +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' @rdname sortByKey +#' @export +setGeneric("sortByKey", + function(x, ascending = TRUE, numPartitions = 1) { + standardGeneric("sortByKey") + }) + +#' @rdname subtract +#' @export +setGeneric("subtract", + function(x, other, numPartitions = 1) { + standardGeneric("subtract") + }) + +#' @rdname subtractByKey +#' @export +setGeneric("subtractByKey", + function(x, other, numPartitions = 1) { + standardGeneric("subtractByKey") + }) + + +################### Broadcast Variable Methods ################# + +#' @rdname broadcast +#' @export +setGeneric("value", function(bcast) { standardGeneric("value") }) + + + +#################### DataFrame Methods ######################## + +#' @rdname schema +#' @export +setGeneric("columns", function(x) {standardGeneric("columns") }) + +#' @rdname schema +#' @export +setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) + +#' @rdname explain +#' @export +setGeneric("explain", function(x, ...) { standardGeneric("explain") }) + +#' @rdname except +#' @export +setGeneric("except", function(x, y) { standardGeneric("except") }) + +#' @rdname filter +#' @export +setGeneric("filter", function(x, condition) { standardGeneric("filter") }) + +#' @rdname DataFrame +#' @export +setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") }) + +#' @rdname insertInto +#' @export +setGeneric("insertInto", function(x, tableName, ...) { standardGeneric("insertInto") }) + +#' @rdname intersect +#' @export +setGeneric("intersect", function(x, y) { standardGeneric("intersect") }) + +#' @rdname isLocal +#' @export +setGeneric("isLocal", function(x) { standardGeneric("isLocal") }) + +#' @rdname limit +#' @export +setGeneric("limit", function(x, num) {standardGeneric("limit") }) + +#' @rdname sortDF +#' @export +setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) + +#' @rdname schema +#' @export +setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) + +#' @rdname registerTempTable +#' @export +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + +#' @rdname sampleDF +#' @export +setGeneric("sampleDF", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleDF") + }) + +#' @rdname saveAsParquetFile +#' @export +setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) + +#' @rdname saveAsTable +#' @export +setGeneric("saveAsTable", function(df, tableName, source, mode, ...) { + standardGeneric("saveAsTable") +}) + +#' @rdname saveAsTable +#' @export +setGeneric("saveDF", function(df, path, source, mode, ...) { standardGeneric("saveDF") }) + +#' @rdname schema +#' @export +setGeneric("schema", function(x) { standardGeneric("schema") }) + +#' @rdname select +#' @export +setGeneric("select", function(x, col, ...) { standardGeneric("select") } ) + +#' @rdname select +#' @export +setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") }) + +#' @rdname showDF +#' @export +setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) + +#' @rdname sortDF +#' @export +setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) + +#' @rdname tojson +#' @export +setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) + +#' @rdname DataFrame +#' @export +setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) + +#' @rdname unionAll +#' @export +setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) + +#' @rdname filter +#' @export +setGeneric("where", function(x, condition) { standardGeneric("where") }) + +#' @rdname withColumn +#' @export +setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn") }) + +#' @rdname withColumnRenamed +#' @export +setGeneric("withColumnRenamed", function(x, existingCol, newCol) { + standardGeneric("withColumnRenamed") }) + + +###################### Column Methods ########################## + +#' @rdname column +#' @export +setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) + +#' @rdname column +#' @export +setGeneric("asc", function(x) { standardGeneric("asc") }) + +#' @rdname column +#' @export +setGeneric("avg", function(x, ...) { standardGeneric("avg") }) + +#' @rdname column +#' @export +setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) + +#' @rdname column +#' @export +setGeneric("contains", function(x, ...) { standardGeneric("contains") }) +#' @rdname column +#' @export +setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) + +#' @rdname column +#' @export +setGeneric("desc", function(x) { standardGeneric("desc") }) + +#' @rdname column +#' @export +setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") }) + +#' @rdname column +#' @export +setGeneric("getField", function(x, ...) { standardGeneric("getField") }) + +#' @rdname column +#' @export +setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) + +#' @rdname column +#' @export +setGeneric("isNull", function(x) { standardGeneric("isNull") }) + +#' @rdname column +#' @export +setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) + +#' @rdname column +#' @export +setGeneric("last", function(x) { standardGeneric("last") }) + +#' @rdname column +#' @export +setGeneric("like", function(x, ...) { standardGeneric("like") }) + +#' @rdname column +#' @export +setGeneric("lower", function(x) { standardGeneric("lower") }) + +#' @rdname column +#' @export +setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) + +#' @rdname column +#' @export +setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") }) + +#' @rdname column +#' @export +setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) + +#' @rdname column +#' @export +setGeneric("upper", function(x) { standardGeneric("upper") }) + diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R new file mode 100644 index 0000000000000..02237b3672d6b --- /dev/null +++ b/R/pkg/R/group.R @@ -0,0 +1,135 @@ +# +# 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. +# + +# group.R - GroupedData class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R schema.R column.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a GroupedData +#' @description GroupedDatas can be created using groupBy() on a DataFrame +#' @rdname GroupedData +#' @seealso groupBy +#' +#' @param sgd A Java object reference to the backing Scala GroupedData +#' @export +setClass("GroupedData", + slots = list(sgd = "jobj")) + +setMethod("initialize", "GroupedData", function(.Object, sgd) { + .Object@sgd <- sgd + .Object +}) + +#' @rdname DataFrame +groupedData <- function(sgd) { + new("GroupedData", sgd) +} + + +#' @rdname show +setMethod("show", "GroupedData", + function(object) { + cat("GroupedData\n") + }) + +#' Count +#' +#' Count the number of rows for each group. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @export +#' @examples +#' \dontrun{ +#' count(groupBy(df, "name")) +#' } +setMethod("count", + signature(x = "GroupedData"), + function(x) { + dataFrame(callJMethod(x@sgd, "count")) + }) + +#' Agg +#' +#' Aggregates on the entire DataFrame without groups. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' df2 <- agg(df, = ) +#' df2 <- agg(df, newColName = aggFunction(column)) +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @rdname agg +#' @examples +#' \dontrun{ +#' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)' +#' df2 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum +#' } +setGeneric("agg", function (x, ...) { standardGeneric("agg") }) + +setMethod("agg", + signature(x = "GroupedData"), + function(x, ...) { + cols = list(...) + stopifnot(length(cols) > 0) + if (is.character(cols[[1]])) { + cols <- varargsToEnv(...) + sdf <- callJMethod(x@sgd, "agg", cols) + } else if (class(cols[[1]]) == "Column") { + ns <- names(cols) + if (!is.null(ns)) { + for (n in ns) { + if (n != "") { + cols[[n]] = alias(cols[[n]], n) + } + } + } + jcols <- lapply(cols, function(c) { c@jc }) + # the GroupedData.agg(col, cols*) API does not contain grouping Column + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "aggWithGrouping", + x@sgd, listToSeq(jcols)) + } else { + stop("agg can only support Column or character") + } + dataFrame(sdf) + }) + + +# sum/mean/avg/min/max +methods <- c("sum", "mean", "avg", "min", "max") + +createMethod <- function(name) { + setMethod(name, + signature(x = "GroupedData"), + function(x, ...) { + sdf <- callJMethod(x@sgd, name, toSeq(...)) + dataFrame(sdf) + }) +} + +createMethods <- function() { + for (name in methods) { + createMethod(name) + } +} + +createMethods() + diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R new file mode 100644 index 0000000000000..a8a25230b636d --- /dev/null +++ b/R/pkg/R/jobj.R @@ -0,0 +1,104 @@ +# +# 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. +# + +# References to objects that exist on the JVM backend +# are maintained using the jobj. + +#' @include generics.R +NULL + +# Maintain a reference count of Java object references +# This allows us to GC the java object when it is safe +.validJobjs <- new.env(parent = emptyenv()) + +# List of object ids to be removed +.toRemoveJobjs <- new.env(parent = emptyenv()) + +# Check if jobj was created with the current SparkContext +isValidJobj <- function(jobj) { + if (exists(".scStartTime", envir = .sparkREnv)) { + jobj$appId == get(".scStartTime", envir = .sparkREnv) + } else { + FALSE + } +} + +getJobj <- function(objId) { + newObj <- jobj(objId) + if (exists(objId, .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] + 1 + } else { + .validJobjs[[objId]] <- 1 + } + newObj +} + +# Handler for a java object that exists on the backend. +jobj <- function(objId) { + if (!is.character(objId)) { + stop("object id must be a character") + } + # NOTE: We need a new env for a jobj as we can only register + # finalizers for environments or external references pointers. + obj <- structure(new.env(parent = emptyenv()), class = "jobj") + obj$id <- objId + obj$appId <- get(".scStartTime", envir = .sparkREnv) + + # Register a finalizer to remove the Java object when this reference + # is garbage collected in R + reg.finalizer(obj, cleanup.jobj) + obj +} + +#' Print a JVM object reference. +#' +#' This function prints the type and id for an object stored +#' in the SparkR JVM backend. +#' +#' @param x The JVM object reference +#' @param ... further arguments passed to or from other methods +print.jobj <- function(x, ...) { + cls <- callJMethod(x, "getClass") + name <- callJMethod(cls, "getName") + cat("Java ref type", name, "id", x$id, "\n", sep = " ") +} + +cleanup.jobj <- function(jobj) { + if (isValidJobj(jobj)) { + objId <- jobj$id + # If we don't know anything about this jobj, ignore it + if (exists(objId, envir = .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] - 1 + + if (.validJobjs[[objId]] == 0) { + rm(list = objId, envir = .validJobjs) + # NOTE: We cannot call removeJObject here as the finalizer may be run + # in the middle of another RPC. Thus we queue up this object Id to be removed + # and then run all the removeJObject when the next RPC is called. + .toRemoveJobjs[[objId]] <- 1 + } + } + } +} + +clearJobjs <- function() { + valid <- ls(.validJobjs) + rm(list = valid, envir = .validJobjs) + + removeList <- ls(.toRemoveJobjs) + rm(list = removeList, envir = .toRemoveJobjs) +} diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R new file mode 100644 index 0000000000000..9791e55791bae --- /dev/null +++ b/R/pkg/R/pairRDD.R @@ -0,0 +1,909 @@ +# +# 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. +# + +# Operations supported on RDDs contains pairs (i.e key, value) +#' @include generics.R jobj.R RDD.R +NULL + +############ Actions and Transformations ############ + +#' Look up elements of a key in an RDD +#' +#' @description +#' \code{lookup} returns a list of values in this RDD for key key. +#' +#' @param x The RDD to collect +#' @param key The key to look up for +#' @return a list of values in this RDD for key key +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +#' rdd <- parallelize(sc, pairs) +#' lookup(rdd, 1) # list(1, 3) +#'} +#' @rdname lookup +#' @aliases lookup,RDD-method +setMethod("lookup", + signature(x = "RDD", key = "ANY"), + function(x, key) { + partitionFunc <- function(part) { + filtered <- part[unlist(lapply(part, function(i) { identical(key, i[[1]]) }))] + lapply(filtered, function(i) { i[[2]] }) + } + valsRDD <- lapplyPartition(x, partitionFunc) + collect(valsRDD) + }) + +#' Count the number of elements for each key, and return the result to the +#' master as lists of (key, count) pairs. +#' +#' Same as countByKey in Spark. +#' +#' @param x The RDD to count keys. +#' @return list of (key, count) pairs, where count is number of each key in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +#' @rdname countByKey +#' @aliases countByKey,RDD-method +setMethod("countByKey", + signature(x = "RDD"), + function(x) { + keys <- lapply(x, function(item) { item[[1]] }) + countByValue(keys) + }) + +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} +#' @rdname keys +#' @aliases keys,RDD +setMethod("keys", + signature(x = "RDD"), + function(x) { + func <- function(k) { + k[[1]] + } + lapply(x, func) + }) + +#' Return an RDD with the values of each tuple. +#' +#' @param x The RDD from which the values of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} +#' @rdname values +#' @aliases values,RDD +setMethod("values", + signature(x = "RDD"), + function(x) { + func <- function(v) { + v[[2]] + } + lapply(x, func) + }) + +#' Applies a function to all values of the elements, without modifying the keys. +#' +#' The same as `mapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' makePairs <- lapply(rdd, function(x) { list(x, x) }) +#' collect(mapValues(makePairs, function(x) { x * 2) }) +#' Output: list(list(1,2), list(2,4), list(3,6), ...) +#'} +#' @rdname mapValues +#' @aliases mapValues,RDD,function-method +setMethod("mapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(x) { + list(x[[1]], FUN(x[[2]])) + } + lapply(X, func) + }) + +#' Pass each value in the key-value pair RDD through a flatMap function without +#' changing the keys; this also retains the original RDD's partitioning. +#' +#' The same as 'flatMapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +#' collect(flatMapValues(rdd, function(x) { x })) +#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#'} +#' @rdname flatMapValues +#' @aliases flatMapValues,RDD,function-method +setMethod("flatMapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + flatMapFunc <- function(x) { + lapply(FUN(x[[2]]), function(v) { list(x[[1]], v) }) + } + flatMap(X, flatMapFunc) + }) + +############ Shuffle Functions ############ + +#' Partition an RDD by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' For each element of this RDD, the partitioner is used to compute a hash +#' function and the RDD is partitioned using this hash value. +#' +#' @param x The RDD to partition. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @param ... Other optional arguments to partitionBy. +#' +#' @param partitionFunc The partition function to use. Uses a default hashCode +#' function if not provided +#' @return An RDD partitioned using the specified partitioner. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- partitionBy(rdd, 2L) +#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#'} +#' @rdname partitionBy +#' @aliases partitionBy,RDD,integer-method +setMethod("partitionBy", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions, partitionFunc = hashCode) { + + #if (missing(partitionFunc)) { + # partitionFunc <- hashCode + #} + + partitionFunc <- cleanClosure(partitionFunc) + serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) + + packageNamesArr <- serialize(.sparkREnv$.packages, + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), function(name) { + get(name, .broadcastNames) }) + jrdd <- getJRDD(x) + + # We create a PairwiseRRDD that extends RDD[(Int, Array[Byte])], + # where the key is the target partition number, the value is + # the content (key-val pairs). + pairwiseRRDD <- newJObject("org.apache.spark.api.r.PairwiseRRDD", + callJMethod(jrdd, "rdd"), + numToInt(numPartitions), + serializedHashFuncBytes, + getSerializedMode(x), + packageNamesArr, + as.character(.sparkREnv$libname), + broadcastArr, + callJMethod(jrdd, "classTag")) + + # Create a corresponding partitioner. + rPartitioner <- newJObject("org.apache.spark.HashPartitioner", + numToInt(numPartitions)) + + # Call partitionBy on the obtained PairwiseRDD. + javaPairRDD <- callJMethod(pairwiseRRDD, "asJavaPairRDD") + javaPairRDD <- callJMethod(javaPairRDD, "partitionBy", rPartitioner) + + # Call .values() on the result to get back the final result, the + # shuffled acutal content key-val pairs. + r <- callJMethod(javaPairRDD, "values") + + RDD(r, serializedMode = "byte") + }) + +#' Group values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and group values for each key in the RDD into a single sequence. +#' +#' @param x The RDD to group. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, list(V)) +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- groupByKey(rdd, 2L) +#' grouped <- collect(parts) +#' grouped[[1]] # Should be a list(1, list(2, 4)) +#'} +#' @rdname groupByKey +#' @aliases groupByKey,RDD,integer-method +setMethod("groupByKey", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions) { + shuffled <- partitionBy(x, numPartitions) + groupVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + appendList <- function(acc, i) { + addItemToAccumulator(acc, i) + acc + } + makeList <- function(i) { + acc <- initAccumulator() + addItemToAccumulator(acc, i) + acc + } + # Each item in the partition is list of (K, V) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, + appendList, makeList) + }) + # extract out data field + vals <- eapply(vals, + function(i) { + length(i$data) <- i$counter + i$data + }) + # Every key in the environment contains a list + # Convert that to list(K, Seq[V]) + convertEnvsToList(keys, vals) + } + lapplyPartition(shuffled, groupVals) + }) + +#' Merge values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, V') where V' is the merged +#' value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- reduceByKey(rdd, "+", 2L) +#' reduced <- collect(parts) +#' reduced[[1]] # Should be a list(1, 6) +#'} +#' @rdname reduceByKey +#' @aliases reduceByKey,RDD,integer-method +setMethod("reduceByKey", + signature(x = "RDD", combineFunc = "ANY", numPartitions = "numeric"), + function(x, combineFunc, numPartitions) { + reduceVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + convertEnvsToList(keys, vals) + } + locallyReduced <- lapplyPartition(x, reduceVals) + shuffled <- partitionBy(locallyReduced, numPartitions) + lapplyPartition(shuffled, reduceVals) + }) + +#' Merge values by key locally +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function, but return the +#' results immediately to the driver as an R list. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @return A list of elements of type list(K, V') where V' is the merged value for each key +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' reduced <- reduceByKeyLocally(rdd, "+") +#' reduced # list(list(1, 6), list(1.1, 3)) +#'} +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +setMethod("reduceByKeyLocally", + signature(x = "RDD", combineFunc = "ANY"), + function(x, combineFunc) { + reducePart <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + list(list(keys, vals)) # return hash to avoid re-compute in merge + } + mergeParts <- function(accum, x) { + pred <- function(item) { + exists(item$hash, accum[[1]]) + } + lapply(ls(x[[1]]), + function(name) { + item <- list(x[[1]][[name]], x[[2]][[name]]) + item$hash <- name + updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, identity) + }) + accum + } + reduced <- mapPartitions(x, reducePart) + merged <- reduce(reduced, mergeParts) + convertEnvsToList(merged[[1]], merged[[2]]) + }) + +#' Combine values by key +#' +#' Generic function to combine the elements for each key using a custom set of +#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +#' for a "combined type" C. Note that V and C can be different -- for example, one +#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). + +#' Users provide three functions: +#' \itemize{ +#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +#' two lists). +#' } +#' +#' @param x The RDD to combine. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param createCombiner Create a combiner (C) given a value (V) +#' @param mergeValue Merge the given value (V) with an existing combiner (C) +#' @param mergeCombiners Merge two combiners and return a new combiner +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, C) where C is the combined type +#' +#' @seealso groupByKey, reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +#' combined <- collect(parts) +#' combined[[1]] # Should be a list(1, 6) +#'} +#' @rdname combineByKey +#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("combineByKey", + signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", + mergeCombiners = "ANY", numPartitions = "numeric"), + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + combineLocally <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) + }) + convertEnvsToList(keys, combiners) + } + locallyCombined <- lapplyPartition(x, combineLocally) + shuffled <- partitionBy(locallyCombined, numPartitions) + mergeAfterShuffle <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) + }) + convertEnvsToList(keys, combiners) + } + lapplyPartition(shuffled, mergeAfterShuffle) + }) + +#' Aggregate a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using given combine functions +#' and a neutral "zero value". This function can return a different result type, +#' U, than the type of the values in this RDD, V. Thus, we need one operation +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument +#' instead of creating a new U. +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the values of each key. It may return +#' a different result type from the type of the values. +#' @param combOp A function to aggregate results of seqOp. +#' @return An RDD containing the aggregation result. +#' @seealso foldByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' # list(list(1, list(3, 2)), list(2, list(7, 2))) +#'} +#' @rdname aggregateByKey +#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("aggregateByKey", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", + combOp = "ANY", numPartitions = "numeric"), + function(x, zeroValue, seqOp, combOp, numPartitions) { + createCombiner <- function(v) { + do.call(seqOp, list(zeroValue, v)) + } + + combineByKey(x, createCombiner, seqOp, combOp, numPartitions) + }) + +#' Fold a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using an associative function "func" +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or +#' 1 for multiplication.). +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param func An associative function for folding values of each key. +#' @return An RDD containing the aggregation result. +#' @seealso aggregateByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#'} +#' @rdname foldByKey +#' @aliases foldByKey,RDD,ANY,ANY,integer-method +setMethod("foldByKey", + signature(x = "RDD", zeroValue = "ANY", + func = "ANY", numPartitions = "numeric"), + function(x, zeroValue, func, numPartitions) { + aggregateByKey(x, zeroValue, func, func, numPartitions) + }) + +############ Binary Functions ############# + +#' Join two RDDs +#' +#' @description +#' \code{join} This function joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with matching keys in +#' two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#'} +#' @rdname join-methods +#' @aliases join,RDD,RDD-method +setMethod("join", + signature(x = "RDD", y = "RDD"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), + doJoin) + }) + +#' Left outer join two RDDs +#' +#' @description +#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' if no elements in rdd2 have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' leftOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#'} +#' @rdname join-methods +#' @aliases leftOuterJoin,RDD,RDD-method +setMethod("leftOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "numeric"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Right outer join two RDDs +#' +#' @description +#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, w) in y, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +#' if no elements in x have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rightOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#'} +#' @rdname join-methods +#' @aliases rightOuterJoin,RDD,RDD-method +setMethod("rightOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "numeric"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Full outer join two RDDs +#' +#' @description +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD +#' will contain all pairs (k, (v, w)) for both (k, v) in x and +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' in x/y have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +#' # list(1, list(3, 1)), +#' # list(2, list(NULL, 4))) +#' # list(3, list(3, NULL)), +#'} +#' @rdname join-methods +#' @aliases fullOuterJoin,RDD,RDD-method +setMethod("fullOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "numeric"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' For each key k in several RDDs, return a resulting RDD that +#' whose values are a list of values for the key in all RDDs. +#' +#' @param ... Several RDDs. +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with values in a list +#' in all RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#'} +#' @rdname cogroup +#' @aliases cogroup,RDD-method +setMethod("cogroup", + "RDD", + function(..., numPartitions) { + rdds <- list(...) + rddsLen <- length(rdds) + for (i in 1:rddsLen) { + rdds[[i]] <- lapply(rdds[[i]], + function(x) { list(x[[1]], list(i, x[[2]])) }) + } + union.rdd <- Reduce(unionRDD, rdds) + group.func <- function(vlist) { + res <- list() + length(res) <- rddsLen + for (x in vlist) { + i <- x[[1]] + acc <- res[[i]] + # Create an accumulator. + if (is.null(acc)) { + acc <- initAccumulator() + } + addItemToAccumulator(acc, x[[2]]) + res[[i]] <- acc + } + lapply(res, function(acc) { + if (is.null(acc)) { + list() + } else { + acc$data + } + }) + } + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + group.func) + }) + +#' Sort a (k, v) pair RDD by k. +#' +#' @param x A (k, v) pair RDD to be sorted. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all (k, v) pair elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#'} +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +setMethod("sortByKey", + signature(x = "RDD"), + function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + rangeBounds <- list() + + if (numPartitions > 1) { + rddSize <- count(x) + # constant from Spark's RangePartitioner + maxSampleSize <- numPartitions * 20 + fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) + + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) + + # Note: the built-in R sort() function only works on atomic vectors + samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) + + if (length(samples) > 0) { + rangeBounds <- lapply(seq_len(numPartitions - 1), + function(i) { + j <- ceiling(length(samples) * i / numPartitions) + samples[j] + }) + } + } + + rangePartitionFunc <- function(key) { + partition <- 0 + + # TODO: Use binary search instead of linear search, similar with Spark + while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { + partition <- partition + 1 + } + + if (ascending) { + partition + } else { + numPartitions - partition - 1 + } + } + + partitionFunc <- function(part) { + sortKeyValueList(part, decreasing = !ascending) + } + + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) + lapplyPartition(newRDD, partitionFunc) + }) + +#' Subtract a pair RDD with another pair RDD. +#' +#' Return an RDD with the pairs from x whose keys are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the pairs from x whose keys are not in other. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +#' list("b", 5), list("a", 2))) +#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +#' collect(subtractByKey(rdd1, rdd2)) +#' # list(list("b", 4), list("b", 5)) +#'} +#' @rdname subtractByKey +#' @aliases subtractByKey,RDD +setMethod("subtractByKey", + signature(x = "RDD", other = "RDD"), + function(x, other, numPartitions = SparkR::numPartitions(x)) { + filterFunction <- function(elem) { + iters <- elem[[2]] + (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) + } + + flatMapValues(filterRDD(cogroup(x, + other, + numPartitions = numPartitions), + filterFunction), + function (v) { v[[1]] }) + }) + +#' Return a subset of this RDD sampled by key. +#' +#' @description +#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates +#' for different keys as specified by fractions, a key to sampling rate map. +#' +#' @param x The RDD to sample elements by key, where each element is +#' list(K, V) or c(K, V). +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3000) +#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +#' fractions <- list(a = 0.2, b = 0.1) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#'} +#' @rdname sampleByKey +#' @aliases sampleByKey,RDD-method +setMethod("sampleByKey", + signature(x = "RDD", withReplacement = "logical", + fractions = "vector", seed = "integer"), + function(x, withReplacement, fractions, seed) { + + for (elem in fractions) { + if (elem < 0.0) { + stop(paste("Negative fraction value ", fractions[which(fractions == elem)])) + } + } + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(partIndex, part) { + set.seed(bitwXor(seed, partIndex)) + res <- vector("list", length(part)) + len <- 0 + + # mixing because the initial seeds are close to each other + runif(10) + + for (elem in part) { + if (elem[[1]] %in% names(fractions)) { + frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) + if (withReplacement) { + count <- rpois(1, frac) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < frac) { + len <- len + 1 + res[[len]] <- elem + } + } + } else { + stop("KeyError: \"", elem[[1]], "\"") + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. (duplicated from sampleRDD) + if (len > 0) { + res[1:len] + } else { + list() + } + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R new file mode 100644 index 0000000000000..e442119086b17 --- /dev/null +++ b/R/pkg/R/schema.R @@ -0,0 +1,162 @@ +# +# 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. +# + +# A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField +# datatypes. These are used to create and interact with DataFrame schemas. + +#' structType +#' +#' Create a structType object that contains the metadata for a DataFrame. Intended for +#' use with createDataFrame and toDF. +#' +#' @param x a structField object (created with the field() function) +#' @param ... additional structField objects +#' @return a structType object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' schema <- structType(structField("a", "integer"), structField("b", "string")) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } +structType <- function(x, ...) { + UseMethod("structType", x) +} + +structType.jobj <- function(x) { + obj <- structure(list(), class = "structType") + obj$jobj <- x + obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } + obj +} + +structType.structField <- function(x, ...) { + fields <- list(x, ...) + if (!all(sapply(fields, inherits, "structField"))) { + stop("All arguments must be structField objects.") + } + sfObjList <- lapply(fields, function(field) { + field$jobj + }) + stObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructType", + listToSeq(sfObjList)) + structType(stObj) +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + cat("StructType\n", + sapply(x$fields(), function(field) { paste("|-", "name = \"", field$name(), + "\", type = \"", field$dataType.toString(), + "\", nullable = ", field$nullable(), "\n", + sep = "") }) + , sep = "") +} + +#' structField +#' +#' Create a structField object that contains the metadata for a single field in a schema. +#' +#' @param x The name of the field +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @return a structField object +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) +#' field1 <- structField("a", "integer", TRUE) +#' field2 <- structField("b", "string", TRUE) +#' schema <- structType(field1, field2) +#' df <- createDataFrame(sqlCtx, rdd, schema) +#' } + +structField <- function(x, ...) { + UseMethod("structField", x) +} + +structField.jobj <- function(x) { + obj <- structure(list(), class = "structField") + obj$jobj <- x + obj$name <- function() { callJMethod(x, "name") } + obj$dataType <- function() { callJMethod(x, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(x, "nullable") } + obj +} + +structField.character <- function(x, type, nullable = TRUE) { + if (class(x) != "character") { + stop("Field name must be a string.") + } + if (class(type) != "character") { + stop("Field type must be a string.") + } + if (class(nullable) != "logical") { + stop("nullable must be either TRUE or FALSE") + } + options <- c("byte", + "integer", + "double", + "numeric", + "character", + "string", + "binary", + "raw", + "logical", + "boolean", + "timestamp", + "date") + dataType <- if (type %in% options) { + type + } else { + stop(paste("Unsupported type for Dataframe:", type)) + } + sfObj <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createStructField", + x, + dataType, + nullable) + structField(sfObj) +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat("StructField(name = \"", x$name(), + "\", type = \"", x$dataType.toString(), + "\", nullable = ", x$nullable(), + ")", + sep = "") +} diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R new file mode 100644 index 0000000000000..c53d0a961016f --- /dev/null +++ b/R/pkg/R/serialize.R @@ -0,0 +1,192 @@ +# +# 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. +# + +# Utility functions to serialize R objects so they can be read in Java. + +# Type mapping from R to Java +# +# NULL -> Void +# integer -> Int +# character -> String +# logical -> Boolean +# double, numeric -> Double +# raw -> Array[Byte] +# Date -> Date +# POSIXct,POSIXlt -> Time +# +# list[T] -> Array[T], where T is one of above mentioned types +# environment -> Map[String, T], where T is a native type +# jobj -> Object, where jobj is an object created in the backend + +writeObject <- function(con, object, writeType = TRUE) { + # NOTE: In R vectors have same type as objects. So we don't support + # passing in vectors as arrays and instead require arrays to be passed + # as lists. + type <- class(object)[[1]] # class of POSIXlt is c("POSIXlt", "POSIXt") + if (writeType) { + writeType(con, type) + } + switch(type, + NULL = writeVoid(con), + integer = writeInt(con, object), + character = writeString(con, object), + logical = writeBoolean(con, object), + double = writeDouble(con, object), + numeric = writeDouble(con, object), + raw = writeRaw(con, object), + list = writeList(con, object), + jobj = writeJobj(con, object), + environment = writeEnv(con, object), + Date = writeDate(con, object), + POSIXlt = writeTime(con, object), + POSIXct = writeTime(con, object), + stop(paste("Unsupported type for serialization", type))) +} + +writeVoid <- function(con) { + # no value for NULL +} + +writeJobj <- function(con, value) { + if (!isValidJobj(value)) { + stop("invalid jobj ", value$id) + } + writeString(con, value$id) +} + +writeString <- function(con, value) { + utfVal <- enc2utf8(value) + writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) + writeBin(utfVal, con, endian = "big") +} + +writeInt <- function(con, value) { + writeBin(as.integer(value), con, endian = "big") +} + +writeDouble <- function(con, value) { + writeBin(value, con, endian = "big") +} + +writeBoolean <- function(con, value) { + # TRUE becomes 1, FALSE becomes 0 + writeInt(con, as.integer(value)) +} + +writeRawSerialize <- function(outputCon, batch) { + outputSer <- serialize(batch, ascii = FALSE, connection = NULL) + writeRaw(outputCon, outputSer) +} + +writeRowSerialize <- function(outputCon, rows) { + invisible(lapply(rows, function(r) { + bytes <- serializeRow(r) + writeRaw(outputCon, bytes) + })) +} + +serializeRow <- function(row) { + rawObj <- rawConnection(raw(0), "wb") + on.exit(close(rawObj)) + writeRow(rawObj, row) + rawConnectionValue(rawObj) +} + +writeRow <- function(con, row) { + numCols <- length(row) + writeInt(con, numCols) + for (i in 1:numCols) { + writeObject(con, row[[i]]) + } +} + +writeRaw <- function(con, batch) { + writeInt(con, length(batch)) + writeBin(batch, con, endian = "big") +} + +writeType <- function(con, class) { + type <- switch(class, + NULL = "n", + integer = "i", + character = "c", + logical = "b", + double = "d", + numeric = "d", + raw = "r", + list = "l", + jobj = "j", + environment = "e", + Date = "D", + POSIXlt = 't', + POSIXct = 't', + stop(paste("Unsupported type for serialization", class))) + writeBin(charToRaw(type), con) +} + +# Used to pass arrays where all the elements are of the same type +writeList <- function(con, arr) { + # All elements should be of same type + elemType <- unique(sapply(arr, function(elem) { class(elem) })) + stopifnot(length(elemType) <= 1) + + # TODO: Empty lists are given type "character" right now. + # This may not work if the Java side expects array of any other type. + if (length(elemType) == 0) { + elemType <- class("somestring") + } + + writeType(con, elemType) + writeInt(con, length(arr)) + + if (length(arr) > 0) { + for (a in arr) { + writeObject(con, a, FALSE) + } + } +} + +# Used to pass in hash maps required on Java side. +writeEnv <- function(con, env) { + len <- length(env) + + writeInt(con, len) + if (len > 0) { + writeList(con, as.list(ls(env))) + vals <- lapply(ls(env), function(x) { env[[x]] }) + writeList(con, as.list(vals)) + } +} + +writeDate <- function(con, date) { + writeString(con, as.character(date)) +} + +writeTime <- function(con, time) { + writeDouble(con, as.double(time)) +} + +# Used to serialize in a list of objects where each +# object can be of a different type. Serialization format is +# for each object +writeArgs <- function(con, args) { + if (length(args) > 0) { + for (a in args) { + writeObject(con, a) + } + } +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R new file mode 100644 index 0000000000000..bc82df01f0fff --- /dev/null +++ b/R/pkg/R/sparkR.R @@ -0,0 +1,266 @@ +# +# 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. +# + +.sparkREnv <- new.env() + +sparkR.onLoad <- function(libname, pkgname) { + .sparkREnv$libname <- libname +} + +# Utility function that returns TRUE if we have an active connection to the +# backend and FALSE otherwise +connExists <- function(env) { + tryCatch({ + exists(".sparkRCon", envir = env) && isOpen(env[[".sparkRCon"]]) + }, error = function(err) { + return(FALSE) + }) +} + +#' Stop the Spark context. +#' +#' Also terminates the backend this R session is connected to +sparkR.stop <- function() { + env <- .sparkREnv + if (exists(".sparkRCon", envir = env)) { + # cat("Stopping SparkR\n") + if (exists(".sparkRjsc", envir = env)) { + sc <- get(".sparkRjsc", envir = env) + callJMethod(sc, "stop") + rm(".sparkRjsc", envir = env) + } + + if (exists(".backendLaunched", envir = env)) { + callJStatic("SparkRHandler", "stopBackend") + } + + # Also close the connection and remove it from our env + conn <- get(".sparkRCon", envir = env) + close(conn) + + rm(".sparkRCon", envir = env) + rm(".scStartTime", envir = env) + } + + if (exists(".monitorConn", envir = env)) { + conn <- get(".monitorConn", envir = env) + close(conn) + rm(".monitorConn", envir = env) + } + + # Clear all broadcast variables we have + # as the jobj will not be valid if we restart the JVM + clearBroadcastVariables() + + # Clear jobj maps + clearJobjs() +} + +#' Initialize a new Spark Context. +#' +#' This function initializes a new SparkContext. +#' +#' @param master The Spark master URL. +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkEnvir Named list of environment variables to set on worker nodes. +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors. +#' @param sparkJars Character string vector of jar files to pass to the worker nodes. +#' @param sparkRLibDir The path where R is installed on the worker nodes. +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark") +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g")) +#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g"), +#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), +#' c("jarfile1.jar","jarfile2.jar")) +#'} + +sparkR.init <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvir = list(), + sparkExecutorEnv = list(), + sparkJars = "", + sparkRLibDir = "") { + + if (exists(".sparkRjsc", envir = .sparkREnv)) { + cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n") + return(get(".sparkRjsc", envir = .sparkREnv)) + } + + sparkMem <- Sys.getenv("SPARK_MEM", "512m") + jars <- suppressWarnings(normalizePath(as.character(sparkJars))) + + # Classpath separator is ";" on Windows + # URI needs four /// as from http://stackoverflow.com/a/18522792 + if (.Platform$OS.type == "unix") { + collapseChar <- ":" + uriSep <- "//" + } else { + collapseChar <- ";" + uriSep <- "////" + } + + existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") + if (existingPort != "") { + backendPort <- existingPort + } else { + path <- tempfile(pattern = "backend_port") + launchBackend( + args = path, + sparkHome = sparkHome, + jars = jars, + sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "sparkr-shell")) + # wait atmost 100 seconds for JVM to launch + wait <- 0.1 + for (i in 1:25) { + Sys.sleep(wait) + if (file.exists(path)) { + break + } + wait <- wait * 1.25 + } + if (!file.exists(path)) { + stop("JVM is not ready after 10 seconds") + } + f <- file(path, open='rb') + backendPort <- readInt(f) + monitorPort <- readInt(f) + close(f) + file.remove(path) + if (length(backendPort) == 0 || backendPort == 0 || + length(monitorPort) == 0 || monitorPort == 0) { + stop("JVM failed to launch") + } + assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) + assign(".backendLaunched", 1, envir = .sparkREnv) + } + + .sparkREnv$backendPort <- backendPort + tryCatch({ + connectBackend("localhost", backendPort) + }, error = function(err) { + stop("Failed to connect JVM\n") + }) + + if (nchar(sparkHome) != 0) { + sparkHome <- normalizePath(sparkHome) + } + + if (nchar(sparkRLibDir) != 0) { + .sparkREnv$libname <- sparkRLibDir + } + + sparkEnvirMap <- new.env() + for (varname in names(sparkEnvir)) { + sparkEnvirMap[[varname]] <- sparkEnvir[[varname]] + } + + sparkExecutorEnvMap <- new.env() + if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) { + sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + } + for (varname in names(sparkExecutorEnv)) { + sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]] + } + + nonEmptyJars <- Filter(function(x) { x != "" }, jars) + localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) + + # Set the start time to identify jobjs + # Seconds resolution is good enough for this purpose, so use ints + assign(".scStartTime", as.integer(Sys.time()), envir = .sparkREnv) + + assign( + ".sparkRjsc", + callJStatic( + "org.apache.spark.api.r.RRDD", + "createSparkContext", + master, + appName, + as.character(sparkHome), + as.list(localJarPaths), + sparkEnvirMap, + sparkExecutorEnvMap), + envir = .sparkREnv + ) + + sc <- get(".sparkRjsc", envir = .sparkREnv) + + # Register a finalizer to sleep 1 seconds on R exit to make RStudio happy + reg.finalizer(.sparkREnv, function(x) { Sys.sleep(1) }, onexit = TRUE) + + sc +} + +#' Initialize a new SQLContext. +#' +#' This function creates a SparkContext from an existing JavaSparkContext and +#' then uses it to initialize a new SQLContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#'} + +sparkRSQL.init <- function(jsc) { + if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + return(get(".sparkRSQLsc", envir = .sparkREnv)) + } + + sqlCtx <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createSQLContext", + jsc) + assign(".sparkRSQLsc", sqlCtx, envir = .sparkREnv) + sqlCtx +} + +#' Initialize a new HiveContext. +#' +#' This function creates a HiveContext from an existing JavaSparkContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRHive.init(sc) +#'} + +sparkRHive.init <- function(jsc) { + if (exists(".sparkRHivesc", envir = .sparkREnv)) { + return(get(".sparkRHivesc", envir = .sparkREnv)) + } + + ssc <- callJMethod(jsc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.HiveContext", ssc) + }, error = function(err) { + stop("Spark SQL is not built with Hive support") + }) + + assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) + hiveCtx +} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R new file mode 100644 index 0000000000000..0e7b7bd5a5b34 --- /dev/null +++ b/R/pkg/R/utils.R @@ -0,0 +1,547 @@ +# +# 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. +# + +# Utilities and Helpers + +# Given a JList, returns an R list containing the same elements, the number +# of which is optionally upper bounded by `logicalUpperBound` (by default, +# return all elements). Takes care of deserializations and type conversions. +convertJListToRList <- function(jList, flatten, logicalUpperBound = NULL, + serializedMode = "byte") { + arrSize <- callJMethod(jList, "size") + + # Datasets with serializedMode == "string" (such as an RDD directly generated by textFile()): + # each partition is not dense-packed into one Array[Byte], and `arrSize` + # here corresponds to number of logical elements. Thus we can prune here. + if (serializedMode == "string" && !is.null(logicalUpperBound)) { + arrSize <- min(arrSize, logicalUpperBound) + } + + results <- if (arrSize > 0) { + lapply(0:(arrSize - 1), + function(index) { + obj <- callJMethod(jList, "get", as.integer(index)) + + # Assume it is either an R object or a Java obj ref. + if (inherits(obj, "jobj")) { + if (isInstanceOf(obj, "scala.Tuple2")) { + # JavaPairRDD[Array[Byte], Array[Byte]]. + + keyBytes = callJMethod(obj, "_1") + valBytes = callJMethod(obj, "_2") + res <- list(unserialize(keyBytes), + unserialize(valBytes)) + } else { + stop(paste("utils.R: convertJListToRList only supports", + "RDD[Array[Byte]] and", + "JavaPairRDD[Array[Byte], Array[Byte]] for now")) + } + } else { + if (inherits(obj, "raw")) { + if (serializedMode == "byte") { + # RDD[Array[Byte]]. `obj` is a whole partition. + res <- unserialize(obj) + # For serialized datasets, `obj` (and `rRaw`) here corresponds to + # one whole partition dense-packed together. We deserialize the + # whole partition first, then cap the number of elements to be returned. + } else if (serializedMode == "row") { + res <- readRowList(obj) + # For DataFrames that have been converted to RRDDs, we call readRowList + # which will read in each row of the RRDD as a list and deserialize + # each element. + flatten <<- FALSE + # Use global assignment to change the flatten flag. This means + # we don't have to worry about the default argument in other functions + # e.g. collect + } + # TODO: is it possible to distinguish element boundary so that we can + # unserialize only what we need? + if (!is.null(logicalUpperBound)) { + res <- head(res, n = logicalUpperBound) + } + } else { + # obj is of a primitive Java type, is simplified to R's + # corresponding type. + res <- list(obj) + } + } + res + }) + } else { + list() + } + + if (flatten) { + as.list(unlist(results, recursive = FALSE)) + } else { + as.list(results) + } +} + +# Returns TRUE if `name` refers to an RDD in the given environment `env` +isRDD <- function(name, env) { + obj <- get(name, envir = env) + inherits(obj, "RDD") +} + +#' Compute the hashCode of an object +#' +#' Java-style function to compute the hashCode for the given object. Returns +#' an integer value. +#' +#' @details +#' This only works for integer, numeric and character types right now. +#' +#' @param key the object to be hashed +#' @return the hash code as an integer +#' @export +#' @examples +#' hashCode(1L) # 1 +#' hashCode(1.0) # 1072693248 +#' hashCode("1") # 49 +hashCode <- function(key) { + if (class(key) == "integer") { + as.integer(key[[1]]) + } else if (class(key) == "numeric") { + # Convert the double to long and then calculate the hash code + rawVec <- writeBin(key[[1]], con = raw()) + intBits <- packBits(rawToBits(rawVec), "integer") + as.integer(bitwXor(intBits[2], intBits[1])) + } else if (class(key) == "character") { + .Call("stringHashCode", key) + } else { + warning(paste("Could not hash object, returning 0", sep = "")) + as.integer(0) + } +} + +# Create a new RDD with serializedMode == "byte". +# Return itself if already in "byte" format. +serializeToBytes <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "byte") { + ser.rdd <- lapply(rdd, function(x) { x }) + return(ser.rdd) + } else { + return(rdd) + } +} + +# Create a new RDD with serializedMode == "string". +# Return itself if already in "string" format. +serializeToString <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "string") { + ser.rdd <- lapply(rdd, function(x) { toString(x) }) + # force it to create jrdd using "string" + getJRDD(ser.rdd, serializedMode = "string") + return(ser.rdd) + } else { + return(rdd) + } +} + +# Fast append to list by using an accumulator. +# http://stackoverflow.com/questions/17046336/here-we-go-again-append-an-element-to-a-list-in-r +# +# The accumulator should has three fields size, counter and data. +# This function amortizes the allocation cost by doubling +# the size of the list every time it fills up. +addItemToAccumulator <- function(acc, item) { + if(acc$counter == acc$size) { + acc$size <- acc$size * 2 + length(acc$data) <- acc$size + } + acc$counter <- acc$counter + 1 + acc$data[[acc$counter]] <- item +} + +initAccumulator <- function() { + acc <- new.env() + acc$counter <- 0 + acc$data <- list(NULL) + acc$size <- 1 + acc +} + +# Utility function to sort a list of key value pairs +# Used in unit tests +sortKeyValueList <- function(kv_list, decreasing = FALSE) { + keys <- sapply(kv_list, function(x) x[[1]]) + kv_list[order(keys, decreasing = decreasing)] +} + +# Utility function to generate compact R lists from grouped rdd +# Used in Join-family functions +# param: +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +genCompactLists <- function(tagged_list, cnull) { + len <- length(tagged_list) + lists <- list(vector("list", len), vector("list", len)) + index <- list(1, 1) + + for (x in tagged_list) { + tag <- x[[1]] + idx <- index[[tag]] + lists[[tag]][[idx]] <- x[[2]] + index[[tag]] <- idx + 1 + } + + len <- lapply(index, function(x) x - 1) + for (i in (1:2)) { + if (cnull[[i]] && len[[i]] == 0) { + lists[[i]] <- list(NULL) + } else { + length(lists[[i]]) <- len[[i]] + } + } + + lists +} + +# Utility function to merge compact R lists +# Used in Join-family functions +# param: +# left/right Two compact lists ready for Cartesian product +mergeCompactLists <- function(left, right) { + result <- list() + length(result) <- length(left) * length(right) + index <- 1 + for (i in left) { + for (j in right) { + result[[index]] <- list(i, j) + index <- index + 1 + } + } + result +} + +# Utility function to wrapper above two operations +# Used in Join-family functions +# param (same as genCompactLists): +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +joinTaggedList <- function(tagged_list, cnull) { + lists <- genCompactLists(tagged_list, cnull) + mergeCompactLists(lists[[1]], lists[[2]]) +} + +# Utility function to reduce a key-value list with predicate +# Used in *ByKey functions +# param +# pair key-value pair +# keys/vals env of key/value with hashes +# updateOrCreatePred predicate function +# updateFn update or merge function for existing pair, similar with `mergeVal` @combineByKey +# createFn create function for new pair, similar with `createCombiner` @combinebykey +updateOrCreatePair <- function(pair, keys, vals, updateOrCreatePred, updateFn, createFn) { + # assume hashVal bind to `$hash`, key/val with index 1/2 + hashVal <- pair$hash + key <- pair[[1]] + val <- pair[[2]] + if (updateOrCreatePred(pair)) { + assign(hashVal, do.call(updateFn, list(get(hashVal, envir = vals), val)), envir = vals) + } else { + assign(hashVal, do.call(createFn, list(val)), envir = vals) + assign(hashVal, key, envir = keys) + } +} + +# Utility function to convert key&values envs into key-val list +convertEnvsToList <- function(keys, vals) { + lapply(ls(keys), + function(name) { + list(keys[[name]], vals[[name]]) + }) +} + +# Utility function to capture the varargs into environment object +varargsToEnv <- function(...) { + pairs <- as.list(substitute(list(...)))[-1L] + env <- new.env() + for (name in names(pairs)) { + env[[name]] <- pairs[[name]] + } + env +} + +getStorageLevel <- function(newLevel = c("DISK_ONLY", + "DISK_ONLY_2", + "MEMORY_AND_DISK", + "MEMORY_AND_DISK_2", + "MEMORY_AND_DISK_SER", + "MEMORY_AND_DISK_SER_2", + "MEMORY_ONLY", + "MEMORY_ONLY_2", + "MEMORY_ONLY_SER", + "MEMORY_ONLY_SER_2", + "OFF_HEAP")) { + match.arg(newLevel) + storageLevel <- switch(newLevel, + "DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"), + "DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"), + "MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"), + "MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"), + "MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"), + "MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"), + "MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"), + "MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"), + "MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"), + "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), + "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) +} + +# Utility function for functions where an argument needs to be integer but we want to allow +# the user to type (for example) `5` instead of `5L` to avoid a confusing error message. +numToInt <- function(num) { + if (as.integer(num) != num) { + warning(paste("Coercing", as.list(sys.call())[[2]], "to integer.")) + } + as.integer(num) +} + +# create a Seq in JVM +toSeq <- function(...) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", list(...)) +} + +# create a Seq in JVM from a list +listToSeq <- function(l) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", l) +} + +# Utility function to recursively traverse the Abstract Syntax Tree (AST) of a +# user defined function (UDF), and to examine variables in the UDF to decide +# if their values should be included in the new function environment. +# param +# node The current AST node in the traversal. +# oldEnv The original function environment. +# defVars An Accumulator of variables names defined in the function's calling environment, +# including function argument and local variable names. +# checkedFunc An environment of function objects examined during cleanClosure. It can +# be considered as a "name"-to-"list of functions" mapping. +# newEnv A new function environment to store necessary function dependencies, an output argument. +processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { + nodeLen <- length(node) + + if (nodeLen > 1 && typeof(node) == "language") { + # Recursive case: current AST node is an internal node, check for its children. + if (length(node[[1]]) > 1) { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else { # if node[[1]] is length of 1, check for some R special functions. + nodeChar <- as.character(node[[1]]) + if (nodeChar == "{" || nodeChar == "(") { # Skip start symbol. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "<-" || nodeChar == "=" || + nodeChar == "<<-") { # Assignment Ops. + defVar <- node[[2]] + if (length(defVar) == 1 && typeof(defVar) == "symbol") { + # Add the defined variable name into defVars. + addItemToAccumulator(defVars, as.character(defVar)) + } else { + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "function") { # Function definition. + # Add parameter names. + newArgs <- names(node[[2]]) + lapply(newArgs, function(arg) { addItemToAccumulator(defVars, arg) }) + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "$") { # Skip the field. + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } else if (nodeChar == "::" || nodeChar == ":::") { + processClosure(node[[3]], oldEnv, defVars, checkedFuncs, newEnv) + } else { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } + } + } else if (nodeLen == 1 && + (typeof(node) == "symbol" || typeof(node) == "language")) { + # Base case: current AST node is a leaf node and a symbol or a function call. + nodeChar <- as.character(node) + if (!nodeChar %in% defVars$data) { # Not a function parameter or local variable. + func.env <- oldEnv + topEnv <- parent.env(.GlobalEnv) + # Search in function environment, and function's enclosing environments + # up to global environment. There is no need to look into package environments + # above the global or namespace environment that is not SparkR below the global, + # as they are assumed to be loaded on workers. + while (!identical(func.env, topEnv)) { + # Namespaces other than "SparkR" will not be searched. + if (!isNamespace(func.env) || + (getNamespaceName(func.env) == "SparkR" && + !(nodeChar %in% getNamespaceExports("SparkR")))) { # Only include SparkR internals. + # Set parameter 'inherits' to FALSE since we do not need to search in + # attached package environments. + if (tryCatch(exists(nodeChar, envir = func.env, inherits = FALSE), + error = function(e) { FALSE })) { + obj <- get(nodeChar, envir = func.env, inherits = FALSE) + if (is.function(obj)) { # If the node is a function call. + funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, + ifnotfound = list(list(NULL)))[[1]] + found <- sapply(funcList, function(func) { + ifelse(identical(func, obj), TRUE, FALSE) + }) + if (sum(found) > 0) { # If function has been examined, ignore. + break + } + # Function has not been examined, record it and recursively clean its closure. + assign(nodeChar, + if (is.null(funcList[[1]])) { + list(obj) + } else { + append(funcList, obj) + }, + envir = checkedFuncs) + obj <- cleanClosure(obj, checkedFuncs) + } + assign(nodeChar, obj, envir = newEnv) + break + } + } + + # Continue to search in enclosure. + func.env <- parent.env(func.env) + } + } + } +} + +# Utility function to get user defined function (UDF) dependencies (closure). +# More specifically, this function captures the values of free variables defined +# outside a UDF, and stores them in the function's environment. +# param +# func A function whose closure needs to be captured. +# checkedFunc An environment of function objects examined during cleanClosure. It can be +# considered as a "name"-to-"list of functions" mapping. +# return value +# a new version of func that has an correct environment (closure). +cleanClosure <- function(func, checkedFuncs = new.env()) { + if (is.function(func)) { + newEnv <- new.env(parent = .GlobalEnv) + func.body <- body(func) + oldEnv <- environment(func) + # defVars is an Accumulator of variables names defined in the function's calling + # environment. First, function's arguments are added to defVars. + defVars <- initAccumulator() + argNames <- names(as.list(args(func))) + for (i in 1:(length(argNames) - 1)) { # Remove the ending NULL in pairlist. + addItemToAccumulator(defVars, argNames[i]) + } + # Recursively examine variables in the function body. + processClosure(func.body, oldEnv, defVars, checkedFuncs, newEnv) + environment(func) <- newEnv + } + func +} + +# Append partition lengths to each partition in two input RDDs if needed. +# param +# x An RDD. +# Other An RDD. +# return value +# A list of two result RDDs. +appendPartitionLengths <- function(x, other) { + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # know the boundary of elements from x and other. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + len <- length(part) + part[[len + 1]] <- len + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + list (x, other) +} + +# Perform zip or cartesian between elements from two RDDs in each partition +# param +# rdd An RDD. +# zip A boolean flag indicating this call is for zip operation or not. +# return value +# A result RDD. +mergePartitions <- function(rdd, zip) { + serializerMode <- getSerializedMode(rdd) + partitionFunc <- function(partIndex, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + if (zip && lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + } else { + keys <- list() + } + if (lengthOfValues > 1) { + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + values <- list() + } + + if (!zip) { + return(mergeCompactLists(keys, values)) + } + } else { + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { list(k, v) }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(rdd, partitionFunc) +} diff --git a/R/pkg/R/zzz.R b/R/pkg/R/zzz.R new file mode 100644 index 0000000000000..80d796d467943 --- /dev/null +++ b/R/pkg/R/zzz.R @@ -0,0 +1,21 @@ +# +# 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. +# + +.onLoad <- function(libname, pkgname) { + sparkR.onLoad(libname, pkgname) +} + diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R new file mode 100644 index 0000000000000..8fe711b622086 --- /dev/null +++ b/R/pkg/inst/profile/general.R @@ -0,0 +1,22 @@ +# +# 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. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) +} diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R new file mode 100644 index 0000000000000..7a7f2031152a0 --- /dev/null +++ b/R/pkg/inst/profile/shell.R @@ -0,0 +1,31 @@ +# +# 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. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) + + library(utils) + library(SparkR) + sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) + assign("sc", sc, envir=.GlobalEnv) + sqlCtx <- sparkRSQL.init(sc) + assign("sqlCtx", sqlCtx, envir=.GlobalEnv) + cat("\n Welcome to SparkR!") + cat("\n Spark context is available as sc, SQL context is available as sqlCtx\n") +} diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/inst/tests/test_binaryFile.R new file mode 100644 index 0000000000000..ca4218f3819f8 --- /dev/null +++ b/R/pkg/inst/tests/test_binaryFile.R @@ -0,0 +1,90 @@ +# +# 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. +# + +context("functions on binary files") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("saveAsObjectFile()/objectFile() following textFile() works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1, 1) + saveAsObjectFile(rdd, fileName2) + rdd <- objectFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + + l <- list(1, 2, 3) + rdd <- parallelize(sc, l, 1) + saveAsObjectFile(rdd, fileName) + rdd <- objectFile(sc, fileName) + expect_equal(collect(rdd), l) + + unlink(fileName, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() following RDD transformations works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsObjectFile(counts, fileName2) + counts <- objectFile(sc, fileName2) + + output <- collect(counts) + expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), + list("is", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works with multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + + rdd1 <- parallelize(sc, "Spark is pretty.") + saveAsObjectFile(rdd1, fileName1) + rdd2 <- parallelize(sc, "Spark is awesome.") + saveAsObjectFile(rdd2, fileName2) + + rdd <- objectFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1, recursive = TRUE) + unlink(fileName2, recursive = TRUE) +}) + diff --git a/R/pkg/inst/tests/test_binary_function.R b/R/pkg/inst/tests/test_binary_function.R new file mode 100644 index 0000000000000..6785a7bdae8cb --- /dev/null +++ b/R/pkg/inst/tests/test_binary_function.R @@ -0,0 +1,101 @@ +# +# 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. +# + +context("binary functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +# File content +mockFile <- c("Spark is pretty.", "Spark is awesome.") + +test_that("union on two RDDs", { + actual <- collect(unionRDD(rdd, rdd)) + expect_equal(actual, as.list(rep(nums, 2))) + + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, c(as.list(nums), mockFile)) + expect_true(getSerializedMode(union.rdd) == "byte") + + rdd<- map(text.rdd, function(x) {x}) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, as.list(c(mockFile, mockFile))) + expect_true(getSerializedMode(union.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cogroup on two RDDs", { + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + expect_equal(actual, + list(list(1, list(list(1), list(2, 3))), list(2, list(list(4), list())))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("a", 4))) + rdd2 <- parallelize(sc, list(list("b", 2), list("a", 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + + expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3)))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) + +test_that("zipPartitions() on RDDs", { + rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 + rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 + rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 + actual <- collect(zipPartitions(rdd1, rdd2, rdd3, + func = function(x, y, z) { list(list(x, y, z))} )) + expect_equal(actual, + list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6)))) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName, 1) + actual <- collect(zipPartitions(rdd, rdd, + func = function(x, y) { list(paste(x, y, sep = "\n")) })) + expected <- list(paste(mockFile, mockFile, sep = "\n")) + expect_equal(actual, expected) + + rdd1 <- parallelize(sc, 0:1, 1) + actual <- collect(zipPartitions(rdd1, rdd, + func = function(x, y) { list(x + nchar(y)) })) + expected <- list(0:1 + nchar(mockFile)) + expect_equal(actual, expected) + + rdd <- map(rdd, function(x) { x }) + actual <- collect(zipPartitions(rdd, rdd1, + func = function(x, y) { list(y + nchar(x)) })) + expect_equal(actual, expected) + + unlink(fileName) +}) diff --git a/R/pkg/inst/tests/test_broadcast.R b/R/pkg/inst/tests/test_broadcast.R new file mode 100644 index 0000000000000..fee91a427d6d5 --- /dev/null +++ b/R/pkg/inst/tests/test_broadcast.R @@ -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. +# + +context("broadcast variables") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rrdd <- parallelize(sc, nums, 2L) + +test_that("using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + randomMatBr <- broadcast(sc, randomMat) + + useBroadcast <- function(x) { + sum(value(randomMatBr) * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) + +test_that("without using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + + useBroadcast <- function(x) { + sum(randomMat * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) diff --git a/R/pkg/inst/tests/test_context.R b/R/pkg/inst/tests/test_context.R new file mode 100644 index 0000000000000..e4aab37436a74 --- /dev/null +++ b/R/pkg/inst/tests/test_context.R @@ -0,0 +1,50 @@ +# +# 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. +# + +context("test functions in sparkR.R") + +test_that("repeatedly starting and stopping SparkR", { + for (i in 1:4) { + sc <- sparkR.init() + rdd <- parallelize(sc, 1:20, 2L) + expect_equal(count(rdd), 20) + sparkR.stop() + } +}) + +test_that("rdd GC across sparkR.stop", { + sparkR.stop() + sc <- sparkR.init() # sc should get id 0 + rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1 + rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2 + sparkR.stop() + + sc <- sparkR.init() # sc should get id 0 again + + # GC rdd1 before creating rdd3 and rdd2 after + rm(rdd1) + gc() + + rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now + rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now + + rm(rdd2) + gc() + + count(rdd3) + count(rdd4) +}) diff --git a/R/pkg/inst/tests/test_includePackage.R b/R/pkg/inst/tests/test_includePackage.R new file mode 100644 index 0000000000000..8152b448d0870 --- /dev/null +++ b/R/pkg/inst/tests/test_includePackage.R @@ -0,0 +1,57 @@ +# +# 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. +# + +context("include R packages") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rdd <- parallelize(sc, nums, 2L) + +test_that("include inside function", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + suppressPackageStartupMessages(library(plyr)) + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) + +test_that("use include package", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + includePackage(sc, plyr) + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) diff --git a/R/pkg/inst/tests/test_parallelize_collect.R b/R/pkg/inst/tests/test_parallelize_collect.R new file mode 100644 index 0000000000000..fff028657db37 --- /dev/null +++ b/R/pkg/inst/tests/test_parallelize_collect.R @@ -0,0 +1,109 @@ +# +# 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. +# + +context("parallelize() and collect()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +numPairs <- list(list(1, 1), list(1, 2), list(2, 2), list(2, 3)) +strPairs <- list(list(strList, strList), list(strList, strList)) + +# JavaSparkContext handle +jsc <- sparkR.init() + +# Tests + +test_that("parallelize() on simple vectors and lists returns an RDD", { + numVectorRDD <- parallelize(jsc, numVector, 1) + numVectorRDD2 <- parallelize(jsc, numVector, 10) + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + + rdds <- c(numVectorRDD, + numVectorRDD2, + numListRDD, + numListRDD2, + strVectorRDD, + strVectorRDD2, + strListRDD, + strListRDD2) + + for (rdd in rdds) { + expect_true(inherits(rdd, "RDD")) + expect_true(.hasSlot(rdd, "jrdd") + && inherits(rdd@jrdd, "jobj") + && isInstanceOf(rdd@jrdd, "org.apache.spark.api.java.JavaRDD")) + } +}) + +test_that("collect(), following a parallelize(), gives back the original collections", { + numVectorRDD <- parallelize(jsc, numVector, 10) + expect_equal(collect(numVectorRDD), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(collect(numListRDD), as.list(numList)) + expect_equal(collect(numListRDD2), as.list(numList)) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(collect(strVectorRDD), as.list(strVector)) + expect_equal(collect(strVectorRDD2), as.list(strVector)) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(collect(strListRDD), as.list(strList)) + expect_equal(collect(strListRDD2), as.list(strList)) +}) + +test_that("regression: collect() following a parallelize() does not drop elements", { + # 10 %/% 6 = 1, ceiling(10 / 6) = 2 + collLen <- 10 + numPart <- 6 + expected <- runif(collLen) + actual <- collect(parallelize(jsc, expected, numPart)) + expect_equal(actual, as.list(expected)) +}) + +test_that("parallelize() and collect() work for lists of pairs (pairwise data)", { + # use the pairwise logical to indicate pairwise data + numPairsRDDD1 <- parallelize(jsc, numPairs, 1) + numPairsRDDD2 <- parallelize(jsc, numPairs, 2) + numPairsRDDD3 <- parallelize(jsc, numPairs, 3) + expect_equal(collect(numPairsRDDD1), numPairs) + expect_equal(collect(numPairsRDDD2), numPairs) + expect_equal(collect(numPairsRDDD3), numPairs) + # can also leave out the parameter name, if the params are supplied in order + strPairsRDDD1 <- parallelize(jsc, strPairs, 1) + strPairsRDDD2 <- parallelize(jsc, strPairs, 2) + expect_equal(collect(strPairsRDDD1), strPairs) + expect_equal(collect(strPairsRDDD2), strPairs) +}) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R new file mode 100644 index 0000000000000..03207353c31c6 --- /dev/null +++ b/R/pkg/inst/tests/test_rdd.R @@ -0,0 +1,789 @@ +# +# 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. +# + +context("basic RDD functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +test_that("get number of partitions in RDD", { + expect_equal(numPartitions(rdd), 2) + expect_equal(numPartitions(intRdd), 2) +}) + +test_that("first on RDD", { + expect_true(first(rdd) == 1) + newrdd <- lapply(rdd, function(x) x + 1) + expect_true(first(newrdd) == 2) +}) + +test_that("count and length on RDD", { + expect_equal(count(rdd), 10) + expect_equal(length(rdd), 10) +}) + +test_that("count by values and keys", { + mods <- lapply(rdd, function(x) { x %% 3 }) + actual <- countByValue(mods) + expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + actual <- countByKey(intRdd) + expected <- list(list(2L, 2L), list(1L, 2L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("lapply on RDD", { + multiples <- lapply(rdd, function(x) { 2 * x }) + actual <- collect(multiples) + expect_equal(actual, as.list(nums * 2)) +}) + +test_that("lapplyPartition on RDD", { + sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("mapPartitions on RDD", { + sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("flatMap() on RDDs", { + flat <- flatMap(intRdd, function(x) { list(x, x) }) + actual <- collect(flat) + expect_equal(actual, rep(intPairs, each=2)) +}) + +test_that("filterRDD on RDD", { + filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list(2, 4, 6, 8, 10)) + + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) + actual <- collect(filtered.rdd) + expect_equal(actual, list(list(1L, -1))) + + # Filter out all elements. + filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list()) +}) + +test_that("lookup on RDD", { + vals <- lookup(intRdd, 1L) + expect_equal(vals, list(-1, 200)) + + vals <- lookup(intRdd, 3L) + expect_equal(vals, list()) +}) + +test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { + rdd2 <- rdd + for (i in 1:12) + rdd2 <- lapplyPartitionsWithIndex( + rdd2, function(partIndex, part) { + part <- as.list(unlist(part) * partIndex + i) + }) + rdd2 <- lapply(rdd2, function(x) x + x) + actual <- collect(rdd2) + expected <- list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176) + expect_equal(actual, expected) +}) + +test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { + # RDD + rdd2 <- rdd + # PipelinedRDD + rdd2 <- lapplyPartitionsWithIndex( + rdd2, + function(partIndex, part) { + part <- as.list(unlist(part) * partIndex) + }) + + cache(rdd2) + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + persist(rdd2, "MEMORY_AND_DISK") + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + tempDir <- tempfile(pattern = "checkpoint") + setCheckpointDir(sc, tempDir) + checkpoint(rdd2) + expect_true(rdd2@env$isCheckpointed) + + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + expect_false(rdd2@env$isCheckpointed) + + # make sure the data is collectable + collect(rdd2) + + unlink(tempDir) +}) + +test_that("reduce on RDD", { + sum <- reduce(rdd, "+") + expect_equal(sum, 55) + + # Also test with an inline function + sumInline <- reduce(rdd, function(x, y) { x + y }) + expect_equal(sumInline, 55) +}) + +test_that("lapply with dependency", { + fa <- 5 + multiples <- lapply(rdd, function(x) { fa * x }) + actual <- collect(multiples) + + expect_equal(actual, as.list(nums * 5)) +}) + +test_that("lapplyPartitionsWithIndex on RDDs", { + func <- function(partIndex, part) { list(partIndex, Reduce("+", part)) } + actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) + expect_equal(actual, list(list(0, 15), list(1, 40))) + + pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L) + partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 } + mkTup <- function(partIndex, part) { list(partIndex, part) } + actual <- collect(lapplyPartitionsWithIndex( + partitionBy(pairsRDD, 2L, partitionByParity), + mkTup), + FALSE) + expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), + list(1, list(list(4, 8))))) +}) + +test_that("sampleRDD() on RDDs", { + expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) +}) + +test_that("takeSample() on RDDs", { + # ported from RDDSuite.scala, modified seeds + data <- parallelize(sc, 1:100, 2L) + for (seed in 4:5) { + s <- takeSample(data, FALSE, 20L, seed) + expect_equal(length(s), 20L) + expect_equal(length(unique(s)), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, FALSE, 200L, seed) + expect_equal(length(s), 100L) + expect_equal(length(unique(s)), 100L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 20L, seed) + expect_equal(length(s), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 100L, seed) + expect_equal(length(s), 100L) + # Chance of getting all distinct elements is astronomically low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 200L, seed) + expect_equal(length(s), 200L) + # Chance of getting all distinct elements is still quite low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } +}) + +test_that("mapValues() on pairwise RDDs", { + multiples <- mapValues(intRdd, function(x) { x * 2 }) + actual <- collect(multiples) + expected <- lapply(intPairs, function(x) { + list(x[[1]], x[[2]] * 2) + }) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("flatMapValues() on pairwise RDDs", { + l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) + actual <- collect(flatMapValues(l, function(x) { x })) + expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) + + # Generate x to x+1 for every value + actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) + expect_equal(actual, + list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), + list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) +}) + +test_that("reduceByKeyLocally() on PairwiseRDDs", { + pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, 6), list(1.1, 3)))) + + pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3), + list("bb", 5)), 4L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5)))) +}) + +test_that("distinct() on RDDs", { + nums.rep2 <- rep(1:10, 2) + rdd.rep2 <- parallelize(sc, nums.rep2, 2L) + uniques <- distinct(rdd.rep2) + actual <- sort(unlist(collect(uniques))) + expect_equal(actual, nums) +}) + +test_that("maximum() on RDDs", { + max <- maximum(rdd) + expect_equal(max, 10) +}) + +test_that("minimum() on RDDs", { + min <- minimum(rdd) + expect_equal(min, 1) +}) + +test_that("sumRDD() on RDDs", { + sum <- sumRDD(rdd) + expect_equal(sum, 55) +}) + +test_that("keyBy on RDDs", { + func <- function(x) { x*x } + keys <- keyBy(rdd, func) + actual <- collect(keys) + expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) +}) + +test_that("repartition/coalesce on RDDs", { + rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements + + # repartition + r1 <- repartition(rdd, 2) + expect_equal(numPartitions(r1), 2L) + count <- length(collectPartition(r1, 0L)) + expect_true(count >= 8 && count <= 12) + + r2 <- repartition(rdd, 6) + expect_equal(numPartitions(r2), 6L) + count <- length(collectPartition(r2, 0L)) + expect_true(count >=0 && count <= 4) + + # coalesce + r3 <- coalesce(rdd, 1) + expect_equal(numPartitions(r3), 1L) + count <- length(collectPartition(r3, 0L)) + expect_equal(count, 20) +}) + +test_that("sortBy() on RDDs", { + sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) + actual <- collect(sortedRdd) + expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) + actual <- collect(sortedRdd2) + expect_equal(actual, as.list(nums)) +}) + +test_that("takeOrdered() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l)))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l)))[1:3]) +}) + +test_that("top() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- top(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- top(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3]) +}) + +test_that("fold() on RDDs", { + actual <- fold(rdd, 0, "+") + expect_equal(actual, Reduce("+", nums, 0)) + + rdd <- parallelize(sc, list()) + actual <- fold(rdd, 0, "+") + expect_equal(actual, 0) +}) + +test_that("aggregateRDD() on RDDs", { + rdd <- parallelize(sc, list(1, 2, 3, 4)) + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(10, 4)) + + rdd <- parallelize(sc, list()) + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(0, 0)) +}) + +test_that("zipWithUniqueId() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 3), list("c", 1), + list("d", 4), list("e", 2)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("zipWithIndex() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("glom() on RDD", { + rdd <- parallelize(sc, as.list(1:4), 2L) + actual <- collect(glom(rdd)) + expect_equal(actual, list(list(1, 2), list(3, 4))) +}) + +test_that("keys() on RDDs", { + keys <- keys(intRdd) + actual <- collect(keys) + expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) +}) + +test_that("values() on RDDs", { + values <- values(intRdd) + actual <- collect(values) + expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) +}) + +test_that("pipeRDD() on RDDs", { + actual <- collect(pipeRDD(rdd, "more")) + expected <- as.list(as.character(1:10)) + expect_equal(actual, expected) + + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) + actual <- collect(pipeRDD(trailed.rdd, "sort")) + expected <- list("", "1", "2", "3") + expect_equal(actual, expected) + + rev.nums <- 9:0 + rev.rdd <- parallelize(sc, rev.nums, 2L) + actual <- collect(pipeRDD(rev.rdd, "sort")) + expected <- as.list(as.character(c(5:9, 0:4))) + expect_equal(actual, expected) +}) + +test_that("zipRDD() on RDDs", { + rdd1 <- parallelize(sc, 0:4, 2) + rdd2 <- parallelize(sc, 1000:1004, 2) + actual <- collect(zipRDD(rdd1, rdd2)) + expect_equal(actual, + list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName, 1) + actual <- collect(zipRDD(rdd, rdd)) + expected <- lapply(mockFile, function(x) { list(x ,x) }) + expect_equal(actual, expected) + + rdd1 <- parallelize(sc, 0:1, 1) + actual <- collect(zipRDD(rdd1, rdd)) + expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) }) + expect_equal(actual, expected) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(zipRDD(rdd, rdd1)) + expected <- lapply(mockFile, function(x) { list(x, x) }) + expect_equal(actual, expected) + + unlink(fileName) +}) + +test_that("cartesian() on RDDs", { + rdd <- parallelize(sc, 1:3) + actual <- collect(cartesian(rdd, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(1, 1), list(1, 2), list(1, 3), + list(2, 1), list(2, 2), list(2, 3), + list(3, 1), list(3, 2), list(3, 3))) + + # test case where one RDD is empty + emptyRdd <- parallelize(sc, list()) + actual <- collect(cartesian(rdd, emptyRdd)) + expect_equal(actual, list()) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + actual <- collect(cartesian(rdd, rdd)) + expected <- list( + list("Spark is awesome.", "Spark is pretty."), + list("Spark is awesome.", "Spark is awesome."), + list("Spark is pretty.", "Spark is pretty."), + list("Spark is pretty.", "Spark is awesome.")) + expect_equal(sortKeyValueList(actual), expected) + + rdd1 <- parallelize(sc, 0:1) + actual <- collect(cartesian(rdd1, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(0, "Spark is pretty."), + list(0, "Spark is awesome."), + list(1, "Spark is pretty."), + list(1, "Spark is awesome."))) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(cartesian(rdd, rdd1)) + expect_equal(sortKeyValueList(actual), expected) + + unlink(fileName) +}) + +test_that("subtract() on RDDs", { + l <- list(1, 1, 2, 2, 3, 4) + rdd1 <- parallelize(sc, l) + + # subtract by itself + actual <- collect(subtract(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtract by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + l) + + rdd2 <- parallelize(sc, list(2, 4)) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + list(1, 1, 3)) + + l <- list("a", "a", "b", "b", "c", "d") + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list("b", "d")) + actual <- collect(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode="character"))), + list("a", "a", "c")) +}) + +test_that("subtractByKey() on pairwise RDDs", { + l <- list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2)) + rdd1 <- parallelize(sc, l) + + # subtractByKey by itself + actual <- collect(subtractByKey(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtractByKey by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(l)) + + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list("b", 4), list("b", 5))) + + l <- list(list(1, 1), list(2, 4), + list(2, 5), list(1, 2)) + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) + actual <- collect(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list(2, 4), list(2, 5))) +}) + +test_that("intersection() on RDDs", { + # intersection with self + actual <- collect(intersection(rdd, rdd)) + expect_equal(sort(as.integer(actual)), nums) + + # intersection with an empty RDD + emptyRdd <- parallelize(sc, list()) + actual <- collect(intersection(rdd, emptyRdd)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) + rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) + actual <- collect(intersection(rdd1, rdd2)) + expect_equal(sort(as.integer(actual)), 1:3) +}) + +test_that("join() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) +}) + +test_that("leftOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) + +test_that("rightOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("fullOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("sortByKey() on pairwise RDDs", { + numPairsRdd <- map(rdd, function(x) { list (x, x) }) + sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) + actual <- collect(sortedRdd) + numPairs <- lapply(nums, function(x) { list (x, x) }) + expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE)) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + numPairsRdd2 <- map(rdd2, function(x) { list (x, x) }) + sortedRdd2 <- sortByKey(numPairsRdd2) + actual <- collect(sortedRdd2) + expect_equal(actual, numPairs) + + # sort by string keys + l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5)) + rdd3 <- parallelize(sc, l, 2L) + sortedRdd3 <- sortByKey(rdd3) + actual <- collect(sortedRdd3) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # test on the boundary cases + + # boundary case 1: the RDD to be sorted has only 1 partition + rdd4 <- parallelize(sc, l, 1L) + sortedRdd4 <- sortByKey(rdd4) + actual <- collect(sortedRdd4) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 2: the sorted RDD has only 1 partition + rdd5 <- parallelize(sc, l, 2L) + sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L) + actual <- collect(sortedRdd5) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 3: the RDD to be sorted has only 1 element + l2 <- list(list("a", 1)) + rdd6 <- parallelize(sc, l2, 2L) + sortedRdd6 <- sortByKey(rdd6) + actual <- collect(sortedRdd6) + expect_equal(actual, l2) + + # boundary case 4: the RDD to be sorted has 0 element + l3 <- list() + rdd7 <- parallelize(sc, l3, 2L) + sortedRdd7 <- sortByKey(rdd7) + actual <- collect(sortedRdd7) + expect_equal(actual, l3) +}) + +test_that("collectAsMap() on a pairwise RDD", { + rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = 2, `3` = 4)) + + rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(a = 1, b = 2)) + + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) + + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = "a", `2` = "b")) +}) + +test_that("show()", { + rdd <- parallelize(sc, list(1:10)) + expect_output(show(rdd), "ParallelCollectionRDD\\[\\d+\\] at parallelize at RRDD\\.scala:\\d+") +}) + +test_that("sampleByKey() on pairwise RDDs", { + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) + fractions <- list(a = 0.2, b = 0.1) + sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) + expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) + expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) + expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) + expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) + expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) + + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) + fractions <- list(`2` = 0.2, `3` = 0.1) + sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) + expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) + expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) + expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) + expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) + expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) +}) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R new file mode 100644 index 0000000000000..d7dedda553c56 --- /dev/null +++ b/R/pkg/inst/tests/test_shuffle.R @@ -0,0 +1,221 @@ +# +# 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. +# + +context("partitionBy, groupByKey, reduceByKey etc.") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +doublePairs <- list(list(1.5, -1), list(2.5, 100), list(2.5, 1), list(1.5, 200)) +doubleRdd <- parallelize(sc, doublePairs, 2L) + +numPairs <- list(list(1L, 100), list(2L, 200), list(4L, -1), list(3L, 1), + list(3L, 0)) +numPairsRdd <- parallelize(sc, numPairs, length(numPairs)) + +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge and ", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ") +strListRDD <- parallelize(sc, strList, 4) + +test_that("groupByKey for integers", { + grouped <- groupByKey(intRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("groupByKey for doubles", { + grouped <- groupByKey(doubleRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for ints", { + reduced <- reduceByKey(intRdd, "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for doubles", { + reduced <- reduceByKey(doubleRdd, "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for ints", { + reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for doubles", { + reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for characters", { + stringKeyRDD <- parallelize(sc, + list(list("max", 1L), list("min", 2L), + list("other", 3L), list("max", 4L)), 2L) + reduced <- combineByKey(stringKeyRDD, + function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("aggregateByKey", { + # test aggregateByKey for int keys + rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list(1, list(3, 2)), list(2, list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test aggregateByKey for string keys + rdd <- parallelize(sc, list(list("a", 1), list("a", 2), list("b", 3), list("b", 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list("a", list(3, 2)), list("b", list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("foldByKey", { + # test foldByKey for int keys + folded <- foldByKey(intRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for double keys + folded <- foldByKey(doubleRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for string keys + stringKeyPairs <- list(list("a", -1), list("b", 100), list("b", 1), list("a", 200)) + + stringKeyRDD <- parallelize(sc, stringKeyPairs) + folded <- foldByKey(stringKeyRDD, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list("b", 101), list("a", 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for empty pair RDD + rdd <- parallelize(sc, list()) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list() + expect_equal(actual, expected) + + # test foldByKey for RDD with only 1 pair + rdd <- parallelize(sc, list(list(1, 1))) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list(list(1, 1)) + expect_equal(actual, expected) +}) + +test_that("partitionBy() partitions data correctly", { + # Partition by magnitude + partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 } + + resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) + + expected_first <- list(list(1, 100), list(2, 200)) # key < 3 + expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3 + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("partitionBy works with dependencies", { + kOne <- 1 + partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 } + + # Partition by parity + resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity) + + # keys even; 100 %% 2 == 0 + expected_first <- list(list(2, 200), list(4, -1)) + # keys odd; 3 %% 2 == 1 + expected_second <- list(list(1, 100), list(3, 1), list(3, 0)) + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("test partitionBy with string keys", { + words <- flatMap(strListRDD, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + resultRDD <- partitionBy(wordCount, 2L) + expected_first <- list(list("Dexter", 1), list("Dexter", 1)) + expected_second <- list(list("and", 1), list("and", 1)) + + actual_first <- Filter(function(item) { item[[1]] == "Dexter" }, + collectPartition(resultRDD, 0L)) + actual_second <- Filter(function(item) { item[[1]] == "and" }, + collectPartition(resultRDD, 1L)) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R new file mode 100644 index 0000000000000..f82e56fdd8278 --- /dev/null +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -0,0 +1,709 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("SparkSQL functions") + +# Tests for SparkSQL functions in SparkR + +sc <- sparkR.init() + +sqlCtx <- sparkRSQL.init(sc) + +mockLines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}") +jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") +parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet") +writeLines(mockLines, jsonPath) + +test_that("infer types", { + expect_equal(infer_type(1L), "integer") + expect_equal(infer_type(1.0), "double") + expect_equal(infer_type("abc"), "string") + expect_equal(infer_type(TRUE), "boolean") + expect_equal(infer_type(as.Date("2015-03-11")), "date") + expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp") + expect_equal(infer_type(c(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(a = 1L, b = "2")), + structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE))) + e <- new.env() + assign("a", 1L, envir = e) + expect_equal(infer_type(e), + list(type = "map", keyType = "string", valueType = "integer", + valueContainsNull = TRUE)) +}) + +test_that("structType and structField", { + testField <- structField("a", "string") + expect_true(inherits(testField, "structField")) + expect_true(testField$name() == "a") + expect_true(testField$nullable()) + + testSchema <- structType(testField, structField("b", "integer")) + expect_true(inherits(testSchema, "structType")) + expect_true(inherits(testSchema$fields()[[2]], "structField")) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "StringType") +}) + +test_that("create DataFrame from RDD", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) + df <- createDataFrame(sqlCtx, rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("toDF", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- toDF(rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) + df <- toDF(rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("create DataFrame from list or data.frame", { + l <- list(list(1, 2), list(3, 4)) + df <- createDataFrame(sqlCtx, l, c("a", "b")) + expect_equal(columns(df), c("a", "b")) + + l <- list(list(a=1, b=2), list(a=3, b=4)) + df <- createDataFrame(sqlCtx, l) + expect_equal(columns(df), c("a", "b")) + + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- createDataFrame(sqlCtx, ldf) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) +}) + +test_that("create DataFrame with different data types", { + l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"), + f = as.POSIXct("2015-03-15 12:13:14.056")) + df <- createDataFrame(sqlCtx, list(l)) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"), + c("d", "string"), c("e", "date"), c("f", "timestamp"))) + expect_equal(count(df), 1) + expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) +}) + +# TODO: enable this test after fix serialization for nested object +#test_that("create DataFrame with nested array and struct", { +# e <- new.env() +# assign("n", 3L, envir = e) +# l <- list(1:10, list("a", "b"), e, list(a="aa", b=3L)) +# df <- createDataFrame(sqlCtx, list(l), c("a", "b", "c", "d")) +# expect_equal(dtypes(df), list(c("a", "array"), c("b", "array"), +# c("c", "map"), c("d", "struct"))) +# expect_equal(count(df), 1) +# ldf <- collect(df) +# expect_equal(ldf[1,], l[[1]]) +#}) + +test_that("jsonFile() on a local file returns a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("jsonRDD() on a RDD with json string", { + rdd <- parallelize(sc, mockLines) + expect_true(count(rdd) == 3) + df <- jsonRDD(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + + rdd2 <- flatMap(rdd, function(x) c(x, x)) + df <- jsonRDD(sqlCtx, rdd2) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 6) +}) + +test_that("test cache, uncache and clearCache", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + cacheTable(sqlCtx, "table1") + uncacheTable(sqlCtx, "table1") + clearCache(sqlCtx) + dropTempTable(sqlCtx, "table1") +}) + +test_that("test tableNames and tables", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + expect_true(length(tableNames(sqlCtx)) == 1) + df <- tables(sqlCtx) + expect_true(count(df) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + newdf <- sql(sqlCtx, "SELECT * FROM table1 where name = 'Michael'") + expect_true(inherits(newdf, "DataFrame")) + expect_true(count(newdf) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("insertInto() on a registered table", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", "overwrite") + dfParquet <- loadDF(sqlCtx, parquetPath, "parquet") + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + saveDF(df2, parquetPath2, "parquet", "overwrite") + dfParquet2 <- loadDF(sqlCtx, parquetPath2, "parquet") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1") + expect_true(count(sql(sqlCtx, "select * from table1")) == 5) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Michael") + dropTempTable(sqlCtx, "table1") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1", overwrite = TRUE) + expect_true(count(sql(sqlCtx, "select * from table1")) == 2) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Bob") + dropTempTable(sqlCtx, "table1") +}) + +test_that("table() returns a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + tabledf <- table(sqlCtx, "table1") + expect_true(inherits(tabledf, "DataFrame")) + expect_true(count(tabledf) == 3) + dropTempTable(sqlCtx, "table1") +}) + +test_that("toRDD() returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toRDD(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(count(testRDD) == 3) +}) + +test_that("union on two RDDs created from DataFrames returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + RDD1 <- toRDD(df) + RDD2 <- toRDD(df) + unioned <- unionRDD(RDD1, RDD2) + expect_true(inherits(unioned, "RDD")) + expect_true(SparkR:::getSerializedMode(unioned) == "byte") + expect_true(collect(unioned)[[2]]$name == "Andy") +}) + +test_that("union on mixed serialization types correctly returns a byte RRDD", { + # Byte RDD + nums <- 1:10 + rdd <- parallelize(sc, nums, 2L) + + # String RDD + textLines <- c("Michael", + "Andy, 30", + "Justin, 19") + textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp") + writeLines(textLines, textPath) + textRDD <- textFile(sc, textPath) + + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + + unionByte <- unionRDD(rdd, dfRDD) + expect_true(inherits(unionByte, "RDD")) + expect_true(SparkR:::getSerializedMode(unionByte) == "byte") + expect_true(collect(unionByte)[[1]] == 1) + expect_true(collect(unionByte)[[12]]$name == "Andy") + + unionString <- unionRDD(textRDD, dfRDD) + expect_true(inherits(unionString, "RDD")) + expect_true(SparkR:::getSerializedMode(unionString) == "byte") + expect_true(collect(unionString)[[1]] == "Michael") + expect_true(collect(unionString)[[5]]$name == "Andy") +}) + +test_that("objectFile() works with row serialization", { + objectPath <- tempfile(pattern="spark-test", fileext=".tmp") + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) + objectIn <- objectFile(sc, objectPath) + + expect_true(inherits(objectIn, "RDD")) + expect_equal(SparkR:::getSerializedMode(objectIn), "byte") + expect_equal(collect(objectIn)[[2]]$age, 30) +}) + +test_that("lapply() on a DataFrame returns an RDD with the correct columns", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- lapply(df, function(row) { + row$newCol <- row$age + 5 + row + }) + expect_true(inherits(testRDD, "RDD")) + collected <- collect(testRDD) + expect_true(collected[[1]]$name == "Michael") + expect_true(collected[[2]]$newCol == "35") +}) + +test_that("collect() returns a data.frame", { + df <- jsonFile(sqlCtx, jsonPath) + rdf <- collect(df) + expect_true(is.data.frame(rdf)) + expect_true(names(rdf)[1] == "age") + expect_true(nrow(rdf) == 3) + expect_true(ncol(rdf) == 2) +}) + +test_that("limit() returns DataFrame with the correct number of rows", { + df <- jsonFile(sqlCtx, jsonPath) + dfLimited <- limit(df, 2) + expect_true(inherits(dfLimited, "DataFrame")) + expect_true(count(dfLimited) == 2) +}) + +test_that("collect() and take() on a DataFrame return the same number of rows and columns", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(nrow(collect(df)) == nrow(take(df, 10))) + expect_true(ncol(collect(df)) == ncol(take(df, 10))) +}) + +test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + first <- lapply(df, function(row) { + row$age <- row$age + 5 + row + }) + second <- lapply(first, function(row) { + row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE + row + }) + expect_true(inherits(second, "RDD")) + expect_true(count(second) == 3) + expect_true(collect(second)[[2]]$age == 35) + expect_true(collect(second)[[2]]$testCol) + expect_false(collect(second)[[3]]$testCol) +}) + +test_that("cache(), persist(), and unpersist() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(df@env$isCached) + cache(df) + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + persist(df, "MEMORY_AND_DISK") + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + # make sure the data is collectable + expect_true(is.data.frame(collect(df))) +}) + +test_that("schema(), dtypes(), columns(), names() return the correct values/format", { + df <- jsonFile(sqlCtx, jsonPath) + testSchema <- schema(df) + expect_true(length(testSchema$fields()) == 2) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "LongType") + expect_true(testSchema$fields()[[2]]$dataType.simpleString() == "string") + expect_true(testSchema$fields()[[1]]$name() == "age") + + testTypes <- dtypes(df) + expect_true(length(testTypes[[1]]) == 2) + expect_true(testTypes[[1]][1] == "age") + + testCols <- columns(df) + expect_true(length(testCols) == 2) + expect_true(testCols[2] == "name") + + testNames <- names(df) + expect_true(length(testNames) == 2) + expect_true(testNames[2] == "name") +}) + +test_that("head() and first() return the correct data", { + df <- jsonFile(sqlCtx, jsonPath) + testHead <- head(df) + expect_true(nrow(testHead) == 3) + expect_true(ncol(testHead) == 2) + + testHead2 <- head(df, 2) + expect_true(nrow(testHead2) == 2) + expect_true(ncol(testHead2) == 2) + + testFirst <- first(df) + expect_true(nrow(testFirst) == 1) +}) + +test_that("distinct() on DataFrames", { + lines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}", + "{\"name\":\"Justin\", \"age\":19}") + jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPathWithDup) + + df <- jsonFile(sqlCtx, jsonPathWithDup) + uniques <- distinct(df) + expect_true(inherits(uniques, "DataFrame")) + expect_true(count(uniques) == 3) +}) + +test_that("sampleDF on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sampled <- sampleDF(df, FALSE, 1.0) + expect_equal(nrow(collect(sampled)), count(df)) + expect_true(inherits(sampled, "DataFrame")) + sampled2 <- sampleDF(df, FALSE, 0.1) + expect_true(count(sampled2) < 3) +}) + +test_that("select operators", { + df <- select(jsonFile(sqlCtx, jsonPath), "name", "age") + expect_true(inherits(df$name, "Column")) + expect_true(inherits(df[[2]], "Column")) + expect_true(inherits(df[["age"]], "Column")) + + expect_true(inherits(df[,1], "DataFrame")) + expect_equal(columns(df[,1]), c("name")) + expect_equal(columns(df[,"age"]), c("age")) + df2 <- df[,c("age", "name")] + expect_true(inherits(df2, "DataFrame")) + expect_equal(columns(df2), c("age", "name")) + + df$age2 <- df$age + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age)), 2) + df$age2 <- df$age * 2 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 2)), 2) + + df$age2 <- NULL + expect_equal(columns(df), c("name", "age")) + df$age3 <- NULL + expect_equal(columns(df), c("name", "age")) +}) + +test_that("select with column", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- select(df, "name") + expect_true(columns(df1) == c("name")) + expect_true(count(df1) == 3) + + df2 <- select(df, df$age) + expect_true(columns(df2) == c("age")) + expect_true(count(df2) == 3) +}) + +test_that("selectExpr() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + selected <- selectExpr(df, "age * 2") + expect_true(names(selected) == "(age * 2)") + expect_equal(collect(selected), collect(select(df, df$age * 2L))) + + selected2 <- selectExpr(df, "name as newName", "abs(age) as age") + expect_equal(names(selected2), c("newName", "age")) + expect_true(count(selected2) == 3) +}) + +test_that("column calculation", { + df <- jsonFile(sqlCtx, jsonPath) + d <- collect(select(df, alias(df$age + 1, "age2"))) + expect_true(names(d) == c("age2")) + df2 <- select(df, lower(df$name), abs(df$age)) + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("load() from json file", { + df <- loadDF(sqlCtx, jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("save() as parquet file", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", mode="overwrite") + df2 <- loadDF(sqlCtx, parquetPath, "parquet") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("test HiveContext", { + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + df <- createExternalTable(hiveCtx, "json", jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + df2 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) + + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + saveAsTable(df, "json", "json", "append", path = jsonPath2) + df3 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df3, "DataFrame")) + expect_true(count(df3) == 6) +}) + +test_that("column operators", { + c <- SparkR:::col("a") + c2 <- (- c + 1 - 2) * 3 / 4.0 + c3 <- (c + c2 - c2) * c2 %% c2 + c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3) +}) + +test_that("column functions", { + c <- SparkR:::col("a") + c2 <- min(c) + max(c) + sum(c) + avg(c) + count(c) + abs(c) + sqrt(c) + c3 <- lower(c) + upper(c) + first(c) + last(c) + c4 <- approxCountDistinct(c) + countDistinct(c) + cast(c, "string") +}) + +test_that("string operators", { + df <- jsonFile(sqlCtx, jsonPath) + expect_equal(count(where(df, like(df$name, "A%"))), 1) + expect_equal(count(where(df, startsWith(df$name, "A"))), 1) + expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") + expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") +}) + +test_that("group by", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- agg(df, name = "max", age = "sum") + expect_true(1 == count(df1)) + df1 <- agg(df, age2 = max(df$age)) + expect_true(1 == count(df1)) + expect_equal(columns(df1), c("age2")) + + gd <- groupBy(df, "name") + expect_true(inherits(gd, "GroupedData")) + df2 <- count(gd) + expect_true(inherits(df2, "DataFrame")) + expect_true(3 == count(df2)) + + df3 <- agg(gd, age = "sum") + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + + df3 <- agg(gd, age = sum(df$age)) + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + expect_equal(columns(df3), c("name", "age")) + + df4 <- sum(gd, "age") + expect_true(inherits(df4, "DataFrame")) + expect_true(3 == count(df4)) + expect_true(3 == count(mean(gd, "age"))) + expect_true(3 == count(max(gd, "age"))) +}) + +test_that("sortDF() and orderBy() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sorted <- sortDF(df, df$age) + expect_true(collect(sorted)[1,2] == "Michael") + + sorted2 <- sortDF(df, "name") + expect_true(collect(sorted2)[2,"age"] == 19) + + sorted3 <- orderBy(df, asc(df$age)) + expect_true(is.na(first(sorted3)$age)) + expect_true(collect(sorted3)[2, "age"] == 19) + + sorted4 <- orderBy(df, desc(df$name)) + expect_true(first(sorted4)$name == "Michael") + expect_true(collect(sorted4)[3,"name"] == "Andy") +}) + +test_that("filter() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + filtered <- filter(df, "age > 20") + expect_true(count(filtered) == 1) + expect_true(collect(filtered)$name == "Andy") + filtered2 <- where(df, df$name != "Michael") + expect_true(count(filtered2) == 2) + expect_true(collect(filtered2)$age[2] == 19) +}) + +test_that("join() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", + "{\"name\":\"Andy\", \"test\": \"no\"}", + "{\"name\":\"Justin\", \"test\": \"yes\"}", + "{\"name\":\"Bob\", \"test\": \"yes\"}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(mockLines2, jsonPath2) + df2 <- jsonFile(sqlCtx, jsonPath2) + + joined <- join(df, df2) + expect_equal(names(joined), c("age", "name", "name", "test")) + expect_true(count(joined) == 12) + + joined2 <- join(df, df2, df$name == df2$name) + expect_equal(names(joined2), c("age", "name", "name", "test")) + expect_true(count(joined2) == 3) + + joined3 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined3), c("age", "name", "name", "test")) + expect_true(count(joined3) == 4) + expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) + + joined4 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined4), c("newAge", "name", "test")) + expect_true(count(joined4) == 4) + expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) +}) + +test_that("toJSON() returns an RDD of the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toJSON(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(SparkR:::getSerializedMode(testRDD) == "string") + expect_equal(collect(testRDD)[[1]], mockLines[1]) +}) + +test_that("showDF()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_output(showDF(df), "+----+-------+\n| age| name|\n+----+-------+\n|null|Michael|\n| 30| Andy|\n| 19| Justin|\n+----+-------+\n") +}) + +test_that("isLocal()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(isLocal(df)) +}) + +test_that("unionAll(), except(), and intersect() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + + unioned <- sortDF(unionAll(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(unioned) == 6) + expect_true(first(unioned)$name == "Michael") + + excepted <- sortDF(except(df, df2), desc(df$age)) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(excepted) == 2) + expect_true(first(excepted)$name == "Justin") + + intersected <- sortDF(intersect(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(intersected) == 1) + expect_true(first(intersected)$name == "Andy") +}) + +test_that("withColumn() and withColumnRenamed()", { + df <- jsonFile(sqlCtx, jsonPath) + newDF <- withColumn(df, "newAge", df$age + 2) + expect_true(length(columns(newDF)) == 3) + expect_true(columns(newDF)[3] == "newAge") + expect_true(first(filter(newDF, df$name != "Michael"))$newAge == 32) + + newDF2 <- withColumnRenamed(df, "age", "newerAge") + expect_true(length(columns(newDF2)) == 2) + expect_true(columns(newDF2)[1] == "newerAge") +}) + +test_that("saveDF() on DataFrame and works with parquetFile", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath) + expect_true(inherits(parquetDF, "DataFrame")) + expect_equal(count(df), count(parquetDF)) +}) + +test_that("parquetFile works with multiple input paths", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + saveDF(df, parquetPath2, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath, parquetPath2) + expect_true(inherits(parquetDF, "DataFrame")) + expect_true(count(parquetDF) == count(df)*2) +}) + +unlink(parquetPath) +unlink(jsonPath) diff --git a/R/pkg/inst/tests/test_take.R b/R/pkg/inst/tests/test_take.R new file mode 100644 index 0000000000000..7f4c7c315d787 --- /dev/null +++ b/R/pkg/inst/tests/test_take.R @@ -0,0 +1,67 @@ +# +# 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. +# + +context("tests RDD function take()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +# JavaSparkContext handle +jsc <- sparkR.init() + +test_that("take() gives back the original elements in correct count and order", { + numVectorRDD <- parallelize(jsc, numVector, 10) + # case: number of elements to take is less than the size of the first partition + expect_equal(take(numVectorRDD, 1), as.list(head(numVector, n = 1))) + # case: number of elements to take is the same as the size of the first partition + expect_equal(take(numVectorRDD, 11), as.list(head(numVector, n = 11))) + # case: number of elements to take is greater than all elements + expect_equal(take(numVectorRDD, length(numVector)), as.list(numVector)) + expect_equal(take(numVectorRDD, length(numVector) + 1), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(take(numListRDD, 3), take(numListRDD2, 3)) + expect_equal(take(numListRDD, 5), take(numListRDD2, 5)) + expect_equal(take(numListRDD, 1), as.list(head(numList, n = 1))) + expect_equal(take(numListRDD2, 999), numList) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(take(strVectorRDD, 4), as.list(strVector)) + expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2))) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3))) + expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1))) + + expect_true(length(take(strListRDD, 0)) == 0) + expect_true(length(take(strVectorRDD, 0)) == 0) + expect_true(length(take(numListRDD, 0)) == 0) + expect_true(length(take(numVectorRDD, 0)) == 0) +}) + diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R new file mode 100644 index 0000000000000..6b87b4b3e0b08 --- /dev/null +++ b/R/pkg/inst/tests/test_textFile.R @@ -0,0 +1,162 @@ +# +# 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. +# + +context("the textFile() function") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("textFile() on a local file returns an RDD", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_true(inherits(rdd, "RDD")) + expect_true(count(rdd) > 0) + expect_true(count(rdd) == 2) + + unlink(fileName) +}) + +test_that("textFile() followed by a collect() returns the same content", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName) +}) + +test_that("textFile() word count works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + output <- collect(counts) + expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), + list("Spark", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName) +}) + +test_that("several transformations on RDD created by textFile()", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) # RDD + for (i in 1:10) { + # PipelinedRDD initially created from RDD + rdd <- lapply(rdd, function(x) paste(x, x)) + } + collect(rdd) + + unlink(fileName) +}) + +test_that("textFile() followed by a saveAsTextFile() returns the same content", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1, 1L) + saveAsTextFile(rdd, fileName2) + rdd <- textFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("saveAsTextFile() on a parallelized list works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + l <- list(1, 2, 3) + rdd <- parallelize(sc, l, 1L) + saveAsTextFile(rdd, fileName) + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) + + unlink(fileName) +}) + +test_that("textFile() and saveAsTextFile() word count works as expected", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsTextFile(counts, fileName2) + rdd <- textFile(sc, fileName2) + + output <- collect(rdd) + expected <- list(list("awesome.", 1), list("Spark", 2), + list("pretty.", 1), list("is", 2)) + expectedStr <- lapply(expected, function(x) { toString(x) }) + expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("textFile() on multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines("Spark is pretty.", fileName1) + writeLines("Spark is awesome.", fileName2) + + rdd <- textFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("Pipelined operations on RDDs created using textFile", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + lengths <- lapply(rdd, function(x) { length(x) }) + expect_equal(collect(lengths), list(1, 1)) + + lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) + expect_equal(collect(lengthsPipelined), list(11, 11)) + + lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) + expect_equal(collect(lengths30), list(31, 31)) + + lengths20 <- lapply(lengths, function(x) { x + 20 }) + expect_equal(collect(lengths20), list(21, 21)) + + unlink(fileName) +}) + diff --git a/R/pkg/inst/tests/test_utils.R b/R/pkg/inst/tests/test_utils.R new file mode 100644 index 0000000000000..9c5bb427932b4 --- /dev/null +++ b/R/pkg/inst/tests/test_utils.R @@ -0,0 +1,137 @@ +# +# 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. +# + +context("functions in utils.R") + +# JavaSparkContext handle +sc <- sparkR.init() + +test_that("convertJListToRList() gives back (deserializes) the original JLists + of strings and integers", { + # It's hard to manually create a Java List using rJava, since it does not + # support generics well. Instead, we rely on collect() returning a + # JList. + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 1L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, nums) + + strs <- as.list("hello", "spark") + rdd <- parallelize(sc, strs, 2L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, strs) +}) + +test_that("serializeToBytes on RDD", { + # File content + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + expect_true(getSerializedMode(text.rdd) == "string") + ser.rdd <- serializeToBytes(text.rdd) + expect_equal(collect(ser.rdd), as.list(mockFile)) + expect_true(getSerializedMode(ser.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cleanClosure on R functions", { + y <- c(1, 2, 3) + g <- function(x) { x + 1 } + f <- function(x) { g(x) + y } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # y, g + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + # Test for nested enclosures and package variables. + env2 <- new.env() + funcEnv <- new.env(parent = env2) + f <- function(x) { log(g(x) + y) } + environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # "min" should not be included + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + base <- c(1, 2, 3) + l <- list(field = matrix(1)) + field <- matrix(2) + defUse <- 3 + g <- function(x) { x + y } + f <- function(x) { + defUse <- base::as.integer(x) + 1 # Test for access operators `::`. + lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. + l$field[1,1] <- 3 # Test for access operators `$`. + res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol. + f(res) # Test for recursive calls. + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". + expect_true("g" %in% ls(env)) + expect_true("l" %in% ls(env)) + expect_true("f" %in% ls(env)) + expect_equal(get("l", envir = env, inherits = FALSE), l) + # "y" should be in the environemnt of g. + newG <- get("g", envir = env, inherits = FALSE) + env <- environment(newG) + expect_equal(length(ls(env)), 1) + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + + # Test for function (and variable) definitions. + f <- function(x) { + g <- function(y) { y * 2 } + g(x) + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. + + # Test for overriding variables in base namespace (Issue: SparkR-196). + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 2L) + t = 4 # Override base::t in .GlobalEnv. + f <- function(x) { x > t } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(ls(env), "t") + expect_equal(get("t", envir = env, inherits = FALSE), t) + actual <- collect(lapply(rdd, f)) + expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) + expect_equal(actual, expected) + + # Test for broadcast variables. + a <- matrix(nrow=10, ncol=10, data=rnorm(100)) + aBroadcast <- broadcast(sc, a) + normMultiply <- function(x) { norm(aBroadcast$value) * x } + newnormMultiply <- SparkR:::cleanClosure(normMultiply) + env <- environment(newnormMultiply) + expect_equal(ls(env), "aBroadcast") + expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) +}) diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R new file mode 100644 index 0000000000000..3584b418a71a9 --- /dev/null +++ b/R/pkg/inst/worker/daemon.R @@ -0,0 +1,52 @@ +# +# 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. +# + +# Worker daemon + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") + +# preload SparkR package, speedup worker +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) + +while (TRUE) { + ready <- socketSelect(list(inputCon)) + if (ready) { + port <- SparkR:::readInt(inputCon) + # There is a small chance that it could be interrupted by signal, retry one time + if (length(port) == 0) { + port <- SparkR:::readInt(inputCon) + if (length(port) == 0) { + cat("quitting daemon\n") + quit(save = "no") + } + } + p <- parallel:::mcfork() + if (inherits(p, "masterProcess")) { + close(inputCon) + Sys.setenv(SPARKR_WORKER_PORT = port) + source(script) + # Set SIGUSR1 so that child can exit + tools::pskill(Sys.getpid(), tools::SIGUSR1) + parallel:::mcexit(0L) + } + } +} diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R new file mode 100644 index 0000000000000..014bf7bd7b3fe --- /dev/null +++ b/R/pkg/inst/worker/worker.R @@ -0,0 +1,177 @@ +# +# 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. +# + +# Worker class + +# Get current system time +currentTimeSecs <- function() { + as.numeric(Sys.time()) +} + +# Get elapsed time +elapsedSecs <- function() { + proc.time()[3] +} + +# Constants +specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) + +# Timing R process boot +bootTime <- currentTimeSecs() +bootElap <- elapsedSecs() + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +# Set libPaths to include SparkR package as loadNamespace needs this +# TODO: Figure out if we can avoid this by not loading any objects that require +# SparkR namespace +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") +outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") + +# read the index of the current partition inside the RDD +partition <- SparkR:::readInt(inputCon) + +deserializer <- SparkR:::readString(inputCon) +serializer <- SparkR:::readString(inputCon) + +# Include packages as required +packageNames <- unserialize(SparkR:::readRaw(inputCon)) +for (pkg in packageNames) { + suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) +} + +# read function dependencies +funcLen <- SparkR:::readInt(inputCon) +computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) +env <- environment(computeFunc) +parent.env(env) <- .GlobalEnv # Attach under global environment. + +# Timing init envs for computing +initElap <- elapsedSecs() + +# Read and set broadcast variables +numBroadcastVars <- SparkR:::readInt(inputCon) +if (numBroadcastVars > 0) { + for (bcast in seq(1:numBroadcastVars)) { + bcastId <- SparkR:::readInt(inputCon) + value <- unserialize(SparkR:::readRaw(inputCon)) + setBroadcastValue(bcastId, value) + } +} + +# Timing broadcast +broadcastElap <- elapsedSecs() + +# If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int +# as number of partitions to create. +numPartitions <- SparkR:::readInt(inputCon) + +isEmpty <- SparkR:::readInt(inputCon) + +if (isEmpty != 0) { + + if (numPartitions == -1) { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- as.list(readLines(inputCon)) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + # Timing reading input data for execution + inputElap <- elapsedSecs() + + output <- computeFunc(partition, data) + # Timing computing + computeElap <- elapsedSecs() + + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) + } + # Timing output + outputElap <- elapsedSecs() + } else { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- readLines(inputCon) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + # Timing reading input data for execution + inputElap <- elapsedSecs() + + res <- new.env() + + # Step 1: hash the data to an environment + hashTupleToEnvir <- function(tuple) { + # NOTE: execFunction is the hash function here + hashVal <- computeFunc(tuple[[1]]) + bucket <- as.character(hashVal %% numPartitions) + acc <- res[[bucket]] + # Create a new accumulator + if (is.null(acc)) { + acc <- SparkR:::initAccumulator() + } + SparkR:::addItemToAccumulator(acc, tuple) + res[[bucket]] <- acc + } + invisible(lapply(data, hashTupleToEnvir)) + # Timing computing + computeElap <- elapsedSecs() + + # Step 2: write out all of the environment as key-value pairs. + for (name in ls(res)) { + SparkR:::writeInt(outputCon, 2L) + SparkR:::writeInt(outputCon, as.integer(name)) + # Truncate the accumulator list to the number of elements we have + length(res[[name]]$data) <- res[[name]]$counter + SparkR:::writeRawSerialize(outputCon, res[[name]]$data) + } + # Timing output + outputElap <- elapsedSecs() + } +} else { + inputElap <- broadcastElap + computeElap <- broadcastElap + outputElap <- broadcastElap +} + +# Report timing +SparkR:::writeInt(outputCon, specialLengths$TIMING_DATA) +SparkR:::writeDouble(outputCon, bootTime) +SparkR:::writeDouble(outputCon, initElap - bootElap) # init +SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast +SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input +SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute +SparkR:::writeDouble(outputCon, outputElap - computeElap) # output + +# End of output +SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) + +close(outputCon) +close(inputCon) diff --git a/R/pkg/src/Makefile b/R/pkg/src/Makefile new file mode 100644 index 0000000000000..a55a56fe80e10 --- /dev/null +++ b/R/pkg/src/Makefile @@ -0,0 +1,27 @@ +# +# 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. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.so string_hash_code.c + +clean: + rm -f *.o + rm -f *.so + +.PHONY: all clean diff --git a/R/pkg/src/Makefile.win b/R/pkg/src/Makefile.win new file mode 100644 index 0000000000000..aa486d8228371 --- /dev/null +++ b/R/pkg/src/Makefile.win @@ -0,0 +1,27 @@ +# +# 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. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.dll string_hash_code.c + +clean: + rm -f *.o + rm -f *.dll + +.PHONY: all clean diff --git a/R/pkg/src/string_hash_code.c b/R/pkg/src/string_hash_code.c new file mode 100644 index 0000000000000..e3274b9a0c547 --- /dev/null +++ b/R/pkg/src/string_hash_code.c @@ -0,0 +1,49 @@ +/* + 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. +*/ + +/* + * A C function for R extension which implements the Java String hash algorithm. + * Refer to http://en.wikipedia.org/wiki/Java_hashCode%28%29#The_java.lang.String_hash_function + * + */ + +#include +#include + +/* for compatibility with R before 3.1 */ +#ifndef IS_SCALAR +#define IS_SCALAR(x, type) (TYPEOF(x) == (type) && XLENGTH(x) == 1) +#endif + +SEXP stringHashCode(SEXP string) { + const char* str; + R_xlen_t len, i; + int hashCode = 0; + + if (!IS_SCALAR(string, STRSXP)) { + error("invalid input"); + } + + str = CHAR(asChar(string)); + len = XLENGTH(asChar(string)); + + for (i = 0; i < len; i++) { + hashCode = (hashCode << 5) - hashCode + *str++; + } + + return ScalarInteger(hashCode); +} diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R new file mode 100644 index 0000000000000..4f8a1ed2d83ef --- /dev/null +++ b/R/pkg/tests/run-all.R @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) +library(SparkR) + +test_package("SparkR") diff --git a/R/run-tests.sh b/R/run-tests.sh new file mode 100755 index 0000000000000..e82ad0ba2cd06 --- /dev/null +++ b/R/run-tests.sh @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# 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. +# + +FWDIR="$(cd `dirname $0`; pwd)" + +FAILED=0 +LOGFILE=$FWDIR/unit-tests.out +rm -f $LOGFILE + +SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +FAILED=$((PIPESTATUS[0]||$FAILED)) + +if [[ $FAILED != 0 ]]; then + cat $LOGFILE + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 +else + echo -en "\033[32m" # Green + echo "Tests passed." + echo -en "\033[0m" # No color +fi diff --git a/assembly/pom.xml b/assembly/pom.xml index f1f8b0d3682e2..2b4d0a990bf22 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -194,7 +194,6 @@ org.apache.maven.plugins maven-assembly-plugin - 2.4 dist @@ -213,16 +212,6 @@ - - kinesis-asl - - - org.apache.httpcomponents - httpclient - ${commons.httpclient.version} - - - diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd new file mode 100644 index 0000000000000..36d932c453b6f --- /dev/null +++ b/bin/load-spark-env.cmd @@ -0,0 +1,59 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script loads spark-env.cmd if it exists, and ensures it is only loaded once. +rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's +rem conf/ subdirectory. + +if [%SPARK_ENV_LOADED%] == [] ( + set SPARK_ENV_LOADED=1 + + if not [%SPARK_CONF_DIR%] == [] ( + set user_conf_dir=%SPARK_CONF_DIR% + ) else ( + set user_conf_dir=%~dp0..\..\conf + ) + + call :LoadSparkEnv +) + +rem Setting SPARK_SCALA_VERSION if not already set. + +set ASSEMBLY_DIR2=%SPARK_HOME%/assembly/target/scala-2.11 +set ASSEMBLY_DIR1=%SPARK_HOME%/assembly/target/scala-2.10 + +if [%SPARK_SCALA_VERSION%] == [] ( + + if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( + echo "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." + echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd." + exit 1 + ) + if exist %ASSEMBLY_DIR2% ( + set SPARK_SCALA_VERSION=2.11 + ) else ( + set SPARK_SCALA_VERSION=2.10 + ) +) +exit /b 0 + +:LoadSparkEnv +if exist "%user_conf_dir%\spark-env.cmd" ( + call "%user_conf_dir%\spark-env.cmd" +) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 2d7070c25d328..95779e9ddbb18 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -20,6 +20,7 @@ # This script loads spark-env.sh if it exists, and ensures it is only loaded once. # spark-env.sh is loaded from SPARK_CONF_DIR if set, or within the current directory's # conf/ subdirectory. +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 @@ -41,8 +42,8 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="$SPARK_HOME/assembly/target/scala-2.11" - ASSEMBLY_DIR1="$SPARK_HOME/assembly/target/scala-2.10" + ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 diff --git a/bin/pyspark b/bin/pyspark index 776b28dc41099..8acad6113797d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,6 +89,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYTHONHASHSEED=0 if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 4f5eb5e20614d..09b4149c2a439 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -20,8 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Figure out which Python to use. if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index b49d0dcb4ff2d..c3e0221fb62e3 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -25,8 +25,7 @@ set FWDIR=%~dp0..\ rem Export this as SPARK_HOME set SPARK_HOME=%FWDIR% -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Test that an argument was given if not "x%1"=="x" goto arg_given diff --git a/bin/spark-class b/bin/spark-class index e29b234afaf96..c49d97ce5cf25 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -40,44 +40,64 @@ else fi fi -# Look for the launcher. In non-release mode, add the compiled classes directly to the classpath -# instead of looking for a jar file. -SPARK_LAUNCHER_CP= -if [ -f $SPARK_HOME/RELEASE ]; then - LAUNCHER_DIR="$SPARK_HOME/lib" - num_jars="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" | wc -l)" - if [ "$num_jars" -eq "0" -a -z "$SPARK_LAUNCHER_CP" ]; then - echo "Failed to find Spark launcher in $LAUNCHER_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 - exit 1 - fi +# Find assembly jar +SPARK_ASSEMBLY_JAR= +if [ -f "$SPARK_HOME/RELEASE" ]; then + ASSEMBLY_DIR="$SPARK_HOME/lib" +else + ASSEMBLY_DIR="$SPARK_HOME/assembly/target/scala-$SPARK_SCALA_VERSION" +fi - LAUNCHER_JARS="$(ls -1 "$LAUNCHER_DIR" | grep "^spark-launcher.*\.jar$" || true)" - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark launcher jars in $LAUNCHER_DIR:" 1>&2 - echo "$LAUNCHER_JARS" 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 - fi +num_jars="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" | wc -l)" +if [ "$num_jars" -eq "0" -a -z "$SPARK_ASSEMBLY_JAR" ]; then + echo "Failed to find Spark assembly in $ASSEMBLY_DIR." 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 +fi +ASSEMBLY_JARS="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" || true)" +if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark assembly jars in $ASSEMBLY_DIR:" 1>&2 + echo "$ASSEMBLY_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 + exit 1 +fi + +SPARK_ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}" - SPARK_LAUNCHER_CP="${LAUNCHER_DIR}/${LAUNCHER_JARS}" +# Verify that versions of java used to build the jars and run Spark are compatible +if [ -n "$JAVA_HOME" ]; then + JAR_CMD="$JAVA_HOME/bin/jar" else - LAUNCHER_DIR="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION" - if [ ! -d "$LAUNCHER_DIR/classes" ]; then - echo "Failed to find Spark launcher classes in $LAUNCHER_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 + JAR_CMD="jar" +fi + +if [ $(command -v "$JAR_CMD") ] ; then + jar_error_check=$("$JAR_CMD" -tf "$SPARK_ASSEMBLY_JAR" nonexistent/class/path 2>&1) + if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " 1>&2 + echo "This is likely because Spark was compiled with Java 7 and run " 1>&2 + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " 1>&2 + echo "or build Spark with Java 6." 1>&2 exit 1 fi - SPARK_LAUNCHER_CP="$LAUNCHER_DIR/classes" fi +LAUNCH_CLASSPATH="$SPARK_ASSEMBLY_JAR" + +# Add the launcher build dir to the classpath if requested. +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + LAUNCH_CLASSPATH="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" +fi + +export _SPARK_ASSEMBLY="$SPARK_ASSEMBLY_JAR" + # 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. CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$SPARK_LAUNCHER_CP" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 37d22215a0e7e..db09fa27e51a6 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -20,8 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%SPARK_HOME%\conf\spark-env.cmd" call "%SPARK_HOME%\conf\spark-env.cmd" +call %SPARK_HOME%\bin\load-spark-env.cmd rem Test that an argument was given if "x%1"=="x" ( @@ -29,42 +28,43 @@ if "x%1"=="x" ( exit /b 1 ) -set LAUNCHER_CP=0 -if exist %SPARK_HOME%\RELEASE goto find_release_launcher +rem Find assembly jar +set SPARK_ASSEMBLY_JAR=0 -rem Look for the Spark launcher in both Scala build directories. The launcher doesn't use Scala so -rem it doesn't really matter which one is picked up. Add the compiled classes directly to the -rem classpath instead of looking for a jar file, since it's very common for people using sbt to use -rem the "assembly" target instead of "package". -set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.10\classes -if exist %LAUNCHER_CLASSES% ( - set LAUNCHER_CP=%LAUNCHER_CLASSES% +if exist "%SPARK_HOME%\RELEASE" ( + set ASSEMBLY_DIR=%SPARK_HOME%\lib +) else ( + set ASSEMBLY_DIR=%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION% ) -set LAUNCHER_CLASSES=%SPARK_HOME%\launcher\target\scala-2.11\classes -if exist %LAUNCHER_CLASSES% ( - set LAUNCHER_CP=%LAUNCHER_CLASSES% -) -goto check_launcher -:find_release_launcher -for %%d in (%SPARK_HOME%\lib\spark-launcher*.jar) do ( - set LAUNCHER_CP=%%d +for %%d in (%ASSEMBLY_DIR%\spark-assembly*hadoop*.jar) do ( + set SPARK_ASSEMBLY_JAR=%%d ) - -:check_launcher -if "%LAUNCHER_CP%"=="0" ( - echo Failed to find Spark launcher JAR. +if "%SPARK_ASSEMBLY_JAR%"=="0" ( + echo Failed to find Spark assembly JAR. echo You need to build Spark before running this program. exit /b 1 ) +set LAUNCH_CLASSPATH=%SPARK_ASSEMBLY_JAR% + +rem Add the launcher build dir to the classpath if requested. +if not "x%SPARK_PREPEND_CLASSES%"=="x" ( + set LAUNCH_CLASSPATH=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH% +) + +set _SPARK_ASSEMBLY=%SPARK_ASSEMBLY_JAR% + rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCHER_CP% org.apache.spark.launcher.Main %*"') do ( +set LAUNCHER_OUTPUT=%temp%\spark-class-launcher-output-%RANDOM%.txt +"%RUNNER%" -cp %LAUNCH_CLASSPATH% org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% +for /f "tokens=*" %%i in (%LAUNCHER_OUTPUT%) do ( set SPARK_CMD=%%i ) +del %LAUNCHER_OUTPUT% %SPARK_CMD% diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index 02f51fe59a911..00fd30fa38d36 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -19,7 +19,7 @@ rem set SPARK_HOME=%~dp0.. -echo "%*" | findstr " --help -h" >nul +echo "%*" | findstr " \<--help\> \<-h\>" >nul if %ERRORLEVEL% equ 0 ( call :usage exit /b 0 diff --git a/bin/spark-submit b/bin/spark-submit index bcff78edd51ca..0e0afe71a0f05 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,6 +19,9 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +# disable randomized hash for string in Python 3.3+ +export PYTHONHASHSEED=0 + # Only define a usage function if an upstream script hasn't done so. if ! type -t usage >/dev/null 2>&1; then usage() { diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 08ddb185742d2..d3fc4a5cc3f6e 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -20,6 +20,9 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. +rem disable randomized hash for string in Python 3.3+ +set PYTHONHASHSEED=0 + set CLASS=org.apache.spark.deploy.SparkSubmit call %~dp0spark-class2.cmd %CLASS% %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% diff --git a/bin/sparkR b/bin/sparkR new file mode 100755 index 0000000000000..8c918e2b09aef --- /dev/null +++ b/bin/sparkR @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# 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. +# + +# Figure out where Spark is installed +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +source "$SPARK_HOME"/bin/load-spark-env.sh + +function usage() { + if [ -n "$1" ]; then + echo $1 + fi + echo "Usage: ./bin/sparkR [options]" 1>&2 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit $2 +} +export -f usage + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage +fi + +exec "$SPARK_HOME"/bin/spark-submit sparkr-shell-main "$@" diff --git a/bin/sparkR.cmd b/bin/sparkR.cmd new file mode 100644 index 0000000000000..d7b60183ca8e0 --- /dev/null +++ b/bin/sparkR.cmd @@ -0,0 +1,23 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This is the entry point for running SparkR. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0sparkR2.cmd %* diff --git a/bin/sparkR2.cmd b/bin/sparkR2.cmd new file mode 100644 index 0000000000000..e47f22c7300bb --- /dev/null +++ b/bin/sparkR2.cmd @@ -0,0 +1,26 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem Figure out where the Spark framework is installed +set SPARK_HOME=%~dp0.. + +call %SPARK_HOME%\bin\load-spark-env.cmd + + +call %SPARK_HOME%\bin\spark-submit2.cmd sparkr-shell-main %* diff --git a/conf/docker.properties.template b/conf/docker.properties.template new file mode 100644 index 0000000000000..26e3bfd9c5b9b --- /dev/null +++ b/conf/docker.properties.template @@ -0,0 +1,3 @@ +spark.mesos.executor.docker.image: +spark.mesos.executor.docker.volumes: /usr/local/lib:/host/usr/local/lib:ro +spark.mesos.executor.home: /opt/spark diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index 89eec7d4b7f61..3a2a88219818f 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 67f81d33361e1..43c4288912b18 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -3,7 +3,7 @@ # This file is sourced when running various Spark programs. # Copy it as spark-env.sh and edit that to configure Spark for your site. -# Options read when launching programs locally with +# Options read when launching programs locally with # ./bin/run-example or ./bin/spark-submit # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node @@ -39,6 +39,7 @@ # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") +# - SPARK_SHUFFLE_OPTS, to set config properties only for the external shuffle service (e.g. "-Dx=y") # - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y") # - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/pom.xml b/core/pom.xml index 6cd1965ec37c2..fc42f48973fe9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -74,6 +74,10 @@ javax.servlet servlet-api + + org.codehaus.jackson + jackson-mapper-asl + @@ -91,6 +95,11 @@ spark-network-shuffle_${scala.binary.version} ${project.version} + + org.apache.spark + spark-unsafe_${scala.binary.version} + ${project.version} + net.java.dev.jets3t jets3t @@ -219,6 +228,14 @@ json4s-jackson_${scala.binary.version} 3.2.10 + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-core + org.apache.mesos mesos @@ -264,7 +281,6 @@ org.apache.ivy ivy - ${ivy.version} oro @@ -275,7 +291,7 @@ org.tachyonproject tachyon-client - 0.5.0 + 0.6.4 org.apache.hadoop @@ -353,7 +369,7 @@ org.spark-project pyrolite - 2.0.1 + 4.4 net.sf.py4j @@ -442,4 +458,54 @@ + + + Windows + + + Windows + + + + \ + .bat + + + + unix + + + unix + + + + / + .sh + + + + sparkr + + + + org.codehaus.mojo + exec-maven-plugin + + + sparkr-pkg + compile + + exec + + + + + ..${path.separator}R${path.separator}install-dev${script.extension} + + + + + + + diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java index 6e161313702bb..0287fb79f8dd2 100644 --- a/core/src/main/java/org/apache/spark/JobExecutionStatus.java +++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java @@ -17,9 +17,15 @@ package org.apache.spark; +import org.apache.spark.util.EnumUtil; + public enum JobExecutionStatus { RUNNING, SUCCEEDED, FAILED, - UNKNOWN + UNKNOWN; + + public static JobExecutionStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(JobExecutionStatus.class, str); + } } diff --git a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala b/core/src/main/java/org/apache/spark/api/java/function/Function0.java similarity index 75% rename from core/src/main/scala/org/apache/spark/TaskContextHelper.scala rename to core/src/main/java/org/apache/spark/api/java/function/Function0.java index 4636c4600a01a..38e410c5debe6 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/Function0.java @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.api.java.function; + +import java.io.Serializable; /** - * This class exists to restrict the visibility of TaskContext setters. + * A zero-argument function that returns an R. */ -private [spark] object TaskContextHelper { - - def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) - - def unset(): Unit = TaskContext.unset() - +public interface Function0 extends Serializable { + public R call() throws Exception; } diff --git a/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java new file mode 100644 index 0000000000000..8c7dcf776fda8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java @@ -0,0 +1,30 @@ +/* + * 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.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +public enum ApplicationStatus { + COMPLETED, + RUNNING; + + public static ApplicationStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(ApplicationStatus.class, str); + } + +} diff --git a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java new file mode 100644 index 0000000000000..9dbb565aab707 --- /dev/null +++ b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java @@ -0,0 +1,31 @@ +/* + * 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.status.api.v1; + +import org.apache.spark.util.EnumUtil; + +public enum StageStatus { + ACTIVE, + COMPLETE, + FAILED, + PENDING; + + public static StageStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(StageStatus.class, str); + } +} diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java similarity index 54% rename from project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala rename to core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java index 3d43c35299555..f19ed01d5aebf 100644 --- a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala +++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java @@ -15,25 +15,34 @@ * limitations under the License. */ +package org.apache.spark.status.api.v1; -package org.apache.spark.scalastyle +import org.apache.spark.util.EnumUtil; -import java.util.regex.Pattern +import java.util.HashSet; +import java.util.Set; -import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} +public enum TaskSorting { + ID, + INCREASING_RUNTIME("runtime"), + DECREASING_RUNTIME("-runtime"); -import scalariform.lexer.Token -import scalariform.parser.CompilationUnit - -class NonASCIICharacterChecker extends ScalariformChecker { - val errorKey: String = "non.ascii.character.disallowed" - - override def verify(ast: CompilationUnit): List[ScalastyleError] = { - ast.tokens.filter(hasNonAsciiChars).map(x => PositionError(x.offset)).toList + private final Set alternateNames; + private TaskSorting(String... names) { + alternateNames = new HashSet(); + for (String n: names) { + alternateNames.add(n); + } } - private def hasNonAsciiChars(x: Token) = - x.rawText.trim.nonEmpty && !Pattern.compile( """\p{ASCII}+""", Pattern.DOTALL) - .matcher(x.text.trim).matches() + public static TaskSorting fromString(String str) { + String lower = str.toLowerCase(); + for (TaskSorting t: values()) { + if (t.alternateNames.contains(lower)) { + return t; + } + } + return EnumUtil.parseIgnoreCase(TaskSorting.class, str); + } } diff --git a/core/src/main/java/org/apache/spark/util/EnumUtil.java b/core/src/main/java/org/apache/spark/util/EnumUtil.java new file mode 100644 index 0000000000000..c40c7e727613c --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/EnumUtil.java @@ -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.util; + +import com.google.common.base.Joiner; +import org.apache.spark.annotation.Private; + +@Private +public class EnumUtil { + public static > E parseIgnoreCase(Class clz, String str) { + E[] constants = clz.getEnumConstants(); + if (str == null) { + return null; + } + for (E e : constants) { + if (e.name().equalsIgnoreCase(str)) { + return e; + } + } + throw new IllegalArgumentException( + String.format("Illegal type='%s'. Supported type values: %s", + str, Joiner.on(", ").join(constants))); + } +} diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 89eec7d4b7f61..3a2a88219818f 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index 14ba37d7c9bd9..013db8df9b363 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -30,7 +30,7 @@ $(function() { stripeSummaryTable(); - $("input:checkbox").click(function() { + $('input[type="checkbox"]').click(function() { var column = "table ." + $(this).attr("name"); $(column).toggle(); stripeSummaryTable(); @@ -39,15 +39,15 @@ $(function() { $("#select-all-metrics").click(function() { if (this.checked) { // Toggle all un-checked options. - $('input:checkbox:not(:checked)').trigger('click'); + $('input[type="checkbox"]:not(:checked)').trigger('click'); } else { // Toggle all checked options. - $('input:checkbox:checked').trigger('click'); + $('input[type="checkbox"]:checked').trigger('click'); } }); // Trigger a click on the checkbox if a user clicks the label next to it. $("span.additional-metric-title").click(function() { - $(this).parent().find('input:checkbox').trigger('click'); + $(this).parent().find('input[type="checkbox"]').trigger('click'); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/d3.min.js new file mode 100644 index 0000000000000..30cd292198b91 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/d3.min.js @@ -0,0 +1,5 @@ +/*v3.5.5*/!function(){function n(n){return n&&(n.ownerDocument||n.document||n).documentElement}function t(n){return n&&(n.ownerDocument&&n.ownerDocument.defaultView||n.document&&n||n.defaultView)}function e(n,t){return t>n?-1:n>t?1:n>=t?0:0/0}function r(n){return null===n?0/0:+n}function u(n){return!isNaN(n)}function i(n){return{left:function(t,e,r,u){for(arguments.length<3&&(r=0),arguments.length<4&&(u=t.length);u>r;){var i=r+u>>>1;n(t[i],e)<0?r=i+1:u=i}return r},right:function(t,e,r,u){for(arguments.length<3&&(r=0),arguments.length<4&&(u=t.length);u>r;){var i=r+u>>>1;n(t[i],e)>0?u=i:r=i+1}return r}}}function o(n){return n.length}function a(n){for(var t=1;n*t%1;)t*=10;return t}function c(n,t){for(var e in t)Object.defineProperty(n.prototype,e,{value:t[e],enumerable:!1})}function l(){this._=Object.create(null)}function s(n){return(n+="")===pa||n[0]===va?va+n:n}function f(n){return(n+="")[0]===va?n.slice(1):n}function h(n){return s(n)in this._}function g(n){return(n=s(n))in this._&&delete this._[n]}function p(){var n=[];for(var t in this._)n.push(f(t));return n}function v(){var n=0;for(var t in this._)++n;return n}function d(){for(var n in this._)return!1;return!0}function m(){this._=Object.create(null)}function y(n){return n}function M(n,t,e){return function(){var r=e.apply(t,arguments);return r===t?n:r}}function x(n,t){if(t in n)return t;t=t.charAt(0).toUpperCase()+t.slice(1);for(var e=0,r=da.length;r>e;++e){var u=da[e]+t;if(u in n)return u}}function b(){}function _(){}function w(n){function t(){for(var t,r=e,u=-1,i=r.length;++ue;e++)for(var u,i=n[e],o=0,a=i.length;a>o;o++)(u=i[o])&&t(u,o,e);return n}function Z(n){return ya(n,Sa),n}function V(n){var t,e;return function(r,u,i){var o,a=n[i].update,c=a.length;for(i!=e&&(e=i,t=0),u>=t&&(t=u+1);!(o=a[t])&&++t0&&(n=n.slice(0,a));var l=ka.get(n);return l&&(n=l,c=B),a?t?u:r:t?b:i}function $(n,t){return function(e){var r=ta.event;ta.event=e,t[0]=this.__data__;try{n.apply(this,t)}finally{ta.event=r}}}function B(n,t){var e=$(n,t);return function(n){var t=this,r=n.relatedTarget;r&&(r===t||8&r.compareDocumentPosition(t))||e.call(t,n)}}function W(e){var r=".dragsuppress-"+ ++Aa,u="click"+r,i=ta.select(t(e)).on("touchmove"+r,S).on("dragstart"+r,S).on("selectstart"+r,S);if(null==Ea&&(Ea="onselectstart"in e?!1:x(e.style,"userSelect")),Ea){var o=n(e).style,a=o[Ea];o[Ea]="none"}return function(n){if(i.on(r,null),Ea&&(o[Ea]=a),n){var t=function(){i.on(u,null)};i.on(u,function(){S(),t()},!0),setTimeout(t,0)}}}function J(n,e){e.changedTouches&&(e=e.changedTouches[0]);var r=n.ownerSVGElement||n;if(r.createSVGPoint){var u=r.createSVGPoint();if(0>Na){var i=t(n);if(i.scrollX||i.scrollY){r=ta.select("body").append("svg").style({position:"absolute",top:0,left:0,margin:0,padding:0,border:"none"},"important");var o=r[0][0].getScreenCTM();Na=!(o.f||o.e),r.remove()}}return Na?(u.x=e.pageX,u.y=e.pageY):(u.x=e.clientX,u.y=e.clientY),u=u.matrixTransform(n.getScreenCTM().inverse()),[u.x,u.y]}var a=n.getBoundingClientRect();return[e.clientX-a.left-n.clientLeft,e.clientY-a.top-n.clientTop]}function G(){return ta.event.changedTouches[0].identifier}function K(n){return n>0?1:0>n?-1:0}function Q(n,t,e){return(t[0]-n[0])*(e[1]-n[1])-(t[1]-n[1])*(e[0]-n[0])}function nt(n){return n>1?0:-1>n?qa:Math.acos(n)}function tt(n){return n>1?Ra:-1>n?-Ra:Math.asin(n)}function et(n){return((n=Math.exp(n))-1/n)/2}function rt(n){return((n=Math.exp(n))+1/n)/2}function ut(n){return((n=Math.exp(2*n))-1)/(n+1)}function it(n){return(n=Math.sin(n/2))*n}function ot(){}function at(n,t,e){return this instanceof at?(this.h=+n,this.s=+t,void(this.l=+e)):arguments.length<2?n instanceof at?new at(n.h,n.s,n.l):bt(""+n,_t,at):new at(n,t,e)}function ct(n,t,e){function r(n){return n>360?n-=360:0>n&&(n+=360),60>n?i+(o-i)*n/60:180>n?o:240>n?i+(o-i)*(240-n)/60:i}function u(n){return Math.round(255*r(n))}var i,o;return n=isNaN(n)?0:(n%=360)<0?n+360:n,t=isNaN(t)?0:0>t?0:t>1?1:t,e=0>e?0:e>1?1:e,o=.5>=e?e*(1+t):e+t-e*t,i=2*e-o,new mt(u(n+120),u(n),u(n-120))}function lt(n,t,e){return this instanceof lt?(this.h=+n,this.c=+t,void(this.l=+e)):arguments.length<2?n instanceof lt?new lt(n.h,n.c,n.l):n instanceof ft?gt(n.l,n.a,n.b):gt((n=wt((n=ta.rgb(n)).r,n.g,n.b)).l,n.a,n.b):new lt(n,t,e)}function st(n,t,e){return isNaN(n)&&(n=0),isNaN(t)&&(t=0),new ft(e,Math.cos(n*=Da)*t,Math.sin(n)*t)}function ft(n,t,e){return this instanceof ft?(this.l=+n,this.a=+t,void(this.b=+e)):arguments.length<2?n instanceof ft?new ft(n.l,n.a,n.b):n instanceof lt?st(n.h,n.c,n.l):wt((n=mt(n)).r,n.g,n.b):new ft(n,t,e)}function ht(n,t,e){var r=(n+16)/116,u=r+t/500,i=r-e/200;return u=pt(u)*Xa,r=pt(r)*$a,i=pt(i)*Ba,new mt(dt(3.2404542*u-1.5371385*r-.4985314*i),dt(-.969266*u+1.8760108*r+.041556*i),dt(.0556434*u-.2040259*r+1.0572252*i))}function gt(n,t,e){return n>0?new lt(Math.atan2(e,t)*Pa,Math.sqrt(t*t+e*e),n):new lt(0/0,0/0,n)}function pt(n){return n>.206893034?n*n*n:(n-4/29)/7.787037}function vt(n){return n>.008856?Math.pow(n,1/3):7.787037*n+4/29}function dt(n){return Math.round(255*(.00304>=n?12.92*n:1.055*Math.pow(n,1/2.4)-.055))}function mt(n,t,e){return this instanceof mt?(this.r=~~n,this.g=~~t,void(this.b=~~e)):arguments.length<2?n instanceof mt?new mt(n.r,n.g,n.b):bt(""+n,mt,ct):new mt(n,t,e)}function yt(n){return new mt(n>>16,n>>8&255,255&n)}function Mt(n){return yt(n)+""}function xt(n){return 16>n?"0"+Math.max(0,n).toString(16):Math.min(255,n).toString(16)}function bt(n,t,e){var r,u,i,o=0,a=0,c=0;if(r=/([a-z]+)\((.*)\)/i.exec(n))switch(u=r[2].split(","),r[1]){case"hsl":return e(parseFloat(u[0]),parseFloat(u[1])/100,parseFloat(u[2])/100);case"rgb":return t(kt(u[0]),kt(u[1]),kt(u[2]))}return(i=Ga.get(n.toLowerCase()))?t(i.r,i.g,i.b):(null==n||"#"!==n.charAt(0)||isNaN(i=parseInt(n.slice(1),16))||(4===n.length?(o=(3840&i)>>4,o=o>>4|o,a=240&i,a=a>>4|a,c=15&i,c=c<<4|c):7===n.length&&(o=(16711680&i)>>16,a=(65280&i)>>8,c=255&i)),t(o,a,c))}function _t(n,t,e){var r,u,i=Math.min(n/=255,t/=255,e/=255),o=Math.max(n,t,e),a=o-i,c=(o+i)/2;return a?(u=.5>c?a/(o+i):a/(2-o-i),r=n==o?(t-e)/a+(e>t?6:0):t==o?(e-n)/a+2:(n-t)/a+4,r*=60):(r=0/0,u=c>0&&1>c?0:r),new at(r,u,c)}function wt(n,t,e){n=St(n),t=St(t),e=St(e);var r=vt((.4124564*n+.3575761*t+.1804375*e)/Xa),u=vt((.2126729*n+.7151522*t+.072175*e)/$a),i=vt((.0193339*n+.119192*t+.9503041*e)/Ba);return ft(116*u-16,500*(r-u),200*(u-i))}function St(n){return(n/=255)<=.04045?n/12.92:Math.pow((n+.055)/1.055,2.4)}function kt(n){var t=parseFloat(n);return"%"===n.charAt(n.length-1)?Math.round(2.55*t):t}function Et(n){return"function"==typeof n?n:function(){return n}}function At(n){return function(t,e,r){return 2===arguments.length&&"function"==typeof e&&(r=e,e=null),Nt(t,e,n,r)}}function Nt(n,t,e,r){function u(){var n,t=c.status;if(!t&&zt(c)||t>=200&&300>t||304===t){try{n=e.call(i,c)}catch(r){return void o.error.call(i,r)}o.load.call(i,n)}else o.error.call(i,c)}var i={},o=ta.dispatch("beforesend","progress","load","error"),a={},c=new XMLHttpRequest,l=null;return!this.XDomainRequest||"withCredentials"in c||!/^(http(s)?:)?\/\//.test(n)||(c=new XDomainRequest),"onload"in c?c.onload=c.onerror=u:c.onreadystatechange=function(){c.readyState>3&&u()},c.onprogress=function(n){var t=ta.event;ta.event=n;try{o.progress.call(i,c)}finally{ta.event=t}},i.header=function(n,t){return n=(n+"").toLowerCase(),arguments.length<2?a[n]:(null==t?delete a[n]:a[n]=t+"",i)},i.mimeType=function(n){return arguments.length?(t=null==n?null:n+"",i):t},i.responseType=function(n){return arguments.length?(l=n,i):l},i.response=function(n){return e=n,i},["get","post"].forEach(function(n){i[n]=function(){return i.send.apply(i,[n].concat(ra(arguments)))}}),i.send=function(e,r,u){if(2===arguments.length&&"function"==typeof r&&(u=r,r=null),c.open(e,n,!0),null==t||"accept"in a||(a.accept=t+",*/*"),c.setRequestHeader)for(var s in a)c.setRequestHeader(s,a[s]);return null!=t&&c.overrideMimeType&&c.overrideMimeType(t),null!=l&&(c.responseType=l),null!=u&&i.on("error",u).on("load",function(n){u(null,n)}),o.beforesend.call(i,c),c.send(null==r?null:r),i},i.abort=function(){return c.abort(),i},ta.rebind(i,o,"on"),null==r?i:i.get(Ct(r))}function Ct(n){return 1===n.length?function(t,e){n(null==t?e:null)}:n}function zt(n){var t=n.responseType;return t&&"text"!==t?n.response:n.responseText}function qt(){var n=Lt(),t=Tt()-n;t>24?(isFinite(t)&&(clearTimeout(tc),tc=setTimeout(qt,t)),nc=0):(nc=1,rc(qt))}function Lt(){var n=Date.now();for(ec=Ka;ec;)n>=ec.t&&(ec.f=ec.c(n-ec.t)),ec=ec.n;return n}function Tt(){for(var n,t=Ka,e=1/0;t;)t.f?t=n?n.n=t.n:Ka=t.n:(t.t8?function(n){return n/e}:function(n){return n*e},symbol:n}}function Pt(n){var t=n.decimal,e=n.thousands,r=n.grouping,u=n.currency,i=r&&e?function(n,t){for(var u=n.length,i=[],o=0,a=r[0],c=0;u>0&&a>0&&(c+a+1>t&&(a=Math.max(1,t-c)),i.push(n.substring(u-=a,u+a)),!((c+=a+1)>t));)a=r[o=(o+1)%r.length];return i.reverse().join(e)}:y;return function(n){var e=ic.exec(n),r=e[1]||" ",o=e[2]||">",a=e[3]||"-",c=e[4]||"",l=e[5],s=+e[6],f=e[7],h=e[8],g=e[9],p=1,v="",d="",m=!1,y=!0;switch(h&&(h=+h.substring(1)),(l||"0"===r&&"="===o)&&(l=r="0",o="="),g){case"n":f=!0,g="g";break;case"%":p=100,d="%",g="f";break;case"p":p=100,d="%",g="r";break;case"b":case"o":case"x":case"X":"#"===c&&(v="0"+g.toLowerCase());case"c":y=!1;case"d":m=!0,h=0;break;case"s":p=-1,g="r"}"$"===c&&(v=u[0],d=u[1]),"r"!=g||h||(g="g"),null!=h&&("g"==g?h=Math.max(1,Math.min(21,h)):("e"==g||"f"==g)&&(h=Math.max(0,Math.min(20,h)))),g=oc.get(g)||Ut;var M=l&&f;return function(n){var e=d;if(m&&n%1)return"";var u=0>n||0===n&&0>1/n?(n=-n,"-"):"-"===a?"":a;if(0>p){var c=ta.formatPrefix(n,h);n=c.scale(n),e=c.symbol+d}else n*=p;n=g(n,h);var x,b,_=n.lastIndexOf(".");if(0>_){var w=y?n.lastIndexOf("e"):-1;0>w?(x=n,b=""):(x=n.substring(0,w),b=n.substring(w))}else x=n.substring(0,_),b=t+n.substring(_+1);!l&&f&&(x=i(x,1/0));var S=v.length+x.length+b.length+(M?0:u.length),k=s>S?new Array(S=s-S+1).join(r):"";return M&&(x=i(k+x,k.length?s-b.length:1/0)),u+=v,n=x+b,("<"===o?u+n+k:">"===o?k+u+n:"^"===o?k.substring(0,S>>=1)+u+n+k.substring(S):u+(M?n:k+n))+e}}}function Ut(n){return n+""}function jt(){this._=new Date(arguments.length>1?Date.UTC.apply(this,arguments):arguments[0])}function Ft(n,t,e){function r(t){var e=n(t),r=i(e,1);return r-t>t-e?e:r}function u(e){return t(e=n(new cc(e-1)),1),e}function i(n,e){return t(n=new cc(+n),e),n}function o(n,r,i){var o=u(n),a=[];if(i>1)for(;r>o;)e(o)%i||a.push(new Date(+o)),t(o,1);else for(;r>o;)a.push(new Date(+o)),t(o,1);return a}function a(n,t,e){try{cc=jt;var r=new jt;return r._=n,o(r,t,e)}finally{cc=Date}}n.floor=n,n.round=r,n.ceil=u,n.offset=i,n.range=o;var c=n.utc=Ht(n);return c.floor=c,c.round=Ht(r),c.ceil=Ht(u),c.offset=Ht(i),c.range=a,n}function Ht(n){return function(t,e){try{cc=jt;var r=new jt;return r._=t,n(r,e)._}finally{cc=Date}}}function Ot(n){function t(n){function t(t){for(var e,u,i,o=[],a=-1,c=0;++aa;){if(r>=l)return-1;if(u=t.charCodeAt(a++),37===u){if(o=t.charAt(a++),i=C[o in sc?t.charAt(a++):o],!i||(r=i(n,e,r))<0)return-1}else if(u!=e.charCodeAt(r++))return-1}return r}function r(n,t,e){_.lastIndex=0;var r=_.exec(t.slice(e));return r?(n.w=w.get(r[0].toLowerCase()),e+r[0].length):-1}function u(n,t,e){x.lastIndex=0;var r=x.exec(t.slice(e));return r?(n.w=b.get(r[0].toLowerCase()),e+r[0].length):-1}function i(n,t,e){E.lastIndex=0;var r=E.exec(t.slice(e));return r?(n.m=A.get(r[0].toLowerCase()),e+r[0].length):-1}function o(n,t,e){S.lastIndex=0;var r=S.exec(t.slice(e));return r?(n.m=k.get(r[0].toLowerCase()),e+r[0].length):-1}function a(n,t,r){return e(n,N.c.toString(),t,r)}function c(n,t,r){return e(n,N.x.toString(),t,r)}function l(n,t,r){return e(n,N.X.toString(),t,r)}function s(n,t,e){var r=M.get(t.slice(e,e+=2).toLowerCase());return null==r?-1:(n.p=r,e)}var f=n.dateTime,h=n.date,g=n.time,p=n.periods,v=n.days,d=n.shortDays,m=n.months,y=n.shortMonths;t.utc=function(n){function e(n){try{cc=jt;var t=new cc;return t._=n,r(t)}finally{cc=Date}}var r=t(n);return e.parse=function(n){try{cc=jt;var t=r.parse(n);return t&&t._}finally{cc=Date}},e.toString=r.toString,e},t.multi=t.utc.multi=ae;var M=ta.map(),x=Yt(v),b=Zt(v),_=Yt(d),w=Zt(d),S=Yt(m),k=Zt(m),E=Yt(y),A=Zt(y);p.forEach(function(n,t){M.set(n.toLowerCase(),t)});var N={a:function(n){return d[n.getDay()]},A:function(n){return v[n.getDay()]},b:function(n){return y[n.getMonth()]},B:function(n){return m[n.getMonth()]},c:t(f),d:function(n,t){return It(n.getDate(),t,2)},e:function(n,t){return It(n.getDate(),t,2)},H:function(n,t){return It(n.getHours(),t,2)},I:function(n,t){return It(n.getHours()%12||12,t,2)},j:function(n,t){return It(1+ac.dayOfYear(n),t,3)},L:function(n,t){return It(n.getMilliseconds(),t,3)},m:function(n,t){return It(n.getMonth()+1,t,2)},M:function(n,t){return It(n.getMinutes(),t,2)},p:function(n){return p[+(n.getHours()>=12)]},S:function(n,t){return It(n.getSeconds(),t,2)},U:function(n,t){return It(ac.sundayOfYear(n),t,2)},w:function(n){return n.getDay()},W:function(n,t){return It(ac.mondayOfYear(n),t,2)},x:t(h),X:t(g),y:function(n,t){return It(n.getFullYear()%100,t,2)},Y:function(n,t){return It(n.getFullYear()%1e4,t,4)},Z:ie,"%":function(){return"%"}},C={a:r,A:u,b:i,B:o,c:a,d:Qt,e:Qt,H:te,I:te,j:ne,L:ue,m:Kt,M:ee,p:s,S:re,U:Xt,w:Vt,W:$t,x:c,X:l,y:Wt,Y:Bt,Z:Jt,"%":oe};return t}function It(n,t,e){var r=0>n?"-":"",u=(r?-n:n)+"",i=u.length;return r+(e>i?new Array(e-i+1).join(t)+u:u)}function Yt(n){return new RegExp("^(?:"+n.map(ta.requote).join("|")+")","i")}function Zt(n){for(var t=new l,e=-1,r=n.length;++e68?1900:2e3)}function Kt(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.m=r[0]-1,e+r[0].length):-1}function Qt(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.d=+r[0],e+r[0].length):-1}function ne(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+3));return r?(n.j=+r[0],e+r[0].length):-1}function te(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.H=+r[0],e+r[0].length):-1}function ee(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.M=+r[0],e+r[0].length):-1}function re(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.S=+r[0],e+r[0].length):-1}function ue(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+3));return r?(n.L=+r[0],e+r[0].length):-1}function ie(n){var t=n.getTimezoneOffset(),e=t>0?"-":"+",r=ga(t)/60|0,u=ga(t)%60;return e+It(r,"0",2)+It(u,"0",2)}function oe(n,t,e){hc.lastIndex=0;var r=hc.exec(t.slice(e,e+1));return r?e+r[0].length:-1}function ae(n){for(var t=n.length,e=-1;++e=0?1:-1,a=o*e,c=Math.cos(t),l=Math.sin(t),s=i*l,f=u*c+s*Math.cos(a),h=s*o*Math.sin(a);yc.add(Math.atan2(h,f)),r=n,u=c,i=l}var t,e,r,u,i;Mc.point=function(o,a){Mc.point=n,r=(t=o)*Da,u=Math.cos(a=(e=a)*Da/2+qa/4),i=Math.sin(a)},Mc.lineEnd=function(){n(t,e)}}function pe(n){var t=n[0],e=n[1],r=Math.cos(e);return[r*Math.cos(t),r*Math.sin(t),Math.sin(e)]}function ve(n,t){return n[0]*t[0]+n[1]*t[1]+n[2]*t[2]}function de(n,t){return[n[1]*t[2]-n[2]*t[1],n[2]*t[0]-n[0]*t[2],n[0]*t[1]-n[1]*t[0]]}function me(n,t){n[0]+=t[0],n[1]+=t[1],n[2]+=t[2]}function ye(n,t){return[n[0]*t,n[1]*t,n[2]*t]}function Me(n){var t=Math.sqrt(n[0]*n[0]+n[1]*n[1]+n[2]*n[2]);n[0]/=t,n[1]/=t,n[2]/=t}function xe(n){return[Math.atan2(n[1],n[0]),tt(n[2])]}function be(n,t){return ga(n[0]-t[0])a;++a)u.point((e=n[a])[0],e[1]);return void u.lineEnd()}var c=new qe(e,n,null,!0),l=new qe(e,null,c,!1);c.o=l,i.push(c),o.push(l),c=new qe(r,n,null,!1),l=new qe(r,null,c,!0),c.o=l,i.push(c),o.push(l)}}),o.sort(t),ze(i),ze(o),i.length){for(var a=0,c=e,l=o.length;l>a;++a)o[a].e=c=!c;for(var s,f,h=i[0];;){for(var g=h,p=!0;g.v;)if((g=g.n)===h)return;s=g.z,u.lineStart();do{if(g.v=g.o.v=!0,g.e){if(p)for(var a=0,l=s.length;l>a;++a)u.point((f=s[a])[0],f[1]);else r(g.x,g.n.x,1,u);g=g.n}else{if(p){s=g.p.z;for(var a=s.length-1;a>=0;--a)u.point((f=s[a])[0],f[1])}else r(g.x,g.p.x,-1,u);g=g.p}g=g.o,s=g.z,p=!p}while(!g.v);u.lineEnd()}}}function ze(n){if(t=n.length){for(var t,e,r=0,u=n[0];++r0){for(b||(i.polygonStart(),b=!0),i.lineStart();++o1&&2&t&&e.push(e.pop().concat(e.shift())),g.push(e.filter(Te))}var g,p,v,d=t(i),m=u.invert(r[0],r[1]),y={point:o,lineStart:c,lineEnd:l,polygonStart:function(){y.point=s,y.lineStart=f,y.lineEnd=h,g=[],p=[]},polygonEnd:function(){y.point=o,y.lineStart=c,y.lineEnd=l,g=ta.merge(g);var n=Fe(m,p);g.length?(b||(i.polygonStart(),b=!0),Ce(g,De,n,e,i)):n&&(b||(i.polygonStart(),b=!0),i.lineStart(),e(null,null,1,i),i.lineEnd()),b&&(i.polygonEnd(),b=!1),g=p=null},sphere:function(){i.polygonStart(),i.lineStart(),e(null,null,1,i),i.lineEnd(),i.polygonEnd()}},M=Re(),x=t(M),b=!1;return y}}function Te(n){return n.length>1}function Re(){var n,t=[];return{lineStart:function(){t.push(n=[])},point:function(t,e){n.push([t,e])},lineEnd:b,buffer:function(){var e=t;return t=[],n=null,e},rejoin:function(){t.length>1&&t.push(t.pop().concat(t.shift()))}}}function De(n,t){return((n=n.x)[0]<0?n[1]-Ra-Ca:Ra-n[1])-((t=t.x)[0]<0?t[1]-Ra-Ca:Ra-t[1])}function Pe(n){var t,e=0/0,r=0/0,u=0/0;return{lineStart:function(){n.lineStart(),t=1},point:function(i,o){var a=i>0?qa:-qa,c=ga(i-e);ga(c-qa)0?Ra:-Ra),n.point(u,r),n.lineEnd(),n.lineStart(),n.point(a,r),n.point(i,r),t=0):u!==a&&c>=qa&&(ga(e-u)Ca?Math.atan((Math.sin(t)*(i=Math.cos(r))*Math.sin(e)-Math.sin(r)*(u=Math.cos(t))*Math.sin(n))/(u*i*o)):(t+r)/2}function je(n,t,e,r){var u;if(null==n)u=e*Ra,r.point(-qa,u),r.point(0,u),r.point(qa,u),r.point(qa,0),r.point(qa,-u),r.point(0,-u),r.point(-qa,-u),r.point(-qa,0),r.point(-qa,u);else if(ga(n[0]-t[0])>Ca){var i=n[0]a;++a){var l=t[a],s=l.length;if(s)for(var f=l[0],h=f[0],g=f[1]/2+qa/4,p=Math.sin(g),v=Math.cos(g),d=1;;){d===s&&(d=0),n=l[d];var m=n[0],y=n[1]/2+qa/4,M=Math.sin(y),x=Math.cos(y),b=m-h,_=b>=0?1:-1,w=_*b,S=w>qa,k=p*M;if(yc.add(Math.atan2(k*_*Math.sin(w),v*x+k*Math.cos(w))),i+=S?b+_*La:b,S^h>=e^m>=e){var E=de(pe(f),pe(n));Me(E);var A=de(u,E);Me(A);var N=(S^b>=0?-1:1)*tt(A[2]);(r>N||r===N&&(E[0]||E[1]))&&(o+=S^b>=0?1:-1)}if(!d++)break;h=m,p=M,v=x,f=n}}return(-Ca>i||Ca>i&&0>yc)^1&o}function He(n){function t(n,t){return Math.cos(n)*Math.cos(t)>i}function e(n){var e,i,c,l,s;return{lineStart:function(){l=c=!1,s=1},point:function(f,h){var g,p=[f,h],v=t(f,h),d=o?v?0:u(f,h):v?u(f+(0>f?qa:-qa),h):0;if(!e&&(l=c=v)&&n.lineStart(),v!==c&&(g=r(e,p),(be(e,g)||be(p,g))&&(p[0]+=Ca,p[1]+=Ca,v=t(p[0],p[1]))),v!==c)s=0,v?(n.lineStart(),g=r(p,e),n.point(g[0],g[1])):(g=r(e,p),n.point(g[0],g[1]),n.lineEnd()),e=g;else if(a&&e&&o^v){var m;d&i||!(m=r(p,e,!0))||(s=0,o?(n.lineStart(),n.point(m[0][0],m[0][1]),n.point(m[1][0],m[1][1]),n.lineEnd()):(n.point(m[1][0],m[1][1]),n.lineEnd(),n.lineStart(),n.point(m[0][0],m[0][1])))}!v||e&&be(e,p)||n.point(p[0],p[1]),e=p,c=v,i=d},lineEnd:function(){c&&n.lineEnd(),e=null},clean:function(){return s|(l&&c)<<1}}}function r(n,t,e){var r=pe(n),u=pe(t),o=[1,0,0],a=de(r,u),c=ve(a,a),l=a[0],s=c-l*l;if(!s)return!e&&n;var f=i*c/s,h=-i*l/s,g=de(o,a),p=ye(o,f),v=ye(a,h);me(p,v);var d=g,m=ve(p,d),y=ve(d,d),M=m*m-y*(ve(p,p)-1);if(!(0>M)){var x=Math.sqrt(M),b=ye(d,(-m-x)/y);if(me(b,p),b=xe(b),!e)return b;var _,w=n[0],S=t[0],k=n[1],E=t[1];w>S&&(_=w,w=S,S=_);var A=S-w,N=ga(A-qa)A;if(!N&&k>E&&(_=k,k=E,E=_),C?N?k+E>0^b[1]<(ga(b[0]-w)qa^(w<=b[0]&&b[0]<=S)){var z=ye(d,(-m+x)/y);return me(z,p),[b,xe(z)]}}}function u(t,e){var r=o?n:qa-n,u=0;return-r>t?u|=1:t>r&&(u|=2),-r>e?u|=4:e>r&&(u|=8),u}var i=Math.cos(n),o=i>0,a=ga(i)>Ca,c=gr(n,6*Da);return Le(t,e,c,o?[0,-n]:[-qa,n-qa])}function Oe(n,t,e,r){return function(u){var i,o=u.a,a=u.b,c=o.x,l=o.y,s=a.x,f=a.y,h=0,g=1,p=s-c,v=f-l;if(i=n-c,p||!(i>0)){if(i/=p,0>p){if(h>i)return;g>i&&(g=i)}else if(p>0){if(i>g)return;i>h&&(h=i)}if(i=e-c,p||!(0>i)){if(i/=p,0>p){if(i>g)return;i>h&&(h=i)}else if(p>0){if(h>i)return;g>i&&(g=i)}if(i=t-l,v||!(i>0)){if(i/=v,0>v){if(h>i)return;g>i&&(g=i)}else if(v>0){if(i>g)return;i>h&&(h=i)}if(i=r-l,v||!(0>i)){if(i/=v,0>v){if(i>g)return;i>h&&(h=i)}else if(v>0){if(h>i)return;g>i&&(g=i)}return h>0&&(u.a={x:c+h*p,y:l+h*v}),1>g&&(u.b={x:c+g*p,y:l+g*v}),u}}}}}}function Ie(n,t,e,r){function u(r,u){return ga(r[0]-n)0?0:3:ga(r[0]-e)0?2:1:ga(r[1]-t)0?1:0:u>0?3:2}function i(n,t){return o(n.x,t.x)}function o(n,t){var e=u(n,1),r=u(t,1);return e!==r?e-r:0===e?t[1]-n[1]:1===e?n[0]-t[0]:2===e?n[1]-t[1]:t[0]-n[0]}return function(a){function c(n){for(var t=0,e=d.length,r=n[1],u=0;e>u;++u)for(var i,o=1,a=d[u],c=a.length,l=a[0];c>o;++o)i=a[o],l[1]<=r?i[1]>r&&Q(l,i,n)>0&&++t:i[1]<=r&&Q(l,i,n)<0&&--t,l=i;return 0!==t}function l(i,a,c,l){var s=0,f=0;if(null==i||(s=u(i,c))!==(f=u(a,c))||o(i,a)<0^c>0){do l.point(0===s||3===s?n:e,s>1?r:t);while((s=(s+c+4)%4)!==f)}else l.point(a[0],a[1])}function s(u,i){return u>=n&&e>=u&&i>=t&&r>=i}function f(n,t){s(n,t)&&a.point(n,t)}function h(){C.point=p,d&&d.push(m=[]),S=!0,w=!1,b=_=0/0}function g(){v&&(p(y,M),x&&w&&A.rejoin(),v.push(A.buffer())),C.point=f,w&&a.lineEnd()}function p(n,t){n=Math.max(-Tc,Math.min(Tc,n)),t=Math.max(-Tc,Math.min(Tc,t));var e=s(n,t);if(d&&m.push([n,t]),S)y=n,M=t,x=e,S=!1,e&&(a.lineStart(),a.point(n,t));else if(e&&w)a.point(n,t);else{var r={a:{x:b,y:_},b:{x:n,y:t}};N(r)?(w||(a.lineStart(),a.point(r.a.x,r.a.y)),a.point(r.b.x,r.b.y),e||a.lineEnd(),k=!1):e&&(a.lineStart(),a.point(n,t),k=!1)}b=n,_=t,w=e}var v,d,m,y,M,x,b,_,w,S,k,E=a,A=Re(),N=Oe(n,t,e,r),C={point:f,lineStart:h,lineEnd:g,polygonStart:function(){a=A,v=[],d=[],k=!0},polygonEnd:function(){a=E,v=ta.merge(v);var t=c([n,r]),e=k&&t,u=v.length;(e||u)&&(a.polygonStart(),e&&(a.lineStart(),l(null,null,1,a),a.lineEnd()),u&&Ce(v,i,t,l,a),a.polygonEnd()),v=d=m=null}};return C}}function Ye(n){var t=0,e=qa/3,r=ir(n),u=r(t,e);return u.parallels=function(n){return arguments.length?r(t=n[0]*qa/180,e=n[1]*qa/180):[t/qa*180,e/qa*180]},u}function Ze(n,t){function e(n,t){var e=Math.sqrt(i-2*u*Math.sin(t))/u;return[e*Math.sin(n*=u),o-e*Math.cos(n)]}var r=Math.sin(n),u=(r+Math.sin(t))/2,i=1+r*(2*u-r),o=Math.sqrt(i)/u;return e.invert=function(n,t){var e=o-t;return[Math.atan2(n,e)/u,tt((i-(n*n+e*e)*u*u)/(2*u))]},e}function Ve(){function n(n,t){Dc+=u*n-r*t,r=n,u=t}var t,e,r,u;Hc.point=function(i,o){Hc.point=n,t=r=i,e=u=o},Hc.lineEnd=function(){n(t,e)}}function Xe(n,t){Pc>n&&(Pc=n),n>jc&&(jc=n),Uc>t&&(Uc=t),t>Fc&&(Fc=t)}function $e(){function n(n,t){o.push("M",n,",",t,i)}function t(n,t){o.push("M",n,",",t),a.point=e}function e(n,t){o.push("L",n,",",t)}function r(){a.point=n}function u(){o.push("Z")}var i=Be(4.5),o=[],a={point:n,lineStart:function(){a.point=t},lineEnd:r,polygonStart:function(){a.lineEnd=u},polygonEnd:function(){a.lineEnd=r,a.point=n},pointRadius:function(n){return i=Be(n),a},result:function(){if(o.length){var n=o.join("");return o=[],n}}};return a}function Be(n){return"m0,"+n+"a"+n+","+n+" 0 1,1 0,"+-2*n+"a"+n+","+n+" 0 1,1 0,"+2*n+"z"}function We(n,t){_c+=n,wc+=t,++Sc}function Je(){function n(n,r){var u=n-t,i=r-e,o=Math.sqrt(u*u+i*i);kc+=o*(t+n)/2,Ec+=o*(e+r)/2,Ac+=o,We(t=n,e=r)}var t,e;Ic.point=function(r,u){Ic.point=n,We(t=r,e=u)}}function Ge(){Ic.point=We}function Ke(){function n(n,t){var e=n-r,i=t-u,o=Math.sqrt(e*e+i*i);kc+=o*(r+n)/2,Ec+=o*(u+t)/2,Ac+=o,o=u*n-r*t,Nc+=o*(r+n),Cc+=o*(u+t),zc+=3*o,We(r=n,u=t)}var t,e,r,u;Ic.point=function(i,o){Ic.point=n,We(t=r=i,e=u=o)},Ic.lineEnd=function(){n(t,e)}}function Qe(n){function t(t,e){n.moveTo(t+o,e),n.arc(t,e,o,0,La)}function e(t,e){n.moveTo(t,e),a.point=r}function r(t,e){n.lineTo(t,e)}function u(){a.point=t}function i(){n.closePath()}var o=4.5,a={point:t,lineStart:function(){a.point=e},lineEnd:u,polygonStart:function(){a.lineEnd=i},polygonEnd:function(){a.lineEnd=u,a.point=t},pointRadius:function(n){return o=n,a},result:b};return a}function nr(n){function t(n){return(a?r:e)(n)}function e(t){return rr(t,function(e,r){e=n(e,r),t.point(e[0],e[1])})}function r(t){function e(e,r){e=n(e,r),t.point(e[0],e[1])}function r(){M=0/0,S.point=i,t.lineStart()}function i(e,r){var i=pe([e,r]),o=n(e,r);u(M,x,y,b,_,w,M=o[0],x=o[1],y=e,b=i[0],_=i[1],w=i[2],a,t),t.point(M,x)}function o(){S.point=e,t.lineEnd()}function c(){r(),S.point=l,S.lineEnd=s}function l(n,t){i(f=n,h=t),g=M,p=x,v=b,d=_,m=w,S.point=i}function s(){u(M,x,y,b,_,w,g,p,f,v,d,m,a,t),S.lineEnd=o,o()}var f,h,g,p,v,d,m,y,M,x,b,_,w,S={point:e,lineStart:r,lineEnd:o,polygonStart:function(){t.polygonStart(),S.lineStart=c +},polygonEnd:function(){t.polygonEnd(),S.lineStart=r}};return S}function u(t,e,r,a,c,l,s,f,h,g,p,v,d,m){var y=s-t,M=f-e,x=y*y+M*M;if(x>4*i&&d--){var b=a+g,_=c+p,w=l+v,S=Math.sqrt(b*b+_*_+w*w),k=Math.asin(w/=S),E=ga(ga(w)-1)i||ga((y*z+M*q)/x-.5)>.3||o>a*g+c*p+l*v)&&(u(t,e,r,a,c,l,N,C,E,b/=S,_/=S,w,d,m),m.point(N,C),u(N,C,E,b,_,w,s,f,h,g,p,v,d,m))}}var i=.5,o=Math.cos(30*Da),a=16;return t.precision=function(n){return arguments.length?(a=(i=n*n)>0&&16,t):Math.sqrt(i)},t}function tr(n){var t=nr(function(t,e){return n([t*Pa,e*Pa])});return function(n){return or(t(n))}}function er(n){this.stream=n}function rr(n,t){return{point:t,sphere:function(){n.sphere()},lineStart:function(){n.lineStart()},lineEnd:function(){n.lineEnd()},polygonStart:function(){n.polygonStart()},polygonEnd:function(){n.polygonEnd()}}}function ur(n){return ir(function(){return n})()}function ir(n){function t(n){return n=a(n[0]*Da,n[1]*Da),[n[0]*h+c,l-n[1]*h]}function e(n){return n=a.invert((n[0]-c)/h,(l-n[1])/h),n&&[n[0]*Pa,n[1]*Pa]}function r(){a=Ae(o=lr(m,M,x),i);var n=i(v,d);return c=g-n[0]*h,l=p+n[1]*h,u()}function u(){return s&&(s.valid=!1,s=null),t}var i,o,a,c,l,s,f=nr(function(n,t){return n=i(n,t),[n[0]*h+c,l-n[1]*h]}),h=150,g=480,p=250,v=0,d=0,m=0,M=0,x=0,b=Lc,_=y,w=null,S=null;return t.stream=function(n){return s&&(s.valid=!1),s=or(b(o,f(_(n)))),s.valid=!0,s},t.clipAngle=function(n){return arguments.length?(b=null==n?(w=n,Lc):He((w=+n)*Da),u()):w},t.clipExtent=function(n){return arguments.length?(S=n,_=n?Ie(n[0][0],n[0][1],n[1][0],n[1][1]):y,u()):S},t.scale=function(n){return arguments.length?(h=+n,r()):h},t.translate=function(n){return arguments.length?(g=+n[0],p=+n[1],r()):[g,p]},t.center=function(n){return arguments.length?(v=n[0]%360*Da,d=n[1]%360*Da,r()):[v*Pa,d*Pa]},t.rotate=function(n){return arguments.length?(m=n[0]%360*Da,M=n[1]%360*Da,x=n.length>2?n[2]%360*Da:0,r()):[m*Pa,M*Pa,x*Pa]},ta.rebind(t,f,"precision"),function(){return i=n.apply(this,arguments),t.invert=i.invert&&e,r()}}function or(n){return rr(n,function(t,e){n.point(t*Da,e*Da)})}function ar(n,t){return[n,t]}function cr(n,t){return[n>qa?n-La:-qa>n?n+La:n,t]}function lr(n,t,e){return n?t||e?Ae(fr(n),hr(t,e)):fr(n):t||e?hr(t,e):cr}function sr(n){return function(t,e){return t+=n,[t>qa?t-La:-qa>t?t+La:t,e]}}function fr(n){var t=sr(n);return t.invert=sr(-n),t}function hr(n,t){function e(n,t){var e=Math.cos(t),a=Math.cos(n)*e,c=Math.sin(n)*e,l=Math.sin(t),s=l*r+a*u;return[Math.atan2(c*i-s*o,a*r-l*u),tt(s*i+c*o)]}var r=Math.cos(n),u=Math.sin(n),i=Math.cos(t),o=Math.sin(t);return e.invert=function(n,t){var e=Math.cos(t),a=Math.cos(n)*e,c=Math.sin(n)*e,l=Math.sin(t),s=l*i-c*o;return[Math.atan2(c*i+l*o,a*r+s*u),tt(s*r-a*u)]},e}function gr(n,t){var e=Math.cos(n),r=Math.sin(n);return function(u,i,o,a){var c=o*t;null!=u?(u=pr(e,u),i=pr(e,i),(o>0?i>u:u>i)&&(u+=o*La)):(u=n+o*La,i=n-.5*c);for(var l,s=u;o>0?s>i:i>s;s-=c)a.point((l=xe([e,-r*Math.cos(s),-r*Math.sin(s)]))[0],l[1])}}function pr(n,t){var e=pe(t);e[0]-=n,Me(e);var r=nt(-e[1]);return((-e[2]<0?-r:r)+2*Math.PI-Ca)%(2*Math.PI)}function vr(n,t,e){var r=ta.range(n,t-Ca,e).concat(t);return function(n){return r.map(function(t){return[n,t]})}}function dr(n,t,e){var r=ta.range(n,t-Ca,e).concat(t);return function(n){return r.map(function(t){return[t,n]})}}function mr(n){return n.source}function yr(n){return n.target}function Mr(n,t,e,r){var u=Math.cos(t),i=Math.sin(t),o=Math.cos(r),a=Math.sin(r),c=u*Math.cos(n),l=u*Math.sin(n),s=o*Math.cos(e),f=o*Math.sin(e),h=2*Math.asin(Math.sqrt(it(r-t)+u*o*it(e-n))),g=1/Math.sin(h),p=h?function(n){var t=Math.sin(n*=h)*g,e=Math.sin(h-n)*g,r=e*c+t*s,u=e*l+t*f,o=e*i+t*a;return[Math.atan2(u,r)*Pa,Math.atan2(o,Math.sqrt(r*r+u*u))*Pa]}:function(){return[n*Pa,t*Pa]};return p.distance=h,p}function xr(){function n(n,u){var i=Math.sin(u*=Da),o=Math.cos(u),a=ga((n*=Da)-t),c=Math.cos(a);Yc+=Math.atan2(Math.sqrt((a=o*Math.sin(a))*a+(a=r*i-e*o*c)*a),e*i+r*o*c),t=n,e=i,r=o}var t,e,r;Zc.point=function(u,i){t=u*Da,e=Math.sin(i*=Da),r=Math.cos(i),Zc.point=n},Zc.lineEnd=function(){Zc.point=Zc.lineEnd=b}}function br(n,t){function e(t,e){var r=Math.cos(t),u=Math.cos(e),i=n(r*u);return[i*u*Math.sin(t),i*Math.sin(e)]}return e.invert=function(n,e){var r=Math.sqrt(n*n+e*e),u=t(r),i=Math.sin(u),o=Math.cos(u);return[Math.atan2(n*i,r*o),Math.asin(r&&e*i/r)]},e}function _r(n,t){function e(n,t){o>0?-Ra+Ca>t&&(t=-Ra+Ca):t>Ra-Ca&&(t=Ra-Ca);var e=o/Math.pow(u(t),i);return[e*Math.sin(i*n),o-e*Math.cos(i*n)]}var r=Math.cos(n),u=function(n){return Math.tan(qa/4+n/2)},i=n===t?Math.sin(n):Math.log(r/Math.cos(t))/Math.log(u(t)/u(n)),o=r*Math.pow(u(n),i)/i;return i?(e.invert=function(n,t){var e=o-t,r=K(i)*Math.sqrt(n*n+e*e);return[Math.atan2(n,e)/i,2*Math.atan(Math.pow(o/r,1/i))-Ra]},e):Sr}function wr(n,t){function e(n,t){var e=i-t;return[e*Math.sin(u*n),i-e*Math.cos(u*n)]}var r=Math.cos(n),u=n===t?Math.sin(n):(r-Math.cos(t))/(t-n),i=r/u+n;return ga(u)u;u++){for(;r>1&&Q(n[e[r-2]],n[e[r-1]],n[u])<=0;)--r;e[r++]=u}return e.slice(0,r)}function zr(n,t){return n[0]-t[0]||n[1]-t[1]}function qr(n,t,e){return(e[0]-t[0])*(n[1]-t[1])<(e[1]-t[1])*(n[0]-t[0])}function Lr(n,t,e,r){var u=n[0],i=e[0],o=t[0]-u,a=r[0]-i,c=n[1],l=e[1],s=t[1]-c,f=r[1]-l,h=(a*(c-l)-f*(u-i))/(f*o-a*s);return[u+h*o,c+h*s]}function Tr(n){var t=n[0],e=n[n.length-1];return!(t[0]-e[0]||t[1]-e[1])}function Rr(){tu(this),this.edge=this.site=this.circle=null}function Dr(n){var t=el.pop()||new Rr;return t.site=n,t}function Pr(n){Xr(n),Qc.remove(n),el.push(n),tu(n)}function Ur(n){var t=n.circle,e=t.x,r=t.cy,u={x:e,y:r},i=n.P,o=n.N,a=[n];Pr(n);for(var c=i;c.circle&&ga(e-c.circle.x)s;++s)l=a[s],c=a[s-1],Kr(l.edge,c.site,l.site,u);c=a[0],l=a[f-1],l.edge=Jr(c.site,l.site,null,u),Vr(c),Vr(l)}function jr(n){for(var t,e,r,u,i=n.x,o=n.y,a=Qc._;a;)if(r=Fr(a,o)-i,r>Ca)a=a.L;else{if(u=i-Hr(a,o),!(u>Ca)){r>-Ca?(t=a.P,e=a):u>-Ca?(t=a,e=a.N):t=e=a;break}if(!a.R){t=a;break}a=a.R}var c=Dr(n);if(Qc.insert(t,c),t||e){if(t===e)return Xr(t),e=Dr(t.site),Qc.insert(c,e),c.edge=e.edge=Jr(t.site,c.site),Vr(t),void Vr(e);if(!e)return void(c.edge=Jr(t.site,c.site));Xr(t),Xr(e);var l=t.site,s=l.x,f=l.y,h=n.x-s,g=n.y-f,p=e.site,v=p.x-s,d=p.y-f,m=2*(h*d-g*v),y=h*h+g*g,M=v*v+d*d,x={x:(d*y-g*M)/m+s,y:(h*M-v*y)/m+f};Kr(e.edge,l,p,x),c.edge=Jr(l,n,null,x),e.edge=Jr(n,p,null,x),Vr(t),Vr(e)}}function Fr(n,t){var e=n.site,r=e.x,u=e.y,i=u-t;if(!i)return r;var o=n.P;if(!o)return-1/0;e=o.site;var a=e.x,c=e.y,l=c-t;if(!l)return a;var s=a-r,f=1/i-1/l,h=s/l;return f?(-h+Math.sqrt(h*h-2*f*(s*s/(-2*l)-c+l/2+u-i/2)))/f+r:(r+a)/2}function Hr(n,t){var e=n.N;if(e)return Fr(e,t);var r=n.site;return r.y===t?r.x:1/0}function Or(n){this.site=n,this.edges=[]}function Ir(n){for(var t,e,r,u,i,o,a,c,l,s,f=n[0][0],h=n[1][0],g=n[0][1],p=n[1][1],v=Kc,d=v.length;d--;)if(i=v[d],i&&i.prepare())for(a=i.edges,c=a.length,o=0;c>o;)s=a[o].end(),r=s.x,u=s.y,l=a[++o%c].start(),t=l.x,e=l.y,(ga(r-t)>Ca||ga(u-e)>Ca)&&(a.splice(o,0,new Qr(Gr(i.site,s,ga(r-f)Ca?{x:f,y:ga(t-f)Ca?{x:ga(e-p)Ca?{x:h,y:ga(t-h)Ca?{x:ga(e-g)=-za)){var g=c*c+l*l,p=s*s+f*f,v=(f*g-l*p)/h,d=(c*p-s*g)/h,f=d+a,m=rl.pop()||new Zr;m.arc=n,m.site=u,m.x=v+o,m.y=f+Math.sqrt(v*v+d*d),m.cy=f,n.circle=m;for(var y=null,M=tl._;M;)if(m.yd||d>=a)return;if(h>p){if(i){if(i.y>=l)return}else i={x:d,y:c};e={x:d,y:l}}else{if(i){if(i.yr||r>1)if(h>p){if(i){if(i.y>=l)return}else i={x:(c-u)/r,y:c};e={x:(l-u)/r,y:l}}else{if(i){if(i.yg){if(i){if(i.x>=a)return}else i={x:o,y:r*o+u};e={x:a,y:r*a+u}}else{if(i){if(i.xi||f>o||r>h||u>g)){if(p=n.point){var p,v=t-n.x,d=e-n.y,m=v*v+d*d;if(c>m){var y=Math.sqrt(c=m);r=t-y,u=e-y,i=t+y,o=e+y,a=p}}for(var M=n.nodes,x=.5*(s+h),b=.5*(f+g),_=t>=x,w=e>=b,S=w<<1|_,k=S+4;k>S;++S)if(n=M[3&S])switch(3&S){case 0:l(n,s,f,x,b);break;case 1:l(n,x,f,h,b);break;case 2:l(n,s,b,x,g);break;case 3:l(n,x,b,h,g)}}}(n,r,u,i,o),a}function gu(n,t){n=ta.rgb(n),t=ta.rgb(t);var e=n.r,r=n.g,u=n.b,i=t.r-e,o=t.g-r,a=t.b-u;return function(n){return"#"+xt(Math.round(e+i*n))+xt(Math.round(r+o*n))+xt(Math.round(u+a*n))}}function pu(n,t){var e,r={},u={};for(e in n)e in t?r[e]=mu(n[e],t[e]):u[e]=n[e];for(e in t)e in n||(u[e]=t[e]);return function(n){for(e in r)u[e]=r[e](n);return u}}function vu(n,t){return n=+n,t=+t,function(e){return n*(1-e)+t*e}}function du(n,t){var e,r,u,i=il.lastIndex=ol.lastIndex=0,o=-1,a=[],c=[];for(n+="",t+="";(e=il.exec(n))&&(r=ol.exec(t));)(u=r.index)>i&&(u=t.slice(i,u),a[o]?a[o]+=u:a[++o]=u),(e=e[0])===(r=r[0])?a[o]?a[o]+=r:a[++o]=r:(a[++o]=null,c.push({i:o,x:vu(e,r)})),i=ol.lastIndex;return ir;++r)a[(e=c[r]).i]=e.x(n);return a.join("")})}function mu(n,t){for(var e,r=ta.interpolators.length;--r>=0&&!(e=ta.interpolators[r](n,t)););return e}function yu(n,t){var e,r=[],u=[],i=n.length,o=t.length,a=Math.min(n.length,t.length);for(e=0;a>e;++e)r.push(mu(n[e],t[e]));for(;i>e;++e)u[e]=n[e];for(;o>e;++e)u[e]=t[e];return function(n){for(e=0;a>e;++e)u[e]=r[e](n);return u}}function Mu(n){return function(t){return 0>=t?0:t>=1?1:n(t)}}function xu(n){return function(t){return 1-n(1-t)}}function bu(n){return function(t){return.5*(.5>t?n(2*t):2-n(2-2*t))}}function _u(n){return n*n}function wu(n){return n*n*n}function Su(n){if(0>=n)return 0;if(n>=1)return 1;var t=n*n,e=t*n;return 4*(.5>n?e:3*(n-t)+e-.75)}function ku(n){return function(t){return Math.pow(t,n)}}function Eu(n){return 1-Math.cos(n*Ra)}function Au(n){return Math.pow(2,10*(n-1))}function Nu(n){return 1-Math.sqrt(1-n*n)}function Cu(n,t){var e;return arguments.length<2&&(t=.45),arguments.length?e=t/La*Math.asin(1/n):(n=1,e=t/4),function(r){return 1+n*Math.pow(2,-10*r)*Math.sin((r-e)*La/t)}}function zu(n){return n||(n=1.70158),function(t){return t*t*((n+1)*t-n)}}function qu(n){return 1/2.75>n?7.5625*n*n:2/2.75>n?7.5625*(n-=1.5/2.75)*n+.75:2.5/2.75>n?7.5625*(n-=2.25/2.75)*n+.9375:7.5625*(n-=2.625/2.75)*n+.984375}function Lu(n,t){n=ta.hcl(n),t=ta.hcl(t);var e=n.h,r=n.c,u=n.l,i=t.h-e,o=t.c-r,a=t.l-u;return isNaN(o)&&(o=0,r=isNaN(r)?t.c:r),isNaN(i)?(i=0,e=isNaN(e)?t.h:e):i>180?i-=360:-180>i&&(i+=360),function(n){return st(e+i*n,r+o*n,u+a*n)+""}}function Tu(n,t){n=ta.hsl(n),t=ta.hsl(t);var e=n.h,r=n.s,u=n.l,i=t.h-e,o=t.s-r,a=t.l-u;return isNaN(o)&&(o=0,r=isNaN(r)?t.s:r),isNaN(i)?(i=0,e=isNaN(e)?t.h:e):i>180?i-=360:-180>i&&(i+=360),function(n){return ct(e+i*n,r+o*n,u+a*n)+""}}function Ru(n,t){n=ta.lab(n),t=ta.lab(t);var e=n.l,r=n.a,u=n.b,i=t.l-e,o=t.a-r,a=t.b-u;return function(n){return ht(e+i*n,r+o*n,u+a*n)+""}}function Du(n,t){return t-=n,function(e){return Math.round(n+t*e)}}function Pu(n){var t=[n.a,n.b],e=[n.c,n.d],r=ju(t),u=Uu(t,e),i=ju(Fu(e,t,-u))||0;t[0]*e[1]180?s+=360:s-l>180&&(l+=360),u.push({i:r.push(r.pop()+"rotate(",null,")")-2,x:vu(l,s)})):s&&r.push(r.pop()+"rotate("+s+")"),f!=h?u.push({i:r.push(r.pop()+"skewX(",null,")")-2,x:vu(f,h)}):h&&r.push(r.pop()+"skewX("+h+")"),g[0]!=p[0]||g[1]!=p[1]?(e=r.push(r.pop()+"scale(",null,",",null,")"),u.push({i:e-4,x:vu(g[0],p[0])},{i:e-2,x:vu(g[1],p[1])})):(1!=p[0]||1!=p[1])&&r.push(r.pop()+"scale("+p+")"),e=u.length,function(n){for(var t,i=-1;++i=0;)e.push(u[r])}function Qu(n,t){for(var e=[n],r=[];null!=(n=e.pop());)if(r.push(n),(i=n.children)&&(u=i.length))for(var u,i,o=-1;++oe;++e)(t=n[e][1])>u&&(r=e,u=t);return r}function si(n){return n.reduce(fi,0)}function fi(n,t){return n+t[1]}function hi(n,t){return gi(n,Math.ceil(Math.log(t.length)/Math.LN2+1))}function gi(n,t){for(var e=-1,r=+n[0],u=(n[1]-r)/t,i=[];++e<=t;)i[e]=u*e+r;return i}function pi(n){return[ta.min(n),ta.max(n)]}function vi(n,t){return n.value-t.value}function di(n,t){var e=n._pack_next;n._pack_next=t,t._pack_prev=n,t._pack_next=e,e._pack_prev=t}function mi(n,t){n._pack_next=t,t._pack_prev=n}function yi(n,t){var e=t.x-n.x,r=t.y-n.y,u=n.r+t.r;return.999*u*u>e*e+r*r}function Mi(n){function t(n){s=Math.min(n.x-n.r,s),f=Math.max(n.x+n.r,f),h=Math.min(n.y-n.r,h),g=Math.max(n.y+n.r,g)}if((e=n.children)&&(l=e.length)){var e,r,u,i,o,a,c,l,s=1/0,f=-1/0,h=1/0,g=-1/0;if(e.forEach(xi),r=e[0],r.x=-r.r,r.y=0,t(r),l>1&&(u=e[1],u.x=u.r,u.y=0,t(u),l>2))for(i=e[2],wi(r,u,i),t(i),di(r,i),r._pack_prev=i,di(i,u),u=r._pack_next,o=3;l>o;o++){wi(r,u,i=e[o]);var p=0,v=1,d=1;for(a=u._pack_next;a!==u;a=a._pack_next,v++)if(yi(a,i)){p=1;break}if(1==p)for(c=r._pack_prev;c!==a._pack_prev&&!yi(c,i);c=c._pack_prev,d++);p?(d>v||v==d&&u.ro;o++)i=e[o],i.x-=m,i.y-=y,M=Math.max(M,i.r+Math.sqrt(i.x*i.x+i.y*i.y));n.r=M,e.forEach(bi)}}function xi(n){n._pack_next=n._pack_prev=n}function bi(n){delete n._pack_next,delete n._pack_prev}function _i(n,t,e,r){var u=n.children;if(n.x=t+=r*n.x,n.y=e+=r*n.y,n.r*=r,u)for(var i=-1,o=u.length;++i=0;)t=u[i],t.z+=e,t.m+=e,e+=t.s+(r+=t.c)}function Ci(n,t,e){return n.a.parent===t.parent?n.a:e}function zi(n){return 1+ta.max(n,function(n){return n.y})}function qi(n){return n.reduce(function(n,t){return n+t.x},0)/n.length}function Li(n){var t=n.children;return t&&t.length?Li(t[0]):n}function Ti(n){var t,e=n.children;return e&&(t=e.length)?Ti(e[t-1]):n}function Ri(n){return{x:n.x,y:n.y,dx:n.dx,dy:n.dy}}function Di(n,t){var e=n.x+t[3],r=n.y+t[0],u=n.dx-t[1]-t[3],i=n.dy-t[0]-t[2];return 0>u&&(e+=u/2,u=0),0>i&&(r+=i/2,i=0),{x:e,y:r,dx:u,dy:i}}function Pi(n){var t=n[0],e=n[n.length-1];return e>t?[t,e]:[e,t]}function Ui(n){return n.rangeExtent?n.rangeExtent():Pi(n.range())}function ji(n,t,e,r){var u=e(n[0],n[1]),i=r(t[0],t[1]);return function(n){return i(u(n))}}function Fi(n,t){var e,r=0,u=n.length-1,i=n[r],o=n[u];return i>o&&(e=r,r=u,u=e,e=i,i=o,o=e),n[r]=t.floor(i),n[u]=t.ceil(o),n}function Hi(n){return n?{floor:function(t){return Math.floor(t/n)*n},ceil:function(t){return Math.ceil(t/n)*n}}:ml}function Oi(n,t,e,r){var u=[],i=[],o=0,a=Math.min(n.length,t.length)-1;for(n[a]2?Oi:ji,c=r?Iu:Ou;return o=u(n,t,c,e),a=u(t,n,c,mu),i}function i(n){return o(n)}var o,a;return i.invert=function(n){return a(n)},i.domain=function(t){return arguments.length?(n=t.map(Number),u()):n},i.range=function(n){return arguments.length?(t=n,u()):t},i.rangeRound=function(n){return i.range(n).interpolate(Du)},i.clamp=function(n){return arguments.length?(r=n,u()):r},i.interpolate=function(n){return arguments.length?(e=n,u()):e},i.ticks=function(t){return Xi(n,t)},i.tickFormat=function(t,e){return $i(n,t,e)},i.nice=function(t){return Zi(n,t),u()},i.copy=function(){return Ii(n,t,e,r)},u()}function Yi(n,t){return ta.rebind(n,t,"range","rangeRound","interpolate","clamp")}function Zi(n,t){return Fi(n,Hi(Vi(n,t)[2]))}function Vi(n,t){null==t&&(t=10);var e=Pi(n),r=e[1]-e[0],u=Math.pow(10,Math.floor(Math.log(r/t)/Math.LN10)),i=t/r*u;return.15>=i?u*=10:.35>=i?u*=5:.75>=i&&(u*=2),e[0]=Math.ceil(e[0]/u)*u,e[1]=Math.floor(e[1]/u)*u+.5*u,e[2]=u,e}function Xi(n,t){return ta.range.apply(ta,Vi(n,t))}function $i(n,t,e){var r=Vi(n,t);if(e){var u=ic.exec(e);if(u.shift(),"s"===u[8]){var i=ta.formatPrefix(Math.max(ga(r[0]),ga(r[1])));return u[7]||(u[7]="."+Bi(i.scale(r[2]))),u[8]="f",e=ta.format(u.join("")),function(n){return e(i.scale(n))+i.symbol}}u[7]||(u[7]="."+Wi(u[8],r)),e=u.join("")}else e=",."+Bi(r[2])+"f";return ta.format(e)}function Bi(n){return-Math.floor(Math.log(n)/Math.LN10+.01)}function Wi(n,t){var e=Bi(t[2]);return n in yl?Math.abs(e-Bi(Math.max(ga(t[0]),ga(t[1]))))+ +("e"!==n):e-2*("%"===n)}function Ji(n,t,e,r){function u(n){return(e?Math.log(0>n?0:n):-Math.log(n>0?0:-n))/Math.log(t)}function i(n){return e?Math.pow(t,n):-Math.pow(t,-n)}function o(t){return n(u(t))}return o.invert=function(t){return i(n.invert(t))},o.domain=function(t){return arguments.length?(e=t[0]>=0,n.domain((r=t.map(Number)).map(u)),o):r},o.base=function(e){return arguments.length?(t=+e,n.domain(r.map(u)),o):t},o.nice=function(){var t=Fi(r.map(u),e?Math:xl);return n.domain(t),r=t.map(i),o},o.ticks=function(){var n=Pi(r),o=[],a=n[0],c=n[1],l=Math.floor(u(a)),s=Math.ceil(u(c)),f=t%1?2:t;if(isFinite(s-l)){if(e){for(;s>l;l++)for(var h=1;f>h;h++)o.push(i(l)*h);o.push(i(l))}else for(o.push(i(l));l++0;h--)o.push(i(l)*h);for(l=0;o[l]c;s--);o=o.slice(l,s)}return o},o.tickFormat=function(n,t){if(!arguments.length)return Ml;arguments.length<2?t=Ml:"function"!=typeof t&&(t=ta.format(t));var r,a=Math.max(.1,n/o.ticks().length),c=e?(r=1e-12,Math.ceil):(r=-1e-12,Math.floor);return function(n){return n/i(c(u(n)+r))<=a?t(n):""}},o.copy=function(){return Ji(n.copy(),t,e,r)},Yi(o,n)}function Gi(n,t,e){function r(t){return n(u(t))}var u=Ki(t),i=Ki(1/t);return r.invert=function(t){return i(n.invert(t))},r.domain=function(t){return arguments.length?(n.domain((e=t.map(Number)).map(u)),r):e},r.ticks=function(n){return Xi(e,n)},r.tickFormat=function(n,t){return $i(e,n,t)},r.nice=function(n){return r.domain(Zi(e,n))},r.exponent=function(o){return arguments.length?(u=Ki(t=o),i=Ki(1/t),n.domain(e.map(u)),r):t},r.copy=function(){return Gi(n.copy(),t,e)},Yi(r,n)}function Ki(n){return function(t){return 0>t?-Math.pow(-t,n):Math.pow(t,n)}}function Qi(n,t){function e(e){return i[((u.get(e)||("range"===t.t?u.set(e,n.push(e)):0/0))-1)%i.length]}function r(t,e){return ta.range(n.length).map(function(n){return t+e*n})}var u,i,o;return e.domain=function(r){if(!arguments.length)return n;n=[],u=new l;for(var i,o=-1,a=r.length;++oe?[0/0,0/0]:[e>0?a[e-1]:n[0],et?0/0:t/i+n,[t,t+1/i]},r.copy=function(){return to(n,t,e)},u()}function eo(n,t){function e(e){return e>=e?t[ta.bisect(n,e)]:void 0}return e.domain=function(t){return arguments.length?(n=t,e):n},e.range=function(n){return arguments.length?(t=n,e):t},e.invertExtent=function(e){return e=t.indexOf(e),[n[e-1],n[e]]},e.copy=function(){return eo(n,t)},e}function ro(n){function t(n){return+n}return t.invert=t,t.domain=t.range=function(e){return arguments.length?(n=e.map(t),t):n},t.ticks=function(t){return Xi(n,t)},t.tickFormat=function(t,e){return $i(n,t,e)},t.copy=function(){return ro(n)},t}function uo(){return 0}function io(n){return n.innerRadius}function oo(n){return n.outerRadius}function ao(n){return n.startAngle}function co(n){return n.endAngle}function lo(n){return n&&n.padAngle}function so(n,t,e,r){return(n-e)*t-(t-r)*n>0?0:1}function fo(n,t,e,r,u){var i=n[0]-t[0],o=n[1]-t[1],a=(u?r:-r)/Math.sqrt(i*i+o*o),c=a*o,l=-a*i,s=n[0]+c,f=n[1]+l,h=t[0]+c,g=t[1]+l,p=(s+h)/2,v=(f+g)/2,d=h-s,m=g-f,y=d*d+m*m,M=e-r,x=s*g-h*f,b=(0>m?-1:1)*Math.sqrt(M*M*y-x*x),_=(x*m-d*b)/y,w=(-x*d-m*b)/y,S=(x*m+d*b)/y,k=(-x*d+m*b)/y,E=_-p,A=w-v,N=S-p,C=k-v;return E*E+A*A>N*N+C*C&&(_=S,w=k),[[_-c,w-l],[_*e/M,w*e/M]]}function ho(n){function t(t){function o(){l.push("M",i(n(s),a))}for(var c,l=[],s=[],f=-1,h=t.length,g=Et(e),p=Et(r);++f1&&u.push("H",r[0]),u.join("")}function mo(n){for(var t=0,e=n.length,r=n[0],u=[r[0],",",r[1]];++t1){a=t[1],i=n[c],c++,r+="C"+(u[0]+o[0])+","+(u[1]+o[1])+","+(i[0]-a[0])+","+(i[1]-a[1])+","+i[0]+","+i[1];for(var l=2;l9&&(u=3*t/Math.sqrt(u),o[a]=u*e,o[a+1]=u*r));for(a=-1;++a<=c;)u=(n[Math.min(c,a+1)][0]-n[Math.max(0,a-1)][0])/(6*(1+o[a]*o[a])),i.push([u||0,o[a]*u||0]);return i}function To(n){return n.length<3?go(n):n[0]+_o(n,Lo(n))}function Ro(n){for(var t,e,r,u=-1,i=n.length;++ur)return s();var u=i[i.active];u&&(--i.count,delete i[i.active],u.event&&u.event.interrupt.call(n,n.__data__,u.index)),i.active=r,o.event&&o.event.start.call(n,n.__data__,t),o.tween.forEach(function(e,r){(r=r.call(n,n.__data__,t))&&v.push(r)}),h=o.ease,f=o.duration,ta.timer(function(){return p.c=l(e||1)?Ne:l,1},0,a)}function l(e){if(i.active!==r)return 1;for(var u=e/f,a=h(u),c=v.length;c>0;)v[--c].call(n,a);return u>=1?(o.event&&o.event.end.call(n,n.__data__,t),s()):void 0}function s(){return--i.count?delete i[r]:delete n[e],1}var f,h,g=o.delay,p=ec,v=[];return p.t=g+a,u>=g?c(u-g):void(p.c=c)},0,a)}}function Bo(n,t,e){n.attr("transform",function(n){var r=t(n);return"translate("+(isFinite(r)?r:e(n))+",0)"})}function Wo(n,t,e){n.attr("transform",function(n){var r=t(n);return"translate(0,"+(isFinite(r)?r:e(n))+")"})}function Jo(n){return n.toISOString()}function Go(n,t,e){function r(t){return n(t)}function u(n,e){var r=n[1]-n[0],u=r/e,i=ta.bisect(Vl,u);return i==Vl.length?[t.year,Vi(n.map(function(n){return n/31536e6}),e)[2]]:i?t[u/Vl[i-1]1?{floor:function(t){for(;e(t=n.floor(t));)t=Ko(t-1);return t},ceil:function(t){for(;e(t=n.ceil(t));)t=Ko(+t+1);return t}}:n))},r.ticks=function(n,t){var e=Pi(r.domain()),i=null==n?u(e,10):"number"==typeof n?u(e,n):!n.range&&[{range:n},t];return i&&(n=i[0],t=i[1]),n.range(e[0],Ko(+e[1]+1),1>t?1:t)},r.tickFormat=function(){return e},r.copy=function(){return Go(n.copy(),t,e)},Yi(r,n)}function Ko(n){return new Date(n)}function Qo(n){return JSON.parse(n.responseText)}function na(n){var t=ua.createRange();return t.selectNode(ua.body),t.createContextualFragment(n.responseText)}var ta={version:"3.5.5"},ea=[].slice,ra=function(n){return ea.call(n)},ua=this.document;if(ua)try{ra(ua.documentElement.childNodes)[0].nodeType}catch(ia){ra=function(n){for(var t=n.length,e=new Array(t);t--;)e[t]=n[t];return e}}if(Date.now||(Date.now=function(){return+new Date}),ua)try{ua.createElement("DIV").style.setProperty("opacity",0,"")}catch(oa){var aa=this.Element.prototype,ca=aa.setAttribute,la=aa.setAttributeNS,sa=this.CSSStyleDeclaration.prototype,fa=sa.setProperty;aa.setAttribute=function(n,t){ca.call(this,n,t+"")},aa.setAttributeNS=function(n,t,e){la.call(this,n,t,e+"")},sa.setProperty=function(n,t,e){fa.call(this,n,t+"",e)}}ta.ascending=e,ta.descending=function(n,t){return n>t?-1:t>n?1:t>=n?0:0/0},ta.min=function(n,t){var e,r,u=-1,i=n.length;if(1===arguments.length){for(;++u=r){e=r;break}for(;++ur&&(e=r)}else{for(;++u=r){e=r;break}for(;++ur&&(e=r)}return e},ta.max=function(n,t){var e,r,u=-1,i=n.length;if(1===arguments.length){for(;++u=r){e=r;break}for(;++ue&&(e=r)}else{for(;++u=r){e=r;break}for(;++ue&&(e=r)}return e},ta.extent=function(n,t){var e,r,u,i=-1,o=n.length;if(1===arguments.length){for(;++i=r){e=u=r;break}for(;++ir&&(e=r),r>u&&(u=r))}else{for(;++i=r){e=u=r;break}for(;++ir&&(e=r),r>u&&(u=r))}return[e,u]},ta.sum=function(n,t){var e,r=0,i=n.length,o=-1;if(1===arguments.length)for(;++o1?c/(s-1):void 0},ta.deviation=function(){var n=ta.variance.apply(this,arguments);return n?Math.sqrt(n):n};var ha=i(e);ta.bisectLeft=ha.left,ta.bisect=ta.bisectRight=ha.right,ta.bisector=function(n){return i(1===n.length?function(t,r){return e(n(t),r)}:n)},ta.shuffle=function(n,t,e){(i=arguments.length)<3&&(e=n.length,2>i&&(t=0));for(var r,u,i=e-t;i;)u=Math.random()*i--|0,r=n[i+t],n[i+t]=n[u+t],n[u+t]=r;return n},ta.permute=function(n,t){for(var e=t.length,r=new Array(e);e--;)r[e]=n[t[e]];return r},ta.pairs=function(n){for(var t,e=0,r=n.length-1,u=n[0],i=new Array(0>r?0:r);r>e;)i[e]=[t=u,u=n[++e]];return i},ta.zip=function(){if(!(r=arguments.length))return[];for(var n=-1,t=ta.min(arguments,o),e=new Array(t);++n=0;)for(r=n[u],t=r.length;--t>=0;)e[--o]=r[t];return e};var ga=Math.abs;ta.range=function(n,t,e){if(arguments.length<3&&(e=1,arguments.length<2&&(t=n,n=0)),(t-n)/e===1/0)throw new Error("infinite range");var r,u=[],i=a(ga(e)),o=-1;if(n*=i,t*=i,e*=i,0>e)for(;(r=n+e*++o)>t;)u.push(r/i);else for(;(r=n+e*++o)=i.length)return r?r.call(u,o):e?o.sort(e):o;for(var c,s,f,h,g=-1,p=o.length,v=i[a++],d=new l;++g=i.length)return n;var r=[],u=o[e++];return n.forEach(function(n,u){r.push({key:n,values:t(u,e)})}),u?r.sort(function(n,t){return u(n.key,t.key)}):r}var e,r,u={},i=[],o=[];return u.map=function(t,e){return n(e,t,0)},u.entries=function(e){return t(n(ta.map,e,0),0)},u.key=function(n){return i.push(n),u},u.sortKeys=function(n){return o[i.length-1]=n,u},u.sortValues=function(n){return e=n,u},u.rollup=function(n){return r=n,u},u},ta.set=function(n){var t=new m;if(n)for(var e=0,r=n.length;r>e;++e)t.add(n[e]);return t},c(m,{has:h,add:function(n){return this._[s(n+="")]=!0,n},remove:g,values:p,size:v,empty:d,forEach:function(n){for(var t in this._)n.call(this,f(t))}}),ta.behavior={},ta.rebind=function(n,t){for(var e,r=1,u=arguments.length;++r=0&&(r=n.slice(e+1),n=n.slice(0,e)),n)return arguments.length<2?this[n].on(r):this[n].on(r,t);if(2===arguments.length){if(null==t)for(n in this)this.hasOwnProperty(n)&&this[n].on(r,null);return this}},ta.event=null,ta.requote=function(n){return n.replace(ma,"\\$&")};var ma=/[\\\^\$\*\+\?\|\[\]\(\)\.\{\}]/g,ya={}.__proto__?function(n,t){n.__proto__=t}:function(n,t){for(var e in t)n[e]=t[e]},Ma=function(n,t){return t.querySelector(n)},xa=function(n,t){return t.querySelectorAll(n)},ba=function(n,t){var e=n.matches||n[x(n,"matchesSelector")];return(ba=function(n,t){return e.call(n,t)})(n,t)};"function"==typeof Sizzle&&(Ma=function(n,t){return Sizzle(n,t)[0]||null},xa=Sizzle,ba=Sizzle.matchesSelector),ta.selection=function(){return ta.select(ua.documentElement)};var _a=ta.selection.prototype=[];_a.select=function(n){var t,e,r,u,i=[];n=N(n);for(var o=-1,a=this.length;++o=0&&(e=n.slice(0,t),n=n.slice(t+1)),wa.hasOwnProperty(e)?{space:wa[e],local:n}:n}},_a.attr=function(n,t){if(arguments.length<2){if("string"==typeof n){var e=this.node();return n=ta.ns.qualify(n),n.local?e.getAttributeNS(n.space,n.local):e.getAttribute(n)}for(t in n)this.each(z(t,n[t]));return this}return this.each(z(n,t))},_a.classed=function(n,t){if(arguments.length<2){if("string"==typeof n){var e=this.node(),r=(n=T(n)).length,u=-1;if(t=e.classList){for(;++uu){if("string"!=typeof n){2>u&&(e="");for(r in n)this.each(P(r,n[r],e));return this}if(2>u){var i=this.node();return t(i).getComputedStyle(i,null).getPropertyValue(n)}r=""}return this.each(P(n,e,r))},_a.property=function(n,t){if(arguments.length<2){if("string"==typeof n)return this.node()[n];for(t in n)this.each(U(t,n[t]));return this}return this.each(U(n,t))},_a.text=function(n){return arguments.length?this.each("function"==typeof n?function(){var t=n.apply(this,arguments);this.textContent=null==t?"":t}:null==n?function(){this.textContent=""}:function(){this.textContent=n}):this.node().textContent},_a.html=function(n){return arguments.length?this.each("function"==typeof n?function(){var t=n.apply(this,arguments);this.innerHTML=null==t?"":t}:null==n?function(){this.innerHTML=""}:function(){this.innerHTML=n}):this.node().innerHTML},_a.append=function(n){return n=j(n),this.select(function(){return this.appendChild(n.apply(this,arguments))})},_a.insert=function(n,t){return n=j(n),t=N(t),this.select(function(){return this.insertBefore(n.apply(this,arguments),t.apply(this,arguments)||null)})},_a.remove=function(){return this.each(F)},_a.data=function(n,t){function e(n,e){var r,u,i,o=n.length,f=e.length,h=Math.min(o,f),g=new Array(f),p=new Array(f),v=new Array(o);if(t){var d,m=new l,y=new Array(o);for(r=-1;++rr;++r)p[r]=H(e[r]);for(;o>r;++r)v[r]=n[r]}p.update=g,p.parentNode=g.parentNode=v.parentNode=n.parentNode,a.push(p),c.push(g),s.push(v)}var r,u,i=-1,o=this.length;if(!arguments.length){for(n=new Array(o=(r=this[0]).length);++ii;i++){u.push(t=[]),t.parentNode=(e=this[i]).parentNode;for(var a=0,c=e.length;c>a;a++)(r=e[a])&&n.call(r,r.__data__,a,i)&&t.push(r)}return A(u)},_a.order=function(){for(var n=-1,t=this.length;++n=0;)(e=r[u])&&(i&&i!==e.nextSibling&&i.parentNode.insertBefore(e,i),i=e);return this},_a.sort=function(n){n=I.apply(this,arguments);for(var t=-1,e=this.length;++tn;n++)for(var e=this[n],r=0,u=e.length;u>r;r++){var i=e[r];if(i)return i}return null},_a.size=function(){var n=0;return Y(this,function(){++n}),n};var Sa=[];ta.selection.enter=Z,ta.selection.enter.prototype=Sa,Sa.append=_a.append,Sa.empty=_a.empty,Sa.node=_a.node,Sa.call=_a.call,Sa.size=_a.size,Sa.select=function(n){for(var t,e,r,u,i,o=[],a=-1,c=this.length;++ar){if("string"!=typeof n){2>r&&(t=!1);for(e in n)this.each(X(e,n[e],t));return this}if(2>r)return(r=this.node()["__on"+n])&&r._;e=!1}return this.each(X(n,t,e))};var ka=ta.map({mouseenter:"mouseover",mouseleave:"mouseout"});ua&&ka.forEach(function(n){"on"+n in ua&&ka.remove(n)});var Ea,Aa=0;ta.mouse=function(n){return J(n,k())};var Na=this.navigator&&/WebKit/.test(this.navigator.userAgent)?-1:0;ta.touch=function(n,t,e){if(arguments.length<3&&(e=t,t=k().changedTouches),t)for(var r,u=0,i=t.length;i>u;++u)if((r=t[u]).identifier===e)return J(n,r)},ta.behavior.drag=function(){function n(){this.on("mousedown.drag",i).on("touchstart.drag",o)}function e(n,t,e,i,o){return function(){function a(){var n,e,r=t(h,v);r&&(n=r[0]-M[0],e=r[1]-M[1],p|=n|e,M=r,g({type:"drag",x:r[0]+l[0],y:r[1]+l[1],dx:n,dy:e}))}function c(){t(h,v)&&(m.on(i+d,null).on(o+d,null),y(p&&ta.event.target===f),g({type:"dragend"}))}var l,s=this,f=ta.event.target,h=s.parentNode,g=r.of(s,arguments),p=0,v=n(),d=".drag"+(null==v?"":"-"+v),m=ta.select(e(f)).on(i+d,a).on(o+d,c),y=W(f),M=t(h,v);u?(l=u.apply(s,arguments),l=[l.x-M[0],l.y-M[1]]):l=[0,0],g({type:"dragstart"})}}var r=E(n,"drag","dragstart","dragend"),u=null,i=e(b,ta.mouse,t,"mousemove","mouseup"),o=e(G,ta.touch,y,"touchmove","touchend");return n.origin=function(t){return arguments.length?(u=t,n):u},ta.rebind(n,r,"on")},ta.touches=function(n,t){return arguments.length<2&&(t=k().touches),t?ra(t).map(function(t){var e=J(n,t);return e.identifier=t.identifier,e}):[]};var Ca=1e-6,za=Ca*Ca,qa=Math.PI,La=2*qa,Ta=La-Ca,Ra=qa/2,Da=qa/180,Pa=180/qa,Ua=Math.SQRT2,ja=2,Fa=4;ta.interpolateZoom=function(n,t){function e(n){var t=n*y;if(m){var e=rt(v),o=i/(ja*h)*(e*ut(Ua*t+v)-et(v));return[r+o*l,u+o*s,i*e/rt(Ua*t+v)]}return[r+n*l,u+n*s,i*Math.exp(Ua*t)]}var r=n[0],u=n[1],i=n[2],o=t[0],a=t[1],c=t[2],l=o-r,s=a-u,f=l*l+s*s,h=Math.sqrt(f),g=(c*c-i*i+Fa*f)/(2*i*ja*h),p=(c*c-i*i-Fa*f)/(2*c*ja*h),v=Math.log(Math.sqrt(g*g+1)-g),d=Math.log(Math.sqrt(p*p+1)-p),m=d-v,y=(m||Math.log(c/i))/Ua;return e.duration=1e3*y,e},ta.behavior.zoom=function(){function n(n){n.on(q,f).on(Oa+".zoom",g).on("dblclick.zoom",p).on(R,h)}function e(n){return[(n[0]-k.x)/k.k,(n[1]-k.y)/k.k]}function r(n){return[n[0]*k.k+k.x,n[1]*k.k+k.y]}function u(n){k.k=Math.max(N[0],Math.min(N[1],n))}function i(n,t){t=r(t),k.x+=n[0]-t[0],k.y+=n[1]-t[1]}function o(t,e,r,o){t.__chart__={x:k.x,y:k.y,k:k.k},u(Math.pow(2,o)),i(d=e,r),t=ta.select(t),C>0&&(t=t.transition().duration(C)),t.call(n.event)}function a(){b&&b.domain(x.range().map(function(n){return(n-k.x)/k.k}).map(x.invert)),w&&w.domain(_.range().map(function(n){return(n-k.y)/k.k}).map(_.invert))}function c(n){z++||n({type:"zoomstart"})}function l(n){a(),n({type:"zoom",scale:k.k,translate:[k.x,k.y]})}function s(n){--z||n({type:"zoomend"}),d=null}function f(){function n(){f=1,i(ta.mouse(u),g),l(a)}function r(){h.on(L,null).on(T,null),p(f&&ta.event.target===o),s(a)}var u=this,o=ta.event.target,a=D.of(u,arguments),f=0,h=ta.select(t(u)).on(L,n).on(T,r),g=e(ta.mouse(u)),p=W(u);Dl.call(u),c(a)}function h(){function n(){var n=ta.touches(p);return g=k.k,n.forEach(function(n){n.identifier in d&&(d[n.identifier]=e(n))}),n}function t(){var t=ta.event.target;ta.select(t).on(x,r).on(b,a),_.push(t);for(var e=ta.event.changedTouches,u=0,i=e.length;i>u;++u)d[e[u].identifier]=null;var c=n(),l=Date.now();if(1===c.length){if(500>l-M){var s=c[0];o(p,s,d[s.identifier],Math.floor(Math.log(k.k)/Math.LN2)+1),S()}M=l}else if(c.length>1){var s=c[0],f=c[1],h=s[0]-f[0],g=s[1]-f[1];m=h*h+g*g}}function r(){var n,t,e,r,o=ta.touches(p);Dl.call(p);for(var a=0,c=o.length;c>a;++a,r=null)if(e=o[a],r=d[e.identifier]){if(t)break;n=e,t=r}if(r){var s=(s=e[0]-n[0])*s+(s=e[1]-n[1])*s,f=m&&Math.sqrt(s/m);n=[(n[0]+e[0])/2,(n[1]+e[1])/2],t=[(t[0]+r[0])/2,(t[1]+r[1])/2],u(f*g)}M=null,i(n,t),l(v)}function a(){if(ta.event.touches.length){for(var t=ta.event.changedTouches,e=0,r=t.length;r>e;++e)delete d[t[e].identifier];for(var u in d)return void n()}ta.selectAll(_).on(y,null),w.on(q,f).on(R,h),E(),s(v)}var g,p=this,v=D.of(p,arguments),d={},m=0,y=".zoom-"+ta.event.changedTouches[0].identifier,x="touchmove"+y,b="touchend"+y,_=[],w=ta.select(p),E=W(p);t(),c(v),w.on(q,null).on(R,t)}function g(){var n=D.of(this,arguments);y?clearTimeout(y):(v=e(d=m||ta.mouse(this)),Dl.call(this),c(n)),y=setTimeout(function(){y=null,s(n)},50),S(),u(Math.pow(2,.002*Ha())*k.k),i(d,v),l(n)}function p(){var n=ta.mouse(this),t=Math.log(k.k)/Math.LN2;o(this,n,e(n),ta.event.shiftKey?Math.ceil(t)-1:Math.floor(t)+1)}var v,d,m,y,M,x,b,_,w,k={x:0,y:0,k:1},A=[960,500],N=Ia,C=250,z=0,q="mousedown.zoom",L="mousemove.zoom",T="mouseup.zoom",R="touchstart.zoom",D=E(n,"zoomstart","zoom","zoomend");return Oa||(Oa="onwheel"in ua?(Ha=function(){return-ta.event.deltaY*(ta.event.deltaMode?120:1)},"wheel"):"onmousewheel"in ua?(Ha=function(){return ta.event.wheelDelta},"mousewheel"):(Ha=function(){return-ta.event.detail},"MozMousePixelScroll")),n.event=function(n){n.each(function(){var n=D.of(this,arguments),t=k;Tl?ta.select(this).transition().each("start.zoom",function(){k=this.__chart__||{x:0,y:0,k:1},c(n)}).tween("zoom:zoom",function(){var e=A[0],r=A[1],u=d?d[0]:e/2,i=d?d[1]:r/2,o=ta.interpolateZoom([(u-k.x)/k.k,(i-k.y)/k.k,e/k.k],[(u-t.x)/t.k,(i-t.y)/t.k,e/t.k]);return function(t){var r=o(t),a=e/r[2];this.__chart__=k={x:u-r[0]*a,y:i-r[1]*a,k:a},l(n)}}).each("interrupt.zoom",function(){s(n)}).each("end.zoom",function(){s(n)}):(this.__chart__=k,c(n),l(n),s(n))})},n.translate=function(t){return arguments.length?(k={x:+t[0],y:+t[1],k:k.k},a(),n):[k.x,k.y]},n.scale=function(t){return arguments.length?(k={x:k.x,y:k.y,k:+t},a(),n):k.k},n.scaleExtent=function(t){return arguments.length?(N=null==t?Ia:[+t[0],+t[1]],n):N},n.center=function(t){return arguments.length?(m=t&&[+t[0],+t[1]],n):m},n.size=function(t){return arguments.length?(A=t&&[+t[0],+t[1]],n):A},n.duration=function(t){return arguments.length?(C=+t,n):C},n.x=function(t){return arguments.length?(b=t,x=t.copy(),k={x:0,y:0,k:1},n):b},n.y=function(t){return arguments.length?(w=t,_=t.copy(),k={x:0,y:0,k:1},n):w},ta.rebind(n,D,"on")};var Ha,Oa,Ia=[0,1/0];ta.color=ot,ot.prototype.toString=function(){return this.rgb()+""},ta.hsl=at;var Ya=at.prototype=new ot;Ya.brighter=function(n){return n=Math.pow(.7,arguments.length?n:1),new at(this.h,this.s,this.l/n)},Ya.darker=function(n){return n=Math.pow(.7,arguments.length?n:1),new at(this.h,this.s,n*this.l)},Ya.rgb=function(){return ct(this.h,this.s,this.l)},ta.hcl=lt;var Za=lt.prototype=new ot;Za.brighter=function(n){return new lt(this.h,this.c,Math.min(100,this.l+Va*(arguments.length?n:1)))},Za.darker=function(n){return new lt(this.h,this.c,Math.max(0,this.l-Va*(arguments.length?n:1)))},Za.rgb=function(){return st(this.h,this.c,this.l).rgb()},ta.lab=ft;var Va=18,Xa=.95047,$a=1,Ba=1.08883,Wa=ft.prototype=new ot;Wa.brighter=function(n){return new ft(Math.min(100,this.l+Va*(arguments.length?n:1)),this.a,this.b)},Wa.darker=function(n){return new ft(Math.max(0,this.l-Va*(arguments.length?n:1)),this.a,this.b)},Wa.rgb=function(){return ht(this.l,this.a,this.b)},ta.rgb=mt;var Ja=mt.prototype=new ot;Ja.brighter=function(n){n=Math.pow(.7,arguments.length?n:1);var t=this.r,e=this.g,r=this.b,u=30;return t||e||r?(t&&u>t&&(t=u),e&&u>e&&(e=u),r&&u>r&&(r=u),new mt(Math.min(255,t/n),Math.min(255,e/n),Math.min(255,r/n))):new mt(u,u,u)},Ja.darker=function(n){return n=Math.pow(.7,arguments.length?n:1),new mt(n*this.r,n*this.g,n*this.b)},Ja.hsl=function(){return _t(this.r,this.g,this.b)},Ja.toString=function(){return"#"+xt(this.r)+xt(this.g)+xt(this.b)};var Ga=ta.map({aliceblue:15792383,antiquewhite:16444375,aqua:65535,aquamarine:8388564,azure:15794175,beige:16119260,bisque:16770244,black:0,blanchedalmond:16772045,blue:255,blueviolet:9055202,brown:10824234,burlywood:14596231,cadetblue:6266528,chartreuse:8388352,chocolate:13789470,coral:16744272,cornflowerblue:6591981,cornsilk:16775388,crimson:14423100,cyan:65535,darkblue:139,darkcyan:35723,darkgoldenrod:12092939,darkgray:11119017,darkgreen:25600,darkgrey:11119017,darkkhaki:12433259,darkmagenta:9109643,darkolivegreen:5597999,darkorange:16747520,darkorchid:10040012,darkred:9109504,darksalmon:15308410,darkseagreen:9419919,darkslateblue:4734347,darkslategray:3100495,darkslategrey:3100495,darkturquoise:52945,darkviolet:9699539,deeppink:16716947,deepskyblue:49151,dimgray:6908265,dimgrey:6908265,dodgerblue:2003199,firebrick:11674146,floralwhite:16775920,forestgreen:2263842,fuchsia:16711935,gainsboro:14474460,ghostwhite:16316671,gold:16766720,goldenrod:14329120,gray:8421504,green:32768,greenyellow:11403055,grey:8421504,honeydew:15794160,hotpink:16738740,indianred:13458524,indigo:4915330,ivory:16777200,khaki:15787660,lavender:15132410,lavenderblush:16773365,lawngreen:8190976,lemonchiffon:16775885,lightblue:11393254,lightcoral:15761536,lightcyan:14745599,lightgoldenrodyellow:16448210,lightgray:13882323,lightgreen:9498256,lightgrey:13882323,lightpink:16758465,lightsalmon:16752762,lightseagreen:2142890,lightskyblue:8900346,lightslategray:7833753,lightslategrey:7833753,lightsteelblue:11584734,lightyellow:16777184,lime:65280,limegreen:3329330,linen:16445670,magenta:16711935,maroon:8388608,mediumaquamarine:6737322,mediumblue:205,mediumorchid:12211667,mediumpurple:9662683,mediumseagreen:3978097,mediumslateblue:8087790,mediumspringgreen:64154,mediumturquoise:4772300,mediumvioletred:13047173,midnightblue:1644912,mintcream:16121850,mistyrose:16770273,moccasin:16770229,navajowhite:16768685,navy:128,oldlace:16643558,olive:8421376,olivedrab:7048739,orange:16753920,orangered:16729344,orchid:14315734,palegoldenrod:15657130,palegreen:10025880,paleturquoise:11529966,palevioletred:14381203,papayawhip:16773077,peachpuff:16767673,peru:13468991,pink:16761035,plum:14524637,powderblue:11591910,purple:8388736,rebeccapurple:6697881,red:16711680,rosybrown:12357519,royalblue:4286945,saddlebrown:9127187,salmon:16416882,sandybrown:16032864,seagreen:3050327,seashell:16774638,sienna:10506797,silver:12632256,skyblue:8900331,slateblue:6970061,slategray:7372944,slategrey:7372944,snow:16775930,springgreen:65407,steelblue:4620980,tan:13808780,teal:32896,thistle:14204888,tomato:16737095,turquoise:4251856,violet:15631086,wheat:16113331,white:16777215,whitesmoke:16119285,yellow:16776960,yellowgreen:10145074});Ga.forEach(function(n,t){Ga.set(n,yt(t))}),ta.functor=Et,ta.xhr=At(y),ta.dsv=function(n,t){function e(n,e,i){arguments.length<3&&(i=e,e=null);var o=Nt(n,t,null==e?r:u(e),i);return o.row=function(n){return arguments.length?o.response(null==(e=n)?r:u(n)):e},o}function r(n){return e.parse(n.responseText)}function u(n){return function(t){return e.parse(t.responseText,n)}}function i(t){return t.map(o).join(n)}function o(n){return a.test(n)?'"'+n.replace(/\"/g,'""')+'"':n}var a=new RegExp('["'+n+"\n]"),c=n.charCodeAt(0);return e.parse=function(n,t){var r;return e.parseRows(n,function(n,e){if(r)return r(n,e-1);var u=new Function("d","return {"+n.map(function(n,t){return JSON.stringify(n)+": d["+t+"]"}).join(",")+"}");r=t?function(n,e){return t(u(n),e)}:u})},e.parseRows=function(n,t){function e(){if(s>=l)return o;if(u)return u=!1,i;var t=s;if(34===n.charCodeAt(t)){for(var e=t;e++s;){var r=n.charCodeAt(s++),a=1;if(10===r)u=!0;else if(13===r)u=!0,10===n.charCodeAt(s)&&(++s,++a);else if(r!==c)continue;return n.slice(t,s-a)}return n.slice(t)}for(var r,u,i={},o={},a=[],l=n.length,s=0,f=0;(r=e())!==o;){for(var h=[];r!==i&&r!==o;)h.push(r),r=e();t&&null==(h=t(h,f++))||a.push(h)}return a},e.format=function(t){if(Array.isArray(t[0]))return e.formatRows(t);var r=new m,u=[];return t.forEach(function(n){for(var t in n)r.has(t)||u.push(r.add(t))}),[u.map(o).join(n)].concat(t.map(function(t){return u.map(function(n){return o(t[n])}).join(n)})).join("\n")},e.formatRows=function(n){return n.map(i).join("\n")},e},ta.csv=ta.dsv(",","text/csv"),ta.tsv=ta.dsv(" ","text/tab-separated-values");var Ka,Qa,nc,tc,ec,rc=this[x(this,"requestAnimationFrame")]||function(n){setTimeout(n,17)};ta.timer=function(n,t,e){var r=arguments.length;2>r&&(t=0),3>r&&(e=Date.now());var u=e+t,i={c:n,t:u,f:!1,n:null};Qa?Qa.n=i:Ka=i,Qa=i,nc||(tc=clearTimeout(tc),nc=1,rc(qt))},ta.timer.flush=function(){Lt(),Tt()},ta.round=function(n,t){return t?Math.round(n*(t=Math.pow(10,t)))/t:Math.round(n)};var uc=["y","z","a","f","p","n","\xb5","m","","k","M","G","T","P","E","Z","Y"].map(Dt);ta.formatPrefix=function(n,t){var e=0;return n&&(0>n&&(n*=-1),t&&(n=ta.round(n,Rt(n,t))),e=1+Math.floor(1e-12+Math.log(n)/Math.LN10),e=Math.max(-24,Math.min(24,3*Math.floor((e-1)/3)))),uc[8+e/3]};var ic=/(?:([^{])?([<>=^]))?([+\- ])?([$#])?(0)?(\d+)?(,)?(\.-?\d+)?([a-z%])?/i,oc=ta.map({b:function(n){return n.toString(2)},c:function(n){return String.fromCharCode(n)},o:function(n){return n.toString(8)},x:function(n){return n.toString(16)},X:function(n){return n.toString(16).toUpperCase()},g:function(n,t){return n.toPrecision(t)},e:function(n,t){return n.toExponential(t)},f:function(n,t){return n.toFixed(t)},r:function(n,t){return(n=ta.round(n,Rt(n,t))).toFixed(Math.max(0,Math.min(20,Rt(n*(1+1e-15),t))))}}),ac=ta.time={},cc=Date;jt.prototype={getDate:function(){return this._.getUTCDate()},getDay:function(){return this._.getUTCDay()},getFullYear:function(){return this._.getUTCFullYear()},getHours:function(){return this._.getUTCHours()},getMilliseconds:function(){return this._.getUTCMilliseconds()},getMinutes:function(){return this._.getUTCMinutes()},getMonth:function(){return this._.getUTCMonth()},getSeconds:function(){return this._.getUTCSeconds()},getTime:function(){return this._.getTime()},getTimezoneOffset:function(){return 0},valueOf:function(){return this._.valueOf()},setDate:function(){lc.setUTCDate.apply(this._,arguments)},setDay:function(){lc.setUTCDay.apply(this._,arguments)},setFullYear:function(){lc.setUTCFullYear.apply(this._,arguments)},setHours:function(){lc.setUTCHours.apply(this._,arguments)},setMilliseconds:function(){lc.setUTCMilliseconds.apply(this._,arguments)},setMinutes:function(){lc.setUTCMinutes.apply(this._,arguments)},setMonth:function(){lc.setUTCMonth.apply(this._,arguments)},setSeconds:function(){lc.setUTCSeconds.apply(this._,arguments)},setTime:function(){lc.setTime.apply(this._,arguments)}};var lc=Date.prototype;ac.year=Ft(function(n){return n=ac.day(n),n.setMonth(0,1),n},function(n,t){n.setFullYear(n.getFullYear()+t)},function(n){return n.getFullYear()}),ac.years=ac.year.range,ac.years.utc=ac.year.utc.range,ac.day=Ft(function(n){var t=new cc(2e3,0);return t.setFullYear(n.getFullYear(),n.getMonth(),n.getDate()),t},function(n,t){n.setDate(n.getDate()+t)},function(n){return n.getDate()-1}),ac.days=ac.day.range,ac.days.utc=ac.day.utc.range,ac.dayOfYear=function(n){var t=ac.year(n);return Math.floor((n-t-6e4*(n.getTimezoneOffset()-t.getTimezoneOffset()))/864e5)},["sunday","monday","tuesday","wednesday","thursday","friday","saturday"].forEach(function(n,t){t=7-t;var e=ac[n]=Ft(function(n){return(n=ac.day(n)).setDate(n.getDate()-(n.getDay()+t)%7),n},function(n,t){n.setDate(n.getDate()+7*Math.floor(t))},function(n){var e=ac.year(n).getDay();return Math.floor((ac.dayOfYear(n)+(e+t)%7)/7)-(e!==t)});ac[n+"s"]=e.range,ac[n+"s"].utc=e.utc.range,ac[n+"OfYear"]=function(n){var e=ac.year(n).getDay();return Math.floor((ac.dayOfYear(n)+(e+t)%7)/7)}}),ac.week=ac.sunday,ac.weeks=ac.sunday.range,ac.weeks.utc=ac.sunday.utc.range,ac.weekOfYear=ac.sundayOfYear;var sc={"-":"",_:" ",0:"0"},fc=/^\s*\d+/,hc=/^%/;ta.locale=function(n){return{numberFormat:Pt(n),timeFormat:Ot(n)}};var gc=ta.locale({decimal:".",thousands:",",grouping:[3],currency:["$",""],dateTime:"%a %b %e %X %Y",date:"%m/%d/%Y",time:"%H:%M:%S",periods:["AM","PM"],days:["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],shortDays:["Sun","Mon","Tue","Wed","Thu","Fri","Sat"],months:["January","February","March","April","May","June","July","August","September","October","November","December"],shortMonths:["Jan","Feb","Mar","Apr","May","Jun","Jul","Aug","Sep","Oct","Nov","Dec"]});ta.format=gc.numberFormat,ta.geo={},ce.prototype={s:0,t:0,add:function(n){le(n,this.t,pc),le(pc.s,this.s,this),this.s?this.t+=pc.t:this.s=pc.t +},reset:function(){this.s=this.t=0},valueOf:function(){return this.s}};var pc=new ce;ta.geo.stream=function(n,t){n&&vc.hasOwnProperty(n.type)?vc[n.type](n,t):se(n,t)};var vc={Feature:function(n,t){se(n.geometry,t)},FeatureCollection:function(n,t){for(var e=n.features,r=-1,u=e.length;++rn?4*qa+n:n,Mc.lineStart=Mc.lineEnd=Mc.point=b}};ta.geo.bounds=function(){function n(n,t){M.push(x=[s=n,h=n]),f>t&&(f=t),t>g&&(g=t)}function t(t,e){var r=pe([t*Da,e*Da]);if(m){var u=de(m,r),i=[u[1],-u[0],0],o=de(i,u);Me(o),o=xe(o);var c=t-p,l=c>0?1:-1,v=o[0]*Pa*l,d=ga(c)>180;if(d^(v>l*p&&l*t>v)){var y=o[1]*Pa;y>g&&(g=y)}else if(v=(v+360)%360-180,d^(v>l*p&&l*t>v)){var y=-o[1]*Pa;f>y&&(f=y)}else f>e&&(f=e),e>g&&(g=e);d?p>t?a(s,t)>a(s,h)&&(h=t):a(t,h)>a(s,h)&&(s=t):h>=s?(s>t&&(s=t),t>h&&(h=t)):t>p?a(s,t)>a(s,h)&&(h=t):a(t,h)>a(s,h)&&(s=t)}else n(t,e);m=r,p=t}function e(){b.point=t}function r(){x[0]=s,x[1]=h,b.point=n,m=null}function u(n,e){if(m){var r=n-p;y+=ga(r)>180?r+(r>0?360:-360):r}else v=n,d=e;Mc.point(n,e),t(n,e)}function i(){Mc.lineStart()}function o(){u(v,d),Mc.lineEnd(),ga(y)>Ca&&(s=-(h=180)),x[0]=s,x[1]=h,m=null}function a(n,t){return(t-=n)<0?t+360:t}function c(n,t){return n[0]-t[0]}function l(n,t){return t[0]<=t[1]?t[0]<=n&&n<=t[1]:nyc?(s=-(h=180),f=-(g=90)):y>Ca?g=90:-Ca>y&&(f=-90),x[0]=s,x[1]=h}};return function(n){g=h=-(s=f=1/0),M=[],ta.geo.stream(n,b);var t=M.length;if(t){M.sort(c);for(var e,r=1,u=M[0],i=[u];t>r;++r)e=M[r],l(e[0],u)||l(e[1],u)?(a(u[0],e[1])>a(u[0],u[1])&&(u[1]=e[1]),a(e[0],u[1])>a(u[0],u[1])&&(u[0]=e[0])):i.push(u=e);for(var o,e,p=-1/0,t=i.length-1,r=0,u=i[t];t>=r;u=e,++r)e=i[r],(o=a(u[1],e[0]))>p&&(p=o,s=e[0],h=u[1])}return M=x=null,1/0===s||1/0===f?[[0/0,0/0],[0/0,0/0]]:[[s,f],[h,g]]}}(),ta.geo.centroid=function(n){xc=bc=_c=wc=Sc=kc=Ec=Ac=Nc=Cc=zc=0,ta.geo.stream(n,qc);var t=Nc,e=Cc,r=zc,u=t*t+e*e+r*r;return za>u&&(t=kc,e=Ec,r=Ac,Ca>bc&&(t=_c,e=wc,r=Sc),u=t*t+e*e+r*r,za>u)?[0/0,0/0]:[Math.atan2(e,t)*Pa,tt(r/Math.sqrt(u))*Pa]};var xc,bc,_c,wc,Sc,kc,Ec,Ac,Nc,Cc,zc,qc={sphere:b,point:_e,lineStart:Se,lineEnd:ke,polygonStart:function(){qc.lineStart=Ee},polygonEnd:function(){qc.lineStart=Se}},Lc=Le(Ne,Pe,je,[-qa,-qa/2]),Tc=1e9;ta.geo.clipExtent=function(){var n,t,e,r,u,i,o={stream:function(n){return u&&(u.valid=!1),u=i(n),u.valid=!0,u},extent:function(a){return arguments.length?(i=Ie(n=+a[0][0],t=+a[0][1],e=+a[1][0],r=+a[1][1]),u&&(u.valid=!1,u=null),o):[[n,t],[e,r]]}};return o.extent([[0,0],[960,500]])},(ta.geo.conicEqualArea=function(){return Ye(Ze)}).raw=Ze,ta.geo.albers=function(){return ta.geo.conicEqualArea().rotate([96,0]).center([-.6,38.7]).parallels([29.5,45.5]).scale(1070)},ta.geo.albersUsa=function(){function n(n){var i=n[0],o=n[1];return t=null,e(i,o),t||(r(i,o),t)||u(i,o),t}var t,e,r,u,i=ta.geo.albers(),o=ta.geo.conicEqualArea().rotate([154,0]).center([-2,58.5]).parallels([55,65]),a=ta.geo.conicEqualArea().rotate([157,0]).center([-3,19.9]).parallels([8,18]),c={point:function(n,e){t=[n,e]}};return n.invert=function(n){var t=i.scale(),e=i.translate(),r=(n[0]-e[0])/t,u=(n[1]-e[1])/t;return(u>=.12&&.234>u&&r>=-.425&&-.214>r?o:u>=.166&&.234>u&&r>=-.214&&-.115>r?a:i).invert(n)},n.stream=function(n){var t=i.stream(n),e=o.stream(n),r=a.stream(n);return{point:function(n,u){t.point(n,u),e.point(n,u),r.point(n,u)},sphere:function(){t.sphere(),e.sphere(),r.sphere()},lineStart:function(){t.lineStart(),e.lineStart(),r.lineStart()},lineEnd:function(){t.lineEnd(),e.lineEnd(),r.lineEnd()},polygonStart:function(){t.polygonStart(),e.polygonStart(),r.polygonStart()},polygonEnd:function(){t.polygonEnd(),e.polygonEnd(),r.polygonEnd()}}},n.precision=function(t){return arguments.length?(i.precision(t),o.precision(t),a.precision(t),n):i.precision()},n.scale=function(t){return arguments.length?(i.scale(t),o.scale(.35*t),a.scale(t),n.translate(i.translate())):i.scale()},n.translate=function(t){if(!arguments.length)return i.translate();var l=i.scale(),s=+t[0],f=+t[1];return e=i.translate(t).clipExtent([[s-.455*l,f-.238*l],[s+.455*l,f+.238*l]]).stream(c).point,r=o.translate([s-.307*l,f+.201*l]).clipExtent([[s-.425*l+Ca,f+.12*l+Ca],[s-.214*l-Ca,f+.234*l-Ca]]).stream(c).point,u=a.translate([s-.205*l,f+.212*l]).clipExtent([[s-.214*l+Ca,f+.166*l+Ca],[s-.115*l-Ca,f+.234*l-Ca]]).stream(c).point,n},n.scale(1070)};var Rc,Dc,Pc,Uc,jc,Fc,Hc={point:b,lineStart:b,lineEnd:b,polygonStart:function(){Dc=0,Hc.lineStart=Ve},polygonEnd:function(){Hc.lineStart=Hc.lineEnd=Hc.point=b,Rc+=ga(Dc/2)}},Oc={point:Xe,lineStart:b,lineEnd:b,polygonStart:b,polygonEnd:b},Ic={point:We,lineStart:Je,lineEnd:Ge,polygonStart:function(){Ic.lineStart=Ke},polygonEnd:function(){Ic.point=We,Ic.lineStart=Je,Ic.lineEnd=Ge}};ta.geo.path=function(){function n(n){return n&&("function"==typeof a&&i.pointRadius(+a.apply(this,arguments)),o&&o.valid||(o=u(i)),ta.geo.stream(n,o)),i.result()}function t(){return o=null,n}var e,r,u,i,o,a=4.5;return n.area=function(n){return Rc=0,ta.geo.stream(n,u(Hc)),Rc},n.centroid=function(n){return _c=wc=Sc=kc=Ec=Ac=Nc=Cc=zc=0,ta.geo.stream(n,u(Ic)),zc?[Nc/zc,Cc/zc]:Ac?[kc/Ac,Ec/Ac]:Sc?[_c/Sc,wc/Sc]:[0/0,0/0]},n.bounds=function(n){return jc=Fc=-(Pc=Uc=1/0),ta.geo.stream(n,u(Oc)),[[Pc,Uc],[jc,Fc]]},n.projection=function(n){return arguments.length?(u=(e=n)?n.stream||tr(n):y,t()):e},n.context=function(n){return arguments.length?(i=null==(r=n)?new $e:new Qe(n),"function"!=typeof a&&i.pointRadius(a),t()):r},n.pointRadius=function(t){return arguments.length?(a="function"==typeof t?t:(i.pointRadius(+t),+t),n):a},n.projection(ta.geo.albersUsa()).context(null)},ta.geo.transform=function(n){return{stream:function(t){var e=new er(t);for(var r in n)e[r]=n[r];return e}}},er.prototype={point:function(n,t){this.stream.point(n,t)},sphere:function(){this.stream.sphere()},lineStart:function(){this.stream.lineStart()},lineEnd:function(){this.stream.lineEnd()},polygonStart:function(){this.stream.polygonStart()},polygonEnd:function(){this.stream.polygonEnd()}},ta.geo.projection=ur,ta.geo.projectionMutator=ir,(ta.geo.equirectangular=function(){return ur(ar)}).raw=ar.invert=ar,ta.geo.rotation=function(n){function t(t){return t=n(t[0]*Da,t[1]*Da),t[0]*=Pa,t[1]*=Pa,t}return n=lr(n[0]%360*Da,n[1]*Da,n.length>2?n[2]*Da:0),t.invert=function(t){return t=n.invert(t[0]*Da,t[1]*Da),t[0]*=Pa,t[1]*=Pa,t},t},cr.invert=ar,ta.geo.circle=function(){function n(){var n="function"==typeof r?r.apply(this,arguments):r,t=lr(-n[0]*Da,-n[1]*Da,0).invert,u=[];return e(null,null,1,{point:function(n,e){u.push(n=t(n,e)),n[0]*=Pa,n[1]*=Pa}}),{type:"Polygon",coordinates:[u]}}var t,e,r=[0,0],u=6;return n.origin=function(t){return arguments.length?(r=t,n):r},n.angle=function(r){return arguments.length?(e=gr((t=+r)*Da,u*Da),n):t},n.precision=function(r){return arguments.length?(e=gr(t*Da,(u=+r)*Da),n):u},n.angle(90)},ta.geo.distance=function(n,t){var e,r=(t[0]-n[0])*Da,u=n[1]*Da,i=t[1]*Da,o=Math.sin(r),a=Math.cos(r),c=Math.sin(u),l=Math.cos(u),s=Math.sin(i),f=Math.cos(i);return Math.atan2(Math.sqrt((e=f*o)*e+(e=l*s-c*f*a)*e),c*s+l*f*a)},ta.geo.graticule=function(){function n(){return{type:"MultiLineString",coordinates:t()}}function t(){return ta.range(Math.ceil(i/d)*d,u,d).map(h).concat(ta.range(Math.ceil(l/m)*m,c,m).map(g)).concat(ta.range(Math.ceil(r/p)*p,e,p).filter(function(n){return ga(n%d)>Ca}).map(s)).concat(ta.range(Math.ceil(a/v)*v,o,v).filter(function(n){return ga(n%m)>Ca}).map(f))}var e,r,u,i,o,a,c,l,s,f,h,g,p=10,v=p,d=90,m=360,y=2.5;return n.lines=function(){return t().map(function(n){return{type:"LineString",coordinates:n}})},n.outline=function(){return{type:"Polygon",coordinates:[h(i).concat(g(c).slice(1),h(u).reverse().slice(1),g(l).reverse().slice(1))]}},n.extent=function(t){return arguments.length?n.majorExtent(t).minorExtent(t):n.minorExtent()},n.majorExtent=function(t){return arguments.length?(i=+t[0][0],u=+t[1][0],l=+t[0][1],c=+t[1][1],i>u&&(t=i,i=u,u=t),l>c&&(t=l,l=c,c=t),n.precision(y)):[[i,l],[u,c]]},n.minorExtent=function(t){return arguments.length?(r=+t[0][0],e=+t[1][0],a=+t[0][1],o=+t[1][1],r>e&&(t=r,r=e,e=t),a>o&&(t=a,a=o,o=t),n.precision(y)):[[r,a],[e,o]]},n.step=function(t){return arguments.length?n.majorStep(t).minorStep(t):n.minorStep()},n.majorStep=function(t){return arguments.length?(d=+t[0],m=+t[1],n):[d,m]},n.minorStep=function(t){return arguments.length?(p=+t[0],v=+t[1],n):[p,v]},n.precision=function(t){return arguments.length?(y=+t,s=vr(a,o,90),f=dr(r,e,y),h=vr(l,c,90),g=dr(i,u,y),n):y},n.majorExtent([[-180,-90+Ca],[180,90-Ca]]).minorExtent([[-180,-80-Ca],[180,80+Ca]])},ta.geo.greatArc=function(){function n(){return{type:"LineString",coordinates:[t||r.apply(this,arguments),e||u.apply(this,arguments)]}}var t,e,r=mr,u=yr;return n.distance=function(){return ta.geo.distance(t||r.apply(this,arguments),e||u.apply(this,arguments))},n.source=function(e){return arguments.length?(r=e,t="function"==typeof e?null:e,n):r},n.target=function(t){return arguments.length?(u=t,e="function"==typeof t?null:t,n):u},n.precision=function(){return arguments.length?n:0},n},ta.geo.interpolate=function(n,t){return Mr(n[0]*Da,n[1]*Da,t[0]*Da,t[1]*Da)},ta.geo.length=function(n){return Yc=0,ta.geo.stream(n,Zc),Yc};var Yc,Zc={sphere:b,point:b,lineStart:xr,lineEnd:b,polygonStart:b,polygonEnd:b},Vc=br(function(n){return Math.sqrt(2/(1+n))},function(n){return 2*Math.asin(n/2)});(ta.geo.azimuthalEqualArea=function(){return ur(Vc)}).raw=Vc;var Xc=br(function(n){var t=Math.acos(n);return t&&t/Math.sin(t)},y);(ta.geo.azimuthalEquidistant=function(){return ur(Xc)}).raw=Xc,(ta.geo.conicConformal=function(){return Ye(_r)}).raw=_r,(ta.geo.conicEquidistant=function(){return Ye(wr)}).raw=wr;var $c=br(function(n){return 1/n},Math.atan);(ta.geo.gnomonic=function(){return ur($c)}).raw=$c,Sr.invert=function(n,t){return[n,2*Math.atan(Math.exp(t))-Ra]},(ta.geo.mercator=function(){return kr(Sr)}).raw=Sr;var Bc=br(function(){return 1},Math.asin);(ta.geo.orthographic=function(){return ur(Bc)}).raw=Bc;var Wc=br(function(n){return 1/(1+n)},function(n){return 2*Math.atan(n)});(ta.geo.stereographic=function(){return ur(Wc)}).raw=Wc,Er.invert=function(n,t){return[-t,2*Math.atan(Math.exp(n))-Ra]},(ta.geo.transverseMercator=function(){var n=kr(Er),t=n.center,e=n.rotate;return n.center=function(n){return n?t([-n[1],n[0]]):(n=t(),[n[1],-n[0]])},n.rotate=function(n){return n?e([n[0],n[1],n.length>2?n[2]+90:90]):(n=e(),[n[0],n[1],n[2]-90])},e([0,0,90])}).raw=Er,ta.geom={},ta.geom.hull=function(n){function t(n){if(n.length<3)return[];var t,u=Et(e),i=Et(r),o=n.length,a=[],c=[];for(t=0;o>t;t++)a.push([+u.call(this,n[t],t),+i.call(this,n[t],t),t]);for(a.sort(zr),t=0;o>t;t++)c.push([a[t][0],-a[t][1]]);var l=Cr(a),s=Cr(c),f=s[0]===l[0],h=s[s.length-1]===l[l.length-1],g=[];for(t=l.length-1;t>=0;--t)g.push(n[a[l[t]][2]]);for(t=+f;t=r&&l.x<=i&&l.y>=u&&l.y<=o?[[r,o],[i,o],[i,u],[r,u]]:[];s.point=n[a]}),t}function e(n){return n.map(function(n,t){return{x:Math.round(i(n,t)/Ca)*Ca,y:Math.round(o(n,t)/Ca)*Ca,i:t}})}var r=Ar,u=Nr,i=r,o=u,a=ul;return n?t(n):(t.links=function(n){return iu(e(n)).edges.filter(function(n){return n.l&&n.r}).map(function(t){return{source:n[t.l.i],target:n[t.r.i]}})},t.triangles=function(n){var t=[];return iu(e(n)).cells.forEach(function(e,r){for(var u,i,o=e.site,a=e.edges.sort(Yr),c=-1,l=a.length,s=a[l-1].edge,f=s.l===o?s.r:s.l;++c=l,h=r>=s,g=h<<1|f;n.leaf=!1,n=n.nodes[g]||(n.nodes[g]=su()),f?u=l:a=l,h?o=s:c=s,i(n,t,e,r,u,o,a,c)}var s,f,h,g,p,v,d,m,y,M=Et(a),x=Et(c);if(null!=t)v=t,d=e,m=r,y=u;else if(m=y=-(v=d=1/0),f=[],h=[],p=n.length,o)for(g=0;p>g;++g)s=n[g],s.xm&&(m=s.x),s.y>y&&(y=s.y),f.push(s.x),h.push(s.y);else for(g=0;p>g;++g){var b=+M(s=n[g],g),_=+x(s,g);v>b&&(v=b),d>_&&(d=_),b>m&&(m=b),_>y&&(y=_),f.push(b),h.push(_)}var w=m-v,S=y-d;w>S?y=d+w:m=v+S;var k=su();if(k.add=function(n){i(k,n,+M(n,++g),+x(n,g),v,d,m,y)},k.visit=function(n){fu(n,k,v,d,m,y)},k.find=function(n){return hu(k,n[0],n[1],v,d,m,y)},g=-1,null==t){for(;++g=0?n.slice(0,t):n,r=t>=0?n.slice(t+1):"in";return e=cl.get(e)||al,r=ll.get(r)||y,Mu(r(e.apply(null,ea.call(arguments,1))))},ta.interpolateHcl=Lu,ta.interpolateHsl=Tu,ta.interpolateLab=Ru,ta.interpolateRound=Du,ta.transform=function(n){var t=ua.createElementNS(ta.ns.prefix.svg,"g");return(ta.transform=function(n){if(null!=n){t.setAttribute("transform",n);var e=t.transform.baseVal.consolidate()}return new Pu(e?e.matrix:sl)})(n)},Pu.prototype.toString=function(){return"translate("+this.translate+")rotate("+this.rotate+")skewX("+this.skew+")scale("+this.scale+")"};var sl={a:1,b:0,c:0,d:1,e:0,f:0};ta.interpolateTransform=Hu,ta.layout={},ta.layout.bundle=function(){return function(n){for(var t=[],e=-1,r=n.length;++ea*a/d){if(p>c){var l=t.charge/c;n.px-=i*l,n.py-=o*l}return!0}if(t.point&&c&&p>c){var l=t.pointCharge/c;n.px-=i*l,n.py-=o*l}}return!t.charge}}function t(n){n.px=ta.event.x,n.py=ta.event.y,a.resume()}var e,r,u,i,o,a={},c=ta.dispatch("start","tick","end"),l=[1,1],s=.9,f=fl,h=hl,g=-30,p=gl,v=.1,d=.64,m=[],M=[];return a.tick=function(){if((r*=.99)<.005)return c.end({type:"end",alpha:r=0}),!0;var t,e,a,f,h,p,d,y,x,b=m.length,_=M.length;for(e=0;_>e;++e)a=M[e],f=a.source,h=a.target,y=h.x-f.x,x=h.y-f.y,(p=y*y+x*x)&&(p=r*i[e]*((p=Math.sqrt(p))-u[e])/p,y*=p,x*=p,h.x-=y*(d=f.weight/(h.weight+f.weight)),h.y-=x*d,f.x+=y*(d=1-d),f.y+=x*d);if((d=r*v)&&(y=l[0]/2,x=l[1]/2,e=-1,d))for(;++e0?n:0:n>0&&(c.start({type:"start",alpha:r=n}),ta.timer(a.tick)),a):r},a.start=function(){function n(n,r){if(!e){for(e=new Array(c),a=0;c>a;++a)e[a]=[];for(a=0;s>a;++a){var u=M[a];e[u.source.index].push(u.target),e[u.target.index].push(u.source)}}for(var i,o=e[t],a=-1,l=o.length;++at;++t)(r=m[t]).index=t,r.weight=0;for(t=0;s>t;++t)r=M[t],"number"==typeof r.source&&(r.source=m[r.source]),"number"==typeof r.target&&(r.target=m[r.target]),++r.source.weight,++r.target.weight;for(t=0;c>t;++t)r=m[t],isNaN(r.x)&&(r.x=n("x",p)),isNaN(r.y)&&(r.y=n("y",v)),isNaN(r.px)&&(r.px=r.x),isNaN(r.py)&&(r.py=r.y);if(u=[],"function"==typeof f)for(t=0;s>t;++t)u[t]=+f.call(this,M[t],t);else for(t=0;s>t;++t)u[t]=f;if(i=[],"function"==typeof h)for(t=0;s>t;++t)i[t]=+h.call(this,M[t],t);else for(t=0;s>t;++t)i[t]=h;if(o=[],"function"==typeof g)for(t=0;c>t;++t)o[t]=+g.call(this,m[t],t);else for(t=0;c>t;++t)o[t]=g;return a.resume()},a.resume=function(){return a.alpha(.1)},a.stop=function(){return a.alpha(0)},a.drag=function(){return e||(e=ta.behavior.drag().origin(y).on("dragstart.force",Xu).on("drag.force",t).on("dragend.force",$u)),arguments.length?void this.on("mouseover.force",Bu).on("mouseout.force",Wu).call(e):e},ta.rebind(a,c,"on")};var fl=20,hl=1,gl=1/0;ta.layout.hierarchy=function(){function n(u){var i,o=[u],a=[];for(u.depth=0;null!=(i=o.pop());)if(a.push(i),(l=e.call(n,i,i.depth))&&(c=l.length)){for(var c,l,s;--c>=0;)o.push(s=l[c]),s.parent=i,s.depth=i.depth+1;r&&(i.value=0),i.children=l}else r&&(i.value=+r.call(n,i,i.depth)||0),delete i.children;return Qu(u,function(n){var e,u;t&&(e=n.children)&&e.sort(t),r&&(u=n.parent)&&(u.value+=n.value)}),a}var t=ei,e=ni,r=ti;return n.sort=function(e){return arguments.length?(t=e,n):t},n.children=function(t){return arguments.length?(e=t,n):e},n.value=function(t){return arguments.length?(r=t,n):r},n.revalue=function(t){return r&&(Ku(t,function(n){n.children&&(n.value=0)}),Qu(t,function(t){var e;t.children||(t.value=+r.call(n,t,t.depth)||0),(e=t.parent)&&(e.value+=t.value)})),t},n},ta.layout.partition=function(){function n(t,e,r,u){var i=t.children;if(t.x=e,t.y=t.depth*u,t.dx=r,t.dy=u,i&&(o=i.length)){var o,a,c,l=-1;for(r=t.value?r/t.value:0;++lf?-1:1),p=(f-c*g)/ta.sum(l),v=ta.range(c),d=[];return null!=e&&v.sort(e===pl?function(n,t){return l[t]-l[n]}:function(n,t){return e(o[n],o[t])}),v.forEach(function(n){d[n]={data:o[n],value:a=l[n],startAngle:s,endAngle:s+=a*p+g,padAngle:h}}),d}var t=Number,e=pl,r=0,u=La,i=0;return n.value=function(e){return arguments.length?(t=e,n):t},n.sort=function(t){return arguments.length?(e=t,n):e},n.startAngle=function(t){return arguments.length?(r=t,n):r},n.endAngle=function(t){return arguments.length?(u=t,n):u},n.padAngle=function(t){return arguments.length?(i=t,n):i},n};var pl={};ta.layout.stack=function(){function n(a,c){if(!(h=a.length))return a;var l=a.map(function(e,r){return t.call(n,e,r)}),s=l.map(function(t){return t.map(function(t,e){return[i.call(n,t,e),o.call(n,t,e)]})}),f=e.call(n,s,c);l=ta.permute(l,f),s=ta.permute(s,f);var h,g,p,v,d=r.call(n,s,c),m=l[0].length;for(p=0;m>p;++p)for(u.call(n,l[0][p],v=d[p],s[0][p][1]),g=1;h>g;++g)u.call(n,l[g][p],v+=s[g-1][p][1],s[g][p][1]);return a}var t=y,e=ai,r=ci,u=oi,i=ui,o=ii;return n.values=function(e){return arguments.length?(t=e,n):t},n.order=function(t){return arguments.length?(e="function"==typeof t?t:vl.get(t)||ai,n):e},n.offset=function(t){return arguments.length?(r="function"==typeof t?t:dl.get(t)||ci,n):r},n.x=function(t){return arguments.length?(i=t,n):i},n.y=function(t){return arguments.length?(o=t,n):o},n.out=function(t){return arguments.length?(u=t,n):u},n};var vl=ta.map({"inside-out":function(n){var t,e,r=n.length,u=n.map(li),i=n.map(si),o=ta.range(r).sort(function(n,t){return u[n]-u[t]}),a=0,c=0,l=[],s=[];for(t=0;r>t;++t)e=o[t],c>a?(a+=i[e],l.push(e)):(c+=i[e],s.push(e));return s.reverse().concat(l)},reverse:function(n){return ta.range(n.length).reverse()},"default":ai}),dl=ta.map({silhouette:function(n){var t,e,r,u=n.length,i=n[0].length,o=[],a=0,c=[];for(e=0;i>e;++e){for(t=0,r=0;u>t;t++)r+=n[t][e][1];r>a&&(a=r),o.push(r)}for(e=0;i>e;++e)c[e]=(a-o[e])/2;return c},wiggle:function(n){var t,e,r,u,i,o,a,c,l,s=n.length,f=n[0],h=f.length,g=[];for(g[0]=c=l=0,e=1;h>e;++e){for(t=0,u=0;s>t;++t)u+=n[t][e][1];for(t=0,i=0,a=f[e][0]-f[e-1][0];s>t;++t){for(r=0,o=(n[t][e][1]-n[t][e-1][1])/(2*a);t>r;++r)o+=(n[r][e][1]-n[r][e-1][1])/a;i+=o*n[t][e][1]}g[e]=c-=u?i/u*a:0,l>c&&(l=c)}for(e=0;h>e;++e)g[e]-=l;return g},expand:function(n){var t,e,r,u=n.length,i=n[0].length,o=1/u,a=[];for(e=0;i>e;++e){for(t=0,r=0;u>t;t++)r+=n[t][e][1];if(r)for(t=0;u>t;t++)n[t][e][1]/=r;else for(t=0;u>t;t++)n[t][e][1]=o}for(e=0;i>e;++e)a[e]=0;return a},zero:ci});ta.layout.histogram=function(){function n(n,i){for(var o,a,c=[],l=n.map(e,this),s=r.call(this,l,i),f=u.call(this,s,l,i),i=-1,h=l.length,g=f.length-1,p=t?1:1/h;++i0)for(i=-1;++i=s[0]&&a<=s[1]&&(o=c[ta.bisect(f,a,1,g)-1],o.y+=p,o.push(n[i]));return c}var t=!0,e=Number,r=pi,u=hi;return n.value=function(t){return arguments.length?(e=t,n):e},n.range=function(t){return arguments.length?(r=Et(t),n):r},n.bins=function(t){return arguments.length?(u="number"==typeof t?function(n){return gi(n,t)}:Et(t),n):u},n.frequency=function(e){return arguments.length?(t=!!e,n):t},n},ta.layout.pack=function(){function n(n,i){var o=e.call(this,n,i),a=o[0],c=u[0],l=u[1],s=null==t?Math.sqrt:"function"==typeof t?t:function(){return t};if(a.x=a.y=0,Qu(a,function(n){n.r=+s(n.value)}),Qu(a,Mi),r){var f=r*(t?1:Math.max(2*a.r/c,2*a.r/l))/2;Qu(a,function(n){n.r+=f}),Qu(a,Mi),Qu(a,function(n){n.r-=f})}return _i(a,c/2,l/2,t?1:1/Math.max(2*a.r/c,2*a.r/l)),o}var t,e=ta.layout.hierarchy().sort(vi),r=0,u=[1,1];return n.size=function(t){return arguments.length?(u=t,n):u},n.radius=function(e){return arguments.length?(t=null==e||"function"==typeof e?e:+e,n):t},n.padding=function(t){return arguments.length?(r=+t,n):r},Gu(n,e)},ta.layout.tree=function(){function n(n,u){var s=o.call(this,n,u),f=s[0],h=t(f);if(Qu(h,e),h.parent.m=-h.z,Ku(h,r),l)Ku(f,i);else{var g=f,p=f,v=f;Ku(f,function(n){n.xp.x&&(p=n),n.depth>v.depth&&(v=n)});var d=a(g,p)/2-g.x,m=c[0]/(p.x+a(p,g)/2+d),y=c[1]/(v.depth||1);Ku(f,function(n){n.x=(n.x+d)*m,n.y=n.depth*y})}return s}function t(n){for(var t,e={A:null,children:[n]},r=[e];null!=(t=r.pop());)for(var u,i=t.children,o=0,a=i.length;a>o;++o)r.push((i[o]=u={_:i[o],parent:t,children:(u=i[o].children)&&u.slice()||[],A:null,a:null,z:0,m:0,c:0,s:0,t:null,i:o}).a=u);return e.children[0]}function e(n){var t=n.children,e=n.parent.children,r=n.i?e[n.i-1]:null;if(t.length){Ni(n);var i=(t[0].z+t[t.length-1].z)/2;r?(n.z=r.z+a(n._,r._),n.m=n.z-i):n.z=i}else r&&(n.z=r.z+a(n._,r._));n.parent.A=u(n,r,n.parent.A||e[0])}function r(n){n._.x=n.z+n.parent.m,n.m+=n.parent.m}function u(n,t,e){if(t){for(var r,u=n,i=n,o=t,c=u.parent.children[0],l=u.m,s=i.m,f=o.m,h=c.m;o=Ei(o),u=ki(u),o&&u;)c=ki(c),i=Ei(i),i.a=n,r=o.z+f-u.z-l+a(o._,u._),r>0&&(Ai(Ci(o,n,e),n,r),l+=r,s+=r),f+=o.m,l+=u.m,h+=c.m,s+=i.m;o&&!Ei(i)&&(i.t=o,i.m+=f-s),u&&!ki(c)&&(c.t=u,c.m+=l-h,e=n)}return e}function i(n){n.x*=c[0],n.y=n.depth*c[1]}var o=ta.layout.hierarchy().sort(null).value(null),a=Si,c=[1,1],l=null;return n.separation=function(t){return arguments.length?(a=t,n):a},n.size=function(t){return arguments.length?(l=null==(c=t)?i:null,n):l?null:c},n.nodeSize=function(t){return arguments.length?(l=null==(c=t)?null:i,n):l?c:null},Gu(n,o)},ta.layout.cluster=function(){function n(n,i){var o,a=t.call(this,n,i),c=a[0],l=0;Qu(c,function(n){var t=n.children;t&&t.length?(n.x=qi(t),n.y=zi(t)):(n.x=o?l+=e(n,o):0,n.y=0,o=n)});var s=Li(c),f=Ti(c),h=s.x-e(s,f)/2,g=f.x+e(f,s)/2;return Qu(c,u?function(n){n.x=(n.x-c.x)*r[0],n.y=(c.y-n.y)*r[1]}:function(n){n.x=(n.x-h)/(g-h)*r[0],n.y=(1-(c.y?n.y/c.y:1))*r[1]}),a}var t=ta.layout.hierarchy().sort(null).value(null),e=Si,r=[1,1],u=!1;return n.separation=function(t){return arguments.length?(e=t,n):e},n.size=function(t){return arguments.length?(u=null==(r=t),n):u?null:r},n.nodeSize=function(t){return arguments.length?(u=null!=(r=t),n):u?r:null},Gu(n,t)},ta.layout.treemap=function(){function n(n,t){for(var e,r,u=-1,i=n.length;++ut?0:t),e.area=isNaN(r)||0>=r?0:r}function t(e){var i=e.children;if(i&&i.length){var o,a,c,l=f(e),s=[],h=i.slice(),p=1/0,v="slice"===g?l.dx:"dice"===g?l.dy:"slice-dice"===g?1&e.depth?l.dy:l.dx:Math.min(l.dx,l.dy);for(n(h,l.dx*l.dy/e.value),s.area=0;(c=h.length)>0;)s.push(o=h[c-1]),s.area+=o.area,"squarify"!==g||(a=r(s,v))<=p?(h.pop(),p=a):(s.area-=s.pop().area,u(s,v,l,!1),v=Math.min(l.dx,l.dy),s.length=s.area=0,p=1/0);s.length&&(u(s,v,l,!0),s.length=s.area=0),i.forEach(t)}}function e(t){var r=t.children;if(r&&r.length){var i,o=f(t),a=r.slice(),c=[];for(n(a,o.dx*o.dy/t.value),c.area=0;i=a.pop();)c.push(i),c.area+=i.area,null!=i.z&&(u(c,i.z?o.dx:o.dy,o,!a.length),c.length=c.area=0);r.forEach(e)}}function r(n,t){for(var e,r=n.area,u=0,i=1/0,o=-1,a=n.length;++oe&&(i=e),e>u&&(u=e));return r*=r,t*=t,r?Math.max(t*u*p/r,r/(t*i*p)):1/0}function u(n,t,e,r){var u,i=-1,o=n.length,a=e.x,l=e.y,s=t?c(n.area/t):0;if(t==e.dx){for((r||s>e.dy)&&(s=e.dy);++ie.dx)&&(s=e.dx);++ie&&(t=1),1>e&&(n=0),function(){var e,r,u;do e=2*Math.random()-1,r=2*Math.random()-1,u=e*e+r*r;while(!u||u>1);return n+t*e*Math.sqrt(-2*Math.log(u)/u)}},logNormal:function(){var n=ta.random.normal.apply(ta,arguments);return function(){return Math.exp(n())}},bates:function(n){var t=ta.random.irwinHall(n);return function(){return t()/n}},irwinHall:function(n){return function(){for(var t=0,e=0;n>e;e++)t+=Math.random();return t}}},ta.scale={};var ml={floor:y,ceil:y};ta.scale.linear=function(){return Ii([0,1],[0,1],mu,!1)};var yl={s:1,g:1,p:1,r:1,e:1};ta.scale.log=function(){return Ji(ta.scale.linear().domain([0,1]),10,!0,[1,10])};var Ml=ta.format(".0e"),xl={floor:function(n){return-Math.ceil(-n)},ceil:function(n){return-Math.floor(-n)}};ta.scale.pow=function(){return Gi(ta.scale.linear(),1,[0,1])},ta.scale.sqrt=function(){return ta.scale.pow().exponent(.5)},ta.scale.ordinal=function(){return Qi([],{t:"range",a:[[]]})},ta.scale.category10=function(){return ta.scale.ordinal().range(bl)},ta.scale.category20=function(){return ta.scale.ordinal().range(_l)},ta.scale.category20b=function(){return ta.scale.ordinal().range(wl)},ta.scale.category20c=function(){return ta.scale.ordinal().range(Sl)};var bl=[2062260,16744206,2924588,14034728,9725885,9197131,14907330,8355711,12369186,1556175].map(Mt),_l=[2062260,11454440,16744206,16759672,2924588,10018698,14034728,16750742,9725885,12955861,9197131,12885140,14907330,16234194,8355711,13092807,12369186,14408589,1556175,10410725].map(Mt),wl=[3750777,5395619,7040719,10264286,6519097,9216594,11915115,13556636,9202993,12426809,15186514,15190932,8666169,11356490,14049643,15177372,8077683,10834324,13528509,14589654].map(Mt),Sl=[3244733,7057110,10406625,13032431,15095053,16616764,16625259,16634018,3253076,7652470,10607003,13101504,7695281,10394312,12369372,14342891,6513507,9868950,12434877,14277081].map(Mt);ta.scale.quantile=function(){return no([],[])},ta.scale.quantize=function(){return to(0,1,[0,1])},ta.scale.threshold=function(){return eo([.5],[0,1])},ta.scale.identity=function(){return ro([0,1])},ta.svg={},ta.svg.arc=function(){function n(){var n=Math.max(0,+e.apply(this,arguments)),l=Math.max(0,+r.apply(this,arguments)),s=o.apply(this,arguments)-Ra,f=a.apply(this,arguments)-Ra,h=Math.abs(f-s),g=s>f?0:1;if(n>l&&(p=l,l=n,n=p),h>=Ta)return t(l,g)+(n?t(n,1-g):"")+"Z";var p,v,d,m,y,M,x,b,_,w,S,k,E=0,A=0,N=[];if((m=(+c.apply(this,arguments)||0)/2)&&(d=i===kl?Math.sqrt(n*n+l*l):+i.apply(this,arguments),g||(A*=-1),l&&(A=tt(d/l*Math.sin(m))),n&&(E=tt(d/n*Math.sin(m)))),l){y=l*Math.cos(s+A),M=l*Math.sin(s+A),x=l*Math.cos(f-A),b=l*Math.sin(f-A);var C=Math.abs(f-s-2*A)<=qa?0:1;if(A&&so(y,M,x,b)===g^C){var z=(s+f)/2;y=l*Math.cos(z),M=l*Math.sin(z),x=b=null}}else y=M=0;if(n){_=n*Math.cos(f-E),w=n*Math.sin(f-E),S=n*Math.cos(s+E),k=n*Math.sin(s+E);var q=Math.abs(s-f+2*E)<=qa?0:1;if(E&&so(_,w,S,k)===1-g^q){var L=(s+f)/2;_=n*Math.cos(L),w=n*Math.sin(L),S=k=null}}else _=w=0;if((p=Math.min(Math.abs(l-n)/2,+u.apply(this,arguments)))>.001){v=l>n^g?0:1;var T=null==S?[_,w]:null==x?[y,M]:Lr([y,M],[S,k],[x,b],[_,w]),R=y-T[0],D=M-T[1],P=x-T[0],U=b-T[1],j=1/Math.sin(Math.acos((R*P+D*U)/(Math.sqrt(R*R+D*D)*Math.sqrt(P*P+U*U)))/2),F=Math.sqrt(T[0]*T[0]+T[1]*T[1]);if(null!=x){var H=Math.min(p,(l-F)/(j+1)),O=fo(null==S?[_,w]:[S,k],[y,M],l,H,g),I=fo([x,b],[_,w],l,H,g);p===H?N.push("M",O[0],"A",H,",",H," 0 0,",v," ",O[1],"A",l,",",l," 0 ",1-g^so(O[1][0],O[1][1],I[1][0],I[1][1]),",",g," ",I[1],"A",H,",",H," 0 0,",v," ",I[0]):N.push("M",O[0],"A",H,",",H," 0 1,",v," ",I[0])}else N.push("M",y,",",M);if(null!=S){var Y=Math.min(p,(n-F)/(j-1)),Z=fo([y,M],[S,k],n,-Y,g),V=fo([_,w],null==x?[y,M]:[x,b],n,-Y,g);p===Y?N.push("L",V[0],"A",Y,",",Y," 0 0,",v," ",V[1],"A",n,",",n," 0 ",g^so(V[1][0],V[1][1],Z[1][0],Z[1][1]),",",1-g," ",Z[1],"A",Y,",",Y," 0 0,",v," ",Z[0]):N.push("L",V[0],"A",Y,",",Y," 0 0,",v," ",Z[0])}else N.push("L",_,",",w)}else N.push("M",y,",",M),null!=x&&N.push("A",l,",",l," 0 ",C,",",g," ",x,",",b),N.push("L",_,",",w),null!=S&&N.push("A",n,",",n," 0 ",q,",",1-g," ",S,",",k);return N.push("Z"),N.join("")}function t(n,t){return"M0,"+n+"A"+n+","+n+" 0 1,"+t+" 0,"+-n+"A"+n+","+n+" 0 1,"+t+" 0,"+n}var e=io,r=oo,u=uo,i=kl,o=ao,a=co,c=lo;return n.innerRadius=function(t){return arguments.length?(e=Et(t),n):e},n.outerRadius=function(t){return arguments.length?(r=Et(t),n):r},n.cornerRadius=function(t){return arguments.length?(u=Et(t),n):u},n.padRadius=function(t){return arguments.length?(i=t==kl?kl:Et(t),n):i},n.startAngle=function(t){return arguments.length?(o=Et(t),n):o},n.endAngle=function(t){return arguments.length?(a=Et(t),n):a},n.padAngle=function(t){return arguments.length?(c=Et(t),n):c},n.centroid=function(){var n=(+e.apply(this,arguments)+ +r.apply(this,arguments))/2,t=(+o.apply(this,arguments)+ +a.apply(this,arguments))/2-Ra;return[Math.cos(t)*n,Math.sin(t)*n]},n};var kl="auto";ta.svg.line=function(){return ho(y)};var El=ta.map({linear:go,"linear-closed":po,step:vo,"step-before":mo,"step-after":yo,basis:So,"basis-open":ko,"basis-closed":Eo,bundle:Ao,cardinal:bo,"cardinal-open":Mo,"cardinal-closed":xo,monotone:To});El.forEach(function(n,t){t.key=n,t.closed=/-closed$/.test(n)});var Al=[0,2/3,1/3,0],Nl=[0,1/3,2/3,0],Cl=[0,1/6,2/3,1/6];ta.svg.line.radial=function(){var n=ho(Ro);return n.radius=n.x,delete n.x,n.angle=n.y,delete n.y,n},mo.reverse=yo,yo.reverse=mo,ta.svg.area=function(){return Do(y)},ta.svg.area.radial=function(){var n=Do(Ro);return n.radius=n.x,delete n.x,n.innerRadius=n.x0,delete n.x0,n.outerRadius=n.x1,delete n.x1,n.angle=n.y,delete n.y,n.startAngle=n.y0,delete n.y0,n.endAngle=n.y1,delete n.y1,n},ta.svg.chord=function(){function n(n,a){var c=t(this,i,n,a),l=t(this,o,n,a);return"M"+c.p0+r(c.r,c.p1,c.a1-c.a0)+(e(c,l)?u(c.r,c.p1,c.r,c.p0):u(c.r,c.p1,l.r,l.p0)+r(l.r,l.p1,l.a1-l.a0)+u(l.r,l.p1,c.r,c.p0))+"Z"}function t(n,t,e,r){var u=t.call(n,e,r),i=a.call(n,u,r),o=c.call(n,u,r)-Ra,s=l.call(n,u,r)-Ra;return{r:i,a0:o,a1:s,p0:[i*Math.cos(o),i*Math.sin(o)],p1:[i*Math.cos(s),i*Math.sin(s)]}}function e(n,t){return n.a0==t.a0&&n.a1==t.a1}function r(n,t,e){return"A"+n+","+n+" 0 "+ +(e>qa)+",1 "+t}function u(n,t,e,r){return"Q 0,0 "+r}var i=mr,o=yr,a=Po,c=ao,l=co;return n.radius=function(t){return arguments.length?(a=Et(t),n):a},n.source=function(t){return arguments.length?(i=Et(t),n):i},n.target=function(t){return arguments.length?(o=Et(t),n):o},n.startAngle=function(t){return arguments.length?(c=Et(t),n):c},n.endAngle=function(t){return arguments.length?(l=Et(t),n):l},n},ta.svg.diagonal=function(){function n(n,u){var i=t.call(this,n,u),o=e.call(this,n,u),a=(i.y+o.y)/2,c=[i,{x:i.x,y:a},{x:o.x,y:a},o];return c=c.map(r),"M"+c[0]+"C"+c[1]+" "+c[2]+" "+c[3]}var t=mr,e=yr,r=Uo;return n.source=function(e){return arguments.length?(t=Et(e),n):t},n.target=function(t){return arguments.length?(e=Et(t),n):e},n.projection=function(t){return arguments.length?(r=t,n):r},n},ta.svg.diagonal.radial=function(){var n=ta.svg.diagonal(),t=Uo,e=n.projection;return n.projection=function(n){return arguments.length?e(jo(t=n)):t},n},ta.svg.symbol=function(){function n(n,r){return(zl.get(t.call(this,n,r))||Oo)(e.call(this,n,r))}var t=Ho,e=Fo;return n.type=function(e){return arguments.length?(t=Et(e),n):t},n.size=function(t){return arguments.length?(e=Et(t),n):e},n};var zl=ta.map({circle:Oo,cross:function(n){var t=Math.sqrt(n/5)/2;return"M"+-3*t+","+-t+"H"+-t+"V"+-3*t+"H"+t+"V"+-t+"H"+3*t+"V"+t+"H"+t+"V"+3*t+"H"+-t+"V"+t+"H"+-3*t+"Z"},diamond:function(n){var t=Math.sqrt(n/(2*Ll)),e=t*Ll;return"M0,"+-t+"L"+e+",0 0,"+t+" "+-e+",0Z"},square:function(n){var t=Math.sqrt(n)/2;return"M"+-t+","+-t+"L"+t+","+-t+" "+t+","+t+" "+-t+","+t+"Z"},"triangle-down":function(n){var t=Math.sqrt(n/ql),e=t*ql/2;return"M0,"+e+"L"+t+","+-e+" "+-t+","+-e+"Z"},"triangle-up":function(n){var t=Math.sqrt(n/ql),e=t*ql/2;return"M0,"+-e+"L"+t+","+e+" "+-t+","+e+"Z"}});ta.svg.symbolTypes=zl.keys();var ql=Math.sqrt(3),Ll=Math.tan(30*Da);_a.transition=function(n){for(var t,e,r=Tl||++Ul,u=Xo(n),i=[],o=Rl||{time:Date.now(),ease:Su,delay:0,duration:250},a=-1,c=this.length;++ai;i++){u.push(t=[]);for(var e=this[i],a=0,c=e.length;c>a;a++)(r=e[a])&&n.call(r,r.__data__,a,i)&&t.push(r)}return Yo(u,this.namespace,this.id)},Pl.tween=function(n,t){var e=this.id,r=this.namespace;return arguments.length<2?this.node()[r][e].tween.get(n):Y(this,null==t?function(t){t[r][e].tween.remove(n)}:function(u){u[r][e].tween.set(n,t)})},Pl.attr=function(n,t){function e(){this.removeAttribute(a)}function r(){this.removeAttributeNS(a.space,a.local)}function u(n){return null==n?e:(n+="",function(){var t,e=this.getAttribute(a);return e!==n&&(t=o(e,n),function(n){this.setAttribute(a,t(n))})})}function i(n){return null==n?r:(n+="",function(){var t,e=this.getAttributeNS(a.space,a.local);return e!==n&&(t=o(e,n),function(n){this.setAttributeNS(a.space,a.local,t(n))})})}if(arguments.length<2){for(t in n)this.attr(t,n[t]);return this}var o="transform"==n?Hu:mu,a=ta.ns.qualify(n);return Zo(this,"attr."+n,t,a.local?i:u)},Pl.attrTween=function(n,t){function e(n,e){var r=t.call(this,n,e,this.getAttribute(u));return r&&function(n){this.setAttribute(u,r(n))}}function r(n,e){var r=t.call(this,n,e,this.getAttributeNS(u.space,u.local));return r&&function(n){this.setAttributeNS(u.space,u.local,r(n))}}var u=ta.ns.qualify(n);return this.tween("attr."+n,u.local?r:e)},Pl.style=function(n,e,r){function u(){this.style.removeProperty(n)}function i(e){return null==e?u:(e+="",function(){var u,i=t(this).getComputedStyle(this,null).getPropertyValue(n);return i!==e&&(u=mu(i,e),function(t){this.style.setProperty(n,u(t),r)})})}var o=arguments.length;if(3>o){if("string"!=typeof n){2>o&&(e="");for(r in n)this.style(r,n[r],e);return this}r=""}return Zo(this,"style."+n,e,i)},Pl.styleTween=function(n,e,r){function u(u,i){var o=e.call(this,u,i,t(this).getComputedStyle(this,null).getPropertyValue(n));return o&&function(t){this.style.setProperty(n,o(t),r)}}return arguments.length<3&&(r=""),this.tween("style."+n,u)},Pl.text=function(n){return Zo(this,"text",n,Vo)},Pl.remove=function(){var n=this.namespace;return this.each("end.transition",function(){var t;this[n].count<2&&(t=this.parentNode)&&t.removeChild(this)})},Pl.ease=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].ease:("function"!=typeof n&&(n=ta.ease.apply(ta,arguments)),Y(this,function(r){r[e][t].ease=n}))},Pl.delay=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].delay:Y(this,"function"==typeof n?function(r,u,i){r[e][t].delay=+n.call(r,r.__data__,u,i)}:(n=+n,function(r){r[e][t].delay=n}))},Pl.duration=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].duration:Y(this,"function"==typeof n?function(r,u,i){r[e][t].duration=Math.max(1,n.call(r,r.__data__,u,i))}:(n=Math.max(1,n),function(r){r[e][t].duration=n}))},Pl.each=function(n,t){var e=this.id,r=this.namespace;if(arguments.length<2){var u=Rl,i=Tl;try{Tl=e,Y(this,function(t,u,i){Rl=t[r][e],n.call(t,t.__data__,u,i)})}finally{Rl=u,Tl=i}}else Y(this,function(u){var i=u[r][e];(i.event||(i.event=ta.dispatch("start","end","interrupt"))).on(n,t)});return this},Pl.transition=function(){for(var n,t,e,r,u=this.id,i=++Ul,o=this.namespace,a=[],c=0,l=this.length;l>c;c++){a.push(n=[]);for(var t=this[c],s=0,f=t.length;f>s;s++)(e=t[s])&&(r=e[o][u],$o(e,s,o,i,{time:r.time,ease:r.ease,delay:r.delay+r.duration,duration:r.duration})),n.push(e)}return Yo(a,o,i)},ta.svg.axis=function(){function n(n){n.each(function(){var n,l=ta.select(this),s=this.__chart__||e,f=this.__chart__=e.copy(),h=null==c?f.ticks?f.ticks.apply(f,a):f.domain():c,g=null==t?f.tickFormat?f.tickFormat.apply(f,a):y:t,p=l.selectAll(".tick").data(h,f),v=p.enter().insert("g",".domain").attr("class","tick").style("opacity",Ca),d=ta.transition(p.exit()).style("opacity",Ca).remove(),m=ta.transition(p.order()).style("opacity",1),M=Math.max(u,0)+o,x=Ui(f),b=l.selectAll(".domain").data([0]),_=(b.enter().append("path").attr("class","domain"),ta.transition(b));v.append("line"),v.append("text");var w,S,k,E,A=v.select("line"),N=m.select("line"),C=p.select("text").text(g),z=v.select("text"),q=m.select("text"),L="top"===r||"left"===r?-1:1;if("bottom"===r||"top"===r?(n=Bo,w="x",k="y",S="x2",E="y2",C.attr("dy",0>L?"0em":".71em").style("text-anchor","middle"),_.attr("d","M"+x[0]+","+L*i+"V0H"+x[1]+"V"+L*i)):(n=Wo,w="y",k="x",S="y2",E="x2",C.attr("dy",".32em").style("text-anchor",0>L?"end":"start"),_.attr("d","M"+L*i+","+x[0]+"H0V"+x[1]+"H"+L*i)),A.attr(E,L*u),z.attr(k,L*M),N.attr(S,0).attr(E,L*u),q.attr(w,0).attr(k,L*M),f.rangeBand){var T=f,R=T.rangeBand()/2;s=f=function(n){return T(n)+R}}else s.rangeBand?s=f:d.call(n,f,s);v.call(n,s,f),m.call(n,f,f)})}var t,e=ta.scale.linear(),r=jl,u=6,i=6,o=3,a=[10],c=null;return n.scale=function(t){return arguments.length?(e=t,n):e},n.orient=function(t){return arguments.length?(r=t in Fl?t+"":jl,n):r},n.ticks=function(){return arguments.length?(a=arguments,n):a},n.tickValues=function(t){return arguments.length?(c=t,n):c},n.tickFormat=function(e){return arguments.length?(t=e,n):t},n.tickSize=function(t){var e=arguments.length;return e?(u=+t,i=+arguments[e-1],n):u},n.innerTickSize=function(t){return arguments.length?(u=+t,n):u},n.outerTickSize=function(t){return arguments.length?(i=+t,n):i},n.tickPadding=function(t){return arguments.length?(o=+t,n):o},n.tickSubdivide=function(){return arguments.length&&n},n};var jl="bottom",Fl={top:1,right:1,bottom:1,left:1};ta.svg.brush=function(){function n(t){t.each(function(){var t=ta.select(this).style("pointer-events","all").style("-webkit-tap-highlight-color","rgba(0,0,0,0)").on("mousedown.brush",i).on("touchstart.brush",i),o=t.selectAll(".background").data([0]);o.enter().append("rect").attr("class","background").style("visibility","hidden").style("cursor","crosshair"),t.selectAll(".extent").data([0]).enter().append("rect").attr("class","extent").style("cursor","move");var a=t.selectAll(".resize").data(v,y);a.exit().remove(),a.enter().append("g").attr("class",function(n){return"resize "+n}).style("cursor",function(n){return Hl[n]}).append("rect").attr("x",function(n){return/[ew]$/.test(n)?-3:null}).attr("y",function(n){return/^[ns]/.test(n)?-3:null}).attr("width",6).attr("height",6).style("visibility","hidden"),a.style("display",n.empty()?"none":null);var c,f=ta.transition(t),h=ta.transition(o);l&&(c=Ui(l),h.attr("x",c[0]).attr("width",c[1]-c[0]),r(f)),s&&(c=Ui(s),h.attr("y",c[0]).attr("height",c[1]-c[0]),u(f)),e(f)})}function e(n){n.selectAll(".resize").attr("transform",function(n){return"translate("+f[+/e$/.test(n)]+","+h[+/^s/.test(n)]+")"})}function r(n){n.select(".extent").attr("x",f[0]),n.selectAll(".extent,.n>rect,.s>rect").attr("width",f[1]-f[0])}function u(n){n.select(".extent").attr("y",h[0]),n.selectAll(".extent,.e>rect,.w>rect").attr("height",h[1]-h[0])}function i(){function i(){32==ta.event.keyCode&&(C||(M=null,q[0]-=f[1],q[1]-=h[1],C=2),S())}function v(){32==ta.event.keyCode&&2==C&&(q[0]+=f[1],q[1]+=h[1],C=0,S())}function d(){var n=ta.mouse(b),t=!1;x&&(n[0]+=x[0],n[1]+=x[1]),C||(ta.event.altKey?(M||(M=[(f[0]+f[1])/2,(h[0]+h[1])/2]),q[0]=f[+(n[0]s?(u=r,r=s):u=s),v[0]!=r||v[1]!=u?(e?a=null:o=null,v[0]=r,v[1]=u,!0):void 0}function y(){d(),k.style("pointer-events","all").selectAll(".resize").style("display",n.empty()?"none":null),ta.select("body").style("cursor",null),L.on("mousemove.brush",null).on("mouseup.brush",null).on("touchmove.brush",null).on("touchend.brush",null).on("keydown.brush",null).on("keyup.brush",null),z(),w({type:"brushend"})}var M,x,b=this,_=ta.select(ta.event.target),w=c.of(b,arguments),k=ta.select(b),E=_.datum(),A=!/^(n|s)$/.test(E)&&l,N=!/^(e|w)$/.test(E)&&s,C=_.classed("extent"),z=W(b),q=ta.mouse(b),L=ta.select(t(b)).on("keydown.brush",i).on("keyup.brush",v);if(ta.event.changedTouches?L.on("touchmove.brush",d).on("touchend.brush",y):L.on("mousemove.brush",d).on("mouseup.brush",y),k.interrupt().selectAll("*").interrupt(),C)q[0]=f[0]-q[0],q[1]=h[0]-q[1];else if(E){var T=+/w$/.test(E),R=+/^n/.test(E);x=[f[1-T]-q[0],h[1-R]-q[1]],q[0]=f[T],q[1]=h[R]}else ta.event.altKey&&(M=q.slice());k.style("pointer-events","none").selectAll(".resize").style("display",null),ta.select("body").style("cursor",_.style("cursor")),w({type:"brushstart"}),d()}var o,a,c=E(n,"brushstart","brush","brushend"),l=null,s=null,f=[0,0],h=[0,0],g=!0,p=!0,v=Ol[0];return n.event=function(n){n.each(function(){var n=c.of(this,arguments),t={x:f,y:h,i:o,j:a},e=this.__chart__||t;this.__chart__=t,Tl?ta.select(this).transition().each("start.brush",function(){o=e.i,a=e.j,f=e.x,h=e.y,n({type:"brushstart"})}).tween("brush:brush",function(){var e=yu(f,t.x),r=yu(h,t.y);return o=a=null,function(u){f=t.x=e(u),h=t.y=r(u),n({type:"brush",mode:"resize"})}}).each("end.brush",function(){o=t.i,a=t.j,n({type:"brush",mode:"resize"}),n({type:"brushend"})}):(n({type:"brushstart"}),n({type:"brush",mode:"resize"}),n({type:"brushend"}))})},n.x=function(t){return arguments.length?(l=t,v=Ol[!l<<1|!s],n):l},n.y=function(t){return arguments.length?(s=t,v=Ol[!l<<1|!s],n):s},n.clamp=function(t){return arguments.length?(l&&s?(g=!!t[0],p=!!t[1]):l?g=!!t:s&&(p=!!t),n):l&&s?[g,p]:l?g:s?p:null},n.extent=function(t){var e,r,u,i,c;return arguments.length?(l&&(e=t[0],r=t[1],s&&(e=e[0],r=r[0]),o=[e,r],l.invert&&(e=l(e),r=l(r)),e>r&&(c=e,e=r,r=c),(e!=f[0]||r!=f[1])&&(f=[e,r])),s&&(u=t[0],i=t[1],l&&(u=u[1],i=i[1]),a=[u,i],s.invert&&(u=s(u),i=s(i)),u>i&&(c=u,u=i,i=c),(u!=h[0]||i!=h[1])&&(h=[u,i])),n):(l&&(o?(e=o[0],r=o[1]):(e=f[0],r=f[1],l.invert&&(e=l.invert(e),r=l.invert(r)),e>r&&(c=e,e=r,r=c))),s&&(a?(u=a[0],i=a[1]):(u=h[0],i=h[1],s.invert&&(u=s.invert(u),i=s.invert(i)),u>i&&(c=u,u=i,i=c))),l&&s?[[e,u],[r,i]]:l?[e,r]:s&&[u,i])},n.clear=function(){return n.empty()||(f=[0,0],h=[0,0],o=a=null),n},n.empty=function(){return!!l&&f[0]==f[1]||!!s&&h[0]==h[1]},ta.rebind(n,c,"on")};var Hl={n:"ns-resize",e:"ew-resize",s:"ns-resize",w:"ew-resize",nw:"nwse-resize",ne:"nesw-resize",se:"nwse-resize",sw:"nesw-resize"},Ol=[["n","e","s","w","nw","ne","se","sw"],["e","w"],["n","s"],[]],Il=ac.format=gc.timeFormat,Yl=Il.utc,Zl=Yl("%Y-%m-%dT%H:%M:%S.%LZ");Il.iso=Date.prototype.toISOString&&+new Date("2000-01-01T00:00:00.000Z")?Jo:Zl,Jo.parse=function(n){var t=new Date(n);return isNaN(t)?null:t},Jo.toString=Zl.toString,ac.second=Ft(function(n){return new cc(1e3*Math.floor(n/1e3))},function(n,t){n.setTime(n.getTime()+1e3*Math.floor(t))},function(n){return n.getSeconds()}),ac.seconds=ac.second.range,ac.seconds.utc=ac.second.utc.range,ac.minute=Ft(function(n){return new cc(6e4*Math.floor(n/6e4))},function(n,t){n.setTime(n.getTime()+6e4*Math.floor(t))},function(n){return n.getMinutes()}),ac.minutes=ac.minute.range,ac.minutes.utc=ac.minute.utc.range,ac.hour=Ft(function(n){var t=n.getTimezoneOffset()/60;return new cc(36e5*(Math.floor(n/36e5-t)+t))},function(n,t){n.setTime(n.getTime()+36e5*Math.floor(t))},function(n){return n.getHours()}),ac.hours=ac.hour.range,ac.hours.utc=ac.hour.utc.range,ac.month=Ft(function(n){return n=ac.day(n),n.setDate(1),n},function(n,t){n.setMonth(n.getMonth()+t)},function(n){return n.getMonth()}),ac.months=ac.month.range,ac.months.utc=ac.month.utc.range;var Vl=[1e3,5e3,15e3,3e4,6e4,3e5,9e5,18e5,36e5,108e5,216e5,432e5,864e5,1728e5,6048e5,2592e6,7776e6,31536e6],Xl=[[ac.second,1],[ac.second,5],[ac.second,15],[ac.second,30],[ac.minute,1],[ac.minute,5],[ac.minute,15],[ac.minute,30],[ac.hour,1],[ac.hour,3],[ac.hour,6],[ac.hour,12],[ac.day,1],[ac.day,2],[ac.week,1],[ac.month,1],[ac.month,3],[ac.year,1]],$l=Il.multi([[".%L",function(n){return n.getMilliseconds()}],[":%S",function(n){return n.getSeconds()}],["%I:%M",function(n){return n.getMinutes()}],["%I %p",function(n){return n.getHours()}],["%a %d",function(n){return n.getDay()&&1!=n.getDate()}],["%b %d",function(n){return 1!=n.getDate()}],["%B",function(n){return n.getMonth()}],["%Y",Ne]]),Bl={range:function(n,t,e){return ta.range(Math.ceil(n/e)*e,+t,e).map(Ko)},floor:y,ceil:y};Xl.year=ac.year,ac.scale=function(){return Go(ta.scale.linear(),Xl,$l)};var Wl=Xl.map(function(n){return[n[0].utc,n[1]]}),Jl=Yl.multi([[".%L",function(n){return n.getUTCMilliseconds()}],[":%S",function(n){return n.getUTCSeconds()}],["%I:%M",function(n){return n.getUTCMinutes()}],["%I %p",function(n){return n.getUTCHours()}],["%a %d",function(n){return n.getUTCDay()&&1!=n.getUTCDate()}],["%b %d",function(n){return 1!=n.getUTCDate()}],["%B",function(n){return n.getUTCMonth()}],["%Y",Ne]]);Wl.year=ac.year.utc,ac.scale.utc=function(){return Go(ta.scale.linear(),Wl,Jl)},ta.text=At(function(n){return n.responseText}),ta.json=function(n,t){return Nt(n,"application/json",Qo,t)},ta.html=function(n,t){return Nt(n,"text/html",na,t)},ta.xml=At(function(n){return n.responseXML}),"function"==typeof define&&define.amd?define(ta):"object"==typeof module&&module.exports&&(module.exports=ta),this.d3=ta}(); diff --git a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js new file mode 100644 index 0000000000000..6d2da25024a83 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js @@ -0,0 +1,29 @@ +/*v0.4.3 with 1 additional commit (see http://github.com/andrewor14/dagre-d3)*/(function(f){if(typeof exports==="object"&&typeof module!=="undefined"){module.exports=f()}else if(typeof define==="function"&&define.amd){define([],f)}else{var g;if(typeof window!=="undefined"){g=window}else if(typeof global!=="undefined"){g=global}else if(typeof self!=="undefined"){g=self}else{g=this}g.dagreD3=f()}})(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;i0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph(); + +graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){ +var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){arguments[4][20][0].apply(exports,arguments)},{dup:20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments; + +stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index1?arguments[1]:{},peg$FAILED={},peg$startRuleFunctions={start:peg$parsestart,graphStmt:peg$parsegraphStmt},peg$startRuleFunction=peg$parsestart,peg$c0=[],peg$c1=peg$FAILED,peg$c2=null,peg$c3="{",peg$c4={type:"literal",value:"{",description:'"{"'},peg$c5="}",peg$c6={type:"literal",value:"}",description:'"}"'},peg$c7=function(strict,type,id,stmts){return{type:type,id:id,strict:strict!==null,stmts:stmts}},peg$c8=";",peg$c9={type:"literal",value:";",description:'";"'},peg$c10=function(first,rest){var result=[first];for(var i=0;i",description:'"->"'},peg$c33=function(rhs,rest){var result=[rhs];if(rest){for(var i=0;ipos){peg$cachedPos=0;peg$cachedPosDetails={line:1,column:1,seenCR:false}}advance(peg$cachedPosDetails,peg$cachedPos,pos);peg$cachedPos=pos}return peg$cachedPosDetails}function peg$fail(expected){if(peg$currPospeg$maxFailPos){peg$maxFailPos=peg$currPos;peg$maxFailExpected=[]}peg$maxFailExpected.push(expected)}function peg$buildException(message,expected,pos){function cleanupExpected(expected){var i=1;expected.sort(function(a,b){if(a.descriptionb.description){return 1}else{return 0}});while(i1?expectedDescs.slice(0,-1).join(", ")+" or "+expectedDescs[expected.length-1]:expectedDescs[0];foundDesc=found?'"'+stringEscape(found)+'"':"end of input";return"Expected "+expectedDesc+" but "+foundDesc+" found."}var posDetails=peg$computePosDetails(pos),found=pospeg$currPos){s5=input.charAt(peg$currPos);peg$currPos++}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c110)}}if(s5!==peg$FAILED){s4=[s4,s5];s3=s4}else{peg$currPos=s3;s3=peg$c1}}else{peg$currPos=s3;s3=peg$c1}while(s3!==peg$FAILED){s2.push(s3);s3=peg$currPos;s4=peg$currPos;peg$silentFails++;if(input.substr(peg$currPos,2)===peg$c108){s5=peg$c108;peg$currPos+=2}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c109)}}peg$silentFails--;if(s5===peg$FAILED){s4=peg$c30}else{peg$currPos=s4;s4=peg$c1}if(s4!==peg$FAILED){if(input.length>peg$currPos){s5=input.charAt(peg$currPos);peg$currPos++}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c110)}}if(s5!==peg$FAILED){s4=[s4,s5];s3=s4}else{peg$currPos=s3;s3=peg$c1}}else{peg$currPos=s3;s3=peg$c1}}if(s2!==peg$FAILED){if(input.substr(peg$currPos,2)===peg$c108){s3=peg$c108;peg$currPos+=2}else{s3=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c109)}}if(s3!==peg$FAILED){s1=[s1,s2,s3];s0=s1}else{peg$currPos=s0;s0=peg$c1}}else{peg$currPos=s0;s0=peg$c1}}else{peg$currPos=s0;s0=peg$c1}}peg$silentFails--;if(s0===peg$FAILED){s1=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c101)}}return s0}function peg$parse_(){var s0;s0=peg$parsewhitespace();if(s0===peg$FAILED){s0=peg$parsecomment()}return s0}var _=require("lodash");var directed;peg$result=peg$startRuleFunction();if(peg$result!==peg$FAILED&&peg$currPos===input.length){return peg$result}else{if(peg$result!==peg$FAILED&&peg$currPos":"--",writer=new Writer;if(!g.isMultigraph()){writer.write("strict ")}writer.writeLine((g.isDirected()?"digraph":"graph")+" {");writer.indent();var graphAttrs=g.graph();if(_.isObject(graphAttrs)){_.each(graphAttrs,function(v,k){writer.writeLine(id(k)+"="+id(v)+";")})}writeSubgraph(g,undefined,writer);g.edges().forEach(function(edge){writeEdge(g,edge,ec,writer)});writer.unindent();writer.writeLine("}");return writer.toString()}function writeSubgraph(g,v,writer){var children=g.isCompound()?g.children(v):g.nodes();_.each(children,function(w){if(!g.isCompound()||!g.children(w).length){writeNode(g,w,writer)}else{writer.writeLine("subgraph "+id(w)+" {");writer.indent();if(_.isObject(g.node(w))){_.map(g.node(w),function(val,key){writer.writeLine(id(key)+"="+id(val)+";")})}writeSubgraph(g,w,writer);writer.unindent();writer.writeLine("}")}})}function writeNode(g,v,writer){writer.write(id(v));writeAttrs(g.node(v),writer);writer.writeLine()}function writeEdge(g,edge,ec,writer){var v=edge.v,w=edge.w,attrs=g.edge(edge);writer.write(id(v)+" "+ec+" "+id(w));writeAttrs(attrs,writer);writer.writeLine()}function writeAttrs(attrs,writer){if(_.isObject(attrs)){var attrStrs=_.map(attrs,function(val,key){return id(key)+"="+id(val)});if(attrStrs.length){writer.write(" ["+attrStrs.join(",")+"]")}}}function id(obj){if(typeof obj==="number"||obj.toString().match(UNESCAPED_ID_PATTERN)){return obj}return'"'+obj.toString().replace(/"/g,'\\"')+'"'}function Writer(){this._indent="";this._content="";this._shouldIndent=true}Writer.prototype.INDENT=" ";Writer.prototype.indent=function(){this._indent+=this.INDENT};Writer.prototype.unindent=function(){this._indent=this._indent.slice(this.INDENT.length)};Writer.prototype.writeLine=function(line){this.write((line||"")+"\n");this._shouldIndent=true};Writer.prototype.write=function(str){if(this._shouldIndent){this._shouldIndent=false;this._content+=this._indent}this._content+=str};Writer.prototype.toString=function(){return this._content}},{lodash:28}],9:[function(require,module,exports){var _=require("lodash");module.exports=_.clone(require("./lib"));module.exports.json=require("./lib/json");module.exports.alg=require("./lib/alg")},{"./lib":25,"./lib/alg":16,"./lib/json":26,lodash:28}],10:[function(require,module,exports){var _=require("lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{lodash:28}],11:[function(require,module,exports){var _=require("lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{lodash:28}],12:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"./dijkstra":13,lodash:28}],13:[function(require,module,exports){var _=require("lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":23,lodash:28}],14:[function(require,module,exports){var _=require("lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"./tarjan":21,lodash:28}],15:[function(require,module,exports){var _=require("lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":23,"../graph":24,lodash:28}],21:[function(require,module,exports){var _=require("lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{lodash:28}],22:[function(require,module,exports){var _=require("lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{lodash:28}],23:[function(require,module,exports){var _=require("lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(v,w,value,name){var valueSpecified=arguments.length>2;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{lodash:28}],25:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":24,"./version":27}],26:[function(require,module,exports){var _=require("lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":24,lodash:28}],27:[function(require,module,exports){module.exports="0.8.1"},{}],28:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f "+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index + * div.stage-metadata > + * div.[dot-file | incoming-edge | outgoing-edge] + * + * Output DOM hierarchy: + * div#dag-viz-graph > + * svg > + * g#cluster_stage_[stageId] + * + * Note that the input metadata is populated by o.a.s.ui.UIUtils.showDagViz. + * Any changes in the input format here must be reflected there. + */ +function renderDagViz(forJob) { + + // If there is not a dot file to render, fail fast and report error + if (metadataContainer().empty()) { + graphContainer().append("div").text( + "No visualization information available for this " + (forJob ? "job" : "stage")); + return; + } + + var svg = graphContainer().append("svg"); + if (forJob) { + renderDagVizForJob(svg); + } else { + renderDagVizForStage(svg); + } + + // Find cached RDDs + metadataContainer().selectAll(".cached-rdd").each(function(v) { + var nodeId = VizConstants.nodePrefix + d3.select(this).text(); + graphContainer().selectAll("#" + nodeId).classed("cached", true); + }); + + // Set the appropriate SVG dimensions to ensure that all elements are displayed + var boundingBox = svg.node().getBBox(); + svg.style("width", (boundingBox.width + VizConstants.svgMarginX) + "px"); + svg.style("height", (boundingBox.height + VizConstants.svgMarginY) + "px"); + + // Add labels to clusters because dagre-d3 doesn't do this for us + svg.selectAll("g.cluster rect").each(function() { + var rect = d3.select(this); + var cluster = d3.select(this.parentNode); + // Shift the boxes up a little to make room for the labels + rect.attr("y", toFloat(rect.attr("y")) - 10); + rect.attr("height", toFloat(rect.attr("height")) + 10); + var labelX = toFloat(rect.attr("x")) + toFloat(rect.attr("width")) - 5; + var labelY = toFloat(rect.attr("y")) + 15; + var labelText = cluster.attr("name").replace(VizConstants.clusterPrefix, ""); + cluster.append("text") + .attr("x", labelX) + .attr("y", labelY) + .attr("text-anchor", "end") + .text(labelText); + }); + + // We have shifted a few elements upwards, so we should fix the SVG views + var startX = -VizConstants.svgMarginX; + var startY = -VizConstants.svgMarginY; + var endX = toFloat(svg.style("width")) + VizConstants.svgMarginX; + var endY = toFloat(svg.style("height")) + VizConstants.svgMarginY; + var newViewBox = startX + " " + startY + " " + endX + " " + endY; + svg.attr("viewBox", newViewBox); + + // Lastly, apply some custom style to the DAG + styleDagViz(forJob); +} + +/* Render the RDD DAG visualization for a stage. */ +function renderDagVizForStage(svgContainer) { + var metadata = metadataContainer().select(".stage-metadata"); + var dot = metadata.select(".dot-file").text(); + var containerId = VizConstants.graphPrefix + metadata.attr("stageId"); + var container = svgContainer.append("g").attr("id", containerId); + renderDot(dot, container); +} + +/* + * Render the RDD DAG visualization for a job. + * + * Due to limitations in dagre-d3, each stage is rendered independently so that + * we have more control on how to position them. Unfortunately, this means we + * cannot rely on dagre-d3 to render edges that cross stages and must render + * these manually on our own. + */ +function renderDagVizForJob(svgContainer) { + var crossStageEdges = []; + + metadataContainer().selectAll(".stage-metadata").each(function(d, i) { + var metadata = d3.select(this); + var dot = metadata.select(".dot-file").text(); + var stageId = metadata.attr("stageId"); + var containerId = VizConstants.graphPrefix + stageId; + // TODO: handle stage attempts + var stageLink = + "/stages/stage/?id=" + stageId.replace(VizConstants.stagePrefix, "") + "&attempt=0"; + var container = svgContainer + .append("a").attr("xlink:href", stageLink) + .append("g").attr("id", containerId); + // Now we need to shift the container for this stage so it doesn't overlap + // with existing ones. We do not need to do this for the first stage. + if (i > 0) { + // Take into account the position and width of the last stage's container + var existingStages = stageClusters(); + if (!existingStages.empty()) { + var lastStage = existingStages[0].pop(); + var lastStageId = d3.select(lastStage).attr("id"); + var lastStageWidth = toFloat(d3.select("#" + lastStageId + " rect").attr("width")); + var lastStagePosition = getAbsolutePosition(lastStageId); + var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; + container.attr("transform", "translate(" + offset + ", 0)"); + } + } + renderDot(dot, container); + // If there are any incoming edges into this graph, keep track of them to render + // them separately later. Note that we cannot draw them now because we need to + // put these edges in a separate container that is on top of all stage graphs. + metadata.selectAll(".incoming-edge").each(function(v) { + var edge = d3.select(this).text().split(","); // e.g. 3,4 => [3, 4] + crossStageEdges.push(edge); + }); + }); + + // Draw edges that cross stages + if (crossStageEdges.length > 0) { + var container = svgContainer.append("g").attr("id", "cross-stage-edges"); + for (var i = 0; i < crossStageEdges.length; i++) { + var fromRDDId = crossStageEdges[i][0]; + var toRDDId = crossStageEdges[i][1]; + connectRDDs(fromRDDId, toRDDId, container); + } + } +} + +/* Render the dot file as an SVG in the given container. */ +function renderDot(dot, container) { + var escaped_dot = dot + .replace(/</g, "<") + .replace(/>/g, ">") + .replace(/"/g, "\""); + var g = graphlibDot.read(escaped_dot); + var renderer = new dagreD3.render(); + renderer(container, g); +} + +/* Style the visualization we just rendered. */ +function styleDagViz(forJob) { + graphContainer().selectAll("svg g.cluster rect") + .style("fill", "white") + .style("stroke", VizConstants.rddOperationColor) + .style("stroke-width", "4px") + .style("stroke-opacity", "0.5"); + graphContainer().selectAll("svg g.cluster text") + .attr("fill", VizConstants.clusterLabelColor) + .attr("font-size", "11px"); + graphContainer().selectAll("svg path") + .style("stroke", VizConstants.edgeColor) + .style("stroke-width", VizConstants.edgeWidth); + stageClusters() + .select("rect") + .style("stroke", VizConstants.stageColor) + .style("strokeWidth", "6px"); + + // Put an arrow at the end of every edge + // We need to do this because we manually render some edges ourselves + // For these edges, we borrow the arrow marker generated by dagre-d3 + var dagreD3Marker = graphContainer().select("svg g.edgePaths marker").node(); + graphContainer().select("svg") + .append(function() { return dagreD3Marker.cloneNode(true); }) + .attr("id", "marker-arrow") + .select("path") + .attr("fill", VizConstants.edgeColor) + .attr("strokeWidth", "0px"); + graphContainer().selectAll("svg g > path").attr("marker-end", "url(#marker-arrow)"); + graphContainer().selectAll("svg g.edgePaths def").remove(); // We no longer need these + + // Apply any job or stage specific styles + if (forJob) { + styleDagVizForJob(); + } else { + styleDagVizForStage(); + } +} + +/* Apply job-page-specific style to the visualization. */ +function styleDagVizForJob() { + graphContainer().selectAll("svg g.node circle") + .style("fill", VizConstants.rddColor); + // TODO: add a legend to explain what a highlighted dot means + graphContainer().selectAll("svg g.cached circle") + .style("fill", VizConstants.rddCachedColor); + graphContainer().selectAll("svg g#cross-stage-edges path") + .style("fill", "none"); +} + +/* Apply stage-page-specific style to the visualization. */ +function styleDagVizForStage() { + graphContainer().selectAll("svg g.node rect") + .style("fill", "none") + .style("stroke", VizConstants.rddColor) + .style("stroke-width", "2px") + .attr("rx", "5") // round corners + .attr("ry", "5"); + // TODO: add a legend to explain what a highlighted RDD means + graphContainer().selectAll("svg g.cached rect") + .style("stroke", VizConstants.rddCachedColor); + graphContainer().selectAll("svg g.node g.label text tspan") + .style("fill", VizConstants.rddColor); +} + +/* + * (Job page only) Helper method to compute the absolute + * position of the group element identified by the given ID. + */ +function getAbsolutePosition(groupId) { + var obj = d3.select("#" + groupId).filter("g"); + var _x = 0, _y = 0; + while (!obj.empty()) { + var transformText = obj.attr("transform"); + var translate = d3.transform(transformText).translate + _x += translate[0]; + _y += translate[1]; + obj = d3.select(obj.node().parentNode).filter("g") + } + return { x: _x, y: _y }; +} + +/* (Job page only) Connect two RDD nodes with a curved edge. */ +function connectRDDs(fromRDDId, toRDDId, container) { + var fromNodeId = VizConstants.nodePrefix + fromRDDId; + var toNodeId = VizConstants.nodePrefix + toRDDId + var fromPos = getAbsolutePosition(fromNodeId); + var toPos = getAbsolutePosition(toNodeId); + + // On the job page, RDDs are rendered as dots (circles). When rendering the path, + // we need to account for the radii of these circles. Otherwise the arrow heads + // will bleed into the circle itself. + var delta = toFloat(graphContainer() + .select("g.node#" + toNodeId) + .select("circle") + .attr("r")); + if (fromPos.x < toPos.x) { + fromPos.x += delta; + toPos.x -= delta; + } else if (fromPos.x > toPos.x) { + fromPos.x -= delta; + toPos.x += delta; + } + + if (fromPos.y == toPos.y) { + // If they are on the same rank, curve the middle part of the edge + // upward a little to avoid interference with things in between + // e.g. _______ + // _____/ \_____ + var points = [ + [fromPos.x, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.2, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.3, fromPos.y - 20], + [fromPos.x + (toPos.x - fromPos.x) * 0.7, fromPos.y - 20], + [fromPos.x + (toPos.x - fromPos.x) * 0.8, toPos.y], + [toPos.x, toPos.y] + ]; + } else { + // Otherwise, draw a curved edge that flattens out on both ends + // e.g. _____ + // / + // | + // _____/ + var points = [ + [fromPos.x, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.4, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.6, toPos.y], + [toPos.x, toPos.y] + ]; + } + + var line = d3.svg.line().interpolate("basis"); + container.append("path").datum(points).attr("d", line); +} + +/* Helper d3 accessor to clusters that represent stages. */ +function stageClusters() { + return graphContainer().selectAll("g.cluster").filter(function() { + return d3.select(this).attr("id").indexOf(VizConstants.stageClusterPrefix) > -1; + }); +} + +/* Helper method to convert attributes to numeric values. */ +function toFloat(f) { + return parseFloat(f.replace(/px$/, "")); +} + diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css new file mode 100644 index 0000000000000..35ef14e5aaf1a --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -0,0 +1,182 @@ +/* + * 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. + */ + +div#application-timeline, div#job-timeline { + margin-bottom: 30px; +} + +#application-timeline div.legend-area { + margin-top: 5px; +} + +.vis.timeline div.content { + width: 100%; +} + +.vis.timeline .item.stage { + cursor: pointer; +} + +.vis.timeline .item.stage.succeeded { + background-color: #D5DDF6; +} + +.vis.timeline .item.stage.succeeded.selected { + background-color: #D5DDF6; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.completed-stage-legend { + fill: #D5DDF6; + stroke: #97B0F8; +} + +.vis.timeline .item.stage.failed { + background-color: #FF5475; +} + +.vis.timeline .item.stage.failed.selected { + background-color: #FF5475; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.failed-stage-legend { + fill: #FF5475; + stroke: #97B0F8; +} + +.vis.timeline .item.stage.running { + background-color: #FDFFCA; +} + +.vis.timeline .item.stage.running.selected { + background-color: #FDFFCA; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.active-stage-legend { + fill: #FDFFCA; + stroke: #97B0F8; +} + +.vis.timeline .item.job { + cursor: pointer; +} + +.vis.timeline .item.job.succeeded { + background-color: #D5DDF6; +} + +.vis.timeline .item.job.succeeded.selected { + background-color: #D5DDF6; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.succeeded-job-legend { + fill: #D5DDF6; + stroke: #97B0F8; +} + +.vis.timeline .item.job.failed { + background-color: #FF5475; +} + +.vis.timeline .item.job.failed.selected { + background-color: #FF5475; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.failed-job-legend { + fill: #FF5475; + stroke: #97B0F8; +} + +.vis.timeline .item.job.running { + background-color: #FDFFCA; +} + +.vis.timeline .item.job.running.selected { + background-color: #FDFFCA; + border-color: #97B0F8; + z-index: auto; +} + +.legend-area rect.running-job-legend { + fill: #FDFFCA; + stroke: #97B0F8; +} + +.vis.timeline .item.executor.added { + background-color: #D5DDF6; +} + +.legend-area rect.executor-added-legend { + fill: #D5DDF6; + stroke: #97B0F8; +} + +.vis.timeline .item.executor.removed { + background-color: #EBCA59; +} + +.legend-area rect.executor-removed-legend { + fill: #EBCA59; + stroke: #97B0F8; +} + +.vis.timeline .item.executor.selected { + border-color: #FFC200; + background-color: #FFF785; + z-index: 2; +} + +tr.corresponding-item-hover>td, tr.corresponding-item-hover>th { + background-color: #FFE1FA !important; +} + +#application-timeline.collapsed { + display: none; +} + +#job-timeline.collapsed { + display: none; +} + +.control-panel { + margin-bottom: 5px; +} + +span.expand-application-timeline, span.expand-job-timeline { + cursor: pointer; +} + +.control-panel input+span { + cursor: pointer; +} + +.vis.timeline .item.range .content { + position: unset; +} + +.vis.timeline .item .tooltip-inner { + max-width: unset !important; +} 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 new file mode 100644 index 0000000000000..e4a891d47f035 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -0,0 +1,168 @@ +/* + * 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. + */ + +function drawApplicationTimeline(groupArray, eventObjArray, startTime) { + var groups = new vis.DataSet(groupArray); + var items = new vis.DataSet(eventObjArray); + var container = $("#application-timeline")[0]; + var options = { + groupOrder: function(a, b) { + return a.value - b.value + }, + editable: false, + showCurrentTime: false, + min: startTime, + zoomable: false + }; + + var applicationTimeline = new vis.Timeline(container); + applicationTimeline.setOptions(options); + applicationTimeline.setGroups(groups); + applicationTimeline.setItems(items); + + setupZoomable("#application-timeline-zoom-lock", applicationTimeline); + setupExecutorEventAction(); + + function setupJobEventAction() { + $(".item.range.job.application-timeline-object").each(function() { + var getJobId = function(baseElem) { + var jobIdText = $($(baseElem).find(".application-timeline-content")[0]).text(); + var jobId = jobIdText.match("\\(Job (\\d+)\\)")[1]; + return jobId; + }; + + $(this).click(function() { + window.location.href = "job/?id=" + getJobId(this); + }); + + $(this).hover( + function() { + $("#job-" + getJobId(this)).addClass("corresponding-item-hover"); + $($(this).find("div.application-timeline-content")[0]).tooltip("show"); + }, + function() { + $("#job-" + getJobId(this)).removeClass("corresponding-item-hover"); + $($(this).find("div.application-timeline-content")[0]).tooltip("hide"); + } + ); + }); + } + + setupJobEventAction(); + + $("span.expand-application-timeline").click(function() { + $("#application-timeline").toggleClass('collapsed'); + + // Switch the class of the arrow from open to closed. + $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-open'); + $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-closed'); + }); +} + +function drawJobTimeline(groupArray, eventObjArray, startTime) { + var groups = new vis.DataSet(groupArray); + var items = new vis.DataSet(eventObjArray); + var container = $('#job-timeline')[0]; + var options = { + groupOrder: function(a, b) { + return a.value - b.value; + }, + editable: false, + showCurrentTime: false, + min: startTime, + zoomable: false, + }; + + var jobTimeline = new vis.Timeline(container); + jobTimeline.setOptions(options); + jobTimeline.setGroups(groups); + jobTimeline.setItems(items); + + setupZoomable("#job-timeline-zoom-lock", jobTimeline); + setupExecutorEventAction(); + + function setupStageEventAction() { + $(".item.range.stage.job-timeline-object").each(function() { + var getStageIdAndAttempt = function(baseElem) { + var stageIdText = $($(baseElem).find(".job-timeline-content")[0]).text(); + var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)")[1].split("."); + return stageIdAndAttempt; + }; + + $(this).click(function() { + var idAndAttempt = getStageIdAndAttempt(this); + var id = idAndAttempt[0]; + var attempt = idAndAttempt[1]; + window.location.href = "../../stages/stage/?id=" + id + "&attempt=" + attempt; + }); + + $(this).hover( + function() { + var idAndAttempt = getStageIdAndAttempt(this); + var id = idAndAttempt[0]; + var attempt = idAndAttempt[1]; + $("#stage-" + id + "-" + attempt).addClass("corresponding-item-hover"); + $($(this).find("div.job-timeline-content")[0]).tooltip("show"); + }, + function() { + var idAndAttempt = getStageIdAndAttempt(this); + var id = idAndAttempt[0]; + var attempt = idAndAttempt[1]; + $("#stage-" + id + "-" + attempt).removeClass("corresponding-item-hover"); + $($(this).find("div.job-timeline-content")[0]).tooltip("hide"); + } + ); + }); + } + + setupStageEventAction(); + + $("span.expand-job-timeline").click(function() { + $("#job-timeline").toggleClass('collapsed'); + + // Switch the class of the arrow from open to closed. + $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-open'); + $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-closed'); + }); +} + +function setupExecutorEventAction() { + $(".item.box.executor").each(function () { + $(this).hover( + function() { + $($(this).find(".executor-event-content")[0]).tooltip("show"); + }, + function() { + $($(this).find(".executor-event-content")[0]).tooltip("hide"); + } + ); + }); +} + +function setupZoomable(id, timeline) { + $(id + '>input[type="checkbox"]').click(function() { + if (this.checked) { + timeline.setOptions({zoomable: false}); + } else { + timeline.setOptions({zoomable: true}); + } + }); + + $(id + ">span").click(function() { + $(this).parent().find('input:checkbox').trigger('click'); + }); +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.map b/core/src/main/resources/org/apache/spark/ui/static/vis.map new file mode 100644 index 0000000000000..9be8b618989b7 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.map @@ -0,0 +1 @@ +{"version":3,"file":"vis.map","sources":["./dist/vis.js"],"names":["root","factory","exports","module","define","amd","this","modules","__webpack_require__","moduleId","installedModules","id","loaded","call","m","c","p","util","DOMutil","DataSet","DataView","Queue","Graph3d","graph3d","Camera","Filter","Point2d","Point3d","Slider","StepNumber","Timeline","Graph2d","timeline","DateUtil","DataStep","Range","stack","TimeStep","components","items","Item","BackgroundItem","BoxItem","PointItem","RangeItem","Component","CurrentTime","CustomTime","DataAxis","GraphGroup","Group","BackgroundGroup","ItemSet","Legend","LineGraph","TimeAxis","Network","network","Edge","Groups","Images","Node","Popup","dotparser","gephiParser","Graph","Error","moment","hammer","Hammer","isNumber","object","Number","isString","String","isDate","Date","match","ASPDateRegex","exec","isNaN","parse","isDataTable","google","visualization","DataTable","randomUUID","S4","Math","floor","random","toString","extend","a","i","len","arguments","length","other","prop","hasOwnProperty","selectiveExtend","props","Array","isArray","selectiveDeepExtend","b","TypeError","constructor","Object","undefined","deepExtend","selectiveNotDeepExtend","indexOf","equalArray","convert","type","Boolean","valueOf","isMoment","toDate","getType","toISOString","value","getAbsoluteLeft","elem","getBoundingClientRect","left","getAbsoluteTop","top","addClassName","className","classes","split","push","join","removeClassName","index","splice","forEach","callback","toArray","array","updateProperty","key","addEventListener","element","action","listener","useCapture","navigator","userAgent","attachEvent","removeEventListener","detachEvent","preventDefault","event","window","returnValue","getTarget","target","srcElement","nodeType","parentNode","option","asBoolean","defaultValue","asNumber","asString","asSize","asElement","hexToRGB","hex","shorthandRegex","replace","r","g","result","parseInt","RGBToHex","red","green","blue","slice","parseColor","color","isValidRGB","rgb","substr","isValidHex","hsv","hexToHSV","lighterColorHSV","h","s","v","min","darkerColorHSV","darkerColorHex","HSVToHex","lighterColorHex","background","border","highlight","hover","RGBToHSV","minRGB","maxRGB","max","d","hue","saturation","cssUtil","cssText","styles","style","trim","parts","keys","map","addCssText","currentStyles","newStyles","removeCssText","removeStyles","HSVToRGB","f","q","t","isOk","test","selectiveBridgeObject","fields","referenceObject","objectTo","create","bridgeObject","mergeOptions","mergeTarget","options","enabled","binarySearchCustom","orderedItems","searchFunction","field","field2","maxIterations","iteration","low","high","middle","item","searchResult","binarySearchValue","sidePreference","prevValue","nextValue","easeInOutQuad","start","end","duration","change","easingFunctions","linear","easeInQuad","easeOutQuad","easeInCubic","easeOutCubic","easeInOutCubic","easeInQuart","easeOutQuart","easeInOutQuart","easeInQuint","easeOutQuint","easeInOutQuint","prepareElements","JSONcontainer","elementType","redundant","used","cleanupElements","removeChild","getSVGElement","svgContainer","shift","document","createElementNS","appendChild","getDOMElement","DOMContainer","insertBefore","createElement","drawPoint","x","y","group","point","drawPoints","setAttributeNS","size","drawBar","width","height","rect","data","_options","_data","_fieldId","fieldId","_type","_subscribers","add","setOptions","prototype","queue","_queue","destroy","on","subscribers","subscribe","off","filter","unsubscribe","_trigger","params","senderId","concat","subscriber","addedIds","me","_addItem","columns","_getColumnNames","row","rows","getNumberOfRows","col","cols","getValue","update","updatedIds","updatedData","addOrUpdate","_updateItem","get","ids","firstType","returnType","allowedValues","itemId","_getItem","order","_sort","_filterFields","_appendRow","getIds","getDataSet","mappedItems","filteredItem","name","sort","av","bv","remove","removedId","removedIds","_remove","clear","maxField","itemField","minField","distinct","values","fieldType","count","exists","types","raw","converted","JSON","stringify","dataTable","getNumberOfColumns","getColumnId","getColumnLabel","addRow","setValue","_ids","_onEvent","apply","setData","viewOptions","getArguments","defaultFilter","dataSet","added","updated","removed","delay","Infinity","_timeout","_extended","_flushIfNeeded","flush","methods","original","method","args","fn","context","entry","clearTimeout","setTimeout","container","SyntaxError","containerElement","margin","defaultXCenter","defaultYCenter","xLabel","yLabel","zLabel","passValueFn","xValueLabel","yValueLabel","zValueLabel","filterLabel","legendLabel","STYLE","DOT","showPerspective","showGrid","keepAspectRatio","showShadow","showGrayBottom","showTooltip","verticalRatio","animationInterval","animationPreload","camera","eye","dataPoints","colX","colY","colZ","colValue","colFilter","xMin","xStep","xMax","yMin","yStep","yMax","zMin","zStep","zMax","valueMin","valueMax","xBarWidth","yBarWidth","colorAxis","colorGrid","colorDot","colorDotBorder","getMouseX","clientX","targetTouches","getMouseY","clientY","Emitter","_setScale","scale","z","xCenter","yCenter","zCenter","setArmLocation","_convert3Dto2D","point3d","translation","_convertPointToTranslation","_convertTranslationToScreen","ax","ay","az","cx","getCameraLocation","cy","cz","sinTx","sin","getCameraRotation","cosTx","cos","sinTy","cosTy","sinTz","cosTz","dx","dy","dz","bx","by","ex","ey","ez","getArmLength","xcenter","frame","canvas","clientWidth","ycenter","_setBackgroundColor","backgroundColor","fill","stroke","strokeWidth","borderColor","borderWidth","borderStyle","BAR","BARCOLOR","BARSIZE","DOTLINE","DOTCOLOR","DOTSIZE","GRID","LINE","SURFACE","_getStyleNumber","styleName","_determineColumnIndexes","counter","column","getDistinctValues","distinctValues","getColumnRange","minMax","_dataInitialize","rawData","_onChange","dataFilter","setOnLoadCallback","redraw","withBars","defaultXBarWidth","dataX","defaultYBarWidth","dataY","xRange","defaultXMin","defaultXMax","defaultXStep","yRange","defaultYMin","defaultYMax","defaultYStep","zRange","defaultZMin","defaultZMax","defaultZStep","valueRange","defaultValueMin","defaultValueMax","_getDataPoints","obj","sortNumber","dataMatrix","xIndex","yIndex","trans","screen","bottom","pointRight","pointTop","pointCross","hasChildNodes","firstChild","position","overflow","noCanvas","fontWeight","padding","innerHTML","onmousedown","_onMouseDown","ontouchstart","_onTouchStart","onmousewheel","_onWheel","ontooltip","_onTooltip","onkeydown","setSize","_resizeCanvas","clientHeight","animationStart","slider","play","animationStop","stop","_resizeCenter","charAt","parseFloat","setCameraPosition","pos","horizontal","vertical","setArmRotation","distance","setArmLength","getCameraPosition","getArmRotation","_readData","_redrawFilter","animationAutoStart","cameraPosition","styleNumber","tooltip","showAnimationControls","_redrawSlider","_redrawClear","_redrawAxis","_redrawDataGrid","_redrawDataLine","_redrawDataBar","_redrawDataDot","_redrawInfo","_redrawLegend","ctx","getContext","clearRect","widthMin","widthMax","dotSize","right","lineWidth","font","ymin","ymax","_hsv2rgb","strokeStyle","beginPath","moveTo","lineTo","strokeRect","fillStyle","closePath","gridLineLen","step","getCurrent","next","textAlign","textBaseline","fillText","label","visible","setValues","setPlayInterval","onchange","getIndex","selectValue","setOnChangeCallback","lineStyle","getLabel","getSelectedValue","from","to","prettyStep","text","xText","yText","zText","offset","xOffset","yOffset","xMin2d","xMax2d","gridLenX","gridLenY","textMargin","armAngle","H","S","V","R","G","B","C","Hi","X","abs","cross","topSideVisible","zAvg","transBottom","dist","sortDepth","aDiff","subtract","bDiff","crossproduct","crossProduct","radius","arc","PI","j","surface","corners","xWidth","yWidth","surfaces","center","avg","transCenter","diff","leftButtonDown","_onMouseUp","which","button","touchDown","startMouseX","startMouseY","startStart","startEnd","startArmRotation","cursor","onmousemove","_onMouseMove","onmouseup","diffX","diffY","horizontalNew","verticalNew","snapAngle","snapValue","round","parameters","emit","boundingRect","mouseX","mouseY","tooltipTimeout","_hideTooltip","dataPoint","_dataPointFromXY","_showTooltip","ontouchmove","_onTouchMove","ontouchend","_onTouchEnd","delta","wheelDelta","detail","oldLength","newLength","_insideTriangle","triangle","sign","as","bs","cs","distMax","closestDataPoint","closestDist","triangle1","triangle2","distX","distY","sqrt","content","line","dot","dom","borderRadius","boxShadow","borderLeft","contentWidth","offsetWidth","contentHeight","offsetHeight","lineHeight","dotWidth","dotHeight","armLocation","armRotation","armLength","cameraLocation","cameraRotation","calculateCameraOrientation","rot","graph","onLoadCallback","loadInBackground","isLoaded","getLoadedProgress","getColumn","getValues","dataView","progress","sub","sum","prev","bar","MozBorderRadius","slide","onclick","togglePlay","onChangeCallback","playTimeout","playInterval","playLoop","setIndex","playNext","interval","clearInterval","getPlayInterval","setPlayLoop","doLoop","onChange","indexToLeft","startClientX","startSlideX","leftToIndex","_start","_end","_step","precision","_current","setRange","setStep","calculatePrettyStep","log10","log","LN10","step1","pow","step2","step5","toPrecision","getStep","groups","forthArgument","defaultOptions","autoResize","orientation","maxHeight","minHeight","_create","body","domProps","emitter","bind","hiddenDates","snap","toScreen","_toScreen","toGlobalScreen","_toGlobalScreen","toTime","_toTime","toGlobalTime","_toGlobalTime","range","timeAxis","currentTime","customTime","itemSet","itemsData","groupsData","setGroups","setItems","Core","newDataSet","initialLoad","dataRange","_getDataRange","setWindow","animate","fit","setSelection","focus","getSelection","itemData","e","getItemRange","dataset","minItem","maxStartItem","maxEndItem","linegraph","getLegend","groupId","isGroupVisible","visibility","convertHiddenOptions","repeat","dateItem","updateHiddenDates","centerContainer","totalRange","pixelTime","startDate","endDate","_d","runUntil","clone","day","dayOfYear","year","dayOffset","date","month","console","removeDuplicates","startHidden","isHidden","endHidden","rangeStart","rangeEnd","hidden","startToFront","endToFront","_applyRange","safeDates","printDates","dates","stepOverHiddenDates","timeStep","previousTime","stepInHidden","currentValue","current","newValue","switchedYear","switchedMonth","switchedDay","time","conversion","getHiddenDurationBetween","correctTimeForHidden","hiddenDuration","totalDuration","partialDuration","accumulatedHiddenDuration","getAccumulatedHiddenDuration","newTime","getHiddenDurationBefore","timeOffset","requiredDuration","previousPoint","snapAwayFromHidden","direction","correctionEnabled","minimumStep","containerHeight","customRange","alignZeros","autoScale","stepIndex","marginStart","marginEnd","deadSpace","majorSteps","minorSteps","setMinimumStep","setFirst","safeSize","minimumStepValue","orderOfMagnitude","minorStepIdx","magnitudefactor","solutionFound","stepSize","niceStart","niceEnd","roundToMinor","marginRange","rounded","hasNext","previous","decimals","exp","cnt","isMajor","now","hours","minutes","seconds","milliseconds","deltaDifference","scaleOffset","moveable","zoomable","zoomMin","zoomMax","touch","animateTimer","_onDragStart","_onDrag","_onDragEnd","_onHold","_onMouseWheel","_onTouch","_onPinch","validateDirection","getPointer","byUser","_cancelAnimation","initStart","initEnd","initTime","anyChanged","dragging","done","changed","newStart","newEnd","getRange","totalHidden","previousDelta","allowDragging","deltaX","deltaY","diffRange","safeStart","safeEnd","pointer","pageX","pageY","pointerDate","_pointerToDate","zoom","centerDate","hiddenDurationBefore","hiddenDurationAfter","move","EPSILON","orderByStart","orderByEnd","aTime","bTime","force","iMax","axis","collidingItem","jj","collision","nostack","subgroups","newTop","subgroup","format","FORMAT","minorLabels","millisecond","second","minute","hour","weekday","majorLabels","setFormat","defaultFormat","first","setFullYear","getFullYear","setMonth","setDate","setHours","setMinutes","setSeconds","setMilliseconds","getMilliseconds","getSeconds","getMinutes","getHours","getDate","getMonth","setScale","newScale","newStep","setAutoScale","enable","stepYear","stepMonth","stepDay","stepHour","stepMinute","stepSecond","stepMillisecond","getLabelMinor","getLabelMajor","getClassName","even","today","isSame","currentWeek","currentMonth","currentYear","locale","lang","toLowerCase","_isResized","resized","_previousWidth","_previousHeight","showCurrentTime","locales","parent","backgroundVertical","title","toUpperCase","substring","currentTimeTimer","setCurrentTime","getCurrentTime","showCustomTime","eventParams","drag","setCustomTime","getCustomTime","stopPropagation","svg","linegraphOptions","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","linegraphSVG","DOMelements","lines","labels","conversionFactor","minWidth","stepPixels","stepPixelsForced","zeroCrossing","lineOffset","master","svgElements","iconsRemoved","amountOfGroups","lineContainer","scrollTop","addGroup","graphOptions","updateGroup","removeGroup","hide","show","display","_redrawGroupIcons","iconHeight","iconOffset","drawIcon","_cleanupIcons","backgroundHorizontal","activeGroups","_calculateCharSize","minorLabelHeight","minorCharHeight","majorLabelHeight","majorCharHeight","minorLineWidth","minorLineHeight","majorLineWidth","majorLineHeight","_redrawLabels","_redrawTitle","amountOfSteps","stepDifference","zeroStepDifference","valueAtZero","marginStartPos","maxLabelSize","_redrawLabel","_redrawLine","titleWidth","titleCharHeight","convertValue","invertedValue","convertedValue","characterHeight","largestWidth","majorCharWidth","minorCharWidth","textMinor","createTextNode","measureCharMinor","textMajor","measureCharMajor","textTitle","measureCharTitle","titleCharWidth","groupsUsingDefaultStyles","usingDefaultStyle","zeroPosition","Line","Bar","Points","setZeroPosition","catmullRom","parametrization","alpha","SVGcontainer","path","fillPath","fillHeight","outline","shaded","barWidth","bar1Height","bar2Height","icon","yAxisOrientation","getYRange","groupData","draw","framework","subgroupIndex","subgroupOrderer","subgroupOrder","visibleItems","byStart","byEnd","checkRangedItems","inner","foreground","marker","Element","getLabelWidth","restack","_updateVisibleItems","markerHeight","lastMarkerHeight","dirty","displayed","_calculateHeight","offsetTop","offsetLeft","ii","repositionY","resetSubgroups","labelSet","setParent","orderSubgroups","_checkIfVisible","sortArray","sortField","removeFromDataSet","removeItem","startArray","endArray","oldVisibleItems","visibleItemsLookup","lowerBound","upperBound","_checkIfVisibleWithReference","initialPosByStart","_traceVisible","initialPosByEnd","repositionX","initialPos","breakCondition","isVisible","align","groupOrder","selectable","editable","updateTime","onAdd","onUpdate","onMove","onRemove","onMoving","itemOptions","itemListeners","_onAdd","_onUpdate","_onRemove","groupListeners","_onAddGroups","_onUpdateGroups","_onRemoveGroups","groupIds","selection","stackDirty","touchParams","UNGROUPED","BACKGROUND","box","_updateUngrouped","backgroundGroup","isFirst","_onSelectItem","_onMultiSelectItem","_onAddItem","addCallback","Function","markDirty","unselect","select","getVisibleItems","rawVisibleItems","_deselect","_orderGroups","visibleInterval","zoomed","lastVisibleInterval","lastWidth","firstGroup","_firstGroup","firstMargin","nonFirstMargin","groupMargin","groupResized","firstGroupIndex","firstGroupId","ungrouped","_getGroupId","getLabelSet","oldItemsData","getItems","_order","getGroups","_getType","_removeItem","groupOptions","oldGroupId","oldGroup","_constructByEndArray","itemFromTarget","dragLeftItem","dragRightItem","itemProps","selected","initialX","newProps","initial","groupFromTarget","_updateItemProps","_moveToGroup","changes","ctrlKey","srcEvent","shiftKey","oldSelection","newSelection","xAbs","newItem","_getItemRange","_item","itemSetFromTarget","side","iconSize","iconSpacing","textArea","scrollableHeight","drawLegendIcons","getComputedStyle","paddingTop","defaultGroup","sampling","graphHeight","barChart","handleOverlap","dataAxis","legend","abortedGraphUpdate","updateSVGheight","updateSVGheightOnResize","lastStart","COUNTER","BarGraphFunctions","yAxisLeft","yAxisRight","legendLeft","legendRight","_updateAllGroupData","_updateGroup","groupsContent","ungroupedCounter","forceGraphUpdate","_updateGraph","rangePerPixelInv","preprocessedGroupData","processedGroupData","groupRanges","changeCalled","minDate","maxDate","_getRelevantData","_applySampling","_convertXcoordinates","_getYRanges","_updateYAxis","MAX_CYCLES","_convertYcoordinates","dataContainer","guess","increment","amountOfPoints","xDistance","pointsPerPixel","ceil","sampledData","barCombinedDataLeft","barCombinedDataRight","getStackedBarYRange","minVal","maxVal","yAxisLeftUsed","yAxisRightUsed","minLeft","minRight","maxLeft","maxRight","ignore","_toggleAxisVisiblity","drawIcons","axisUsed","datapoints","xValue","yValue","extractedData","svgHeight","majorTexts","minorTexts","lineTop","parentChanged","foregroundNextSibling","nextSibling","backgroundNextSibling","_repaintLabels","timeLabelsize","cur","prevLine","xPrev","xFirstMajorLabel","_repaintMinorText","_repaintMajorText","_repaintMajorLine","_repaintMinorLine","leftTime","leftText","widthText","arr","pop","childNodes","nodeValue","_repaintDeleteButton","anchor","deleteButton","_updateContents","template","_updateTitle","removeAttribute","_updateDataAttributes","dataAttributes","attributes","setAttribute","_updateStyle","emptyContent","baseClassName","onTop","itemSubgroup","itemSetHeight","marginLeft","maxWidth","_repaintDragLeft","_repaintDragRight","contentLeft","parentWidth","boxWidth","dragLeft","dragRight","_determineBrowserMethod","_initializeMixinLoaders","renderRefreshRate","renderTimestep","renderTime","physicsTime","runDoubleSpeed","physicsDiscreteStepsize","initializing","triggerFunctions","edit","editEdge","connect","del","nodes","mass","radiusMin","radiusMax","shape","image","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","level","borderWidthSelected","edges","widthSelectionMultiplier","hoverWidth","labelAlignment","arrowScaleFactor","dash","gap","altLength","inheritColor","configurePhysics","physics","barnesHut","thetaInverted","gravitationalConstant","centralGravity","springLength","springConstant","damping","repulsion","nodeDistance","hierarchicalRepulsion","clustering","initialMaxNodes","clusterThreshold","reduceToNodes","chainThreshold","clusterEdgeThreshold","sectorThreshold","screenSizeThreshold","fontSizeMultiplier","maxFontSize","forceAmplification","distanceAmplification","edgeGrowth","nodeScaling","maxNodeSizeIncrements","activeAreaBoxSize","clusterLevelDifference","navigation","keyboard","speed","dataManipulation","initiallyVisible","hierarchicalLayout","levelSeparation","nodeSpacing","layout","freezeForStabilization","smoothCurves","dynamic","roundness","maxVelocity","minVelocity","stabilize","stabilizationIterations","zoomExtentOnStabilize","dragNetwork","dragNodes","hideEdgesOnDrag","hideNodesOnDrag","constants","pixelRatio","hoverObj","controlNodesActive","navigationHammers","existing","_new","animationSpeed","animationEasingFunction","easingTime","sourceScale","targetScale","sourceTranslation","targetTranslation","lockedOnNodeId","lockedOnNodeOffset","touchTime","images","setOnloadCallback","_redraw","xIncrement","yIncrement","zoomIncrement","_loadPhysicsSystem","_loadSectorSystem","_loadClusterSystem","_loadSelectionSystem","_loadHierarchySystem","_setTranslation","freezeSimulation","cachedFunctions","startedStabilization","stabilized","draggingNodes","calculationNodes","calculationNodeIndices","nodeIndices","canvasTopLeft","canvasBottomRight","pointerPosition","areaCenter","previousScale","nodesData","edgesData","nodesListeners","_addNodes","_updateNodes","_removeNodes","edgesListeners","_addEdges","_updateEdges","_removeEdges","moving","timer","_setupHierarchicalLayout","zoomExtent","startWithClustering","keycharm","hammerUtil","MixinLoader","Activator","browserType","requiresTimeout","_getScriptPath","scripts","getElementsByTagName","src","_getRange","node","minY","maxY","minX","maxX","nodeId","boundingBox","_findCenter","animationOptions","initialZoom","disableStart","zoomLevel","numberOfNodes","factor","yDistance","xZoomLevel","yZoomLevel","animation","_updateNodeIndexList","_clearNodeIndexList","idx","_createManipulatorBar","dotData","DOTToGraph","gephi","gephiData","parseGephi","_setNodes","_setEdges","_putDataInSector","_resetLevels","_stabilize","onEdit","onEditEdge","onConnect","onDelete","editMode","newColorObj","groupname","clickToUse","activator","_createKeyBinds","_loadNavigationControls","_loadManipulationSystem","_configureSmoothCurves","devicePixelRatio","webkitBackingStorePixelRatio","mozBackingStorePixelRatio","msBackingStorePixelRatio","oBackingStorePixelRatio","backingStorePixelRatio","setTransform","pinch","set","_onTap","_onDoubleTap","onTouch","_onMouseMoveTitle","reset","isActive","_moveUp","_yStopMoving","_moveDown","_moveLeft","_xStopMoving","_moveRight","_zoomIn","_stopZoom","_zoomOut","_deleteSelected","_cleanupPhysicsConfiguration","_recursiveDOMDelete","DOMobject","_getPointer","pinched","_getScale","_handleTouch","_handleDragStart","_getNodeAt","_getTranslation","isSelected","_selectObject","nodeIds","objectId","selectionObj","xFixed","yFixed","_handleOnDrag","releaseNode","_XconvertDOMtoCanvas","_XconvertCanvasToDOM","_YconvertDOMtoCanvas","_YconvertCanvasToDOM","_handleDragEnd","_handleTap","_handleDoubleTap","_handleOnHold","_onRelease","_handleOnRelease","_zoom","scaleOld","preScaleDragPointer","DOMtoCanvas","scaleFrac","tx","ty","updateClustersDefault","postScaleDragPointer","canvasToDOM","popupObj","_checkHidePopup","checkShow","_checkShowPopup","popupTimer","edgeId","_getEdgeAt","_hoverObject","_blurObject","lastPopupNode","nodeUnderCursor","overlappingNodes","isOverlappingWith","getTitle","overlappingEdges","edge","connected","popup","setPosition","setText","emitEvent","oldWidth","oldHeight","oldNodesData","_updateSelection","angle","_updateCalculationNodes","_reconnectEdges","_updateValueRange","updateLabels","changedData","setProperties","properties","oldEdgesData","oldEdge","disconnect","showInternalIds","_createBezierNodes","via","sectors","dynamicEdges","setValueRange","w","save","translate","_doInAllSectors","restore","offsetX","offsetY","_drawNodes","alwaysShow","setScaleAndPos","inArea","sMax","_drawEdges","_drawControlNodes","_freezeDefinedNodes","_physicsTick","_restoreFrozenNodes","fixedData","_isMoving","vmin","isMoving","_discreteStepNodes","nodesPresent","discreteStepLimited","discreteStep","vminCorrected","_revertPhysicsState","revertPosition","_revertPhysicsTick","_doInAllActiveSectors","_doInSupportSector","mainMovingStatus","supportMovingStatus","mainMoving","_animationStep","_handleNavigation","startTime","renderStartTime","requestAnimationFrame","mozRequestAnimationFrame","webkitRequestAnimationFrame","msRequestAnimationFrame","iterations","toggleFreeze","parentEdgeId","internalMultiplier","positionBezierNode","mixin","storePosition","storePositions","dataArray","allowedToMoveX","allowedToMoveY","getPositions","focusOnNode","nodePosition","lockedOnNode","easingFunction","animateView","locked","_transitionRedraw","viewCenter","distanceFromCenter","_classicRedraw","_lockedRedraw","active","getScale","getCenterCoordinates","getBoundingBox","networkConstants","fromId","toId","widthSelected","labelDimensions","yLine","dirtyLabel","fromBackup","toBackup","originalFromId","originalToId","widthFixed","lengthFixed","controlNodesEnabled","controlNodes","positions","connectedNode","_drawLine","_drawArrow","_drawArrowCenter","_drawDashLine","attachEdge","detachEdge","xFrom","yFrom","xTo","yTo","xObj","yObj","_getDistanceToEdge","_getColor","colorObj","_getLineWidth","_line","midpointX","midpointY","_pointOnLine","_label","resize","_circle","_pointOnCircle","networkScaleInv","_getViaCoordinates","xVia","yVia","quadraticCurveTo","lineCount","measureText","_rotateForLabelAlignment","_drawLabelRect","_drawLabelText","angleInDegrees","atan2","rotate","lineMargin","fillRect","lineJoin","strokeText","setLineDash","pattern","lineDashOffset","lineCap","dashedLine","percentage","arrow","_pointOnBezier","_findBorderPosition","distanceToBorder","distanceToNodes","difference","threshold","arrowPos","guidePos","edgeSegmentLength","toBorderDist","toBorderPoint","x1","y1","x2","y2","x3","y3","lastX","lastY","minDistance","_getDistanceToLine","px","py","something","u","nodeIdFrom","nodeIdTo","getControlNodeFromPosition","getControlNodeToPosition","_enableControlNodes","_disableControlNodes","_getSelectedControlNode","fromDistance","toDistance","_restoreControlNodes","controlnodeFromPos","fromBorderDist","fromBorderPoint","controlnodeToPos","defaultIndex","DEFAULT","imageBroken","load","url","brokenUrl","img","Image","onload","onerror","error","imagelist","grouplist","reroutedEdges","fontDrawThreshold","horizontalAlignLeft","verticalAlignTop","baseRadiusValue","radiusFixed","preassignedLevel","hierarchyEnumerated","fx","fy","vx","vy","previousState","resetCluster","dynamicEdgesLength","clusterSession","clusterSizeWidthFactor","clusterSizeHeightFactor","clusterSizeRadiusFactor","growthIndicator","networkScale","formationScale","clusterSize","containedNodes","containedEdges","clusterSessions","originalLabel","triggerFunction","groupObj","imageObj","brokenImage","_drawDatabase","_resizeDatabase","_drawBox","_resizeBox","_drawCircle","_resizeCircle","_drawEllipse","_resizeEllipse","_drawImage","_resizeImage","_drawCircularImage","_resizeCircularImage","_drawText","_resizeText","_drawDot","_resizeShape","_drawSquare","_drawTriangle","_drawTriangleDown","_drawStar","_reset","clearSizeCache","_setForce","_addForce","storeState","isFixed","velocity","getDistance","_drawImageAtPosition","globalAlpha","drawImage","_drawImageLabel","getTextSize","_swapToImageResizeWhenImageLoaded","diameter","centerX","centerY","_drawRawCircle","circle","clip","textSize","clusterLineWidth","selectionLineWidth","roundRect","database","defaultSize","ellipse","_drawShape","radiusMultiplier","baseline","labelUnderNode","inView","clearVelocity","updateVelocity","massBeforeClustering","energyBefore","styleAttr","fontFamily","WebkitBorderRadius","whiteSpace","parseDOT","parseGraph","nextPreview","isAlphaNumeric","regexAlphaNumeric","merge","o","addNode","graphs","attr","addEdge","createEdge","getToken","tokenType","TOKENTYPE","NULL","token","isComment","DELIMITER","c2","DELIMITERS","IDENTIFIER","newSyntaxError","UNKNOWN","chop","strict","parseStatements","parseStatement","subgraph","parseSubgraph","parseEdge","parseAttributeStatement","parseNodeStatement","subgraphs","parseAttributeList","message","maxLength","forEach2","array1","array2","elem1","elem2","graphData","dotNode","graphNode","convertEdge","dotEdge","graphEdge","subEdge","{","}","[","]",";","=",",","->","--","gephiJSON","allowedToMove","gEdges","gNodes","gEdge","source","gNode","propagating","onMouseWheel","leftContainer","rightContainer","shadowTop","shadowBottom","shadowTopLeft","shadowBottomLeft","shadowTopRight","shadowBottomRight","_redrawTimer","touchAction","listeners","events","scrollTopMin","redrawCount","_initAutoResize","component","_stopAutoResize","what","getWindow","borderRootHeight","borderRootWidth","autoHeight","centerWidth","_updateScrollTop","visibilityTop","visibilityBottom","MAX_REDRAWS","repaint","_startAutoResize","_onResize","lastHeight","watchTimer","setInterval","_setScrollTop","_getScrollTop","inputHandler","onRelease","isFinal","offTouch","offRelease","custom","back","editNode","addDescription","edgeDescription","editEdgeDescription","createEdgeError","deleteClusterError","CanvasRenderingContext2D","square","s2","ir","triangleDown","star","n","r2d","kappa","ox","oy","xe","ye","xm","ym","bezierCurveTo","wEllipse","hEllipse","ymb","yeb","xt","yt","xi","yi","xl","yl","xr","yr","dashArray","dashLength","dashCount","slope","distRemaining","dashIndex","_catmullRom","_linear","dFill","_catmullRomUniform","p0","p1","p2","p3","bp1","bp2","normalization","d1","d2","d3","A","N","M","d3powA","d2powA","d3pow2A","d2pow2A","d1pow2A","d1powA","Bargraph","barCombinedData","coreDistance","drawData","combinedData","intersections","barPoints","_getDataIntersections","heightOffset","_getSafeDrawData","nextKey","amount","resolved","prevKey","accumulated","groupLabel","_getStackedBarYRange","xpos","PhysicsMixin","ClusterMixin","SectorsMixin","SelectionMixin","ManipulationMixin","NavigationMixin","HierarchicalLayoutMixin","_loadMixin","sourceVariable","mixinFunction","_clearMixin","_loadSelectedForceSolver","_loadPhysicsConfiguration","hubThreshold","activeSector","drawingNode","blockConnectingEdgeSelection","forceAppendSelection","manipulationDiv","editModeDiv","closeDiv","_cleanNavigation","_loadNavigationElements","overlay","prevent_default","_onTapOverlay","windowHammer","_hasParent","deactivate","escListener","activate","unbind","_callbacks","once","self","removeListener","removeAllListeners","callbacks","cb","hasListeners","__WEBPACK_AMD_DEFINE_RESULT__","global","dfl","hasOwnProp","defaultParsingFlags","empty","unusedTokens","unusedInput","charsLeftOver","nullInput","invalidMonth","invalidFormat","userInvalidated","iso","printMsg","msg","suppressDeprecationWarnings","warn","deprecate","firstTime","deprecateSimple","deprecations","padToken","func","leftZeroFill","ordinalizeToken","period","localeData","ordinal","monthDiff","anchor2","adjust","wholeMonthDiff","meridiemFixWrap","meridiem","isPm","meridiemHour","isPM","Locale","Moment","config","skipOverflow","checkOverflow","copyConfig","updateInProgress","updateOffset","Duration","normalizedInput","normalizeObjectUnits","years","quarters","quarter","months","weeks","week","days","_milliseconds","_days","_months","_locale","_bubble","val","_isAMomentObject","_i","_f","_l","_strict","_tzm","_isUTC","_offset","_pf","momentProperties","absRound","number","targetLength","forceSign","output","positiveMomentsDifference","base","res","isAfter","momentsDifference","makeAs","isBefore","createAdder","dur","tmp","addOrSubtractDurationFromMoment","mom","isAdding","setTime","rawSetter","rawGetter","rawMonthSetter","input","compareArrays","dontConvert","lengthDiff","diffs","toInt","normalizeUnits","units","lowered","unitAliases","camelFunctions","inputObject","normalizedProp","makeList","setter","getter","results","utc","argumentForCoercion","coercedNumber","isFinite","daysInMonth","UTC","getUTCDate","weeksInYear","dow","doy","weekOfYear","daysInYear","isLeapYear","_a","MONTH","DATE","YEAR","HOUR","MINUTE","SECOND","MILLISECOND","_overflowDayOfYear","isValid","_isValid","getTime","bigHour","normalizeLocale","chooseLocale","names","loadLocale","oldLocale","hasModule","code","model","local","removeFormattingTokens","makeFormatFunction","formattingTokens","formatTokenFunctions","formatMoment","expandFormat","formatFunctions","invalidDate","replaceLongDateFormatTokens","longDateFormat","localFormattingTokens","lastIndex","getParseRegexForToken","parseTokenOneDigit","parseTokenThreeDigits","parseTokenFourDigits","parseTokenOneToFourDigits","parseTokenSignedNumber","parseTokenSixDigits","parseTokenOneToSixDigits","parseTokenTwoDigits","parseTokenOneToThreeDigits","parseTokenWord","_meridiemParse","parseTokenOffsetMs","parseTokenTimestampMs","parseTokenTimezone","parseTokenT","parseTokenDigits","parseTokenOneOrTwoDigits","_ordinalParse","_ordinalParseLenient","RegExp","regexpEscape","unescapeFormat","utcOffsetFromString","string","possibleTzMatches","tzChunk","parseTimezoneChunker","addTimeToArrayFromToken","datePartArray","monthsParse","_dayOfYear","parseTwoDigitYear","_meridiem","_useUTC","weekdaysParse","_w","invalidWeekday","dayOfYearFromWeekInfo","weekYear","temp","GG","W","E","_week","gg","dayOfYearFromWeeks","dateFromConfig","currentDate","yearToUse","currentDateArray","makeUTCDate","getUTCMonth","_nextDay","makeDate","setUTCMinutes","getUTCMinutes","dateFromObject","getUTCFullYear","makeDateFromStringAndFormat","ISO_8601","parseISO","parsedInput","tokens","skipped","stringLength","totalParsedInputLength","matched","p4","makeDateFromStringAndArray","tempConfig","bestMoment","scoreToBeat","currentScore","NaN","score","l","isoRegex","isoDates","isoTimes","makeDateFromString","createFromInputFallback","makeDateFromInput","aspNetJsonRegex","ms","setUTCFullYear","parseWeekday","substituteTimeAgo","withoutSuffix","isFuture","relativeTime","posNegDuration","relativeTimeThresholds","firstDayOfWeek","firstDayOfWeekOfYear","adjustedMoment","daysToDayOfWeek","daysToAdd","getUTCDay","makeMoment","invalid","preparse","pickBy","moments","dayOfMonth","unit","makeAccessor","keepTime","daysToYears","yearsToDays","makeDurationGetter","makeGlobal","shouldDeprecate","ender","oldGlobalMoment","globalScope","VERSION","aspNetTimeSpanJsonRegex","isoDurationRegex","isoFormat","unitMillisecondFactors","Milliseconds","Seconds","Minutes","Hours","Days","Months","Years","D","Q","DDD","dayofyear","isoweekday","isoweek","weekyear","isoweekyear","ordinalizeTokens","paddedTokens","MMM","monthsShort","MMMM","dd","weekdaysMin","ddd","weekdaysShort","dddd","weekdays","isoWeek","YY","YYYY","YYYYY","YYYYYY","gggg","ggggg","isoWeekYear","GGGG","GGGGG","isoWeekday","SS","SSS","SSSS","Z","utcOffset","ZZ","zoneAbbr","zz","zoneName","unix","lists","DDDD","_monthsShort","monthName","regex","_monthsParse","_longMonthsParse","_shortMonthsParse","_weekdays","_weekdaysShort","_weekdaysMin","weekdayName","_weekdaysParse","_longDateFormat","LTS","LT","L","LL","LLL","LLLL","isLower","_calendar","sameDay","nextDay","nextWeek","lastDay","lastWeek","sameElse","calendar","_relativeTime","future","past","mm","hh","MM","yy","pastFuture","_ordinal","postformat","firstDayOfYear","_invalidDate","ret","parseIso","diffRes","isDuration","inp","version","relativeTimeThreshold","limit","defineLocale","_abbr","abbr","langData","flags","parseZone","isDSTShifted","parsingFlags","invalidAt","keepLocalTime","_dateUtcOffset","inputString","asFloat","that","zoneDiff","humanize","fromNow","sod","startOf","isDST","getDay","endOf","inputMs","isBetween","zone","localAdjust","_changeInProgress","isLocal","isUtcOffset","isUtc","hasAlignedHourOffset","isoWeeksInYear","weekInfo","newLocaleData","getTimezoneOffset","isoWeeks","toJSON","isUTC","withSuffix","toIsoString","asSeconds","asMilliseconds","asMinutes","asHours","asDays","asWeeks","asMonths","asYears","ordinalParse","require","noGlobal","__WEBPACK_AMD_DEFINE_FACTORY__","__WEBPACK_AMD_DEFINE_ARRAY__","_exportFunctions","_bound","keydown","keyup","_keys","fromCharCode","down","handleEvent","up","keyCode","bound","bindAll","getKey","newBindings","_firstTarget","propagatedHandler","_handled","stopped","firstTarget","_handlers","Manager","PropagatingHammer","_on","_off","_destroy","handler","exportName","setTimeoutContext","timeout","bindFn","invokeArrayArg","arg","each","iterator","dest","inherit","child","childP","baseP","_super","boolOrFn","TYPE_FUNCTION","ifUndefined","val1","val2","addEventListeners","splitStr","removeEventListeners","hasParent","inStr","str","find","inArray","findByKey","uniqueArray","prefixed","property","prefix","camelProp","VENDOR_PREFIXES","uniqueId","_uniqueId","getWindowForElement","doc","ownerDocument","defaultView","parentWindow","Input","manager","inputTarget","domHandler","ev","init","createInputInstance","Type","inputClass","SUPPORT_POINTER_EVENTS","PointerEventInput","SUPPORT_ONLY_TOUCH","TouchInput","SUPPORT_TOUCH","TouchMouseInput","MouseInput","eventType","pointersLen","pointers","changedPointersLen","changedPointers","INPUT_START","INPUT_END","INPUT_CANCEL","session","computeInputData","recognize","prevInput","pointersLength","firstInput","simpleCloneInputData","firstMultiple","offsetCenter","getCenter","timeStamp","deltaTime","getAngle","computeDeltaXY","offsetDirection","getDirection","rotation","getRotation","computeIntervalInputData","offsetDelta","prevDelta","velocityX","velocityY","last","lastInterval","COMPUTE_INTERVAL","getVelocity","DIRECTION_NONE","DIRECTION_LEFT","DIRECTION_RIGHT","DIRECTION_UP","DIRECTION_DOWN","PROPS_XY","PROPS_CLIENT_XY","evEl","MOUSE_ELEMENT_EVENTS","evWin","MOUSE_WINDOW_EVENTS","allow","pressed","POINTER_ELEMENT_EVENTS","POINTER_WINDOW_EVENTS","store","pointerEvents","SingleTouchInput","evTarget","SINGLE_TOUCH_TARGET_EVENTS","SINGLE_TOUCH_WINDOW_EVENTS","started","normalizeSingleTouches","all","touches","changedTouches","TOUCH_TARGET_EVENTS","targetIds","getTouches","allTouches","INPUT_MOVE","identifier","changedTargetTouches","mouse","TouchAction","cleanTouchActions","actions","TOUCH_ACTION_NONE","hasPanX","TOUCH_ACTION_PAN_X","hasPanY","TOUCH_ACTION_PAN_Y","TOUCH_ACTION_MANIPULATION","TOUCH_ACTION_AUTO","Recognizer","defaults","state","STATE_POSSIBLE","simultaneous","requireFail","stateStr","STATE_CANCELLED","STATE_ENDED","STATE_CHANGED","STATE_BEGAN","directionStr","getRecognizerByNameIfManager","otherRecognizer","recognizer","AttrRecognizer","PanRecognizer","pX","pY","PinchRecognizer","PressRecognizer","_timer","_input","RotateRecognizer","SwipeRecognizer","TapRecognizer","pTime","pCenter","recognizers","preset","handlers","toggleCssProps","recognizeWith","requireFailure","cssProps","triggerDomEvent","gestureEvent","createEvent","initEvent","gesture","dispatchEvent","TEST_ELEMENT","MOBILE_REGEX","INPUT_TYPE_TOUCH","INPUT_TYPE_PEN","INPUT_TYPE_MOUSE","INPUT_TYPE_KINECT","DIRECTION_HORIZONTAL","DIRECTION_VERTICAL","DIRECTION_ALL","MOUSE_INPUT_MAP","mousedown","mousemove","mouseup","pointerType","POINTER_INPUT_MAP","pointerdown","pointermove","pointerup","pointercancel","pointerout","IE10_POINTER_TYPE_ENUM",2,3,4,5,"MSPointerEvent","removePointer","eventTypeNormalized","isTouch","storeIndex","pointerId","SINGLE_TOUCH_INPUT_MAP","touchstart","touchmove","touchend","touchcancel","TOUCH_INPUT_MAP","inputEvent","inputData","isMouse","PREFIXED_TOUCH_ACTION","NATIVE_TOUCH_ACTION","TOUCH_ACTION_COMPUTE","compute","getTouchAction","preventDefaults","prevented","hasNone","preventSrc","STATE_RECOGNIZED","STATE_FAILED","dropRecognizeWith","dropRequireFailure","hasRequireFailures","canRecognizeWith","withState","tryEmit","canEmit","inputDataClone","process","attrTest","optionPointers","isRecognized","directionTest","hasMoved","inOut","validPointers","validMovement","validTime","taps","posThreshold","validTouchTime","failTimeout","validInterval","validMultiTap","tapCount","domEvents","userSelect","touchSelect","touchCallout","contentZooming","userDrag","tapHighlightColor","STOP","FORCED_STOP","curRecognizer","Tap","Pan","Swipe","Pinch","Rotate","Press","clusterToFit","maxNumberOfNodes","reposition","maxLevels","forceAggregateHubs","normalizeClusterLevels","increaseClusterLevel","repositionNodes","openCluster","isMovingBeforeClustering","_nodeInActiveArea","_sector","_addSector","decreaseClusterLevel","_expandClusterNode","_updateDynamicEdges","updateClusters","zoomDirection","recursive","doNotStart","amountOfNodes","_collapseSector","_formClusters","_openClusters","_openClustersBySize","_aggregateHubs","handleChains","chainPercentage","_getChainFraction","_reduceAmountOfChains","_getHubSize","_formClustersByHub","openAll","containedNodeId","childNode","_expelChildFromParent","_unselectAll","_releaseContainedEdges","_connectEdgeBackToChild","_validateEdges","othersPresent","childNodeId","_repositionBezierNodes","_formClustersByZoom","_forceClustersByZoom","minLength","_addToCluster","_clusterToSmallestNeighbour","smallestNeighbour","smallestNeighbourNode","neighbour","onlyEqual","_formClusterFromHub","hubNode","absorptionSizeOffset","allowCluster","edgesIdarray","amountOfInitialEdges","_addToContainedEdges","_connectEdgeToCluster","_containCircularEdgesFromNode","massBefore","correction","edgeToId","edgeFromId","k","_addToReroutedEdges","maxLevel","minLevel","clusterLevel","targetLevel","average","averageSquared","hubCounter","largestHub","variance","standardDeviation","fraction","reduceAmount","chains","total","_switchToSector","sectorId","sectorType","_switchToActiveSector","_switchToFrozenSector","_switchToSupportSector","_loadLatestSector","_previousSector","_setActiveSector","newId","_forgetLastSector","_createNewSector","_deleteActiveSector","_deleteFrozenSector","_freezeSector","_activateSector","_mergeThisWithFrozen","_collapseThisToSingleCluster","sector","unqiueIdentifier","previousSector","runFunction","argument","returnValues","_doInAllFrozenSectors","_drawSectorNodes","_drawAllSectorNodes","_getNodesOverlappingWith","_getAllNodesOverlappingWith","_pointerToPositionObject","positionObject","_getEdgesOverlappingWith","_getAllEdgesOverlappingWith","_addToSelection","_addToHover","_removeFromSelection","doNotTrigger","_unselectClusters","_getSelectedNodeCount","_getSelectedNode","_getSelectedEdge","_getSelectedEdgeCount","_getSelectedObjectCount","_selectionIsEmpty","_clusterInSelection","_selectConnectedEdges","_hoverConnectedEdges","_unselectConnectedEdges","append","highlightEdges","overrideSelectable","DOM","_manipulationReleaseOverload","_navigationReleaseOverload","getSelectedNodes","edgeIds","getSelectedEdges","idArray","selectNodes","RangeError","selectEdges","_clearManipulatorBar","manipulationDOM","_restoreOverloadedFunctions","functionName","_toggleEditMode","toolbar","boundFunction","edgeBeingEdited","selectedControlNode","_createAddNodeToolbar","_createAddEdgeToolbar","_editNode","_createEditEdgeToolbar","_addNode","_handleConnect","_finishConnect","_selectControlNode","_controlNodeDrag","_releaseControlNode","newNode","_editEdge","alert","supportNodes","targetNode","connectionEdge","connectFromId","_createEdge","defaultData","finalizedData","sourceNodeId","targetNodeId","selectedNodes","selectedEdges","navigationDivs","navigationDivActions","_stopMovement","_zoomExtent","hubsize","definedLevel","undefinedLevel","_changeConstants","_determineLevels","_determineLevelsDirected","distribution","_getDistribution","_placeNodesByHierarchy","minPos","_placeBranchNodes","maxCount","_setLevel","firstNode","_setLevelDirected","parentId","parentLevel","nodeMoved","_restoreNodes","graphToggleSmoothCurves","graph_toggleSmooth","getElementById","graphRepositionNodes","showValueOfRange","graphGenerateOptions","optionsSpecific","radioButton1","radioButton2","checked","backupConstants","optionsDiv","switchConfigurations","radioButton","querySelector","tableId","table","constantsVariableName","valueId","rangeValue","_overWriteGraphConstants","RepulsionMixin","HierarchialRepulsionMixin","BarnesHutMixin","_toggleBarnesHut","barnesHutTree","_initializeForceCalculation","_calculateForces","_calculateGravitationalForces","_calculateNodeForces","_calculateSpringForcesWithSupport","_calculateHierarchicalSpringForces","_calculateSpringForces","supportNodeId","gravity","gravityForce","edgeLength","springForce","combinedClusterSize","node1","node2","node3","_calculateSpringForce","physicsConfiguration","hierarchicalLayoutDirections","parentElement","rangeElement","radioButton3","graph_repositionNodes","graph_generateOptions","dynamicSmoothCurves","nameArray","webpackContext","req","resolve","repulsingForce","a_base","minimumDistance","steepness","springFx","springFy","totalFx","totalFy","correctionFx","correctionFy","nodeCount","_formBarnesHutTree","_getForceContribution","children","NW","NE","SW","SE","parentBranch","childrenCount","centerOfMass","calcSize","MAX_VALUE","sizeDiff","minimumTreeSize","rootSize","halfRootSize","_splitBranch","_placeInTree","_updateBranchMass","totalMass","totalMassInv","biggestSize","skipMassUpdate","_placeInRegion","region","containedNode","_insertRegion","childSize","_drawTree","_drawBranch","branch","webpackPolyfill","paths","__webpack_amd_options__"],"mappings":";;;;;;;;;;;;;;;;;;;;;;;;AAyBA,cAEA,SAA2CA,EAAMC,GAC1B,gBAAZC,UAA0C,gBAAXC,QACxCA,OAAOD,QAAUD,IACQ,kBAAXG,SAAyBA,OAAOC,IAC9CD,OAAOH,GACmB,gBAAZC,SACdA,QAAa,IAAID,IAEjBD,EAAU,IAAIC,KACbK,KAAM,WACT,MAAgB,UAAUC,GAKhB,QAASC,GAAoBC,GAG5B,GAAGC,EAAiBD,GACnB,MAAOC,GAAiBD,GAAUP,OAGnC,IAAIC,GAASO,EAAiBD,IAC7BP,WACAS,GAAIF,EACJG,QAAQ,EAUT,OANAL,GAAQE,GAAUI,KAAKV,EAAOD,QAASC,EAAQA,EAAOD,QAASM,GAG/DL,EAAOS,QAAS,EAGTT,EAAOD,QAvBf,GAAIQ,KAqCJ,OATAF,GAAoBM,EAAIP,EAGxBC,EAAoBO,EAAIL,EAGxBF,EAAoBQ,EAAI,GAGjBR,EAAoB,KAK/B,SAASL,EAAQD,EAASM,GAG9BN,EAAQe,KAAOT,EAAoB,GACnCN,EAAQgB,QAAUV,EAAoB,GAGtCN,EAAQiB,QAAUX,EAAoB,GACtCN,EAAQkB,SAAWZ,EAAoB,GACvCN,EAAQmB,MAAQb,EAAoB,GAGpCN,EAAQoB,QAAUd,EAAoB,GACtCN,EAAQqB,SACNC,OAAQhB,EAAoB,GAC5BiB,OAAQjB,EAAoB,GAC5BkB,QAASlB,EAAoB,GAC7BmB,QAASnB,EAAoB,IAC7BoB,OAAQpB,EAAoB,IAC5BqB,WAAYrB,EAAoB,KAIlCN,EAAQ4B,SAAWtB,EAAoB,IACvCN,EAAQ6B,QAAUvB,EAAoB,IACtCN,EAAQ8B,UACNC,SAAUzB,EAAoB,IAC9B0B,SAAU1B,EAAoB,IAC9B2B,MAAO3B,EAAoB,IAC3B4B,MAAO5B,EAAoB,IAC3B6B,SAAU7B,EAAoB,IAE9B8B,YACEC,OACEC,KAAMhC,EAAoB,IAC1BiC,eAAgBjC,EAAoB,IACpCkC,QAASlC,EAAoB,IAC7BmC,UAAWnC,EAAoB,IAC/BoC,UAAWpC,EAAoB,KAGjCqC,UAAWrC,EAAoB,IAC/BsC,YAAatC,EAAoB,IACjCuC,WAAYvC,EAAoB,IAChCwC,SAAUxC,EAAoB,IAC9ByC,WAAYzC,EAAoB,IAChC0C,MAAO1C,EAAoB,IAC3B2C,gBAAiB3C,EAAoB,IACrC4C,QAAS5C,EAAoB,IAC7B6C,OAAQ7C,EAAoB,IAC5B8C,UAAW9C,EAAoB,IAC/B+C,SAAU/C,EAAoB,MAKlCN,EAAQsD,QAAUhD,EAAoB,IACtCN,EAAQuD,SACNC,KAAMlD,EAAoB,IAC1BmD,OAAQnD,EAAoB,IAC5BoD,OAAQpD,EAAoB,IAC5BqD,KAAMrD,EAAoB,IAC1BsD,MAAOtD,EAAoB,IAC3BuD,UAAWvD,EAAoB,IAC/BwD,YAAaxD,EAAoB,KAInCN,EAAQ+D,MAAQ,WACd,KAAM,IAAIC,OAAM,+EAIlBhE,EAAQiE,OAAS3D,EAAoB,IACrCN,EAAQkE,OAAS5D,EAAoB,IACrCN,EAAQmE,OAAS7D,EAAoB,KAKjC,SAASL,EAAQD,EAASM,GAM9B,GAAI2D,GAAS3D,EAAoB,GAOjCN,GAAQoE,SAAW,SAASC,GAC1B,MAAQA,aAAkBC,SAA2B,gBAAVD,IAQ7CrE,EAAQuE,SAAW,SAASF,GAC1B,MAAQA,aAAkBG,SAA2B,gBAAVH,IAQ7CrE,EAAQyE,OAAS,SAASJ,GACxB,GAAIA,YAAkBK,MACpB,OAAO,CAEJ,IAAI1E,EAAQuE,SAASF,GAAS,CAEjC,GAAIM,GAAQC,EAAaC,KAAKR,EAC9B,IAAIM,EACF,OAAO,CAEJ,KAAKG,MAAMJ,KAAKK,MAAMV,IACzB,OAAO,EAIX,OAAO,GAQTrE,EAAQgF,YAAc,SAASX,GAC7B,MAA4B,mBAAb,SACVY,OAAoB,eACpBA,OAAOC,cAAuB,WAC9Bb,YAAkBY,QAAOC,cAAcC,WAQ9CnF,EAAQoF,WAAa,WACnB,GAAIC,GAAK,WACP,MAAOC,MAAKC,MACQ,MAAhBD,KAAKE,UACPC,SAAS,IAGb,OACIJ,KAAOA,IAAO,IACVA,IAAO,IACPA,IAAO,IACPA,IAAO,IACPA,IAAOA,IAAOA,KAWxBrF,EAAQ0F,OAAS,SAAUC,GACzB,IAAK,GAAIC,GAAI,EAAGC,EAAMC,UAAUC,OAAYF,EAAJD,EAASA,IAAK,CACpD,GAAII,GAAQF,UAAUF,EACtB,KAAK,GAAIK,KAAQD,GACXA,EAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAKtB,MAAON,IAWT3F,EAAQmG,gBAAkB,SAAUC,EAAOT,GACzC,IAAKU,MAAMC,QAAQF,GACjB,KAAM,IAAIpC,OAAM,uDAGlB,KAAK,GAAI4B,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAGpC,IAAK,GAFDI,GAAQF,UAAUF,GAEb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACbkF,GAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAItB,MAAON,IAWT3F,EAAQuG,oBAAsB,SAAUH,EAAOT,EAAGa,GAEhD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIb,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAEpC,IAAK,GADDI,GAAQF,UAAUF,GACb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACjB,IAAIkF,EAAME,eAAeD,GACvB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,IAMpB,MAAON,IAWT3F,EAAQ8G,uBAAyB,SAAUV,EAAOT,EAAGa,GAEnD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,IACQ,IAAvBG,EAAMW,QAAQd,GAChB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAKpB,MAAON,IAST3F,EAAQ6G,WAAa,SAASlB,EAAGa,GAE/B,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAGtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,GACnB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAIlB,MAAON,IAUT3F,EAAQgH,WAAa,SAAUrB,EAAGa,GAChC,GAAIb,EAAEI,QAAUS,EAAET,OAAQ,OAAO,CAEjC,KAAK,GAAIH,GAAI,EAAGC,EAAMF,EAAEI,OAAYF,EAAJD,EAASA,IACvC,GAAID,EAAEC,IAAMY,EAAEZ,GAAI,OAAO,CAG3B,QAAO,GAYT5F,EAAQiH,QAAU,SAAS5C,EAAQ6C,GACjC,GAAIvC,EAEJ,IAAeiC,SAAXvC,EACF,MAAOuC,OAET,IAAe,OAAXvC,EACF,MAAO,KAGT,KAAK6C,EACH,MAAO7C,EAET,IAAsB,gBAAT6C,MAAwBA,YAAgB1C,SACnD,KAAM,IAAIR,OAAM,wBAIlB,QAAQkD,GACN,IAAK,UACL,IAAK,UACH,MAAOC,SAAQ9C,EAEjB,KAAK,SACL,IAAK,SACH,MAAOC,QAAOD,EAAO+C,UAEvB,KAAK,SACL,IAAK,SACH,MAAO5C,QAAOH,EAEhB,KAAK,OACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAElB,IAAIA,YAAkBK,MACpB,MAAO,IAAIA,MAAKL,EAAO+C,UAEpB,IAAInD,EAAOoD,SAAShD,GACvB,MAAO,IAAIK,MAAKL,EAAO+C,UAEzB,IAAIpH,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAGtBV,EAAOI,GAAQiD,QAIxB,MAAM,IAAItD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,SACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAOJ,GAAOI,EAEhB,IAAIA,YAAkBK,MACpB,MAAOT,GAAOI,EAAO+C,UAElB,IAAInD,EAAOoD,SAAShD,GACvB,MAAOJ,GAAOI,EAEhB,IAAIrE,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GAGjBJ,EAFLU,EAEYL,OAAOK,EAAM,IAGbN,EAIhB,MAAM,IAAIL,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,UACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAEb,IAAIA,YAAkBK,MACzB,MAAOL,GAAOmD,aAEX,IAAIvD,EAAOoD,SAAShD,GACvB,MAAOA,GAAOiD,SAASE,aAEpB,IAAIxH,EAAQuE,SAASF,GAExB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAAK6C,cAG3B,GAAI9C,MAAKL,GAAQmD,aAI1B,MAAM,IAAIxD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,KAAK,UACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,SAAWA,EAAS,IAExB,IAAIA,YAAkBK,MACzB,MAAO,SAAWL,EAAO+C,UAAY,IAElC,IAAIpH,EAAQuE,SAASF,GAAS,CACjCM,EAAQC,EAAaC,KAAKR,EAC1B,IAAIoD,EAQJ,OALEA,GAFE9C,EAEM,GAAID,MAAKJ,OAAOK,EAAM,KAAKyC,UAG3B,GAAI1C,MAAKL,GAAQ+C,UAEpB,SAAWK,EAAQ,KAG1B,KAAM,IAAIzD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,SACE,KAAM,IAAIL,OAAM,iBAAmBkD,EAAO,MAOhD,IAAItC,GAAe,qBAOnB5E,GAAQuH,QAAU,SAASlD,GACzB,GAAI6C,SAAc7C,EAElB,OAAY,UAAR6C,EACY,MAAV7C,EACK,OAELA,YAAkB8C,SACb,UAEL9C,YAAkBC,QACb,SAELD,YAAkBG,QACb,SAEL6B,MAAMC,QAAQjC,GACT,QAELA,YAAkBK,MACb,OAEF,SAEQ,UAARwC,EACA,SAEQ,WAARA,EACA,UAEQ,UAARA,EACA,SAGFA,GASTlH,EAAQ0H,gBAAkB,SAASC,GACjC,MAAOA,GAAKC,wBAAwBC,MAStC7H,EAAQ8H,eAAiB,SAASH,GAChC,MAAOA,GAAKC,wBAAwBG,KAQtC/H,EAAQgI,aAAe,SAASL,EAAMM,GACpC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,IACD,KAA9BD,EAAQnB,QAAQkB,KAClBC,EAAQE,KAAKH,GACbN,EAAKM,UAAYC,EAAQG,KAAK,OASlCrI,EAAQsI,gBAAkB,SAASX,EAAMM,GACvC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,KAC/BI,EAAQL,EAAQnB,QAAQkB,EACf,KAATM,IACFL,EAAQM,OAAOD,EAAO,GACtBZ,EAAKM,UAAYC,EAAQG,KAAK,OAalCrI,EAAQyI,QAAU,SAASpE,EAAQqE,GACjC,GAAI9C,GACAC,CACJ,IAAIQ,MAAMC,QAAQjC,GAEhB,IAAKuB,EAAI,EAAGC,EAAMxB,EAAO0B,OAAYF,EAAJD,EAASA,IACxC8C,EAASrE,EAAOuB,GAAIA,EAAGvB,OAKzB,KAAKuB,IAAKvB,GACJA,EAAO6B,eAAeN,IACxB8C,EAASrE,EAAOuB,GAAIA,EAAGvB,IAY/BrE,EAAQ2I,QAAU,SAAStE,GACzB,GAAIuE,KAEJ,KAAK,GAAI3C,KAAQ5B,GACXA,EAAO6B,eAAeD,IAAO2C,EAAMR,KAAK/D,EAAO4B,GAGrD,OAAO2C,IAUT5I,EAAQ6I,eAAiB,SAASxE,EAAQyE,EAAKrB,GAC7C,MAAIpD,GAAOyE,KAASrB,GAClBpD,EAAOyE,GAAOrB,GACP,IAGA,GAYXzH,EAAQ+I,iBAAmB,SAASC,EAASC,EAAQC,EAAUC,GACzDH,EAAQD,kBACSnC,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQD,iBAAiBE,EAAQC,EAAUC,IAE3CH,EAAQM,YAAY,KAAOL,EAAQC,IAWvClJ,EAAQuJ,oBAAsB,SAASP,EAASC,EAAQC,EAAUC,GAC5DH,EAAQO,qBAES3C,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQO,oBAAoBN,EAAQC,EAAUC,IAG9CH,EAAQQ,YAAY,KAAOP,EAAQC,IAOvClJ,EAAQyJ,eAAiB,SAAUC,GAC5BA,IACHA,EAAQC,OAAOD,OAEbA,EAAMD,eACRC,EAAMD,iBAGNC,EAAME,aAAc,GASxB5J,EAAQ6J,UAAY,SAASH,GAEtBA,IACHA,EAAQC,OAAOD,MAGjB,IAAII,EAcJ,OAZIJ,GAAMI,OACRA,EAASJ,EAAMI,OAERJ,EAAMK,aACbD,EAASJ,EAAMK,YAGMnD,QAAnBkD,EAAOE,UAA4C,GAAnBF,EAAOE,WAEzCF,EAASA,EAAOG,YAGXH,GAGT9J,EAAQkK,UAQRlK,EAAQkK,OAAOC,UAAY,SAAU1C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACe,GAATA,EAGH2C,GAAgB,MASzBpK,EAAQkK,OAAOG,SAAW,SAAU5C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKnD,OAAOmD,IAAU2C,GAAgB,KAGnCA,GAAgB,MASzBpK,EAAQkK,OAAOI,SAAW,SAAU7C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKjD,OAAOiD,GAGT2C,GAAgB,MASzBpK,EAAQkK,OAAOK,OAAS,SAAU9C,EAAO2C,GAKvC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGNzH,EAAQuE,SAASkD,GACZA,EAEAzH,EAAQoE,SAASqD,GACjBA,EAAQ,KAGR2C,GAAgB,MAU3BpK,EAAQkK,OAAOM,UAAY,SAAU/C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGHA,GAAS2C,GAAgB,MASlCpK,EAAQyK,SAAW,SAASC,GAE1B,GAAIC,GAAiB,kCACrBD,GAAMA,EAAIE,QAAQD,EAAgB,SAAS/J,EAAGiK,EAAGC,EAAGtE,GAChD,MAAOqE,GAAIA,EAAIC,EAAIA,EAAItE,EAAIA,GAE/B,IAAIuE,GAAS,4CAA4ClG,KAAK6F,EAC9D,OAAOK,IACHF,EAAGG,SAASD,EAAO,GAAI,IACvBD,EAAGE,SAASD,EAAO,GAAI,IACvBvE,EAAGwE,SAASD,EAAO,GAAI,KACvB,MAWN/K,EAAQiL,SAAW,SAASC,EAAIC,EAAMC,GACpC,MAAO,MAAQ,GAAK,KAAOF,GAAO,KAAOC,GAAS,GAAKC,GAAM3F,SAAS,IAAI4F,MAAM,IASlFrL,EAAQsL,WAAa,SAASC,GAC5B,GAAI1K,EACJ,IAAIb,EAAQuE,SAASgH,GAAQ,CAC3B,GAAIvL,EAAQwL,WAAWD,GAAQ,CAC7B,GAAIE,GAAMF,EAAMG,OAAO,GAAGA,OAAO,EAAEH,EAAMxF,OAAO,GAAGoC,MAAM,IACzDoD,GAAQvL,EAAQiL,SAASQ,EAAI,GAAGA,EAAI,GAAGA,EAAI,IAE7C,GAAIzL,EAAQ2L,WAAWJ,GAAQ,CAC7B,GAAIK,GAAM5L,EAAQ6L,SAASN,GACvBO,GAAmBC,EAAEH,EAAIG,EAAEC,EAAU,IAARJ,EAAII,EAASC,EAAE3G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,IAC3DE,GAAmBJ,EAAEH,EAAIG,EAAEC,EAAE1G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,GAAUA,EAAQ,GAANL,EAAIK,GAC5DG,EAAkBpM,EAAQqM,SAASF,EAAeJ,EAAGI,EAAeJ,EAAGI,EAAeF,GACtFK,EAAkBtM,EAAQqM,SAASP,EAAgBC,EAAED,EAAgBE,EAAEF,EAAgBG,EAE3FpL,IACE0L,WAAYhB,EACZiB,OAAOJ,EACPK,WACEF,WAAWD,EACXE,OAAOJ,GAETM,OACEH,WAAWD,EACXE,OAAOJ,QAKXvL,IACE0L,WAAWhB,EACXiB,OAAOjB,EACPkB,WACEF,WAAWhB,EACXiB,OAAOjB,GAETmB,OACEH,WAAWhB,EACXiB,OAAOjB,QAMb1K,MACAA,EAAE0L,WAAahB,EAAMgB,YAAc,QACnC1L,EAAE2L,OAASjB,EAAMiB,QAAU3L,EAAE0L,WAEzBvM,EAAQuE,SAASgH,EAAMkB,WACzB5L,EAAE4L,WACAD,OAAQjB,EAAMkB,UACdF,WAAYhB,EAAMkB,YAIpB5L,EAAE4L,aACF5L,EAAE4L,UAAUF,WAAahB,EAAMkB,WAAalB,EAAMkB,UAAUF,YAAc1L,EAAE0L,WAC5E1L,EAAE4L,UAAUD,OAASjB,EAAMkB,WAAalB,EAAMkB,UAAUD,QAAU3L,EAAE2L,QAGlExM,EAAQuE,SAASgH,EAAMmB,OACzB7L,EAAE6L,OACAF,OAAQjB,EAAMmB,MACdH,WAAYhB,EAAMmB,QAIpB7L,EAAE6L,SACF7L,EAAE6L,MAAMH,WAAahB,EAAMmB,OAASnB,EAAMmB,MAAMH,YAAc1L,EAAE0L,WAChE1L,EAAE6L,MAAMF,OAASjB,EAAMmB,OAASnB,EAAMmB,MAAMF,QAAU3L,EAAE2L,OAI5D,OAAO3L,IAYTb,EAAQ2M,SAAW,SAASzB,EAAIC,EAAMC,GACpCF,GAAQ,IAAKC,GAAY,IAAKC,GAAU,GACxC,IAAIwB,GAAStH,KAAK4G,IAAIhB,EAAI5F,KAAK4G,IAAIf,EAAMC,IACrCyB,EAASvH,KAAKwH,IAAI5B,EAAI5F,KAAKwH,IAAI3B,EAAMC,GAGzC,IAAIwB,GAAUC,EACZ,OAAQd,EAAE,EAAEC,EAAE,EAAEC,EAAEW,EAIpB,IAAIG,GAAK7B,GAAK0B,EAAUzB,EAAMC,EAASA,GAAMwB,EAAU1B,EAAIC,EAAQC,EAAKF,EACpEa,EAAKb,GAAK0B,EAAU,EAAMxB,GAAMwB,EAAU,EAAI,EAC9CI,EAAM,IAAIjB,EAAIgB,GAAGF,EAASD,IAAS,IACnCK,GAAcJ,EAASD,GAAQC,EAC/BpF,EAAQoF,CACZ,QAAQd,EAAEiB,EAAIhB,EAAEiB,EAAWhB,EAAExE,GAG/B,IAAIyF,IAEF/E,MAAO,SAAUgF,GACf,GAAIC,KAWJ,OATAD,GAAQhF,MAAM,KAAKM,QAAQ,SAAU4E,GACnC,GAAoB,IAAhBA,EAAMC,OAAc,CACtB,GAAIC,GAAQF,EAAMlF,MAAM,KACpBW,EAAMyE,EAAM,GAAGD,OACf7F,EAAQ8F,EAAM,GAAGD,MACrBF,GAAOtE,GAAOrB,KAIX2F,GAIT/E,KAAM,SAAU+E,GACd,MAAOzG,QAAO6G,KAAKJ,GACdK,IAAI,SAAU3E,GACb,MAAOA,GAAM,KAAOsE,EAAOtE,KAE5BT,KAAK,OASdrI,GAAQ0N,WAAa,SAAU1E,EAASmE,GACtC,GAAIQ,GAAgBT,EAAQ/E,MAAMa,EAAQqE,MAAMF,SAC5CS,EAAYV,EAAQ/E,MAAMgF,GAC1BC,EAASpN,EAAQ0F,OAAOiI,EAAeC,EAE3C5E,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAQvCpN,EAAQ6N,cAAgB,SAAU7E,EAASmE,GACzC,GAAIC,GAASF,EAAQ/E,MAAMa,EAAQqE,MAAMF,SACrCW,EAAeZ,EAAQ/E,MAAMgF,EAEjC,KAAK,GAAIrE,KAAOgF,GACVA,EAAa5H,eAAe4C,UACvBsE,GAAOtE,EAIlBE,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAWvCpN,EAAQ+N,SAAW,SAAShC,EAAGC,EAAGC,GAChC,GAAIpB,GAAGC,EAAGtE,EAENZ,EAAIN,KAAKC,MAAU,EAAJwG,GACfiC,EAAQ,EAAJjC,EAAQnG,EACZ9E,EAAImL,GAAK,EAAID,GACbiC,EAAIhC,GAAK,EAAI+B,EAAIhC,GACjBkC,EAAIjC,GAAK,GAAK,EAAI+B,GAAKhC,EAE3B,QAAQpG,EAAI,GACV,IAAK,GAAGiF,EAAIoB,EAAGnB,EAAIoD,EAAG1H,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAIoD,EAAGnD,EAAImB,EAAGzF,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAI/J,EAAGgK,EAAImB,EAAGzF,EAAI0H,CAAG,MAC7B,KAAK,GAAGrD,EAAI/J,EAAGgK,EAAImD,EAAGzH,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIqD,EAAGpD,EAAIhK,EAAG0F,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIoB,EAAGnB,EAAIhK,EAAG0F,EAAIyH,EAG5B,OAAQpD,EAAEvF,KAAKC,MAAU,IAAJsF,GAAUC,EAAExF,KAAKC,MAAU,IAAJuF,GAAUtE,EAAElB,KAAKC,MAAU,IAAJiB,KAGrExG,EAAQqM,SAAW,SAASN,EAAGC,EAAGC,GAChC,GAAIR,GAAMzL,EAAQ+N,SAAShC,EAAGC,EAAGC,EACjC,OAAOjM,GAAQiL,SAASQ,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ6L,SAAW,SAASnB,GAC1B,GAAIe,GAAMzL,EAAQyK,SAASC,EAC3B,OAAO1K,GAAQ2M,SAASlB,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ2L,WAAa,SAASjB,GAC5B,GAAIyD,GAAO,qCAAqCC,KAAK1D,EACrD,OAAOyD,IAGTnO,EAAQwL,WAAa,SAASC,GAC5BA,EAAMA,EAAIb,QAAQ,IAAI,GACtB,IAAIuD,GAAO,wCAAwCC,KAAK3C,EACxD,OAAO0C,IAUTnO,EAAQqO,sBAAwB,SAASC,EAAQC,GAC/C,GAA8B,gBAAnBA,GAA6B,CAEtC,IAAK,GADDC,GAAW7H,OAAO8H,OAAOF,GACpB3I,EAAI,EAAGA,EAAI0I,EAAOvI,OAAQH,IAC7B2I,EAAgBrI,eAAeoI,EAAO1I,KACC,gBAA9B2I,GAAgBD,EAAO1I,MAChC4I,EAASF,EAAO1I,IAAM5F,EAAQ0O,aAAaH,EAAgBD,EAAO1I,KAIxE,OAAO4I,GAGP,MAAO,OAWXxO,EAAQ0O,aAAe,SAASH,GAC9B,GAA8B,gBAAnBA,GAA6B,CACtC,GAAIC,GAAW7H,OAAO8H,OAAOF,EAC7B,KAAK,GAAI3I,KAAK2I,GACRA,EAAgBrI,eAAeN,IACA,gBAAtB2I,GAAgB3I,KACzB4I,EAAS5I,GAAK5F,EAAQ0O,aAAaH,EAAgB3I,IAIzD,OAAO4I,GAGP,MAAO,OAcXxO,EAAQ2O,aAAe,SAAUC,EAAaC,EAAS3E,GACrD,GAAwBtD,SAApBiI,EAAQ3E,GACV,GAA8B,iBAAnB2E,GAAQ3E,GACjB0E,EAAY1E,GAAQ4E,QAAUD,EAAQ3E,OAEnC,CACH0E,EAAY1E,GAAQ4E,SAAU,CAC9B,KAAK,GAAI7I,KAAQ4I,GAAQ3E,GACnB2E,EAAQ3E,GAAQhE,eAAeD,KACjC2I,EAAY1E,GAAQjE,GAAQ4I,EAAQ3E,GAAQjE,MAmBtDjG,EAAQ+O,mBAAqB,SAASC,EAAcC,EAAgBC,EAAOC,GAMzE,IALA,GAAIC,GAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAEnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAC/C,GAAIG,GAASlK,KAAKC,OAAO+J,EAAMC,GAAQ,GAEnCE,EAAOT,EAAaQ,GACpB/H,EAAoBb,SAAXuI,EAAwBM,EAAKP,GAASO,EAAKP,GAAOC,GAE3DO,EAAeT,EAAexH,EAClC,IAAoB,GAAhBiI,EACF,MAAOF,EAEgB,KAAhBE,EACPJ,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGlBH,IAGF,MAAO,IAeTrP,EAAQ2P,kBAAoB,SAASX,EAAclF,EAAQoF,EAAOU,GAOhE,IANA,GAIIC,GAAWpI,EAAOqI,EAAWN,EAJ7BJ,EAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAGnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAO/C,GALAG,EAASlK,KAAKC,MAAM,IAAKgK,EAAKD,IAC9BO,EAAYb,EAAa1J,KAAKwH,IAAI,EAAE0C,EAAS,IAAIN,GACjDzH,EAAYuH,EAAaQ,GAAQN,GACjCY,EAAYd,EAAa1J,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,IAAIN,GAEjEzH,GAASqC,EACX,MAAO0F,EAEJ,IAAgB1F,EAAZ+F,GAAsBpI,EAAQqC,EACrC,MAAyB,UAAlB8F,EAA6BtK,KAAKwH,IAAI,EAAE0C,EAAS,GAAKA,CAE1D,IAAY1F,EAARrC,GAAkBqI,EAAYhG,EACrC,MAAyB,UAAlB8F,EAA6BJ,EAASlK,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,EAGzE1F,GAARrC,EACF6H,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGpBH,IAIF,MAAO,IAYTrP,EAAQ+P,cAAgB,SAAU7B,EAAG8B,EAAOC,EAAKC,GAC/C,GAAIC,GAASF,EAAMD,CAEnB,OADA9B,IAAKgC,EAAS,EACN,EAAJhC,EAAciC,EAAO,EAAEjC,EAAEA,EAAI8B,GACjC9B,KACQiC,EAAO,GAAKjC,GAAGA,EAAE,GAAK,GAAK8B,IAUrChQ,EAAQoQ,iBAENC,OAAQ,SAAUnC,GAChB,MAAOA,IAGToC,WAAY,SAAUpC,GACpB,MAAOA,GAAIA,GAGbqC,YAAa,SAAUrC,GACrB,MAAOA,IAAK,EAAIA,IAGlB6B,cAAe,SAAU7B,GACvB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAI,IAAM,EAAI,EAAIA,GAAKA,GAGjDsC,YAAa,SAAUtC,GACrB,MAAOA,GAAIA,EAAIA,GAGjBuC,aAAc,SAAUvC,GACtB,QAAUA,EAAKA,EAAIA,EAAI,GAGzBwC,eAAgB,SAAUxC,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,GAAKA,EAAI,IAAM,EAAIA,EAAI,IAAM,EAAIA,EAAI,GAAK,GAGxEyC,YAAa,SAAUzC,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,GAGrB0C,aAAc,SAAU1C,GACtB,MAAO,MAAOA,EAAKA,EAAIA,EAAIA,GAG7B2C,eAAgB,SAAU3C,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,IAAOA,EAAKA,EAAIA,EAAIA,GAG9D4C,YAAa,SAAU5C,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,EAAIA,GAGzB6C,aAAc,SAAU7C,GACtB,MAAO,KAAOA,EAAKA,EAAIA,EAAIA,EAAIA,GAGjC8C,eAAgB,SAAU9C,GACxB,MAAW,GAAJA,EAAS,GAAKA,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,KAAQA,EAAKA,EAAIA,EAAIA,EAAIA,KAMtE,SAASjO,EAAQD,GASrBA,EAAQiR,gBAAkB,SAASC,GAEjC,IAAK,GAAIC,KAAeD,GAClBA,EAAchL,eAAeiL,KAC/BD,EAAcC,GAAaC,UAAYF,EAAcC,GAAaE,KAClEH,EAAcC,GAAaE,UAYjCrR,EAAQsR,gBAAkB,SAASJ,GAEjC,IAAK,GAAIC,KAAeD,GACtB,GAAIA,EAAchL,eAAeiL,IAC3BD,EAAcC,GAAaC,UAAW,CACxC,IAAK,GAAIxL,GAAI,EAAGA,EAAIsL,EAAcC,GAAaC,UAAUrL,OAAQH,IAC/DsL,EAAcC,GAAaC,UAAUxL,GAAGqE,WAAWsH,YAAYL,EAAcC,GAAaC,UAAUxL,GAEtGsL,GAAcC,GAAaC,eAgBnCpR,EAAQwR,cAAgB,SAAUL,EAAaD,EAAeO,GAC5D,GAAIzI,EAqBJ,OAnBIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjEM,EAAaI,YAAY7I,KAK3BA,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjED,EAAcC,IAAgBE,QAAUD,cACxCK,EAAaI,YAAY7I,IAE3BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAcThJ,EAAQ8R,cAAgB,SAAUX,EAAaD,EAAea,EAAcC,GAC1E,GAAIhJ,EA+BJ,OA7BIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASM,cAAcd,GACZvK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,KAM7BA,EAAU2I,SAASM,cAAcd,GACjCD,EAAcC,IAAgBE,QAAUD,cACnBxK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,IAG7BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAkBThJ,EAAQkS,UAAY,SAASC,EAAGC,EAAGC,EAAOnB,EAAeO,GACvD,GAAIa,EAmBJ,OAlBsC,UAAlCD,EAAMxD,QAAQ0D,WAAWlF,OAC3BiF,EAAQtS,EAAQwR,cAAc,SAASN,EAAcO,GACrDa,EAAME,eAAe,KAAM,KAAML,GACjCG,EAAME,eAAe,KAAM,KAAMJ,GACjCE,EAAME,eAAe,KAAM,IAAK,GAAMH,EAAMxD,QAAQ0D,WAAWE,QAG/DH,EAAQtS,EAAQwR,cAAc,OAAON,EAAcO,GACnDa,EAAME,eAAe,KAAM,IAAKL,EAAI,GAAIE,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,IAAKJ,EAAI,GAAIC,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,QAASH,EAAMxD,QAAQ0D,WAAWE,MAC7DH,EAAME,eAAe,KAAM,SAAUH,EAAMxD,QAAQ0D,WAAWE,OAGzB7L,SAApCyL,EAAMxD,QAAQ0D,WAAWnF,QAC1BkF,EAAME,eAAe,KAAM,QAASH,EAAMA,MAAMxD,QAAQ0D,WAAWnF,QAErEkF,EAAME,eAAe,KAAM,QAASH,EAAMpK,UAAY,UAC/CqK,GAUTtS,EAAQ0S,QAAU,SAAUP,EAAGC,EAAGO,EAAOC,EAAQ3K,EAAWiJ,EAAeO,GACzE,GAAc,GAAVmB,EAAa,CACF,EAATA,IACFA,GAAU,GACVR,GAAKQ,EAEP,IAAIC,GAAO7S,EAAQwR,cAAc,OAAON,EAAeO,EACvDoB,GAAKL,eAAe,KAAM,IAAKL,EAAI,GAAMQ,GACzCE,EAAKL,eAAe,KAAM,IAAKJ,GAC/BS,EAAKL,eAAe,KAAM,QAASG,GACnCE,EAAKL,eAAe,KAAM,SAAUI,GACpCC,EAAKL,eAAe,KAAM,QAASvK,MAMnC,SAAShI,EAAQD,EAASM,GAgD9B,QAASW,GAAS6R,EAAMjE,GActB,IAZIiE,GAASzM,MAAMC,QAAQwM,IAAU/R,EAAKiE,YAAY8N,KACpDjE,EAAUiE,EACVA,EAAO,MAGT1S,KAAK2S,SAAWlE,MAChBzO,KAAK4S,SACL5S,KAAK6S,SAAW7S,KAAK2S,SAASG,SAAW,KACzC9S,KAAK+S,SAID/S,KAAK2S,SAAS7L,KAChB,IAAK,GAAIgI,KAAS9O,MAAK2S,SAAS7L,KAC9B,GAAI9G,KAAK2S,SAAS7L,KAAKhB,eAAegJ,GAAQ,CAC5C,GAAIzH,GAAQrH,KAAK2S,SAAS7L,KAAKgI,EAE7B9O,MAAK+S,MAAMjE,GADA,QAATzH,GAA4B,WAATA,GAA+B,WAATA,EACvB,OAGAA,EAO5B,GAAIrH,KAAK2S,SAAS9L,QAChB,KAAM,IAAIjD,OAAM,sDAGlB5D,MAAKgT,gBAGDN,GACF1S,KAAKiT,IAAIP,GAGX1S,KAAKkT,WAAWzE,GAtFlB,GAAI9N,GAAOT,EAAoB,GAC3Ba,EAAQb,EAAoB,EAiGhCW,GAAQsS,UAAUD,WAAa,SAASzE,GAClCA,GAA6BjI,SAAlBiI,EAAQ2E,QACjB3E,EAAQ2E,SAAU,EAEhBpT,KAAKqT,SACPrT,KAAKqT,OAAOC,gBACLtT,MAAKqT,SAKTrT,KAAKqT,SACRrT,KAAKqT,OAAStS,EAAMuE,OAAOtF,MACzBwK,SAAU,MAAO,SAAU,aAIF,gBAAlBiE,GAAQ2E,OACjBpT,KAAKqT,OAAOH,WAAWzE,EAAQ2E,UAevCvS,EAAQsS,UAAUI,GAAK,SAASjK,EAAOhB,GACrC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAC/BkK,KACHA,KACAxT,KAAKgT,aAAa1J,GAASkK,GAG7BA,EAAYxL,MACVM,SAAUA,KAKdzH,EAAQsS,UAAUM,UAAY5S,EAAQsS,UAAUI,GAOhD1S,EAAQsS,UAAUO,IAAM,SAASpK,EAAOhB,GACtC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAChCkK,KACFxT,KAAKgT,aAAa1J,GAASkK,EAAYG,OAAO,SAAU7K,GACtD,MAAQA,GAASR,UAAYA,MAMnCzH,EAAQsS,UAAUS,YAAc/S,EAAQsS,UAAUO,IASlD7S,EAAQsS,UAAUU,SAAW,SAAUvK,EAAOwK,EAAQC,GACpD,GAAa,KAATzK,EACF,KAAM,IAAI1F,OAAM,yBAGlB,IAAI4P,KACAlK,KAAStJ,MAAKgT,eAChBQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa1J,KAEjD,KAAOtJ,MAAKgT,eACdQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa,MAGrD,KAAK,GAAIxN,GAAI,EAAGA,EAAIgO,EAAY7N,OAAQH,IAAK,CAC3C,GAAIyO,GAAaT,EAAYhO,EACzByO,GAAW3L,UACb2L,EAAW3L,SAASgB,EAAOwK,EAAQC,GAAY,QAYrDlT,EAAQsS,UAAUF,IAAM,SAAUP,EAAMqB,GACtC,GACI1T,GADA6T,KAEAC,EAAKnU,IAET,IAAIiG,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CnF,EAAK8T,EAAGC,SAAS1B,EAAKlN,IACtB0O,EAASlM,KAAK3H,OAGb,IAAIM,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCrU,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,OAGb,CAAA,KAAIqS,YAAgBnM,SAMvB,KAAM,IAAI3C,OAAM,mBAJhBvD,GAAK8T,EAAGC,SAAS1B,GACjBwB,EAASlM,KAAK3H,GAUhB,MAJI6T,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAGnCG,GASTrT,EAAQsS,UAAU0B,OAAS,SAAUnC,EAAMqB,GACzC,GAAIG,MACAY,KACAC,KACAZ,EAAKnU,KACL8S,EAAUqB,EAAGtB,SAEbmC,EAAc,SAAU3F,GAC1B,GAAIhP,GAAKgP,EAAKyD,EACVqB,GAAGvB,MAAMvS,IAEXA,EAAK8T,EAAGc,YAAY5F,GACpByF,EAAW9M,KAAK3H,GAChB0U,EAAY/M,KAAKqH,KAIjBhP,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,IAIlB,IAAI4F,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CwP,EAAYtC,EAAKlN,QAGhB,IAAI7E,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCM,EAAY3F,OAGX,CAAA,KAAIqD,YAAgBnM,SAKvB,KAAM,IAAI3C,OAAM,mBAHhBoR,GAAYtC,GAad,MAPIwB,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAEtCe,EAAWnP,QACb3F,KAAK6T,SAAS,UAAW5R,MAAO6S,EAAYpC,KAAMqC,GAAchB,GAG3DG,EAASF,OAAOc,IAsCzBjU,EAAQsS,UAAU+B,IAAM,WACtB,GAGI7U,GAAI8U,EAAK1G,EAASiE,EAHlByB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAE3B/U,EAAKqF,UAAU,GACf+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,IAEG,SAAb0P,GAEPD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAI2P,EACJ,IAAI5G,GAAWA,EAAQ4G,WAAY,CACjC,GAAIC,IAAiB,YAAa,QAAS,SAG3C,IAFAD,EAA0D,IAA7CC,EAAc3O,QAAQ8H,EAAQ4G,YAAoB,QAAU5G,EAAQ4G,WAE7E3C,GAAS2C,GAAc1U,EAAKwG,QAAQuL,GACtC,KAAM,IAAI9O,OAAM,6BAA+BjD,EAAKwG,QAAQuL,GAAQ,sDACVjE,EAAQ3H,KAAO,IAE3E,IAAkB,aAAduO,IAA8B1U,EAAKiE,YAAY8N,GACjD,KAAM,IAAI9O,OAAM,6EAKlByR,GADO3C,GAC6B,aAAtB/R,EAAKwG,QAAQuL,GAAwB,YAGtC,OAIf,IAEgBrD,GAAMkG,EAAQ/P,EAAGC,EAF7BqB,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD6M,EAASlF,GAAWA,EAAQkF,OAC5B1R,IAGJ,IAAUuE,QAANnG,EAEFgP,EAAO8E,EAAGqB,SAASnV,EAAIyG,GACnB6M,IAAWA,EAAOtE,KACpBA,EAAO,UAGN,IAAW7I,QAAP2O,EAEP,IAAK3P,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrC6J,EAAO8E,EAAGqB,SAASL,EAAI3P,GAAIsB,KACtB6M,GAAUA,EAAOtE,KACpBpN,EAAM+F,KAAKqH,OAMf,KAAKkG,IAAUvV,MAAK4S,MACd5S,KAAK4S,MAAM9M,eAAeyP,KAC5BlG,EAAO8E,EAAGqB,SAASD,EAAQzO,KACtB6M,GAAUA,EAAOtE,KACpBpN,EAAM+F,KAAKqH,GAYnB,IALIZ,GAAWA,EAAQgH,OAAejP,QAANnG,GAC9BL,KAAK0V,MAAMzT,EAAOwM,EAAQgH,OAIxBhH,GAAWA,EAAQP,OAAQ,CAC7B,GAAIA,GAASO,EAAQP,MACrB,IAAU1H,QAANnG,EACFgP,EAAOrP,KAAK2V,cAActG,EAAMnB,OAGhC,KAAK1I,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCvD,EAAMuD,GAAKxF,KAAK2V,cAAc1T,EAAMuD,GAAI0I,GAM9C,GAAkB,aAAdmH,EAA2B,CAC7B,GAAIhB,GAAUrU,KAAKsU,gBAAgB5B,EACnC,IAAUlM,QAANnG,EAEF8T,EAAGyB,WAAWlD,EAAM2B,EAAShF,OAI7B,KAAK7J,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5B2O,EAAGyB,WAAWlD,EAAM2B,EAASpS,EAAMuD,GAGvC,OAAOkN,GAEJ,GAAkB,UAAd2C,EAAwB,CAC/B,GAAI1K,KACJ,KAAKnF,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5BmF,EAAO1I,EAAMuD,GAAGnF,IAAM4B,EAAMuD,EAE9B,OAAOmF,GAIP,GAAUnE,QAANnG,EAEF,MAAOgP,EAIP,IAAIqD,EAAM,CAER,IAAKlN,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCkN,EAAK1K,KAAK/F,EAAMuD,GAElB,OAAOkN,GAIP,MAAOzQ,IAcfpB,EAAQsS,UAAU0C,OAAS,SAAUpH,GACnC,GAIIjJ,GACAC,EACApF,EACAgP,EACApN,EARAyQ,EAAO1S,KAAK4S,MACZe,EAASlF,GAAWA,EAAQkF,OAC5B8B,EAAQhH,GAAWA,EAAQgH,MAC3B3O,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAMhDqO,IAEJ,IAAIxB,EAEF,GAAI8B,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACTpN,EAAM+F,KAAKqH,GAOjB,KAFArP,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACT8F,EAAInN,KAAKqH,EAAKrP,KAAK6S,gBAQ3B,IAAI4C,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,IACtB4B,EAAM+F,KAAK0K,EAAKrS,GAMpB,KAFAL,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOqD,EAAKrS,GACZ8U,EAAInN,KAAKqH,EAAKrP,KAAK6S,WAM3B,OAAOsC,IAOTtU,EAAQsS,UAAU2C,WAAa,WAC7B,MAAO9V,OAaTa,EAAQsS,UAAU9K,QAAU,SAAUC,EAAUmG,GAC9C,GAGIY,GACAhP,EAJAsT,EAASlF,GAAWA,EAAQkF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD4L,EAAO1S,KAAK4S,KAIhB,IAAInE,GAAWA,EAAQgH,MAIrB,IAAK,GAFDxT,GAAQjC,KAAKkV,IAAIzG,GAEZjJ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IAC3C6J,EAAOpN,EAAMuD,GACbnF,EAAKgP,EAAKrP,KAAK6S,UACfvK,EAAS+G,EAAMhP,OAKjB,KAAKA,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB/G,EAAS+G,EAAMhP,KAkBzBQ,EAAQsS,UAAU9F,IAAM,SAAU/E,EAAUmG,GAC1C,GAIIY,GAJAsE,EAASlF,GAAWA,EAAQkF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChDiP,KACArD,EAAO1S,KAAK4S,KAIhB,KAAK,GAAIvS,KAAMqS,GACTA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB0G,EAAY/N,KAAKM,EAAS+G,EAAMhP,IAUtC,OAJIoO,IAAWA,EAAQgH,OACrBzV,KAAK0V,MAAMK,EAAatH,EAAQgH,OAG3BM,GAUTlV,EAAQsS,UAAUwC,cAAgB,SAAUtG,EAAMnB,GAChD,GAAI8H,KAEJ,KAAK,GAAIlH,KAASO,GACZA,EAAKvJ,eAAegJ,IAAoC,IAAzBZ,EAAOvH,QAAQmI,KAChDkH,EAAalH,GAASO,EAAKP,GAI/B,OAAOkH,IASTnV,EAAQsS,UAAUuC,MAAQ,SAAUzT,EAAOwT,GACzC,GAAI9U,EAAKwD,SAASsR,GAAQ,CAExB,GAAIQ,GAAOR,CACXxT,GAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,GAAI+P,GAAK5Q,EAAE0Q,GACPG,EAAKhQ,EAAE6P,EACX,OAAQE,GAAKC,EAAM,EAAWA,EAALD,EAAW,GAAK,QAGxC,CAAA,GAAqB,kBAAVV,GAOd,KAAM,IAAIpP,WAAU,uCALpBpE,GAAMiU,KAAKT,KAgBf5U,EAAQsS,UAAUkD,OAAS,SAAUhW,EAAI0T,GACvC,GACIvO,GAAGC,EAAK6Q,EADRC,IAGJ,IAAItQ,MAAMC,QAAQ7F,GAChB,IAAKmF,EAAI,EAAGC,EAAMpF,EAAGsF,OAAYF,EAAJD,EAASA,IACpC8Q,EAAYtW,KAAKwW,QAAQnW,EAAGmF,IACX,MAAb8Q,GACFC,EAAWvO,KAAKsO,OAKpBA,GAAYtW,KAAKwW,QAAQnW,GACR,MAAbiW,GACFC,EAAWvO,KAAKsO,EAQpB,OAJIC,GAAW5Q,QACb3F,KAAK6T,SAAS,UAAW5R,MAAOsU,GAAaxC,GAGxCwC,GAST1V,EAAQsS,UAAUqD,QAAU,SAAUnW,GACpC,GAAIM,EAAKqD,SAAS3D,IAAOM,EAAKwD,SAAS9D,IACrC,GAAIL,KAAK4S,MAAMvS,GAEb,aADOL,MAAK4S,MAAMvS,GACXA,MAGN,IAAIA,YAAckG,QAAQ,CAC7B,GAAIgP,GAASlV,EAAGL,KAAK6S,SACrB,IAAI0C,GAAUvV,KAAK4S,MAAM2C,GAEvB,aADOvV,MAAK4S,MAAM2C,GACXA,EAGX,MAAO,OAQT1U,EAAQsS,UAAUsD,MAAQ,SAAU1C,GAClC,GAAIoB,GAAM5O,OAAO6G,KAAKpN,KAAK4S,MAM3B,OAJA5S,MAAK4S,SAEL5S,KAAK6T,SAAS,UAAW5R,MAAOkT,GAAMpB,GAE/BoB,GAQTtU,EAAQsS,UAAUzG,IAAM,SAAUoC,GAChC,GAAI4D,GAAO1S,KAAK4S,MACZlG,EAAM,KACNgK,EAAW,IAEf,KAAK,GAAIrW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuBjK,GAAOiK,EAAYD,KAC5ChK,EAAM2C,EACNqH,EAAWC,GAKjB,MAAOjK,IAQT7L,EAAQsS,UAAUrH,IAAM,SAAUgD,GAChC,GAAI4D,GAAO1S,KAAK4S,MACZ9G,EAAM,KACN8K,EAAW,IAEf,KAAK,GAAIvW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuB7K,GAAmB8K,EAAZD,KAChC7K,EAAMuD,EACNuH,EAAWD,GAKjB,MAAO7K,IAUTjL,EAAQsS,UAAU0D,SAAW,SAAU/H,GACrC,GAIItJ,GAJAkN,EAAO1S,KAAK4S,MACZkE,KACAC,EAAY/W,KAAK2S,SAAS7L,MAAQ9G,KAAK2S,SAAS7L,KAAKgI,IAAU,KAC/DkI,EAAQ,CAGZ,KAAK,GAAInR,KAAQ6M,GACf,GAAIA,EAAK5M,eAAeD,GAAO,CAC7B,GAAIwJ,GAAOqD,EAAK7M,GACZwB,EAAQgI,EAAKP,GACbmI,GAAS,CACb,KAAKzR,EAAI,EAAOwR,EAAJxR,EAAWA,IACrB,GAAIsR,EAAOtR,IAAM6B,EAAO,CACtB4P,GAAS,CACT,OAGCA,GAAqBzQ,SAAVa,IACdyP,EAAOE,GAAS3P,EAChB2P,KAKN,GAAID,EACF,IAAKvR,EAAI,EAAGA,EAAIsR,EAAOnR,OAAQH,IAC7BsR,EAAOtR,GAAK7E,EAAKkG,QAAQiQ,EAAOtR,GAAIuR,EAIxC,OAAOD,IASTjW,EAAQsS,UAAUiB,SAAW,SAAU/E,GACrC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SAEnB,IAAUrM,QAANnG,GAEF,GAAIL,KAAK4S,MAAMvS,GAEb,KAAM,IAAIuD,OAAM,iCAAmCvD,EAAK,uBAK1DA,GAAKM,EAAKqE,aACVqK,EAAKrP,KAAK6S,UAAYxS,CAGxB,IAAIsM,KACJ,KAAK,GAAImC,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GAAY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAKzC,MAFA/W,MAAK4S,MAAMvS,GAAMsM,EAEVtM,GAUTQ,EAAQsS,UAAUqC,SAAW,SAAUnV,EAAI6W,GACzC,GAAIpI,GAAOzH,EAGP8P,EAAMnX,KAAK4S,MAAMvS,EACrB,KAAK8W,EACH,MAAO,KAIT,IAAIC,KACJ,IAAIF,EACF,IAAKpI,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASnO,EAAKkG,QAAQQ,EAAO6P,EAAMpI,SAMjD,KAAKA,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASzH,EAIzB,OAAO+P,IAWTvW,EAAQsS,UAAU8B,YAAc,SAAU5F,GACxC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SACnB,IAAUrM,QAANnG,EACF,KAAM,IAAIuD,OAAM,6CAA+CyT,KAAKC,UAAUjI,GAAQ,IAExF,IAAI1C,GAAI3M,KAAK4S,MAAMvS,EACnB,KAAKsM,EAEH,KAAM,IAAI/I,OAAM,uCAAyCvD,EAAK,SAIhE,KAAK,GAAIyO,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GAAY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAIzC,MAAO1W,IASTQ,EAAQsS,UAAUmB,gBAAkB,SAAUiD,GAE5C,IAAK,GADDlD,MACKK,EAAM,EAAGC,EAAO4C,EAAUC,qBAA4B7C,EAAND,EAAYA,IACnEL,EAAQK,GAAO6C,EAAUE,YAAY/C,IAAQ6C,EAAUG,eAAehD,EAExE,OAAOL,IAUTxT,EAAQsS,UAAUyC,WAAa,SAAU2B,EAAWlD,EAAShF,GAG3D,IAAK,GAFDkF,GAAMgD,EAAUI,SAEXjD,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpB6C,GAAUK,SAASrD,EAAKG,EAAKrF,EAAKP,MAItCjP,EAAOD,QAAUiB,GAKb,SAAShB,EAAQD,EAASM,GAe9B,QAASY,GAAU4R,EAAMjE,GACvBzO,KAAK4S,MAAQ,KACb5S,KAAK6X,QACL7X,KAAK2S,SAAWlE,MAChBzO,KAAK6S,SAAW,KAChB7S,KAAKgT,eAEL,IAAImB,GAAKnU,IACTA,MAAK8I,SAAW,WACdqL,EAAG2D,SAASC,MAAM5D,EAAIzO,YAGxB1F,KAAKgY,QAAQtF,GAzBf,GAAI/R,GAAOT,EAAoB,GAC3BW,EAAUX,EAAoB,EAkClCY,GAASqS,UAAU6E,QAAU,SAAUtF,GACrC,GAAIyC,GAAK3P,EAAGC,CAEZ,IAAIzF,KAAK4S,MAAO,CAEV5S,KAAK4S,MAAMgB,aACb5T,KAAK4S,MAAMgB,YAAY,IAAK5T,KAAK8I,UAInCqM,IACA,KAAK,GAAI9U,KAAML,MAAK6X,KACd7X,KAAK6X,KAAK/R,eAAezF,IAC3B8U,EAAInN,KAAK3H,EAGbL,MAAK6X,QACL7X,KAAK6T,SAAS,UAAW5R,MAAOkT,IAKlC,GAFAnV,KAAK4S,MAAQF,EAET1S,KAAK4S,MAAO,CAQd,IANA5S,KAAK6S,SAAW7S,KAAK2S,SAASG,SACzB9S,KAAK4S,OAAS5S,KAAK4S,MAAMnE,SAAWzO,KAAK4S,MAAMnE,QAAQqE,SACxD,KAGJqC,EAAMnV,KAAK4S,MAAMiD,QAAQlC,OAAQ3T,KAAK2S,UAAY3S,KAAK2S,SAASgB,SAC3DnO,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACTxF,KAAK6X,KAAKxX,IAAM,CAElBL,MAAK6T,SAAS,OAAQ5R,MAAOkT,IAGzBnV,KAAK4S,MAAMW,IACbvT,KAAK4S,MAAMW,GAAG,IAAKvT,KAAK8I,YAuC9BhI,EAASqS,UAAU+B,IAAM,WACvB,GAGIC,GAAK1G,EAASiE,EAHdyB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAAsC,SAAbA,GAEpDD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAIuS,GAActX,EAAK2E,UAAWtF,KAAK2S,SAAUlE,EAG7CzO,MAAK2S,SAASgB,QAAUlF,GAAWA,EAAQkF,SAC7CsE,EAAYtE,OAAS,SAAUtE,GAC7B,MAAO8E,GAAGxB,SAASgB,OAAOtE,IAASZ,EAAQkF,OAAOtE,IAKtD,IAAI6I,KAOJ,OANW1R,SAAP2O,GACF+C,EAAalQ,KAAKmN,GAEpB+C,EAAalQ,KAAKiQ,GAClBC,EAAalQ,KAAK0K,GAEX1S,KAAK4S,OAAS5S,KAAK4S,MAAMsC,IAAI6C,MAAM/X,KAAK4S,MAAOsF,IAWxDpX,EAASqS,UAAU0C,OAAS,SAAUpH,GACpC,GAAI0G,EAEJ,IAAInV,KAAK4S,MAAO,CACd,GACIe,GADAwE,EAAgBnY,KAAK2S,SAASgB,MAK9BA,GAFAlF,GAAWA,EAAQkF,OACjBwE,EACO,SAAU9I,GACjB,MAAO8I,GAAc9I,IAASZ,EAAQkF,OAAOtE,IAItCZ,EAAQkF,OAIVwE,EAGXhD,EAAMnV,KAAK4S,MAAMiD,QACflC,OAAQA,EACR8B,MAAOhH,GAAWA,EAAQgH,YAI5BN,KAGF,OAAOA,IAQTrU,EAASqS,UAAU2C,WAAa,WAE9B,IADA,GAAIsC,GAAUpY,KACPoY,YAAmBtX,IACxBsX,EAAUA,EAAQxF,KAEpB,OAAOwF,IAAW,MAYpBtX,EAASqS,UAAU2E,SAAW,SAAUxO,EAAOwK,EAAQC,GACrD,GAAIvO,GAAGC,EAAKpF,EAAIgP,EACZ8F,EAAMrB,GAAUA,EAAO7R,MACvByQ,EAAO1S,KAAK4S,MACZyF,KACAC,KACAC,IAEJ,IAAIpD,GAAOzC,EAAM,CACf,OAAQpJ,GACN,IAAK,MAEH,IAAK9D,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GACZgP,IACFrP,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,GAIf,MAEF,KAAK,SAGH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GAEZgP,EACErP,KAAK6X,KAAKxX,GACZiY,EAAQtQ,KAAK3H,IAGbL,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,IAITL,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,GAQnB,MAEF,KAAK,SAEH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACLxF,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,IAOjBgY,EAAM1S,QACR3F,KAAK6T,SAAS,OAAQ5R,MAAOoW,GAAQtE,GAEnCuE,EAAQ3S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOqW,GAAUvE,GAExCwE,EAAQ5S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOsW,GAAUxE,KAMhDjT,EAASqS,UAAUI,GAAK1S,EAAQsS,UAAUI,GAC1CzS,EAASqS,UAAUO,IAAM7S,EAAQsS,UAAUO,IAC3C5S,EAASqS,UAAUU,SAAWhT,EAAQsS,UAAUU,SAGhD/S,EAASqS,UAAUM,UAAY3S,EAASqS,UAAUI,GAClDzS,EAASqS,UAAUS,YAAc9S,EAASqS,UAAUO,IAEpD7T,EAAOD,QAAUkB,GAIb,SAASjB,GAeb,QAASkB,GAAM0N,GAEbzO,KAAKwY,MAAQ,KACbxY,KAAK0M,IAAM+L,IAGXzY,KAAKqT,UACLrT,KAAK0Y,SAAW,KAChB1Y,KAAK2Y,UAAY,KAEjB3Y,KAAKkT,WAAWzE,GAgBlB1N,EAAMoS,UAAUD,WAAa,SAAUzE,GACjCA,GAAoC,mBAAlBA,GAAQ+J,QAC5BxY,KAAKwY,MAAQ/J,EAAQ+J,OAEnB/J,GAAkC,mBAAhBA,GAAQ/B,MAC5B1M,KAAK0M,IAAM+B,EAAQ/B,KAGrB1M,KAAK4Y,kBAsBP7X,EAAMuE,OAAS,SAAUrB,EAAQwK,GAC/B,GAAI2E,GAAQ,GAAIrS,GAAM0N,EAEtB,IAAqBjI,SAAjBvC,EAAO4U,MACT,KAAM,IAAIjV,OAAM,6CAElBK,GAAO4U,MAAQ,WACbzF,EAAMyF,QAGR,IAAIC,KACF7C,KAAM,QACN8C,SAAUvS,QAGZ,IAAIiI,GAAWA,EAAQjE,QACrB,IAAK,GAAIhF,GAAI,EAAGA,EAAIiJ,EAAQjE,QAAQ7E,OAAQH,IAAK,CAC/C,GAAIyQ,GAAOxH,EAAQjE,QAAQhF,EAC3BsT,GAAQ9Q,MACNiO,KAAMA,EACN8C,SAAU9U,EAAOgS,KAEnB7C,EAAM5I,QAAQvG,EAAQgS,GAS1B,MALA7C,GAAMuF,WACJ1U,OAAQA,EACR6U,QAASA,GAGJ1F,GAOTrS,EAAMoS,UAAUG,QAAU,WAGxB,GAFAtT,KAAK6Y,QAED7Y,KAAK2Y,UAAW,CAGlB,IAAK,GAFD1U,GAASjE,KAAK2Y,UAAU1U,OACxB6U,EAAU9Y,KAAK2Y,UAAUG,QACpBtT,EAAI,EAAGA,EAAIsT,EAAQnT,OAAQH,IAAK,CACvC,GAAIwT,GAASF,EAAQtT,EACjBwT,GAAOD,SACT9U,EAAO+U,EAAO/C,MAAQ+C,EAAOD,eAGtB9U,GAAO+U,EAAO/C,MAGzBjW,KAAK2Y,UAAY,OASrB5X,EAAMoS,UAAU3I,QAAU,SAASvG,EAAQ+U,GACzC,GAAI7E,GAAKnU,KACL+Y,EAAW9U,EAAO+U,EACtB,KAAKD,EACH,KAAM,IAAInV,OAAM,UAAYoV,EAAS,aAGvC/U,GAAO+U,GAAU,WAGf,IAAK,GADDC,MACKzT,EAAI,EAAGA,EAAIE,UAAUC,OAAQH,IACpCyT,EAAKzT,GAAKE,UAAUF,EAItB2O,GAAGf,OACD6F,KAAMA,EACNC,GAAIH,EACJI,QAASnZ,SASfe,EAAMoS,UAAUC,MAAQ,SAASgG,GAE7BpZ,KAAKqT,OAAOrL,KADO,kBAAVoR,IACSF,GAAIE,GAGLA,GAGnBpZ,KAAK4Y,kBAOP7X,EAAMoS,UAAUyF,eAAiB,WAQ/B,GANI5Y,KAAKqT,OAAO1N,OAAS3F,KAAK0M,KAC5B1M,KAAK6Y,QAIPQ,aAAarZ,KAAK0Y,UACd1Y,KAAKoT,MAAMzN,OAAS,GAA2B,gBAAf3F,MAAKwY,MAAoB,CAC3D,GAAIrE,GAAKnU,IACTA,MAAK0Y,SAAWY,WAAW,WACzBnF,EAAG0E,SACF7Y,KAAKwY,SAOZzX,EAAMoS,UAAU0F,MAAQ,WACtB,KAAO7Y,KAAKqT,OAAO1N,OAAS,GAAG,CAC7B,GAAIyT,GAAQpZ,KAAKqT,OAAO/B,OACxB8H,GAAMF,GAAGnB,MAAMqB,EAAMD,SAAWC,EAAMF,GAAIE,EAAMH,YAIpDpZ,EAAOD,QAAUmB,GAKb,SAASlB,EAAQD,EAASM,GAwB9B,QAASc,GAAQuY,EAAW7G,EAAMjE,GAChC,KAAMzO,eAAgBgB,IACpB,KAAM,IAAIwY,aAAY,mDAIxBxZ,MAAKyZ,iBAAmBF,EACxBvZ,KAAKuS,MAAQ,QACbvS,KAAKwS,OAAS,QACdxS,KAAK0Z,OAAS,GACd1Z,KAAK2Z,eAAiB,MACtB3Z,KAAK4Z,eAAiB,MAEtB5Z,KAAK6Z,OAAS,IACd7Z,KAAK8Z,OAAS,IACd9Z,KAAK+Z,OAAS,GAEd,IAAIC,GAAc,SAASnO,GAAK,MAAOA,GACvC7L,MAAKia,YAAcD,EACnBha,KAAKka,YAAcF,EACnBha,KAAKma,YAAcH,EAEnBha,KAAKoa,YAAc,OACnBpa,KAAKqa,YAAc,QAEnBra,KAAKiN,MAAQjM,EAAQsZ,MAAMC,IAC3Bva,KAAKwa,iBAAkB,EACvBxa,KAAKya,UAAW,EAChBza,KAAK0a,iBAAkB,EACvB1a,KAAK2a,YAAa,EAClB3a,KAAK4a,gBAAiB,EACtB5a,KAAK6a,aAAc,EACnB7a,KAAK8a,cAAgB,GAErB9a,KAAK+a,kBAAoB,IACzB/a,KAAKgb,kBAAmB,EAExBhb,KAAKib,OAAS,GAAI/Z,GAClBlB,KAAKkb,IAAM,GAAI7Z,GAAQ,EAAG,EAAG,IAE7BrB,KAAKuX,UAAY,KACjBvX,KAAKmb,WAAa,KAGlBnb,KAAKob,KAAO5U,OACZxG,KAAKqb,KAAO7U,OACZxG,KAAKsb,KAAO9U,OACZxG,KAAKub,SAAW/U,OAChBxG,KAAKwb,UAAYhV,OAEjBxG,KAAKyb,KAAO,EACZzb,KAAK0b,MAAQlV,OACbxG,KAAK2b,KAAO,EACZ3b,KAAK4b,KAAO,EACZ5b,KAAK6b,MAAQrV,OACbxG,KAAK8b,KAAO,EACZ9b,KAAK+b,KAAO,EACZ/b,KAAKgc,MAAQxV,OACbxG,KAAKic,KAAO,EACZjc,KAAKkc,SAAW,EAChBlc,KAAKmc,SAAW,EAChBnc,KAAKoc,UAAY,EACjBpc,KAAKqc,UAAY,EAIjBrc,KAAKsc,UAAY,UACjBtc,KAAKuc,UAAY,UACjBvc,KAAKwc,SAAW,UAChBxc,KAAKyc,eAAiB,UAGtBzc,KAAKqO,SAGLrO,KAAKkT,WAAWzE,GAGZiE,GACF1S,KAAKgY,QAAQtF,GAknEjB,QAASgK,GAAWpT,GAClB,MAAI,WAAaA,GAAcA,EAAMqT,QAC9BrT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGD,SAAW,EAQrE,QAASE,GAAWvT,GAClB,MAAI,WAAaA,GAAcA,EAAMwT,QAC9BxT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGE,SAAW,EAnuErE,GAAIC,GAAU7c,EAAoB,IAC9BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BS,EAAOT,EAAoB,GAC3BmB,EAAUnB,EAAoB,IAC9BkB,EAAUlB,EAAoB,GAC9BgB,EAAShB,EAAoB,GAC7BiB,EAASjB,EAAoB,GAC7BoB,EAASpB,EAAoB,IAC7BqB,EAAarB,EAAoB,GAiGrC6c,GAAQ/b,EAAQmS,WAKhBnS,EAAQmS,UAAU6J,UAAY,WAC5Bhd,KAAKid,MAAQ,GAAI5b,GAAQ,GAAKrB,KAAK2b,KAAO3b,KAAKyb,MAC7C,GAAKzb,KAAK8b,KAAO9b,KAAK4b,MACtB,GAAK5b,KAAKic,KAAOjc,KAAK+b,OAGpB/b,KAAK0a,kBACH1a,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,EAE5BhS,KAAKid,MAAMjL,EAAIhS,KAAKid,MAAMlL,EAI1B/R,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,GAK9BhS,KAAKid,MAAMC,GAAKld,KAAK8a,cAIrB9a,KAAKid,MAAM5V,MAAQ,GAAKrH,KAAKmc,SAAWnc,KAAKkc,SAG7C,IAAIiB,IAAWnd,KAAK2b,KAAO3b,KAAKyb,MAAQ,EAAIzb,KAAKid,MAAMlL,EACnDqL,GAAWpd,KAAK8b,KAAO9b,KAAK4b,MAAQ,EAAI5b,KAAKid,MAAMjL,EACnDqL,GAAWrd,KAAKic,KAAOjc,KAAK+b,MAAQ,EAAI/b,KAAKid,MAAMC,CACvDld,MAAKib,OAAOqC,eAAeH,EAASC,EAASC,IAU/Crc,EAAQmS,UAAUoK,eAAiB,SAASC,GAC1C,GAAIC,GAAczd,KAAK0d,2BAA2BF,EAClD,OAAOxd,MAAK2d,4BAA4BF,IAW1Czc,EAAQmS,UAAUuK,2BAA6B,SAASF,GACtD,GAAII,GAAKJ,EAAQzL,EAAI/R,KAAKid,MAAMlL,EAC9B8L,EAAKL,EAAQxL,EAAIhS,KAAKid,MAAMjL,EAC5B8L,EAAKN,EAAQN,EAAIld,KAAKid,MAAMC,EAE5Ba,EAAK/d,KAAKib,OAAO+C,oBAAoBjM,EACrCkM,EAAKje,KAAKib,OAAO+C,oBAAoBhM,EACrCkM,EAAKle,KAAKib,OAAO+C,oBAAoBd,EAGrCiB,EAAQjZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBtM,GACjDuM,EAAQpZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBtM,GACjDyM,EAAQtZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBrM,GACjDyM,EAAQvZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBrM,GACjD0M,EAAQxZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBnB,GACjDyB,EAAQzZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBnB,GAGjD0B,EAAKH,GAASC,GAASb,EAAKI,GAAMU,GAASf,EAAKG,IAAOS,GAASV,EAAKI,GACrEW,EAAKV,GAASM,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQO,GAASK,GAASd,EAAKI,GAAMS,GAASd,EAAGG,IAC9He,EAAKR,GAASG,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQI,GAASQ,GAASd,EAAKI,GAAMS,GAASd,EAAGG,GAEhI,OAAO,IAAI1c,GAAQud,EAAIC,EAAIC,IAU7B9d,EAAQmS,UAAUwK,4BAA8B,SAASF,GACvD,GAQIsB,GACAC,EATAC,EAAKjf,KAAKkb,IAAInJ,EAChBmN,EAAKlf,KAAKkb,IAAIlJ,EACdmN,EAAKnf,KAAKkb,IAAIgC,EACd0B,EAAKnB,EAAY1L,EACjB8M,EAAKpB,EAAYzL,EACjB8M,EAAKrB,EAAYP,CAgBnB,OAXIld,MAAKwa,iBACPuE,GAAMH,EAAKK,IAAOE,EAAKL,GACvBE,GAAMH,EAAKK,IAAOC,EAAKL,KAGvBC,EAAKH,IAAOO,EAAKnf,KAAKib,OAAOmE,gBAC7BJ,EAAKH,IAAOM,EAAKnf,KAAKib,OAAOmE,iBAKxB,GAAIhe,GACTpB,KAAKqf,QAAUN,EAAK/e,KAAKsf,MAAMC,OAAOC,YACtCxf,KAAKyf,QAAUT,EAAKhf,KAAKsf,MAAMC,OAAOC,cAO1Cxe,EAAQmS,UAAUuM,oBAAsB,SAASC,GAC/C,GAAIC,GAAO,QACPC,EAAS,OACTC,EAAc,CAElB,IAAgC,gBAAtB,GACRF,EAAOD,EACPE,EAAS,OACTC,EAAc,MAEX,IAAgC,gBAAtB,GACgBtZ,SAAzBmZ,EAAgBC,OAAuBA,EAAOD,EAAgBC,MACnCpZ,SAA3BmZ,EAAgBE,SAAyBA,EAASF,EAAgBE,QAClCrZ,SAAhCmZ,EAAgBG,cAA2BA,EAAcH,EAAgBG,iBAE1E,IAAyBtZ,SAApBmZ,EAIR,KAAM,qCAGR3f,MAAKsf,MAAMrS,MAAM0S,gBAAkBC,EACnC5f,KAAKsf,MAAMrS,MAAM8S,YAAcF,EAC/B7f,KAAKsf,MAAMrS,MAAM+S,YAAcF,EAAc,KAC7C9f,KAAKsf,MAAMrS,MAAMgT,YAAc,SAKjCjf,EAAQsZ,OACN4F,IAAK,EACLC,SAAU,EACVC,QAAS,EACT7F,IAAM,EACN8F,QAAU,EACVC,SAAU,EACVC,QAAS,EACTC,KAAO,EACPC,KAAM,EACNC,QAAU,GASZ1f,EAAQmS,UAAUwN,gBAAkB,SAASC,GAC3C,OAAQA,GACN,IAAK,MAAW,MAAO5f,GAAQsZ,MAAMC,GACrC,KAAK,WAAa,MAAOvZ,GAAQsZ,MAAM+F,OACvC,KAAK,YAAe,MAAOrf,GAAQsZ,MAAMgG,QACzC,KAAK,WAAa,MAAOtf,GAAQsZ,MAAMiG,OACvC,KAAK,OAAW,MAAOvf,GAAQsZ,MAAMmG,IACrC,KAAK,OAAW,MAAOzf,GAAQsZ,MAAMkG,IACrC,KAAK,UAAa,MAAOxf,GAAQsZ,MAAMoG,OACvC,KAAK,MAAW,MAAO1f,GAAQsZ,MAAM4F,GACrC,KAAK,YAAe,MAAOlf,GAAQsZ,MAAM6F,QACzC,KAAK,WAAa,MAAOnf,GAAQsZ,MAAM8F,QAGzC,MAAO,IAQTpf,EAAQmS,UAAU0N,wBAA0B,SAASnO,GACnD,GAAI1S,KAAKiN,QAAUjM,EAAQsZ,MAAMC,KAC/Bva,KAAKiN,QAAUjM,EAAQsZ,MAAM+F,SAC7BrgB,KAAKiN,QAAUjM,EAAQsZ,MAAMmG,MAC7BzgB,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC7BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,SAC7B1gB,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,IAE7BlgB,KAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW/U,OAEZkM,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,OAGhB,CAAA,GAAIxb,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UACpCtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,SAC7BvgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAY7B,KAAM,kBAAoBpgB,KAAKiN,MAAQ,GAVvCjN,MAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW,EAEZ7I,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,KAQvBxa,EAAQmS,UAAUsB,gBAAkB,SAAS/B,GAC3C,MAAOA,GAAK/M,QAId3E,EAAQmS,UAAUqE,mBAAqB,SAAS9E,GAC9C,GAAIoO,GAAU,CACd,KAAK,GAAIC,KAAUrO,GAAK,GAClBA,EAAK,GAAG5M,eAAeib,IACzBD,GAGJ,OAAOA,IAIT9f,EAAQmS,UAAU6N,kBAAoB,SAAStO,EAAMqO,GAEnD,IAAK,GADDE,MACKzb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IACgB,IAA3Cyb,EAAeta,QAAQ+L,EAAKlN,GAAGub,KACjCE,EAAejZ,KAAK0K,EAAKlN,GAAGub,GAGhC,OAAOE,IAITjgB,EAAQmS,UAAU+N,eAAiB,SAASxO,EAAKqO,GAE/C,IAAK,GADDI,IAAUrV,IAAI4G,EAAK,GAAGqO,GAAQrU,IAAIgG,EAAK,GAAGqO,IACrCvb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B2b,EAAOrV,IAAM4G,EAAKlN,GAAGub,KAAWI,EAAOrV,IAAM4G,EAAKlN,GAAGub,IACrDI,EAAOzU,IAAMgG,EAAKlN,GAAGub,KAAWI,EAAOzU,IAAMgG,EAAKlN,GAAGub,GAE3D,OAAOI,IASTngB,EAAQmS,UAAUiO,gBAAkB,SAAUC,GAC5C,GAAIlN,GAAKnU,IAOT,IAJIA,KAAKoY,SACPpY,KAAKoY,QAAQ1E,IAAI,IAAK1T,KAAKshB,WAGb9a,SAAZ6a,EAAJ,CAGIpb,MAAMC,QAAQmb,KAChBA,EAAU,GAAIxgB,GAAQwgB,GAGxB,IAAI3O,EACJ,MAAI2O,YAAmBxgB,IAAWwgB,YAAmBvgB,IAInD,KAAM,IAAI8C,OAAM,uCAGlB,IANE8O,EAAO2O,EAAQnM,MAME,GAAfxC,EAAK/M,OAAT,CAGA3F,KAAKoY,QAAUiJ,EACfrhB,KAAKuX,UAAY7E,EAGjB1S,KAAKshB,UAAY,WACfnN,EAAG6D,QAAQ7D,EAAGiE,UAEhBpY,KAAKoY,QAAQ7E,GAAG,IAAKvT,KAAKshB,WAS1BthB,KAAKob,KAAO,IACZpb,KAAKqb,KAAO,IACZrb,KAAKsb,KAAO,IACZtb,KAAKub,SAAW,QAChBvb,KAAKwb,UAAY,SAKb9I,EAAK,GAAG5M,eAAe,WACDU,SAApBxG,KAAKuhB,aACPvhB,KAAKuhB,WAAa,GAAIpgB,GAAOkgB,EAASrhB,KAAKwb,UAAWxb,MACtDA,KAAKuhB,WAAWC,kBAAkB,WAAYrN,EAAGsN,WAKrD,IAAIC,GAAW1hB,KAAKiN,OAASjM,EAAQsZ,MAAM4F,KACzClgB,KAAKiN,OAASjM,EAAQsZ,MAAM6F,UAC5BngB,KAAKiN,OAASjM,EAAQsZ,MAAM8F,OAG9B,IAAIsB,EAAU,CACZ,GAA8Blb,SAA1BxG,KAAK2hB,iBACP3hB,KAAKoc,UAAYpc,KAAK2hB,qBAEnB,CACH,GAAIC,GAAQ5hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKob,KAC7Cpb,MAAKoc,UAAawF,EAAM,GAAKA,EAAM,IAAO,EAG5C,GAA8Bpb,SAA1BxG,KAAK6hB,iBACP7hB,KAAKqc,UAAYrc,KAAK6hB,qBAEnB,CACH,GAAIC,GAAQ9hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKqb,KAC7Crb,MAAKqc,UAAayF,EAAM,GAAKA,EAAM,IAAO,GAK9C,GAAIC,GAAS/hB,KAAKkhB,eAAexO,EAAK1S,KAAKob,KACvCsG,KACFK,EAAOjW,KAAO9L,KAAKoc,UAAY,EAC/B2F,EAAOrV,KAAO1M,KAAKoc,UAAY,GAEjCpc,KAAKyb,KAA6BjV,SAArBxG,KAAKgiB,YAA6BhiB,KAAKgiB,YAAcD,EAAOjW,IACzE9L,KAAK2b,KAA6BnV,SAArBxG,KAAKiiB,YAA6BjiB,KAAKiiB,YAAcF,EAAOrV,IACrE1M,KAAK2b,MAAQ3b,KAAKyb,OAAMzb,KAAK2b,KAAO3b,KAAKyb,KAAO,GACpDzb,KAAK0b,MAA+BlV,SAAtBxG,KAAKkiB,aAA8BliB,KAAKkiB,cAAgBliB,KAAK2b,KAAK3b,KAAKyb,MAAM,CAE3F,IAAI0G,GAASniB,KAAKkhB,eAAexO,EAAK1S,KAAKqb,KACvCqG,KACFS,EAAOrW,KAAO9L,KAAKqc,UAAY,EAC/B8F,EAAOzV,KAAO1M,KAAKqc,UAAY,GAEjCrc,KAAK4b,KAA6BpV,SAArBxG,KAAKoiB,YAA6BpiB,KAAKoiB,YAAcD,EAAOrW,IACzE9L,KAAK8b,KAA6BtV,SAArBxG,KAAKqiB,YAA6BriB,KAAKqiB,YAAcF,EAAOzV,IACrE1M,KAAK8b,MAAQ9b,KAAK4b,OAAM5b,KAAK8b,KAAO9b,KAAK4b,KAAO,GACpD5b,KAAK6b,MAA+BrV,SAAtBxG,KAAKsiB,aAA8BtiB,KAAKsiB,cAAgBtiB,KAAK8b,KAAK9b,KAAK4b,MAAM,CAE3F;GAAI2G,GAASviB,KAAKkhB,eAAexO,EAAK1S,KAAKsb,KAM3C,IALAtb,KAAK+b,KAA6BvV,SAArBxG,KAAKwiB,YAA6BxiB,KAAKwiB,YAAcD,EAAOzW,IACzE9L,KAAKic,KAA6BzV,SAArBxG,KAAKyiB,YAA6BziB,KAAKyiB,YAAcF,EAAO7V,IACrE1M,KAAKic,MAAQjc,KAAK+b,OAAM/b,KAAKic,KAAOjc,KAAK+b,KAAO,GACpD/b,KAAKgc,MAA+BxV,SAAtBxG,KAAK0iB,aAA8B1iB,KAAK0iB,cAAgB1iB,KAAKic,KAAKjc,KAAK+b,MAAM,EAErEvV,SAAlBxG,KAAKub,SAAwB,CAC/B,GAAIoH,GAAa3iB,KAAKkhB,eAAexO,EAAK1S,KAAKub,SAC/Cvb,MAAKkc,SAAqC1V,SAAzBxG,KAAK4iB,gBAAiC5iB,KAAK4iB,gBAAkBD,EAAW7W,IACzF9L,KAAKmc,SAAqC3V,SAAzBxG,KAAK6iB,gBAAiC7iB,KAAK6iB,gBAAkBF,EAAWjW,IACrF1M,KAAKmc,UAAYnc,KAAKkc,WAAUlc,KAAKmc,SAAWnc,KAAKkc,SAAW,GAItElc,KAAKgd,eAUPhc,EAAQmS,UAAU2P,eAAiB,SAAUpQ,GAE3C,GAAIX,GAAGC,EAAGxM,EAAG0X,EAAG6F,EAAK7Q,EAEjBiJ,IAEJ,IAAInb,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC/BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,QAAS,CAKtC,GAAIkB,MACAE,IACJ,KAAKtc,EAAI,EAAGA,EAAIxF,KAAKyU,gBAAgB/B,GAAOlN,IAC1CuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAED,KAArBuG,EAAMjb,QAAQoL,IAChB6P,EAAM5Z,KAAK+J,GAEY,KAArB+P,EAAMnb,QAAQqL,IAChB8P,EAAM9Z,KAAKgK,EAIf,IAAIgR,GAAa,SAAUzd,EAAGa,GAC5B,MAAOb,GAAIa,EAEbwb,GAAM1L,KAAK8M,GACXlB,EAAM5L,KAAK8M,EAGX,IAAIC,KACJ,KAAKzd,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAAK,CAChCuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAC1B6B,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,CAE1B,IAAI4H,GAAStB,EAAMjb,QAAQoL,GACvBoR,EAASrB,EAAMnb,QAAQqL,EAEAxL,UAAvByc,EAAWC,KACbD,EAAWC,MAGb,IAAI1F,GAAU,GAAInc,EAClBmc,GAAQzL,EAAIA,EACZyL,EAAQxL,EAAIA,EACZwL,EAAQN,EAAIA,EAEZ6F,KACAA,EAAI7Q,MAAQsL,EACZuF,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OACbuc,EAAIO,OAAS,GAAIjiB,GAAQ0Q,EAAGC,EAAGhS,KAAK+b,MAEpCkH,EAAWC,GAAQC,GAAUJ,EAE7B5H,EAAWnT,KAAK+a,GAIlB,IAAKhR,EAAI,EAAGA,EAAIkR,EAAWtd,OAAQoM,IACjC,IAAKC,EAAI,EAAGA,EAAIiR,EAAWlR,GAAGpM,OAAQqM,IAChCiR,EAAWlR,GAAGC,KAChBiR,EAAWlR,GAAGC,GAAGuR,WAAcxR,EAAIkR,EAAWtd,OAAO,EAAKsd,EAAWlR,EAAE,GAAGC,GAAKxL,OAC/Eyc,EAAWlR,GAAGC,GAAGwR,SAAcxR,EAAIiR,EAAWlR,GAAGpM,OAAO,EAAKsd,EAAWlR,GAAGC,EAAE,GAAKxL,OAClFyc,EAAWlR,GAAGC,GAAGyR,WACd1R,EAAIkR,EAAWtd,OAAO,GAAKqM,EAAIiR,EAAWlR,GAAGpM,OAAO,EACnDsd,EAAWlR,EAAE,GAAGC,EAAE,GAClBxL,YAOV,KAAKhB,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B0M,EAAQ,GAAI7Q,GACZ6Q,EAAMH,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAChClJ,EAAMF,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAChCnJ,EAAMgL,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,EAEV9U,SAAlBxG,KAAKub,WACPrJ,EAAM7K,MAAQqL,EAAKlN,GAAGxF,KAAKub,WAAa,GAG1CwH,KACAA,EAAI7Q,MAAQA,EACZ6Q,EAAIO,OAAS,GAAIjiB,GAAQ6Q,EAAMH,EAAGG,EAAMF,EAAGhS,KAAK+b,MAChDgH,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OAEb2U,EAAWnT,KAAK+a,EAIpB,OAAO5H,IASTna,EAAQmS,UAAU9E,OAAS,WAEzB,KAAOrO,KAAKyZ,iBAAiBiK,iBAC3B1jB,KAAKyZ,iBAAiBtI,YAAYnR,KAAKyZ,iBAAiBkK,WAG1D3jB,MAAKsf,MAAQ/N,SAASM,cAAc,OACpC7R,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKsf,MAAMrS,MAAM4W,SAAW,SAG5B7jB,KAAKsf,MAAMC,OAAShO,SAASM,cAAe,UAC5C7R,KAAKsf,MAAMC,OAAOtS,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMC,OAGhC,IAAIuE,GAAWvS,SAASM,cAAe,MACvCiS,GAAS7W,MAAM9B,MAAQ,MACvB2Y,EAAS7W,MAAM8W,WAAc,OAC7BD,EAAS7W,MAAM+W,QAAW,OAC1BF,EAASG,UAAa,mDACtBjkB,KAAKsf,MAAMC,OAAO9N,YAAYqS,GAGhC9jB,KAAKsf,MAAM3L,OAASpC,SAASM,cAAe,OAC5C7R,KAAKsf,MAAM3L,OAAO1G,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM3L,OAAO1G,MAAMqW,OAAS,MACjCtjB,KAAKsf,MAAM3L,OAAO1G,MAAMxF,KAAO,MAC/BzH,KAAKsf,MAAM3L,OAAO1G,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM3L,OAGlC,IAAIQ,GAAKnU,KACLkkB,EAAc,SAAU5a,GAAQ6K,EAAGgQ,aAAa7a,IAChD8a,EAAe,SAAU9a,GAAQ6K,EAAGkQ,cAAc/a,IAClDgb,EAAe,SAAUhb,GAAQ6K,EAAGoQ,SAASjb,IAC7Ckb,EAAY,SAAUlb,GAAQ6K,EAAGsQ,WAAWnb,GAGhD3I,GAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,UAAWmF,WACpD/jB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,YAAa2E,GACtDvjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc6E,GACvDzjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc+E,GACvD3jB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,YAAaiF,GAGtDxkB,KAAKyZ,iBAAiBhI,YAAYzR,KAAKsf,QAWzCte,EAAQmS,UAAUwR,QAAU,SAASpS,EAAOC,GAC1CxS,KAAKsf,MAAMrS,MAAMsF,MAAQA,EACzBvS,KAAKsf,MAAMrS,MAAMuF,OAASA,EAE1BxS,KAAK4kB,iBAMP5jB,EAAQmS,UAAUyR,cAAgB,WAChC5kB,KAAKsf,MAAMC,OAAOtS,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAMC,OAAOtS,MAAMuF,OAAS,OAEjCxS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAC5Cxf,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAG7C7kB,KAAKsf,MAAM3L,OAAO1G,MAAMsF,MAASvS,KAAKsf,MAAMC,OAAOC,YAAc,GAAU,MAM7Exe,EAAQmS,UAAU2R,eAAiB,WACjC,IAAK9kB,KAAKsf,MAAM3L,SAAW3T,KAAKsf,MAAM3L,OAAOoR,OAC3C,KAAM,wBAER/kB,MAAKsf,MAAM3L,OAAOoR,OAAOC,QAO3BhkB,EAAQmS,UAAU8R,cAAgB,WAC3BjlB,KAAKsf,MAAM3L,QAAW3T,KAAKsf,MAAM3L,OAAOoR,QAE7C/kB,KAAKsf,MAAM3L,OAAOoR,OAAOG,QAU3BlkB,EAAQmS,UAAUgS,cAAgB,WAG9BnlB,KAAKqf,QAD0D,MAA7Drf,KAAK2Z,eAAeyL,OAAOplB,KAAK2Z,eAAehU,OAAO,GAEtD0f,WAAWrlB,KAAK2Z,gBAAkB,IAChC3Z,KAAKsf,MAAMC,OAAOC,YAGP6F,WAAWrlB,KAAK2Z,gBAK/B3Z,KAAKyf,QAD0D,MAA7Dzf,KAAK4Z,eAAewL,OAAOplB,KAAK4Z,eAAejU,OAAO,GAEtD0f,WAAWrlB,KAAK4Z,gBAAkB,KAC/B5Z,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAKsf,MAAM3L,OAAOkR,cAGzCQ,WAAWrlB,KAAK4Z,iBAoBnC5Y,EAAQmS,UAAUmS,kBAAoB,SAASC,GACjC/e,SAAR+e,IAImB/e,SAAnB+e,EAAIC,YAA6Chf,SAAjB+e,EAAIE,UACtCzlB,KAAKib,OAAOyK,eAAeH,EAAIC,WAAYD,EAAIE,UAG5Bjf,SAAjB+e,EAAII,UACN3lB,KAAKib,OAAO2K,aAAaL,EAAII,UAG/B3lB,KAAKyhB,WASPzgB,EAAQmS,UAAU0S,kBAAoB,WACpC,GAAIN,GAAMvlB,KAAKib,OAAO6K,gBAEtB,OADAP,GAAII,SAAW3lB,KAAKib,OAAOmE,eACpBmG,GAMTvkB,EAAQmS,UAAU4S,UAAY,SAASrT,GAErC1S,KAAKohB,gBAAgB1O,EAAM1S,KAAKiN,OAK9BjN,KAAKmb,WAFHnb,KAAKuhB,WAEWvhB,KAAKuhB,WAAWuB,iBAIhB9iB,KAAK8iB,eAAe9iB,KAAKuX,WAI7CvX,KAAKgmB,iBAOPhlB,EAAQmS,UAAU6E,QAAU,SAAUtF,GACpC1S,KAAK+lB,UAAUrT,GACf1S,KAAKyhB,SAGDzhB,KAAKimB,oBAAsBjmB,KAAKuhB,YAClCvhB,KAAK8kB,kBAQT9jB,EAAQmS,UAAUD,WAAa,SAAUzE,GACvC,GAAIyX,GAAiB1f,MAIrB,IAFAxG,KAAKilB,gBAEWze,SAAZiI,EAAuB,CAkBzB,GAhBsBjI,SAAlBiI,EAAQ8D,QAA2BvS,KAAKuS,MAAQ9D,EAAQ8D,OACrC/L,SAAnBiI,EAAQ+D,SAA2BxS,KAAKwS,OAAS/D,EAAQ+D,QAErChM,SAApBiI,EAAQ0O,UAA2Bnd,KAAK2Z,eAAiBlL,EAAQ0O,SAC7C3W,SAApBiI,EAAQ2O,UAA2Bpd,KAAK4Z,eAAiBnL,EAAQ2O,SAEzC5W,SAAxBiI,EAAQ2L,cAA+Bpa,KAAKoa,YAAc3L,EAAQ2L,aAC1C5T,SAAxBiI,EAAQ4L,cAA+Bra,KAAKqa,YAAc5L,EAAQ4L,aAC/C7T,SAAnBiI,EAAQoL,SAA0B7Z,KAAK6Z,OAASpL,EAAQoL,QACrCrT,SAAnBiI,EAAQqL,SAA0B9Z,KAAK8Z,OAASrL,EAAQqL,QACrCtT,SAAnBiI,EAAQsL,SAA0B/Z,KAAK+Z,OAAStL,EAAQsL,QAEhCvT,SAAxBiI,EAAQwL,cAA+Bja,KAAKia,YAAcxL,EAAQwL,aAC1CzT,SAAxBiI,EAAQyL,cAA+Bla,KAAKka,YAAczL,EAAQyL,aAC1C1T,SAAxBiI,EAAQ0L,cAA+Bna,KAAKma,YAAc1L,EAAQ0L,aAEhD3T,SAAlBiI,EAAQxB,MAAqB,CAC/B,GAAIkZ,GAAcnmB,KAAK2gB,gBAAgBlS,EAAQxB,MAC3B,MAAhBkZ,IACFnmB,KAAKiN,MAAQkZ,GAGQ3f,SAArBiI,EAAQgM,WAA6Bza,KAAKya,SAAWhM,EAAQgM,UACjCjU,SAA5BiI,EAAQ+L,kBAAiCxa,KAAKwa,gBAAkB/L,EAAQ+L,iBACjDhU,SAAvBiI,EAAQkM,aAA6B3a,KAAK2a,WAAalM,EAAQkM,YAC3CnU,SAApBiI,EAAQ2X,UAA6BpmB,KAAK6a,YAAcpM,EAAQ2X,SAC9B5f,SAAlCiI,EAAQ4X,wBAAqCrmB,KAAKqmB,sBAAwB5X,EAAQ4X,uBACtD7f,SAA5BiI,EAAQiM,kBAAiC1a,KAAK0a,gBAAkBjM,EAAQiM,iBAC9ClU,SAA1BiI,EAAQqM,gBAA+B9a,KAAK8a,cAAgBrM,EAAQqM,eAEtCtU,SAA9BiI,EAAQsM,oBAAiC/a,KAAK+a,kBAAoBtM,EAAQsM,mBAC7CvU,SAA7BiI,EAAQuM,mBAAiChb,KAAKgb,iBAAmBvM,EAAQuM,kBAC1CxU,SAA/BiI,EAAQwX,qBAAiCjmB,KAAKimB,mBAAqBxX,EAAQwX,oBAErDzf,SAAtBiI,EAAQ2N,YAAyBpc,KAAK2hB,iBAAmBlT,EAAQ2N,WAC3C5V,SAAtBiI,EAAQ4N,YAAyBrc,KAAK6hB,iBAAmBpT,EAAQ4N,WAEhD7V,SAAjBiI,EAAQgN,OAAoBzb,KAAKgiB,YAAcvT,EAAQgN,MACrCjV,SAAlBiI,EAAQiN,QAAqB1b,KAAKkiB,aAAezT,EAAQiN,OACxClV,SAAjBiI,EAAQkN,OAAoB3b,KAAKiiB,YAAcxT,EAAQkN,MACtCnV,SAAjBiI,EAAQmN,OAAoB5b,KAAKoiB,YAAc3T,EAAQmN,MACrCpV,SAAlBiI,EAAQoN,QAAqB7b,KAAKsiB,aAAe7T,EAAQoN,OACxCrV,SAAjBiI,EAAQqN,OAAoB9b,KAAKqiB,YAAc5T,EAAQqN,MACtCtV,SAAjBiI,EAAQsN,OAAoB/b,KAAKwiB,YAAc/T,EAAQsN,MACrCvV,SAAlBiI,EAAQuN,QAAqBhc,KAAK0iB,aAAejU,EAAQuN,OACxCxV,SAAjBiI,EAAQwN,OAAoBjc,KAAKyiB,YAAchU,EAAQwN,MAClCzV,SAArBiI,EAAQyN,WAAwBlc,KAAK4iB,gBAAkBnU,EAAQyN,UAC1C1V,SAArBiI,EAAQ0N,WAAwBnc,KAAK6iB,gBAAkBpU,EAAQ0N,UAEpC3V,SAA3BiI,EAAQyX,iBAA8BA,EAAiBzX,EAAQyX,gBAE5C1f,SAAnB0f,GACFlmB,KAAKib,OAAOyK,eAAeQ,EAAeV,WAAYU,EAAeT,UACrEzlB,KAAKib,OAAO2K,aAAaM,EAAeP,YAGxC3lB,KAAKib,OAAOyK,eAAe,EAAK,IAChC1lB,KAAKib,OAAO2K,aAAa,MAI7B5lB,KAAK0f,oBAAoBjR,GAAWA,EAAQkR,iBAE5C3f,KAAK2kB,QAAQ3kB,KAAKuS,MAAOvS,KAAKwS,QAG1BxS,KAAKuX,WACPvX,KAAKgY,QAAQhY,KAAKuX,WAIhBvX,KAAKimB,oBAAsBjmB,KAAKuhB,YAClCvhB,KAAK8kB,kBAOT9jB,EAAQmS,UAAUsO,OAAS,WACzB,GAAwBjb,SAApBxG,KAAKmb,WACP,KAAM,mCAGRnb,MAAK4kB,gBACL5kB,KAAKmlB,gBACLnlB,KAAKsmB,gBACLtmB,KAAKumB,eACLvmB,KAAKwmB,cAEDxmB,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC/BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,QAC7B1gB,KAAKymB,kBAEEzmB,KAAKiN,QAAUjM,EAAQsZ,MAAMmG,KACpCzgB,KAAK0mB,kBAEE1mB,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,KACpClgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAC7BpgB,KAAK2mB,iBAIL3mB,KAAK4mB,iBAGP5mB,KAAK6mB,cACL7mB,KAAK8mB,iBAMP9lB,EAAQmS,UAAUoT,aAAe,WAC/B,GAAIhH,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAE5BD,GAAIE,UAAU,EAAG,EAAG1H,EAAOhN,MAAOgN,EAAO/M,SAO3CxR,EAAQmS,UAAU2T,cAAgB,WAChC,GAAI9U,EAEJ,IAAIhS,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAC/BtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QAAS,CAEtC,GAEI2G,GAAUC,EAFVC,EAAmC,IAAzBpnB,KAAKsf,MAAME,WAGrBxf,MAAKiN,QAAUjM,EAAQsZ,MAAMiG,SAC/B2G,EAAWE,EAAU,EACrBD,EAAWC,EAAU,EAAc,EAAVA,IAGzBF,EAAW,GACXC,EAAW,GAGb,IAAI3U,GAAStN,KAAKwH,IAA8B,IAA1B1M,KAAKsf,MAAMuF,aAAqB,KAClDld,EAAM3H,KAAK0Z,OACX2N,EAAQrnB,KAAKsf,MAAME,YAAcxf,KAAK0Z,OACtCjS,EAAO4f,EAAQF,EACf7D,EAAS3b,EAAM6K,EAGrB,GAAI+M,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAI5B,IAHAD,EAAIO,UAAY,EAChBP,EAAIQ,KAAO,aAEPvnB,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,SAAU,CAEzC,GAAIkH,GAAO,EACPC,EAAOjV,CACX,KAAKR,EAAIwV,EAAUC,EAAJzV,EAAUA,IAAK,CAC5B,GAAIpE,IAAKoE,EAAIwV,IAASC,EAAOD,GAGzB5a,EAAU,IAAJgB,EACNzC,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,EAElCma,GAAIY,YAAcxc,EAClB4b,EAAIa,YACJb,EAAIc,OAAOpgB,EAAME,EAAMqK,GACvB+U,EAAIe,OAAOT,EAAO1f,EAAMqK,GACxB+U,EAAIlH,SAGNkH,EAAIY,YAAe3nB,KAAKsc,UACxByK,EAAIgB,WAAWtgB,EAAME,EAAKwf,EAAU3U,GAiBtC,GAdIxS,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,UAE/BwG,EAAIY,YAAe3nB,KAAKsc,UACxByK,EAAIiB,UAAahoB,KAAKwc,SACtBuK,EAAIa,YACJb,EAAIc,OAAOpgB,EAAME,GACjBof,EAAIe,OAAOT,EAAO1f,GAClBof,EAAIe,OAAOT,EAAQF,EAAWD,EAAU5D,GACxCyD,EAAIe,OAAOrgB,EAAM6b,GACjByD,EAAIkB,YACJlB,EAAInH,OACJmH,EAAIlH,UAGF7f,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAC/BtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QAAS,CAEtC,GAAI2H,GAAc,EACdC,EAAO,GAAI5mB,GAAWvB,KAAKkc,SAAUlc,KAAKmc,UAAWnc,KAAKmc,SAASnc,KAAKkc,UAAU,GAAG,EAKzF,KAJAiM,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAKkc,UAC3BiM,EAAKE,QAECF,EAAKtY,OACXmC,EAAIsR,GAAU6E,EAAKC,aAAepoB,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY1J,EAErFuU,EAAIa,YACJb,EAAIc,OAAOpgB,EAAOygB,EAAalW,GAC/B+U,EAAIe,OAAOrgB,EAAMuK,GACjB+U,EAAIlH,SAEJkH,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASL,EAAKC,aAAc3gB,EAAO,EAAIygB,EAAalW,GAExDmW,EAAKE,MAGPtB,GAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,KACnB,IAAIE,GAAQzoB,KAAKqa,WACjB0M,GAAIyB,SAASC,EAAOpB,EAAO/D,EAAStjB,KAAK0Z,UAO7C1Y,EAAQmS,UAAU6S,cAAgB,WAGhC,GAFAhmB,KAAKsf,MAAM3L,OAAOsQ,UAAY,GAE1BjkB,KAAKuhB,WAAY,CACnB,GAAI9S,IACFia,QAAW1oB,KAAKqmB,uBAEdtB,EAAS,GAAIzjB,GAAOtB,KAAKsf,MAAM3L,OAAQlF,EAC3CzO,MAAKsf,MAAM3L,OAAOoR,OAASA,EAG3B/kB,KAAKsf,MAAM3L,OAAO1G,MAAM+W,QAAU,OAGlCe,EAAO4D,UAAU3oB,KAAKuhB,WAAWzK,QACjCiO,EAAO6D,gBAAgB5oB,KAAK+a,kBAG5B,IAAI5G,GAAKnU,KACL6oB,EAAW,WACb,GAAI1gB,GAAQ4c,EAAO+D,UAEnB3U,GAAGoN,WAAWwH,YAAY5gB,GAC1BgM,EAAGgH,WAAahH,EAAGoN,WAAWuB,iBAE9B3O,EAAGsN,SAELsD,GAAOiE,oBAAoBH,OAG3B7oB,MAAKsf,MAAM3L,OAAOoR,OAASve,QAO/BxF,EAAQmS,UAAUmT,cAAgB,WACE9f,SAA7BxG,KAAKsf,MAAM3L,OAAOoR,QACrB/kB,KAAKsf,MAAM3L,OAAOoR,OAAOtD,UAQ7BzgB,EAAQmS,UAAU0T,YAAc,WAC9B,GAAI7mB,KAAKuhB,WAAY,CACnB,GAAIhC,GAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAE5BD,GAAIQ,KAAO,aACXR,EAAIkC,UAAY,OAChBlC,EAAIiB,UAAY,OAChBjB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,KAEnB,IAAIxW,GAAI/R,KAAK0Z,OACT1H,EAAIhS,KAAK0Z,MACbqN,GAAIyB,SAASxoB,KAAKuhB,WAAW2H,WAAa,KAAOlpB,KAAKuhB,WAAW4H,mBAAoBpX,EAAGC,KAQ5FhR,EAAQmS,UAAUqT,YAAc,WAC9B,GAEE4C,GAAMC,EAAIlB,EAAMmB,EAChBC,EAAMC,EAAOC,EAAOC,EACpBC,EAAQC,EAASC,EACjBC,EAAQC,EALNxK,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAQ1BD,GAAIQ,KAAO,GAAKvnB,KAAKib,OAAOmE,eAAiB,UAG7C,IAAI4K,GAAW,KAAQhqB,KAAKid,MAAMlL,EAC9BkY,EAAW,KAAQjqB,KAAKid,MAAMjL,EAC9BkY,EAAa,EAAIlqB,KAAKib,OAAOmE,eAC7B+K,EAAWnqB,KAAKib,OAAO6K,iBAAiBN,UAU5C,KAPAuB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAKkiB,aACnBiG,EAAO,GAAI5mB,GAAWvB,KAAKyb,KAAMzb,KAAK2b,KAAM3b,KAAK0b,MAAO4N,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAKyb,MAC3B0M,EAAKE,QAECF,EAAKtY,OAAO,CAClB,GAAIkC,GAAIoW,EAAKC,YAETpoB,MAAKya,UACP2O,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAM5b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAM9b,KAAK+b,OACxDgL,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,WAGJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAM5b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK4b,KAAKoO,EAAUhqB,KAAK+b,OACjEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAM9b,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG/R,KAAK8b,KAAKkO,EAAUhqB,KAAK+b,OACjEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,UAGN4J,EAASvkB,KAAKqZ,IAAI4L,GAAY,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,KACpDyN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQ0Q,EAAG0X,EAAOzpB,KAAK+b,OAClD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,MACnBgB,EAAKvX,GAAKkY,GAEHhlB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS,KAAOxoB,KAAKia,YAAYkO,EAAKC,cAAgB,KAAMmB,EAAKxX,EAAGwX,EAAKvX,GAE7EmW,EAAKE,OAWP,IAPAtB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAKsiB,aACnB6F,EAAO,GAAI5mB,GAAWvB,KAAK4b,KAAM5b,KAAK8b,KAAM9b,KAAK6b,MAAOyN,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAK4b,MAC3BuM,EAAKE,QAECF,EAAKtY,OACP7P,KAAKya,UACP2O,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAM0M,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAMwM,EAAKC,aAAcpoB,KAAK+b,OACxEgL,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,WAGJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAM0M,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAKwO,EAAU9B,EAAKC,aAAcpoB,KAAK+b,OACjFgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAMwM,EAAKC,aAAcpoB,KAAK+b,OAC1EsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAKsO,EAAU9B,EAAKC,aAAcpoB,KAAK+b,OACjFgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,UAGN2J,EAAStkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD4N,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOrB,EAAKC,aAAcpoB,KAAK+b,OAClE7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,MACnBgB,EAAKvX,GAAKkY,GAEHhlB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS,KAAOxoB,KAAKka,YAAYiO,EAAKC,cAAgB,KAAMmB,EAAKxX,EAAGwX,EAAKvX,GAE7EmW,EAAKE,MAaP,KATAtB,EAAIO,UAAY,EAChBgC,EAAoC9iB,SAAtBxG,KAAK0iB,aACnByF,EAAO,GAAI5mB,GAAWvB,KAAK+b,KAAM/b,KAAKic,KAAMjc,KAAKgc,MAAOsN,GACxDnB,EAAKvY,QACDuY,EAAKC,aAAepoB,KAAK+b,MAC3BoM,EAAKE,OAEPmB,EAAStkB,KAAKqZ,IAAI4L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD8N,EAASvkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,MAC7CqM,EAAKtY,OAEXuZ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOtB,EAAKC,eAC1DrB,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOsB,EAAKrX,EAAImY,EAAYd,EAAKpX,GACrC+U,EAAIlH,SAEJkH,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASxoB,KAAKma,YAAYgO,EAAKC,cAAgB,IAAKgB,EAAKrX,EAAI,EAAGqX,EAAKpX,GAEzEmW,EAAKE,MAEPtB,GAAIO,UAAY,EAChB8B,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OAC1DsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAKic,OACxD8K,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAGJkH,EAAIO,UAAY,EAEhBwC,EAAS9pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK4b,KAAM5b,KAAK+b,OACpEgO,EAAS/pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK4b,KAAM5b,KAAK+b,OACpEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOiC,EAAO/X,EAAG+X,EAAO9X,GAC5B+U,EAAIe,OAAOiC,EAAOhY,EAAGgY,EAAO/X,GAC5B+U,EAAIlH,SAEJiK,EAAS9pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK8b,KAAM9b,KAAK+b,OACpEgO,EAAS/pB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK8b,KAAM9b,KAAK+b,OACpEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOiC,EAAO/X,EAAG+X,EAAO9X,GAC5B+U,EAAIe,OAAOiC,EAAOhY,EAAGgY,EAAO/X,GAC5B+U,EAAIlH,SAGJkH,EAAIO,UAAY,EAEhB8B,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK4b,KAAM5b,KAAK+b,OAClEsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAKyb,KAAMzb,KAAK8b,KAAM9b,KAAK+b,OAChEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,SAEJuJ,EAAOppB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK4b,KAAM5b,KAAK+b,OAClEsN,EAAKrpB,KAAKud,eAAe,GAAIlc,GAAQrB,KAAK2b,KAAM3b,KAAK8b,KAAM9b,KAAK+b,OAChEgL,EAAIY,YAAc3nB,KAAKsc,UACvByK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAOuB,EAAGtX,EAAGsX,EAAGrX,GACpB+U,EAAIlH,QAGJ,IAAIhG,GAAS7Z,KAAK6Z,MACdA,GAAOlU,OAAS,IAClBkkB,EAAU,GAAM7pB,KAAKid,MAAMjL,EAC3BwX,GAASxpB,KAAKyb,KAAOzb,KAAK2b,MAAQ,EAClC8N,EAASvkB,KAAKqZ,IAAI4L,GAAY,EAAKnqB,KAAK4b,KAAOiO,EAAS7pB,KAAK8b,KAAO+N,EACpEN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OACtD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,OAEZrjB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS3O,EAAQ0P,EAAKxX,EAAGwX,EAAKvX,GAIpC,IAAI8H,GAAS9Z,KAAK8Z,MACdA,GAAOnU,OAAS,IAClBikB,EAAU,GAAM5pB,KAAKid,MAAMlL,EAC3ByX,EAAStkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAKyb,KAAOmO,EAAU5pB,KAAK2b,KAAOiO,EACtEH,GAASzpB,KAAK4b,KAAO5b,KAAK8b,MAAQ,EAClCyN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOzpB,KAAK+b,OACtD7W,KAAKqZ,IAAe,EAAX4L,GAAgB,GAC3BpD,EAAIuB,UAAY,SAChBvB,EAAIwB,aAAe,OAEZrjB,KAAKkZ,IAAe,EAAX+L,GAAgB,GAChCpD,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,WAGnBxB,EAAIuB,UAAY,OAChBvB,EAAIwB,aAAe,UAErBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAAS1O,EAAQyP,EAAKxX,EAAGwX,EAAKvX,GAIpC,IAAI+H,GAAS/Z,KAAK+Z,MACdA,GAAOpU,OAAS,IAClBgkB,EAAS,GACTH,EAAStkB,KAAKqZ,IAAI4L,GAAa,EAAKnqB,KAAKyb,KAAOzb,KAAK2b,KACrD8N,EAASvkB,KAAKkZ,IAAI+L,GAAa,EAAKnqB,KAAK4b,KAAO5b,KAAK8b,KACrD4N,GAAS1pB,KAAK+b,KAAO/b,KAAKic,MAAQ,EAClCsN,EAAOvpB,KAAKud,eAAe,GAAIlc,GAAQmoB,EAAOC,EAAOC,IACrD3C,EAAIuB,UAAY,QAChBvB,EAAIwB,aAAe,SACnBxB,EAAIiB,UAAYhoB,KAAKsc,UACrByK,EAAIyB,SAASzO,EAAQwP,EAAKxX,EAAI4X,EAAQJ,EAAKvX,KAU/ChR,EAAQmS,UAAUuU,SAAW,SAAS0C,EAAGC,EAAGC,GAC1C,GAAIC,GAAGC,EAAGC,EAAGC,EAAGC,EAAIC,CAMpB,QAJAF,EAAIJ,EAAID,EACRM,EAAKzlB,KAAKC,MAAMilB,EAAE,IAClBQ,EAAIF,GAAK,EAAIxlB,KAAK2lB,IAAMT,EAAE,GAAM,EAAK,IAE7BO,GACN,IAAK,GAAGJ,EAAIG,EAAGF,EAAII,EAAGH,EAAI,CAAG,MAC7B,KAAK,GAAGF,EAAIK,EAAGJ,EAAIE,EAAGD,EAAI,CAAG,MAC7B,KAAK,GAAGF,EAAI,EAAGC,EAAIE,EAAGD,EAAIG,CAAG,MAC7B,KAAK,GAAGL,EAAI,EAAGC,EAAII,EAAGH,EAAIC,CAAG,MAC7B,KAAK,GAAGH,EAAIK,EAAGJ,EAAI,EAAGC,EAAIC,CAAG,MAC7B,KAAK,GAAGH,EAAIG,EAAGF,EAAI,EAAGC,EAAIG,CAAG,MAE7B,SAASL,EAAI,EAAGC,EAAI,EAAGC,EAAI,EAG7B,MAAO,OAAS7f,SAAW,IAAF2f,GAAS,IAAM3f,SAAW,IAAF4f,GAAS,IAAM5f,SAAW,IAAF6f,GAAS,KAQpFzpB,EAAQmS,UAAUsT,gBAAkB,WAClC,GAEEvU,GAAOmV,EAAO1f,EAAKmjB,EACnBtlB,EACAulB,EAAgB/C,EAAWL,EAAaL,EACxC3b,EAAGC,EAAGC,EAAGmf,EALPzL,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAO1B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAE9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAIpB,IAFAlrB,KAAKmb,WAAWjF,KAAKiV,GAEjBnrB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,SAC/B,IAAKlb,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAMtC,GALA0M,EAAQlS,KAAKmb,WAAW3V,GACxB6hB,EAAQrnB,KAAKmb,WAAW3V,GAAG+d,WAC3B5b,EAAQ3H,KAAKmb,WAAW3V,GAAGge,SAC3BsH,EAAQ9qB,KAAKmb,WAAW3V,GAAGie,WAEbjd,SAAV0L,GAAiC1L,SAAV6gB,GAA+B7gB,SAARmB,GAA+BnB,SAAVskB,EAAqB,CAE1F,GAAI9qB,KAAK4a,gBAAkB5a,KAAK2a,WAAY,CAK1C,GAAIyQ,GAAQ/pB,EAAQgqB,SAASP,EAAM1H,MAAOlR,EAAMkR,OAC5CkI,EAAQjqB,EAAQgqB,SAAS1jB,EAAIyb,MAAOiE,EAAMjE,OAC1CmI,EAAelqB,EAAQmqB,aAAaJ,EAAOE,GAC3C7lB,EAAM8lB,EAAa5lB,QAGvBolB,GAAkBQ,EAAarO,EAAI,MAGnC6N,IAAiB,CAGfA,IAEFC,GAAQ9Y,EAAMA,MAAMgL,EAAImK,EAAMnV,MAAMgL,EAAIvV,EAAIuK,MAAMgL,EAAI4N,EAAM5Y,MAAMgL,GAAK,EACvEvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eACnDlP,EAAI,EAEA5L,KAAK2a,YACP9O,EAAI3G,KAAK4G,IAAI,EAAKyf,EAAaxZ,EAAItM,EAAO,EAAG,GAC7CuiB,EAAYhoB,KAAK0nB,SAAS/b,EAAGC,EAAGC,GAChC8b,EAAcK,IAGdnc,EAAI,EACJmc,EAAYhoB,KAAK0nB,SAAS/b,EAAGC,EAAGC,GAChC8b,EAAc3nB,KAAKsc,aAIrB0L,EAAY,OACZL,EAAc3nB,KAAKsc,WAErBgL,EAAY,GAEZP,EAAIO,UAAYA,EAChBP,EAAIiB,UAAYA,EAChBjB,EAAIY,YAAcA,EAClBZ,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOT,EAAMhE,OAAOtR,EAAGsV,EAAMhE,OAAOrR,GACxC+U,EAAIe,OAAOgD,EAAMzH,OAAOtR,EAAG+Y,EAAMzH,OAAOrR,GACxC+U,EAAIe,OAAOngB,EAAI0b,OAAOtR,EAAGpK,EAAI0b,OAAOrR,GACpC+U,EAAIkB,YACJlB,EAAInH,OACJmH,EAAIlH,cAKR,KAAKra,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IACtC0M,EAAQlS,KAAKmb,WAAW3V,GACxB6hB,EAAQrnB,KAAKmb,WAAW3V,GAAG+d,WAC3B5b,EAAQ3H,KAAKmb,WAAW3V,GAAGge,SAEbhd,SAAV0L,IAEAoV,EADEtnB,KAAKwa,gBACK,GAAKtI,EAAMkR,MAAMlG,EAGjB,IAAMld,KAAKkb,IAAIgC,EAAIld,KAAKib,OAAOmE,iBAIjC5Y,SAAV0L,GAAiC1L,SAAV6gB,IAEzB2D,GAAQ9Y,EAAMA,MAAMgL,EAAImK,EAAMnV,MAAMgL,GAAK,EACzCvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAEnDiM,EAAIO,UAAYA,EAChBP,EAAIY,YAAc3nB,KAAK0nB,SAAS/b,EAAG,EAAG,GACtCob,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOT,EAAMhE,OAAOtR,EAAGsV,EAAMhE,OAAOrR,GACxC+U,EAAIlH,UAGQrZ,SAAV0L,GAA+B1L,SAARmB,IAEzBqjB,GAAQ9Y,EAAMA,MAAMgL,EAAIvV,EAAIuK,MAAMgL,GAAK,EACvCvR,EAAoE,KAA/D,GAAKqf,EAAOhrB,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAEnDiM,EAAIO,UAAYA,EAChBP,EAAIY,YAAc3nB,KAAK0nB,SAAS/b,EAAG,EAAG,GACtCob,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIe,OAAOngB,EAAI0b,OAAOtR,EAAGpK,EAAI0b,OAAOrR,GACpC+U,EAAIlH,YAWZ7e,EAAQmS,UAAUyT,eAAiB,WACjC,GAEIphB,GAFA+Z,EAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAG5B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAC9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAEpBlrB,MAAKmb,WAAWjF,KAAKiV,EAGrB,IAAI/D,GAAmC,IAAzBpnB,KAAKsf,MAAME,WACzB,KAAKha,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI0M,GAAQlS,KAAKmb,WAAW3V,EAE5B,IAAIxF,KAAKiN,QAAUjM,EAAQsZ,MAAM+F,QAAS,CAGxC,GAAI+I,GAAOppB,KAAKud,eAAerL,EAAMoR,OACrCyD,GAAIO,UAAY,EAChBP,EAAIY,YAAc3nB,KAAKuc,UACvBwK,EAAIa,YACJb,EAAIc,OAAOuB,EAAKrX,EAAGqX,EAAKpX,GACxB+U,EAAIe,OAAO5V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,GACxC+U,EAAIlH,SAIN,GAAIxN,EAEFA,GADErS,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,QACxB6G,EAAQ,EAAI,EAAEA,GAAWlV,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAGpFkL,CAGT,IAAIqE,EAEFA,GADEzrB,KAAKwa,gBACEnI,GAAQH,EAAMkR,MAAMlG,EAGpB7K,IAASrS,KAAKkb,IAAIgC,EAAIld,KAAKib,OAAOmE,gBAEhC,EAATqM,IACFA,EAAS,EAGX,IAAI7e,GAAKzB,EAAO4U,CACZ/f,MAAKiN,QAAUjM,EAAQsZ,MAAMgG,UAE/B1T,EAAqE,KAA9D,GAAKsF,EAAMA,MAAM7K,MAAQrH,KAAKkc,UAAYlc,KAAKid,MAAM5V,OAC5D8D,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAE7B5M,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,SACpCpV,EAAQnL,KAAKwc,SACbuD,EAAc/f,KAAKyc,iBAInB7P,EAA+E,KAAxE,GAAKsF,EAAMA,MAAMgL,EAAIld,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAC9D3P,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAItCma,EAAIO,UAAY,EAChBP,EAAIY,YAAc5H,EAClBgH,EAAIiB,UAAY7c,EAChB4b,EAAIa,YACJb,EAAI2E,IAAIxZ,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,EAAGyZ,EAAQ,EAAW,EAARvmB,KAAKymB,IAAM,GAC9D5E,EAAInH,OACJmH,EAAIlH,YAQR7e,EAAQmS,UAAUwT,eAAiB,WACjC,GAEInhB,GAAGomB,EAAGC,EAASC,EAFfvM,EAASvf,KAAKsf,MAAMC,OACpBwH,EAAMxH,EAAOyH,WAAW,KAG5B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAC9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,CAG5B,IAAI4H,GAAcjrB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG8d,OACrEtjB,MAAKmb,WAAW3V,GAAG0lB,KAAOlrB,KAAKwa,gBAAkByQ,EAAYtlB,UAAYslB,EAAY/N,EAIvF,GAAIiO,GAAY,SAAU5lB,EAAGa,GAC3B,MAAOA,GAAE8kB,KAAO3lB,EAAE2lB,KAEpBlrB,MAAKmb,WAAWjF,KAAKiV,EAGrB,IAAIY,GAAS/rB,KAAKoc,UAAY,EAC1B4P,EAAShsB,KAAKqc,UAAY,CAC9B,KAAK7W,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAGIoH,GAAKzB,EAAO4U,EAHZ7N,EAAQlS,KAAKmb,WAAW3V,EAIxBxF,MAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAE/BvT,EAAqE,KAA9D,GAAKsF,EAAMA,MAAM7K,MAAQrH,KAAKkc,UAAYlc,KAAKid,MAAM5V,OAC5D8D,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAE7B5M,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,SACpCjV,EAAQnL,KAAKwc,SACbuD,EAAc/f,KAAKyc,iBAInB7P,EAA+E,KAAxE,GAAKsF,EAAMA,MAAMgL,EAAIld,KAAK+b,MAAQ/b,KAAKid,MAAMC,EAAKld,KAAK8a,eAC9D3P,EAAQnL,KAAK0nB,SAAS9a,EAAK,EAAG,GAC9BmT,EAAc/f,KAAK0nB,SAAS9a,EAAK,EAAG,KAIlC5M,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,UAC/B2L,EAAU/rB,KAAKoc,UAAY,IAAOlK,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY,GAAM,IAC/G8P,EAAUhsB,KAAKqc,UAAY,IAAOnK,EAAMA,MAAM7K,MAAQrH,KAAKkc,WAAalc,KAAKmc,SAAWnc,KAAKkc,UAAY,GAAM,IAIjH,IAAI/H,GAAKnU,KACLwd,EAAUtL,EAAMA,MAChBvK,IACDuK,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KACnEhL,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQxO,EAAQN,KAElEoG,IACDpR,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,QAChE7J,MAAO,GAAI7Q,GAAQmc,EAAQzL,EAAIga,EAAQvO,EAAQxL,EAAIga,EAAQhsB,KAAK+b,OAInEpU,GAAIU,QAAQ,SAAU0a,GACpBA,EAAIM,OAASlP,EAAGoJ,eAAewF,EAAI7Q,SAErCoR,EAAOjb,QAAQ,SAAU0a,GACvBA,EAAIM,OAASlP,EAAGoJ,eAAewF,EAAI7Q,QAIrC,IAAI+Z,KACDH,QAASnkB,EAAKukB,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAC7D4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,SAChG4Z,SAAUnkB,EAAI,GAAIA,EAAI,GAAI2b,EAAO,GAAIA,EAAO,IAAK4I,OAAQ7qB,EAAQ8qB,IAAI7I,EAAO,GAAGpR,MAAOoR,EAAO,GAAGpR,QAKnG,KAHAA,EAAM+Z,SAAWA,EAGZL,EAAI,EAAGA,EAAIK,EAAStmB,OAAQimB,IAAK,CACpCC,EAAUI,EAASL,EACnB,IAAIQ,GAAcpsB,KAAK0d,2BAA2BmO,EAAQK,OAC1DL,GAAQX,KAAOlrB,KAAKwa,gBAAkB4R,EAAYzmB,UAAYymB,EAAYlP,EAwB5E,IAjBA+O,EAAS/V,KAAK,SAAU3Q,EAAGa,GACzB,GAAIimB,GAAOjmB,EAAE8kB,KAAO3lB,EAAE2lB,IACtB,OAAImB,GAAaA,EAGb9mB,EAAEumB,UAAYnkB,EAAY,EAC1BvB,EAAE0lB,UAAYnkB,EAAY,GAGvB,IAITof,EAAIO,UAAY,EAChBP,EAAIY,YAAc5H,EAClBgH,EAAIiB,UAAY7c,EAEXygB,EAAI,EAAGA,EAAIK,EAAStmB,OAAQimB,IAC/BC,EAAUI,EAASL,GACnBE,EAAUD,EAAQC,QAClB/E,EAAIa,YACJb,EAAIc,OAAOiE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAIe,OAAOgE,EAAQ,GAAGzI,OAAOtR,EAAG+Z,EAAQ,GAAGzI,OAAOrR,GAClD+U,EAAInH,OACJmH,EAAIlH,YAUV7e,EAAQmS,UAAUuT,gBAAkB,WAClC,GAEExU,GAAO1M,EAFL+Z,EAASvf,KAAKsf,MAAMC,OACtBwH,EAAMxH,EAAOyH,WAAW,KAG1B,MAAwBxgB,SAApBxG,KAAKmb,YAA4Bnb,KAAKmb,WAAWxV,QAAU,GAA/D,CAIA,IAAKH,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C,GAAI4d,GAAQpjB,KAAK0d,2BAA2B1d,KAAKmb,WAAW3V,GAAG0M,OAC3DmR,EAASrjB,KAAK2d,4BAA4ByF,EAE9CpjB,MAAKmb,WAAW3V,GAAG4d,MAAQA,EAC3BpjB,KAAKmb,WAAW3V,GAAG6d,OAASA,EAc9B,IAVIrjB,KAAKmb,WAAWxV,OAAS,IAC3BuM,EAAQlS,KAAKmb,WAAW,GAExB4L,EAAIO,UAAY,EAChBP,EAAIY,YAAc,OAClBZ,EAAIa,YACJb,EAAIc,OAAO3V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,IAIrCxM,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IACtC0M,EAAQlS,KAAKmb,WAAW3V,GACxBuhB,EAAIe,OAAO5V,EAAMmR,OAAOtR,EAAGG,EAAMmR,OAAOrR,EAItChS,MAAKmb,WAAWxV,OAAS,GAC3BohB,EAAIlH,WASR7e,EAAQmS,UAAUgR,aAAe,SAAS7a,GAWxC,GAVAA,EAAQA,GAASC,OAAOD,MAIpBtJ,KAAKssB,gBACPtsB,KAAKusB,WAAWjjB,GAIlBtJ,KAAKssB,eAAiBhjB,EAAMkjB,MAAyB,IAAhBljB,EAAMkjB,MAAiC,IAAjBljB,EAAMmjB,OAC5DzsB,KAAKssB,gBAAmBtsB,KAAK0sB,UAAlC,CAGA1sB,KAAK2sB,YAAcjQ,EAAUpT,GAC7BtJ,KAAK4sB,YAAc/P,EAAUvT,GAE7BtJ,KAAK6sB,WAAa,GAAIvoB,MAAKtE,KAAK4P,OAChC5P,KAAK8sB,SAAW,GAAIxoB,MAAKtE,KAAK6P,KAC9B7P,KAAK+sB,iBAAmB/sB,KAAKib,OAAO6K,iBAEpC9lB,KAAKsf,MAAMrS,MAAM+f,OAAS,MAK1B,IAAI7Y,GAAKnU,IACTA,MAAKitB,YAAc,SAAU3jB,GAAQ6K,EAAG+Y,aAAa5jB,IACrDtJ,KAAKmtB,UAAc,SAAU7jB,GAAQ6K,EAAGoY,WAAWjjB,IACnD3I,EAAKgI,iBAAiB4I,SAAU,YAAa4C,EAAG8Y,aAChDtsB,EAAKgI,iBAAiB4I,SAAU,UAAW4C,EAAGgZ,WAC9CxsB,EAAK0I,eAAeC,KAStBtI,EAAQmS,UAAU+Z,aAAe,SAAU5jB,GACzCA,EAAQA,GAASC,OAAOD,KAGxB,IAAI8jB,GAAQ/H,WAAW3I,EAAUpT,IAAUtJ,KAAK2sB,YAC5CU,EAAQhI,WAAWxI,EAAUvT,IAAUtJ,KAAK4sB,YAE5CU,EAAgBttB,KAAK+sB,iBAAiBvH,WAAa4H,EAAQ,IAC3DG,EAAcvtB,KAAK+sB,iBAAiBtH,SAAW4H,EAAQ,IAEvDG,EAAY,EACZC,EAAYvoB,KAAKkZ,IAAIoP,EAAY,IAAM,EAAItoB,KAAKymB,GAIhDzmB,MAAK2lB,IAAI3lB,KAAKkZ,IAAIkP,IAAkBG,IACtCH,EAAgBpoB,KAAKwoB,MAAOJ,EAAgBpoB,KAAKymB,IAAOzmB,KAAKymB,GAAK,MAEhEzmB,KAAK2lB,IAAI3lB,KAAKqZ,IAAI+O,IAAkBG,IACtCH,GAAiBpoB,KAAKwoB,MAAOJ,EAAepoB,KAAKymB,GAAK,IAAQ,IAAOzmB,KAAKymB,GAAK,MAI7EzmB,KAAK2lB,IAAI3lB,KAAKkZ,IAAImP,IAAgBE,IACpCF,EAAcroB,KAAKwoB,MAAOH,EAAcroB,KAAKymB,IAAOzmB,KAAKymB,IAEvDzmB,KAAK2lB,IAAI3lB,KAAKqZ,IAAIgP,IAAgBE,IACpCF,GAAeroB,KAAKwoB,MAAOH,EAAaroB,KAAKymB,GAAK,IAAQ,IAAOzmB,KAAKymB,IAGxE3rB,KAAKib,OAAOyK,eAAe4H,EAAeC,GAC1CvtB,KAAKyhB,QAGL,IAAIkM,GAAa3tB,KAAK6lB,mBACtB7lB,MAAK4tB,KAAK,uBAAwBD,GAElChtB,EAAK0I,eAAeC,IAStBtI,EAAQmS,UAAUoZ,WAAa,SAAUjjB,GACvCtJ,KAAKsf,MAAMrS,MAAM+f,OAAS,OAC1BhtB,KAAKssB,gBAAiB,EAGtB3rB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKitB,aACrDtsB,EAAKwI,oBAAoBoI,SAAU,UAAavR,KAAKmtB,WACrDxsB,EAAK0I,eAAeC,IAOtBtI,EAAQmS,UAAUsR,WAAa,SAAUnb,GACvC,GAAIkP,GAAQ,IACRqV,EAAe7tB,KAAKsf,MAAM9X,wBAC1BsmB,EAASpR,EAAUpT,GAASukB,EAAapmB,KACzCsmB,EAASlR,EAAUvT,GAASukB,EAAalmB,GAE7C,IAAK3H,KAAK6a,YAAV,CASA,GALI7a,KAAKguB,gBACP3U,aAAarZ,KAAKguB,gBAIhBhuB,KAAKssB,eAEP,WADAtsB,MAAKiuB,cAIP,IAAIjuB,KAAKomB,SAAWpmB,KAAKomB,QAAQ8H,UAAW,CAE1C,GAAIA,GAAYluB,KAAKmuB,iBAAiBL,EAAQC,EAC1CG,KAAcluB,KAAKomB,QAAQ8H,YAEzBA,EACFluB,KAAKouB,aAAaF,GAGlBluB,KAAKiuB,oBAIN,CAEH,GAAI9Z,GAAKnU,IACTA,MAAKguB,eAAiB1U,WAAW,WAC/BnF,EAAG6Z,eAAiB,IAGpB,IAAIE,GAAY/Z,EAAGga,iBAAiBL,EAAQC,EACxCG,IACF/Z,EAAGia,aAAaF,IAEjB1V,MAOPxX,EAAQmS,UAAUkR,cAAgB,SAAS/a,GACzCtJ,KAAK0sB,WAAY,CAEjB,IAAIvY,GAAKnU,IACTA,MAAKquB,YAAc,SAAU/kB,GAAQ6K,EAAGma,aAAahlB,IACrDtJ,KAAKuuB,WAAc,SAAUjlB,GAAQ6K,EAAGqa,YAAYllB,IACpD3I,EAAKgI,iBAAiB4I,SAAU,YAAa4C,EAAGka,aAChD1tB,EAAKgI,iBAAiB4I,SAAU,WAAY4C,EAAGoa,YAE/CvuB,KAAKmkB,aAAa7a,IAMpBtI,EAAQmS,UAAUmb,aAAe,SAAShlB,GACxCtJ,KAAKktB,aAAa5jB,IAMpBtI,EAAQmS,UAAUqb,YAAc,SAASllB,GACvCtJ,KAAK0sB,WAAY,EAEjB/rB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKquB,aACrD1tB,EAAKwI,oBAAoBoI,SAAU,WAAcvR,KAAKuuB,YAEtDvuB,KAAKusB,WAAWjjB,IASlBtI,EAAQmS,UAAUoR,SAAW,SAASjb,GAC/BA,IACHA,EAAQC,OAAOD,MAGjB,IAAImlB,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAW,IAChBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAO,GAMpBF,EAAO,CACT,GAAIG,GAAY5uB,KAAKib,OAAOmE,eACxByP,EAAYD,GAAa,EAAIH,EAAQ,GAEzCzuB,MAAKib,OAAO2K,aAAaiJ,GACzB7uB,KAAKyhB,SAELzhB,KAAKiuB,eAIP,GAAIN,GAAa3tB,KAAK6lB,mBACtB7lB,MAAK4tB,KAAK,uBAAwBD,GAKlChtB,EAAK0I,eAAeC,IAUtBtI,EAAQmS,UAAU2b,gBAAkB,SAAU5c,EAAO6c,GAKnD,QAASC,GAAMjd,GACb,MAAOA,GAAI,EAAI,EAAQ,EAAJA,EAAQ,GAAK,EALlC,GAAIxM,GAAIwpB,EAAS,GACf3oB,EAAI2oB,EAAS,GACbtuB,EAAIsuB,EAAS,GAMXE,EAAKD,GAAM5oB,EAAE2L,EAAIxM,EAAEwM,IAAMG,EAAMF,EAAIzM,EAAEyM,IAAM5L,EAAE4L,EAAIzM,EAAEyM,IAAME,EAAMH,EAAIxM,EAAEwM,IACrEmd,EAAKF,GAAMvuB,EAAEsR,EAAI3L,EAAE2L,IAAMG,EAAMF,EAAI5L,EAAE4L,IAAMvR,EAAEuR,EAAI5L,EAAE4L,IAAME,EAAMH,EAAI3L,EAAE2L,IACrEod,EAAKH,GAAMzpB,EAAEwM,EAAItR,EAAEsR,IAAMG,EAAMF,EAAIvR,EAAEuR,IAAMzM,EAAEyM,EAAIvR,EAAEuR,IAAME,EAAMH,EAAItR,EAAEsR,GAGzE,SAAc,GAANkd,GAAiB,GAANC,GAAWD,GAAMC,GAC3B,GAANA,GAAiB,GAANC,GAAWD,GAAMC,GACtB,GAANF,GAAiB,GAANE,GAAWF,GAAME,IAUjCnuB,EAAQmS,UAAUgb,iBAAmB,SAAUpc,EAAGC,GAChD,GAAIxM,GACF4pB,EAAU,IACVlB,EAAY,KACZmB,EAAmB,KACnBC,EAAc,KACdpD,EAAS,GAAI9qB,GAAQ2Q,EAAGC,EAE1B,IAAIhS,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,KAC/BlgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAE7B,IAAK5a,EAAIxF,KAAKmb,WAAWxV,OAAS,EAAGH,GAAK,EAAGA,IAAK,CAChD0oB,EAAYluB,KAAKmb,WAAW3V,EAC5B,IAAIymB,GAAYiC,EAAUjC,QAC1B,IAAIA,EACF,IAAK,GAAIrgB,GAAIqgB,EAAStmB,OAAS,EAAGiG,GAAK,EAAGA,IAAK,CAE7C,GAAIigB,GAAUI,EAASrgB,GACnBkgB,EAAUD,EAAQC,QAClByD,GAAazD,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,QAC9DmM,GAAa1D,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAAQyI,EAAQ,GAAGzI,OAClE,IAAIrjB,KAAK8uB,gBAAgB5C,EAAQqD,IAC/BvvB,KAAK8uB,gBAAgB5C,EAAQsD,GAE7B,MAAOtB,QAQf,KAAK1oB,EAAI,EAAGA,EAAIxF,KAAKmb,WAAWxV,OAAQH,IAAK,CAC3C0oB,EAAYluB,KAAKmb,WAAW3V,EAC5B,IAAI0M,GAAQgc,EAAU7K,MACtB,IAAInR,EAAO,CACT,GAAIud,GAAQvqB,KAAK2lB,IAAI9Y,EAAIG,EAAMH,GAC3B2d,EAAQxqB,KAAK2lB,IAAI7Y,EAAIE,EAAMF,GAC3BkZ,EAAQhmB,KAAKyqB,KAAKF,EAAQA,EAAQC,EAAQA,IAEzB,OAAhBJ,GAA+BA,EAAPpE,IAA8BkE,EAAPlE,IAClDoE,EAAcpE,EACdmE,EAAmBnB,IAO3B,MAAOmB,IAQTruB,EAAQmS,UAAUib,aAAe,SAAUF,GACzC,GAAI0B,GAASC,EAAMC,CAEd9vB,MAAKomB,SAiCRwJ,EAAU5vB,KAAKomB,QAAQ2J,IAAIH,QAC3BC,EAAQ7vB,KAAKomB,QAAQ2J,IAAIF,KACzBC,EAAQ9vB,KAAKomB,QAAQ2J,IAAID,MAlCzBF,EAAUre,SAASM,cAAc,OACjC+d,EAAQ3iB,MAAM2W,SAAW,WACzBgM,EAAQ3iB,MAAM+W,QAAU,OACxB4L,EAAQ3iB,MAAMb,OAAS,oBACvBwjB,EAAQ3iB,MAAM9B,MAAQ,UACtBykB,EAAQ3iB,MAAMd,WAAa,wBAC3ByjB,EAAQ3iB,MAAM+iB,aAAe,MAC7BJ,EAAQ3iB,MAAMgjB,UAAY,qCAE1BJ,EAAOte,SAASM,cAAc,OAC9Bge,EAAK5iB,MAAM2W,SAAW,WACtBiM,EAAK5iB,MAAMuF,OAAS,OACpBqd,EAAK5iB,MAAMsF,MAAQ,IACnBsd,EAAK5iB,MAAMijB,WAAa,oBAExBJ,EAAMve,SAASM,cAAc,OAC7Bie,EAAI7iB,MAAM2W,SAAW,WACrBkM,EAAI7iB,MAAMuF,OAAS,IACnBsd,EAAI7iB,MAAMsF,MAAQ,IAClBud,EAAI7iB,MAAMb,OAAS,oBACnB0jB,EAAI7iB,MAAM+iB,aAAe,MAEzBhwB,KAAKomB,SACH8H,UAAW,KACX6B,KACEH,QAASA,EACTC,KAAMA,EACNC,IAAKA,KAUX9vB,KAAKiuB,eAELjuB,KAAKomB,QAAQ8H,UAAYA,EAEvB0B,EAAQ3L,UADsB,kBAArBjkB,MAAK6a,YACM7a,KAAK6a,YAAYqT,EAAUhc,OAG3B,6BACMgc,EAAUhc,MAAMH,EAAI,gCACpBmc,EAAUhc,MAAMF,EAAI,gCACpBkc,EAAUhc,MAAMgL,EAAI,qBAIhD0S,EAAQ3iB,MAAMxF,KAAQ,IACtBmoB,EAAQ3iB,MAAMtF,IAAQ,IACtB3H,KAAKsf,MAAM7N,YAAYme,GACvB5vB,KAAKsf,MAAM7N,YAAYoe,GACvB7vB,KAAKsf,MAAM7N,YAAYqe,EAGvB,IAAIK,GAAgBP,EAAQQ,YACxBC,EAAkBT,EAAQU,aAC1BC,EAAgBV,EAAKS,aACrBE,EAAcV,EAAIM,YAClBK,EAAgBX,EAAIQ,aAEpB7oB,EAAOymB,EAAU7K,OAAOtR,EAAIoe,EAAe,CAC/C1oB,GAAOvC,KAAK4G,IAAI5G,KAAKwH,IAAIjF,EAAM,IAAKzH,KAAKsf,MAAME,YAAc,GAAK2Q,GAElEN,EAAK5iB,MAAMxF,KAASymB,EAAU7K,OAAOtR,EAAI,KACzC8d,EAAK5iB,MAAMtF,IAAUumB,EAAU7K,OAAOrR,EAAIue,EAAc,KACxDX,EAAQ3iB,MAAMxF,KAAQA,EAAO,KAC7BmoB,EAAQ3iB,MAAMtF,IAASumB,EAAU7K,OAAOrR,EAAIue,EAAaF,EAAiB,KAC1EP,EAAI7iB,MAAMxF,KAAWymB,EAAU7K,OAAOtR,EAAIye,EAAW,EAAK,KAC1DV,EAAI7iB,MAAMtF,IAAWumB,EAAU7K,OAAOrR,EAAIye,EAAY,EAAK,MAO7DzvB,EAAQmS,UAAU8a,aAAe,WAC/B,GAAIjuB,KAAKomB,QAAS,CAChBpmB,KAAKomB,QAAQ8H,UAAY,IAEzB,KAAK,GAAIroB,KAAQ7F,MAAKomB,QAAQ2J,IAC5B,GAAI/vB,KAAKomB,QAAQ2J,IAAIjqB,eAAeD,GAAO,CACzC,GAAI0B,GAAOvH,KAAKomB,QAAQ2J,IAAIlqB,EACxB0B,IAAQA,EAAKsC,YACftC,EAAKsC,WAAWsH,YAAY5J,MA8BtC1H,EAAOD,QAAUoB,GAKb,SAASnB,EAAQD,EAASM,GAc9B,QAASgB,KACPlB,KAAK0wB,YAAc,GAAIrvB,GACvBrB,KAAK2wB,eACL3wB,KAAK2wB,YAAYnL,WAAa,EAC9BxlB,KAAK2wB,YAAYlL,SAAW,EAC5BzlB,KAAK4wB,UAAY,IAEjB5wB,KAAK6wB,eAAiB,GAAIxvB,GAC1BrB,KAAK8wB,eAAkB,GAAIzvB,GAAQ,GAAI6D,KAAKymB,GAAI,EAAG,GAEnD3rB,KAAK+wB,6BAtBP,GAAI1vB,GAAUnB,EAAoB,GA+BlCgB,GAAOiS,UAAUmK,eAAiB,SAASvL,EAAGC,EAAGkL,GAC/Cld,KAAK0wB,YAAY3e,EAAIA,EACrB/R,KAAK0wB,YAAY1e,EAAIA,EACrBhS,KAAK0wB,YAAYxT,EAAIA,EAErBld,KAAK+wB,8BAWP7vB,EAAOiS,UAAUuS,eAAiB,SAASF,EAAYC,GAClCjf,SAAfgf,IACFxlB,KAAK2wB,YAAYnL,WAAaA,GAGfhf,SAAbif,IACFzlB,KAAK2wB,YAAYlL,SAAWA,EACxBzlB,KAAK2wB,YAAYlL,SAAW,IAAGzlB,KAAK2wB,YAAYlL,SAAW,GAC3DzlB,KAAK2wB,YAAYlL,SAAW,GAAIvgB,KAAKymB,KAAI3rB,KAAK2wB,YAAYlL,SAAW,GAAIvgB,KAAKymB,MAGjEnlB,SAAfgf,GAAyChf,SAAbif,IAC9BzlB,KAAK+wB,8BAQT7vB,EAAOiS,UAAU2S,eAAiB,WAChC,GAAIkL,KAIJ,OAHAA,GAAIxL,WAAaxlB,KAAK2wB,YAAYnL,WAClCwL,EAAIvL,SAAWzlB,KAAK2wB,YAAYlL,SAEzBuL,GAOT9vB,EAAOiS,UAAUyS,aAAe,SAASjgB,GACxBa,SAAXb,IAGJ3F,KAAK4wB,UAAYjrB,EAKb3F,KAAK4wB,UAAY,MAAM5wB,KAAK4wB,UAAY,KACxC5wB,KAAK4wB,UAAY,IAAK5wB,KAAK4wB,UAAY,GAE3C5wB,KAAK+wB,+BAOP7vB,EAAOiS,UAAUiM,aAAe,WAC9B,MAAOpf,MAAK4wB,WAOd1vB,EAAOiS,UAAU6K,kBAAoB,WACnC,MAAOhe,MAAK6wB,gBAOd3vB,EAAOiS,UAAUkL,kBAAoB,WACnC,MAAOre,MAAK8wB,gBAOd5vB,EAAOiS,UAAU4d,2BAA6B,WAE5C/wB,KAAK6wB,eAAe9e,EAAI/R,KAAK0wB,YAAY3e,EAAI/R,KAAK4wB,UAAY1rB,KAAKkZ,IAAIpe,KAAK2wB,YAAYnL,YAActgB,KAAKqZ,IAAIve,KAAK2wB,YAAYlL,UAChIzlB,KAAK6wB,eAAe7e,EAAIhS,KAAK0wB,YAAY1e,EAAIhS,KAAK4wB,UAAY1rB,KAAKqZ,IAAIve,KAAK2wB,YAAYnL,YAActgB,KAAKqZ,IAAIve,KAAK2wB,YAAYlL,UAChIzlB,KAAK6wB,eAAe3T,EAAIld,KAAK0wB,YAAYxT,EAAIld,KAAK4wB,UAAY1rB,KAAKkZ,IAAIpe,KAAK2wB,YAAYlL,UAGxFzlB,KAAK8wB,eAAe/e,EAAI7M,KAAKymB,GAAG,EAAI3rB,KAAK2wB,YAAYlL,SACrDzlB,KAAK8wB,eAAe9e,EAAI,EACxBhS,KAAK8wB,eAAe5T,GAAKld,KAAK2wB,YAAYnL,YAG5C3lB,EAAOD,QAAUsB,GAIb,SAASrB,EAAQD,EAASM,GAW9B,QAASiB,GAAQuR,EAAMqO,EAAQkQ,GAC7BjxB,KAAK0S,KAAOA,EACZ1S,KAAK+gB,OAASA,EACd/gB,KAAKixB,MAAQA,EAEbjxB,KAAKmI,MAAQ3B,OACbxG,KAAKqH,MAAQb,OAGbxG,KAAK8W,OAASma,EAAMjQ,kBAAkBtO,EAAKwC,MAAOlV,KAAK+gB,QAGvD/gB,KAAK8W,OAAOZ,KAAK,SAAU3Q,EAAGa,GAC5B,MAAOb,GAAIa,EAAI,EAAQA,EAAJb,EAAQ,GAAK,IAG9BvF,KAAK8W,OAAOnR,OAAS,GACvB3F,KAAK+oB,YAAY,GAInB/oB,KAAKmb,cAELnb,KAAKM,QAAS,EACdN,KAAKkxB,eAAiB1qB,OAElByqB,EAAMjW,kBACRhb,KAAKM,QAAS,EACdN,KAAKmxB,oBAGLnxB,KAAKM,QAAS,EAxClB,GAAIQ,GAAWZ,EAAoB,EAiDnCiB,GAAOgS,UAAUie,SAAW,WAC1B,MAAOpxB,MAAKM,QAQda,EAAOgS,UAAUke,kBAAoB,WAInC,IAHA,GAAI5rB,GAAMzF,KAAK8W,OAAOnR,OAElBH,EAAI,EACDxF,KAAKmb,WAAW3V,IACrBA,GAGF,OAAON,MAAKwoB,MAAMloB,EAAIC,EAAM,MAQ9BtE,EAAOgS,UAAU+V,SAAW,WAC1B,MAAOlpB,MAAKixB,MAAM7W,aAQpBjZ,EAAOgS,UAAUme,UAAY,WAC3B,MAAOtxB,MAAK+gB,QAOd5f,EAAOgS,UAAUgW,iBAAmB,WAClC,MAAmB3iB,UAAfxG,KAAKmI,MACA3B,OAEFxG,KAAK8W,OAAO9W,KAAKmI,QAO1BhH,EAAOgS,UAAUoe,UAAY,WAC3B,MAAOvxB,MAAK8W,QAQd3V,EAAOgS,UAAUyB,SAAW,SAASzM,GACnC,GAAIA,GAASnI,KAAK8W,OAAOnR,OACvB,KAAM,2BAER,OAAO3F,MAAK8W,OAAO3O,IASrBhH,EAAOgS,UAAU2P,eAAiB,SAAS3a,GAIzC,GAHc3B,SAAV2B,IACFA,EAAQnI,KAAKmI,OAED3B,SAAV2B,EACF,QAEF,IAAIgT,EACJ,IAAInb,KAAKmb,WAAWhT,GAClBgT,EAAanb,KAAKmb,WAAWhT,OAE1B,CACH,GAAIyF,KACJA,GAAEmT,OAAS/gB,KAAK+gB,OAChBnT,EAAEvG,MAAQrH,KAAK8W,OAAO3O,EAEtB,IAAIqpB,GAAW,GAAI1wB,GAASd,KAAK0S,MAAMiB,OAAQ,SAAUtE,GAAO,MAAQA,GAAKzB,EAAEmT,SAAWnT,EAAEvG,SAAW6N,KACvGiG,GAAanb,KAAKixB,MAAMnO,eAAe0O,GAEvCxxB,KAAKmb,WAAWhT,GAASgT,EAG3B,MAAOA,IAQTha,EAAOgS,UAAUqO,kBAAoB,SAASlZ,GAC5CtI,KAAKkxB,eAAiB5oB,GASxBnH,EAAOgS,UAAU4V,YAAc,SAAS5gB,GACtC,GAAIA,GAASnI,KAAK8W,OAAOnR,OACvB,KAAM,2BAER3F,MAAKmI,MAAQA,EACbnI,KAAKqH,MAAQrH,KAAK8W,OAAO3O,IAO3BhH,EAAOgS,UAAUge,iBAAmB,SAAShpB,GAC7B3B,SAAV2B,IACFA,EAAQ,EAEV,IAAImX,GAAQtf,KAAKixB,MAAM3R,KAEvB,IAAInX,EAAQnI,KAAK8W,OAAOnR,OAAQ,CAC9B,CAAqB3F,KAAK8iB,eAAe3a,GAIlB3B,SAAnB8Y,EAAMmS,WACRnS,EAAMmS,SAAWlgB,SAASM,cAAc,OACxCyN,EAAMmS,SAASxkB,MAAM2W,SAAW,WAChCtE,EAAMmS,SAASxkB,MAAM9B,MAAQ,OAC7BmU,EAAM7N,YAAY6N,EAAMmS,UAE1B,IAAIA,GAAWzxB,KAAKqxB,mBACpB/R,GAAMmS,SAASxN,UAAY,wBAA0BwN,EAAW,IAEhEnS,EAAMmS,SAASxkB,MAAMqW,OAAS,OAC9BhE,EAAMmS,SAASxkB,MAAMxF,KAAO,MAE5B,IAAI0M,GAAKnU,IACTsZ,YAAW,WAAYnF,EAAGgd,iBAAiBhpB,EAAM,IAAM,IACvDnI,KAAKM,QAAS,MAGdN,MAAKM,QAAS,EAGSkG,SAAnB8Y,EAAMmS,WACRnS,EAAMnO,YAAYmO,EAAMmS,UACxBnS,EAAMmS,SAAWjrB,QAGfxG,KAAKkxB,gBACPlxB,KAAKkxB,kBAIXrxB,EAAOD,QAAUuB,GAKb,SAAStB,GAOb,QAASuB,GAAS2Q,EAAGC,GACnBhS,KAAK+R,EAAUvL,SAANuL,EAAkBA,EAAI,EAC/B/R,KAAKgS,EAAUxL,SAANwL,EAAkBA,EAAI,EAGjCnS,EAAOD,QAAUwB,GAKb,SAASvB,GAQb,QAASwB,GAAQ0Q,EAAGC,EAAGkL,GACrBld,KAAK+R,EAAUvL,SAANuL,EAAkBA,EAAI,EAC/B/R,KAAKgS,EAAUxL,SAANwL,EAAkBA,EAAI,EAC/BhS,KAAKkd,EAAU1W,SAAN0W,EAAkBA,EAAI,EASjC7b,EAAQgqB,SAAW,SAAS9lB,EAAGa,GAC7B,GAAIsrB,GAAM,GAAIrwB,EAId,OAHAqwB,GAAI3f,EAAIxM,EAAEwM,EAAI3L,EAAE2L,EAChB2f,EAAI1f,EAAIzM,EAAEyM,EAAI5L,EAAE4L,EAChB0f,EAAIxU,EAAI3X,EAAE2X,EAAI9W,EAAE8W,EACTwU,GASTrwB,EAAQ4R,IAAM,SAAS1N,EAAGa,GACxB,GAAIurB,GAAM,GAAItwB,EAId;MAHAswB,GAAI5f,EAAIxM,EAAEwM,EAAI3L,EAAE2L,EAChB4f,EAAI3f,EAAIzM,EAAEyM,EAAI5L,EAAE4L,EAChB2f,EAAIzU,EAAI3X,EAAE2X,EAAI9W,EAAE8W,EACTyU,GASTtwB,EAAQ8qB,IAAM,SAAS5mB,EAAGa,GACxB,MAAO,IAAI/E,IACFkE,EAAEwM,EAAI3L,EAAE2L,GAAK,GACbxM,EAAEyM,EAAI5L,EAAE4L,GAAK,GACbzM,EAAE2X,EAAI9W,EAAE8W,GAAK,IAWxB7b,EAAQmqB,aAAe,SAASjmB,EAAGa,GACjC,GAAImlB,GAAe,GAAIlqB,EAMvB,OAJAkqB,GAAaxZ,EAAIxM,EAAEyM,EAAI5L,EAAE8W,EAAI3X,EAAE2X,EAAI9W,EAAE4L,EACrCuZ,EAAavZ,EAAIzM,EAAE2X,EAAI9W,EAAE2L,EAAIxM,EAAEwM,EAAI3L,EAAE8W,EACrCqO,EAAarO,EAAI3X,EAAEwM,EAAI3L,EAAE4L,EAAIzM,EAAEyM,EAAI5L,EAAE2L,EAE9BwZ,GAQTlqB,EAAQ8R,UAAUxN,OAAS,WACzB,MAAOT,MAAKyqB,KACJ3vB,KAAK+R,EAAI/R,KAAK+R,EACd/R,KAAKgS,EAAIhS,KAAKgS,EACdhS,KAAKkd,EAAIld,KAAKkd,IAIxBrd,EAAOD,QAAUyB,GAKb,SAASxB,EAAQD,EAASM,GAa9B,QAASoB,GAAOiY,EAAW9K,GACzB,GAAkBjI,SAAd+S,EACF,KAAM,qCAKR,IAHAvZ,KAAKuZ,UAAYA,EACjBvZ,KAAK0oB,QAAWja,GAA8BjI,QAAnBiI,EAAQia,QAAwBja,EAAQia,SAAU,EAEzE1oB,KAAK0oB,QAAS,CAChB1oB,KAAKsf,MAAQ/N,SAASM,cAAc,OAEpC7R,KAAKsf,MAAMrS,MAAMsF,MAAQ,OACzBvS,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKuZ,UAAU9H,YAAYzR,KAAKsf,OAEhCtf,KAAKsf,MAAMsS,KAAOrgB,SAASM,cAAc,SACzC7R,KAAKsf,MAAMsS,KAAK9qB,KAAO,SACvB9G,KAAKsf,MAAMsS,KAAKvqB,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMsS,MAElC5xB,KAAKsf,MAAM0F,KAAOzT,SAASM,cAAc,SACzC7R,KAAKsf,MAAM0F,KAAKle,KAAO,SACvB9G,KAAKsf,MAAM0F,KAAK3d,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM0F,MAElChlB,KAAKsf,MAAM+I,KAAO9W,SAASM,cAAc,SACzC7R,KAAKsf,MAAM+I,KAAKvhB,KAAO,SACvB9G,KAAKsf,MAAM+I,KAAKhhB,MAAQ,OACxBrH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM+I,MAElCroB,KAAKsf,MAAMuS,IAAMtgB,SAASM,cAAc,SACxC7R,KAAKsf,MAAMuS,IAAI/qB,KAAO,SACtB9G,KAAKsf,MAAMuS,IAAI5kB,MAAM2W,SAAW,WAChC5jB,KAAKsf,MAAMuS,IAAI5kB,MAAMb,OAAS,gBAC9BpM,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,MAAQ,QAC7BvS,KAAKsf,MAAMuS,IAAI5kB,MAAMuF,OAAS,MAC9BxS,KAAKsf,MAAMuS,IAAI5kB,MAAM+iB,aAAe,MACpChwB,KAAKsf,MAAMuS,IAAI5kB,MAAM6kB,gBAAkB,MACvC9xB,KAAKsf,MAAMuS,IAAI5kB,MAAMb,OAAS,oBAC9BpM,KAAKsf,MAAMuS,IAAI5kB,MAAM0S,gBAAkB,UACvC3f,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMuS,KAElC7xB,KAAKsf,MAAMyS,MAAQxgB,SAASM,cAAc,SAC1C7R,KAAKsf,MAAMyS,MAAMjrB,KAAO,SACxB9G,KAAKsf,MAAMyS,MAAM9kB,MAAMyM,OAAS,MAChC1Z,KAAKsf,MAAMyS,MAAM1qB,MAAQ,IACzBrH,KAAKsf,MAAMyS,MAAM9kB,MAAM2W,SAAW,WAClC5jB,KAAKsf,MAAMyS,MAAM9kB,MAAMxF,KAAO,SAC9BzH,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMyS,MAGlC,IAAI5d,GAAKnU,IACTA,MAAKsf,MAAMyS,MAAM7N,YAAc,SAAU5a,GAAQ6K,EAAGgQ,aAAa7a,IACjEtJ,KAAKsf,MAAMsS,KAAKI,QAAU,SAAU1oB,GAAQ6K,EAAGyd,KAAKtoB,IACpDtJ,KAAKsf,MAAM0F,KAAKgN,QAAU,SAAU1oB,GAAQ6K,EAAG8d,WAAW3oB,IAC1DtJ,KAAKsf,MAAM+I,KAAK2J,QAAU,SAAU1oB,GAAQ6K,EAAGkU,KAAK/e,IAGtDtJ,KAAKkyB,iBAAmB1rB,OAExBxG,KAAK8W,UACL9W,KAAKmI,MAAQ3B,OAEbxG,KAAKmyB,YAAc3rB,OACnBxG,KAAKoyB,aAAe,IACpBpyB,KAAKqyB,UAAW,EA3ElB,GAAI1xB,GAAOT,EAAoB,EAiF/BoB,GAAO6R,UAAUye,KAAO,WACtB,GAAIzpB,GAAQnI,KAAK8oB,UACb3gB,GAAQ,IACVA,IACAnI,KAAKsyB,SAASnqB,KAOlB7G,EAAO6R,UAAUkV,KAAO,WACtB,GAAIlgB,GAAQnI,KAAK8oB,UACb3gB,GAAQnI,KAAK8W,OAAOnR,OAAS,IAC/BwC,IACAnI,KAAKsyB,SAASnqB,KAOlB7G,EAAO6R,UAAUof,SAAW,WAC1B,GAAI3iB,GAAQ,GAAItL,MAEZ6D,EAAQnI,KAAK8oB,UACb3gB,GAAQnI,KAAK8W,OAAOnR,OAAS,GAC/BwC,IACAnI,KAAKsyB,SAASnqB,IAEPnI,KAAKqyB,WAEZlqB,EAAQ,EACRnI,KAAKsyB,SAASnqB,GAGhB,IAAI0H,GAAM,GAAIvL,MACV+nB,EAAQxc,EAAMD,EAId4iB,EAAWttB,KAAKwH,IAAI1M,KAAKoyB,aAAe/F,EAAM,GAG9ClY,EAAKnU,IACTA,MAAKmyB,YAAc7Y,WAAW,WAAYnF,EAAGoe,YAAcC,IAM7DlxB,EAAO6R,UAAU8e,WAAa,WACHzrB,SAArBxG,KAAKmyB,YACPnyB,KAAKglB,OAELhlB,KAAKklB,QAOT5jB,EAAO6R,UAAU6R,KAAO,WAElBhlB,KAAKmyB,cAETnyB,KAAKuyB,WAEDvyB,KAAKsf,QACPtf,KAAKsf,MAAM0F,KAAK3d,MAAQ,UAO5B/F,EAAO6R,UAAU+R,KAAO,WACtBuN,cAAczyB,KAAKmyB,aACnBnyB,KAAKmyB,YAAc3rB,OAEfxG,KAAKsf,QACPtf,KAAKsf,MAAM0F,KAAK3d,MAAQ,SAQ5B/F,EAAO6R,UAAU6V,oBAAsB,SAAS1gB,GAC9CtI,KAAKkyB,iBAAmB5pB,GAO1BhH,EAAO6R,UAAUyV,gBAAkB,SAAS4J,GAC1CxyB,KAAKoyB,aAAeI,GAOtBlxB,EAAO6R,UAAUuf,gBAAkB,WACjC,MAAO1yB,MAAKoyB,cASd9wB,EAAO6R,UAAUwf,YAAc,SAASC,GACtC5yB,KAAKqyB,SAAWO,GAOlBtxB,EAAO6R,UAAU0f,SAAW,WACIrsB,SAA1BxG,KAAKkyB,kBACPlyB,KAAKkyB,oBAOT5wB,EAAO6R,UAAUsO,OAAS,WACxB,GAAIzhB,KAAKsf,MAAO,CAEdtf,KAAKsf,MAAMuS,IAAI5kB,MAAMtF,IAAO3H,KAAKsf,MAAMuF,aAAa,EAChD7kB,KAAKsf,MAAMuS,IAAIvB,aAAa,EAAK,KACrCtwB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,MAASvS,KAAKsf,MAAME,YACrCxf,KAAKsf,MAAMsS,KAAKpS,YAChBxf,KAAKsf,MAAM0F,KAAKxF,YAChBxf,KAAKsf,MAAM+I,KAAK7I,YAAc,GAAO,IAGzC,IAAI/X,GAAOzH,KAAK8yB,YAAY9yB,KAAKmI,MACjCnI,MAAKsf,MAAMyS,MAAM9kB,MAAMxF,KAAO,EAAS,OAS3CnG,EAAO6R,UAAUwV,UAAY,SAAS7R,GACpC9W,KAAK8W,OAASA,EAEV9W,KAAK8W,OAAOnR,OAAS,EACvB3F,KAAKsyB,SAAS,GAEdtyB,KAAKmI,MAAQ3B,QAOjBlF,EAAO6R,UAAUmf,SAAW,SAASnqB,GACnC,KAAIA,EAAQnI,KAAK8W,OAAOnR,QAOtB,KAAM,2BANN3F,MAAKmI,MAAQA,EAEbnI,KAAKyhB,SACLzhB,KAAK6yB,YAWTvxB,EAAO6R,UAAU2V,SAAW,WAC1B,MAAO9oB,MAAKmI,OAQd7G,EAAO6R,UAAU+B,IAAM,WACrB,MAAOlV,MAAK8W,OAAO9W,KAAKmI,QAI1B7G,EAAO6R,UAAUgR,aAAe,SAAS7a,GAEvC,GAAIgjB,GAAiBhjB,EAAMkjB,MAAyB,IAAhBljB,EAAMkjB,MAAiC,IAAjBljB,EAAMmjB,MAChE,IAAKH,EAAL,CAEAtsB,KAAK+yB,aAAezpB,EAAMqT,QAC1B3c,KAAKgzB,YAAc3N,WAAWrlB,KAAKsf,MAAMyS,MAAM9kB,MAAMxF,MAErDzH,KAAKsf,MAAMrS,MAAM+f,OAAS,MAK1B,IAAI7Y,GAAKnU,IACTA,MAAKitB,YAAc,SAAU3jB,GAAQ6K,EAAG+Y,aAAa5jB,IACrDtJ,KAAKmtB,UAAc,SAAU7jB,GAAQ6K,EAAGoY,WAAWjjB,IACnD3I,EAAKgI,iBAAiB4I,SAAU,YAAavR,KAAKitB,aAClDtsB,EAAKgI,iBAAiB4I,SAAU,UAAavR,KAAKmtB,WAClDxsB,EAAK0I,eAAeC,KAItBhI,EAAO6R,UAAU8f,YAAc,SAAUxrB,GACvC,GAAI8K,GAAQ8S,WAAWrlB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,OACxCvS,KAAKsf,MAAMyS,MAAMvS,YAAc,GAC/BzN,EAAItK,EAAO,EAEXU,EAAQjD,KAAKwoB,MAAM3b,EAAIQ,GAASvS,KAAK8W,OAAOnR,OAAO,GAIvD,OAHY,GAARwC,IAAWA,EAAQ,GACnBA,EAAQnI,KAAK8W,OAAOnR,OAAO,IAAGwC,EAAQnI,KAAK8W,OAAOnR,OAAO,GAEtDwC,GAGT7G,EAAO6R,UAAU2f,YAAc,SAAU3qB,GACvC,GAAIoK,GAAQ8S,WAAWrlB,KAAKsf,MAAMuS,IAAI5kB,MAAMsF,OACxCvS,KAAKsf,MAAMyS,MAAMvS,YAAc,GAE/BzN,EAAI5J,GAASnI,KAAK8W,OAAOnR,OAAO,GAAK4M,EACrC9K,EAAOsK,EAAI,CAEf,OAAOtK,IAKTnG,EAAO6R,UAAU+Z,aAAe,SAAU5jB,GACxC,GAAI+iB,GAAO/iB,EAAMqT,QAAU3c,KAAK+yB,aAC5BhhB,EAAI/R,KAAKgzB,YAAc3G,EAEvBlkB,EAAQnI,KAAKizB,YAAYlhB,EAE7B/R,MAAKsyB,SAASnqB,GAEdxH,EAAK0I,kBAIP/H,EAAO6R,UAAUoZ,WAAa,WAC5BvsB,KAAKsf,MAAMrS,MAAM+f,OAAS,OAG1BrsB,EAAKwI,oBAAoBoI,SAAU,YAAavR,KAAKitB,aACrDtsB,EAAKwI,oBAAoBoI,SAAU,UAAWvR,KAAKmtB,WAEnDxsB,EAAK0I,kBAGPxJ,EAAOD,QAAU0B,GAKb,SAASzB,GA2Bb,QAAS0B,GAAWqO,EAAOC,EAAKsY,EAAMmB,GAEpCtpB,KAAKkzB,OAAS,EACdlzB,KAAKmzB,KAAO,EACZnzB,KAAKozB,MAAQ,EACbpzB,KAAKspB,YAAa,EAClBtpB,KAAKqzB,UAAY,EAEjBrzB,KAAKszB,SAAW,EAChBtzB,KAAKuzB,SAAS3jB,EAAOC,EAAKsY,EAAMmB,GAYlC/nB,EAAW4R,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKsY,EAAMmB,GACzDtpB,KAAKkzB,OAAStjB,EAAQA,EAAQ,EAC9B5P,KAAKmzB,KAAOtjB,EAAMA,EAAM,EAExB7P,KAAKwzB,QAAQrL,EAAMmB,IASrB/nB,EAAW4R,UAAUqgB,QAAU,SAASrL,EAAMmB,GAC/B9iB,SAAT2hB,GAA8B,GAARA,IAGP3hB,SAAf8iB,IACFtpB,KAAKspB,WAAaA,GAGlBtpB,KAAKozB,MADHpzB,KAAKspB,cAAe,EACT/nB,EAAWkyB,oBAAoBtL,GAE/BA,IAUjB5mB,EAAWkyB,oBAAsB,SAAUtL,GACzC,GAAIuL,GAAQ,SAAU3hB,GAAI,MAAO7M,MAAKyuB,IAAI5hB,GAAK7M,KAAK0uB,MAGhDC,EAAQ3uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,KACtC4L,EAAQ,EAAI7uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,EAAO,KACjD6L,EAAQ,EAAI9uB,KAAK4uB,IAAI,GAAI5uB,KAAKwoB,MAAMgG,EAAMvL,EAAO,KAGjDmB,EAAauK,CASjB,OARI3uB,MAAK2lB,IAAIkJ,EAAQ5L,IAASjjB,KAAK2lB,IAAIvB,EAAanB,KAAOmB,EAAayK,GACpE7uB,KAAK2lB,IAAImJ,EAAQ7L,IAASjjB,KAAK2lB,IAAIvB,EAAanB,KAAOmB,EAAa0K,GAGtD,GAAd1K,IACFA,EAAa,GAGRA,GAOT/nB,EAAW4R,UAAUiV,WAAa,WAChC,MAAO/C,YAAWrlB,KAAKszB,SAASW,YAAYj0B,KAAKqzB,aAOnD9xB,EAAW4R,UAAU+gB,QAAU,WAC7B,MAAOl0B,MAAKozB,OAOd7xB,EAAW4R,UAAUvD,MAAQ,WAC3B5P,KAAKszB,SAAWtzB,KAAKkzB,OAASlzB,KAAKkzB,OAASlzB,KAAKozB,OAMnD7xB,EAAW4R,UAAUkV,KAAO,WAC1BroB,KAAKszB,UAAYtzB,KAAKozB,OAOxB7xB,EAAW4R,UAAUtD,IAAM,WACzB,MAAQ7P,MAAKszB,SAAWtzB,KAAKmzB,MAG/BtzB,EAAOD,QAAU2B,GAKb,SAAS1B,EAAQD,EAASM,GAuB9B,QAASsB,GAAU+X,EAAWtX,EAAOkyB,EAAQ1lB,GAC3C,KAAMzO,eAAgBwB,IACpB,KAAM,IAAIgY,aAAY,mDAIxB,MAAMvT,MAAMC,QAAQiuB,IAAWA,YAAkBtzB,KAAYszB,YAAkB5tB,QAAQ,CACrF,GAAI6tB,GAAgB3lB,CACpBA,GAAU0lB,EACVA,EAASC,EAGX,GAAIjgB,GAAKnU,IACTA,MAAKq0B,gBACHzkB,MAAO,KACPC,IAAO,KAEPykB,YAAY,EAEZC,YAAa,SACbhiB,MAAO,KACPC,OAAQ,KACRgiB,UAAW,KACXC,UAAW,MAEbz0B,KAAKyO,QAAU9N,EAAK8F,cAAezG,KAAKq0B,gBAGxCr0B,KAAK00B,QAAQnb,GAGbvZ,KAAKgC,cAELhC,KAAK20B,MACH5E,IAAK/vB,KAAK+vB,IACV6E,SAAU50B,KAAKgG,MACf6uB,SACEthB,GAAIvT,KAAKuT,GAAGuhB,KAAK90B,MACjB0T,IAAK1T,KAAK0T,IAAIohB,KAAK90B,MACnB4tB,KAAM5tB,KAAK4tB,KAAKkH,KAAK90B,OAEvB+0B,eACAp0B,MACEq0B,KAAM,KACNC,SAAU9gB,EAAG+gB,UAAUJ,KAAK3gB,GAC5BghB,eAAgBhhB,EAAGihB,gBAAgBN,KAAK3gB,GACxCkhB,OAAQlhB,EAAGmhB,QAAQR,KAAK3gB,GACxBohB,aAAephB,EAAGqhB,cAAcV,KAAK3gB,KAKzCnU,KAAKy1B,MAAQ,GAAI5zB,GAAM7B,KAAK20B,MAC5B30B,KAAKgC,WAAWgG,KAAKhI,KAAKy1B,OAC1Bz1B,KAAK20B,KAAKc,MAAQz1B,KAAKy1B,MAGvBz1B,KAAK01B,SAAW,GAAIzyB,GAASjD,KAAK20B,MAClC30B,KAAKgC,WAAWgG,KAAKhI,KAAK01B,UAC1B11B,KAAK20B,KAAKh0B,KAAKq0B,KAAOh1B,KAAK01B,SAASV,KAAKF,KAAK90B,KAAK01B,UAGnD11B,KAAK21B,YAAc,GAAInzB,GAAYxC,KAAK20B,MACxC30B,KAAKgC,WAAWgG,KAAKhI,KAAK21B,aAI1B31B,KAAK41B,WAAa,GAAInzB,GAAWzC,KAAK20B,MACtC30B,KAAKgC,WAAWgG,KAAKhI,KAAK41B,YAG1B51B,KAAK61B,QAAU,GAAI/yB,GAAQ9C,KAAK20B,MAChC30B,KAAKgC,WAAWgG,KAAKhI,KAAK61B,SAE1B71B,KAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGdtnB,GACFzO,KAAKkT,WAAWzE,GAId0lB,GACFn0B,KAAKg2B,UAAU7B,GAIblyB,EACFjC,KAAKi2B,SAASh0B,GAGdjC,KAAKyhB,SAjHT,GAEI9gB,IAFUT,EAAoB,IACrBA,EAAoB,IACtBA,EAAoB,IAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/B2B,EAAQ3B,EAAoB,IAC5Bg2B,EAAOh2B,EAAoB,IAC3B+C,EAAW/C,EAAoB,IAC/BsC,EAActC,EAAoB,IAClCuC,EAAavC,EAAoB,IACjC4C,EAAU5C,EAAoB,GA4GlCsB,GAAS2R,UAAY,GAAI+iB,GAMzB10B,EAAS2R,UAAU8iB,SAAW,SAASh0B,GACrC,GAGIk0B,GAHAC,EAAiC,MAAlBp2B,KAAK81B,SAwBxB,IAhBEK,EAJGl0B,EAGIA,YAAiBpB,IAAWoB,YAAiBnB,GACvCmB,EAIA,GAAIpB,GAAQoB,GACvB6E,MACE8I,MAAO,OACPC,IAAK,UAVI,KAgBf7P,KAAK81B,UAAYK,EACjBn2B,KAAK61B,SAAW71B,KAAK61B,QAAQI,SAASE,GAElCC,EACF,GAA0B5vB,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAAkB,CACpE,GAA0BrJ,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAClD,GAAIwmB,GAAYr2B,KAAKs2B,eAGvB,IAAI1mB,GAA8BpJ,QAAtBxG,KAAKyO,QAAQmB,MAAqB5P,KAAKyO,QAAQmB,MAAQymB,EAAUzmB,MACzEC,EAA4BrJ,QAApBxG,KAAKyO,QAAQoB,IAAqB7P,KAAKyO,QAAQoB,IAAQwmB,EAAUxmB,GAE7E7P,MAAKu2B,UAAU3mB,EAAOC,GAAM2mB,SAAS,QAGrCx2B,MAAKy2B,KAAKD,SAAS,KASzBh1B,EAAS2R,UAAU6iB,UAAY,SAAS7B,GAEtC,GAAIgC,EAKFA,GAJGhC,EAGIA,YAAkBtzB,IAAWszB,YAAkBrzB,GACzCqzB,EAIA,GAAItzB,GAAQszB,GAPZ,KAUfn0B,KAAK+1B,WAAaI,EAClBn2B,KAAK61B,QAAQG,UAAUG,IAmBzB30B,EAAS2R,UAAUujB,aAAe,SAASvhB,EAAK1G,GAC9CzO,KAAK61B,SAAW71B,KAAK61B,QAAQa,aAAavhB,GAEtC1G,GAAWA,EAAQkoB,OACrB32B,KAAK22B,MAAMxhB,EAAK1G,IAQpBjN,EAAS2R,UAAUyjB,aAAe,WAChC,MAAO52B,MAAK61B,SAAW71B,KAAK61B,QAAQe,oBAetCp1B,EAAS2R,UAAUwjB,MAAQ,SAASt2B,EAAIoO,GACtC,GAAKzO,KAAK81B,WAAmBtvB,QAANnG,EAAvB,CAEA,GAAI8U,GAAMlP,MAAMC,QAAQ7F,GAAMA,GAAMA,GAGhCy1B,EAAY91B,KAAK81B,UAAUhgB,aAAaZ,IAAIC,GAC9CrO,MACE8I,MAAO,OACPC,IAAK,UAKLD,EAAQ,KACRC,EAAM,IAcV,IAbAimB,EAAUztB,QAAQ,SAAUwuB,GAC1B,GAAIjrB,GAAIirB,EAASjnB,MAAM5I,UACnB8vB,EAAI,OAASD,GAAWA,EAAShnB,IAAI7I,UAAY6vB,EAASjnB,MAAM5I,WAEtD,OAAV4I,GAAsBA,EAAJhE,KACpBgE,EAAQhE,IAGE,OAARiE,GAAgBinB,EAAIjnB,KACtBA,EAAMinB,KAII,OAAVlnB,GAA0B,OAARC,EAAc,CAElC,GAAIT,IAAUQ,EAAQC,GAAO,EACzB2iB,EAAWttB,KAAKwH,IAAK1M,KAAKy1B,MAAM5lB,IAAM7P,KAAKy1B,MAAM7lB,MAAwB,KAAfC,EAAMD,IAEhE4mB,EAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7Ex2B,MAAKy1B,MAAMlC,SAASnkB,EAASojB,EAAW,EAAGpjB,EAASojB,EAAW,EAAGgE,MAUtEh1B,EAAS2R,UAAU4jB,aAAe,WAEhC,GAAIC,GAAUh3B,KAAK81B,UAAUhgB,aAC3BhK,EAAM,KACNY,EAAM,IAER,IAAIsqB,EAAS,CAEX,GAAIC,GAAUD,EAAQlrB,IAAI,QAC1BA,GAAMmrB,EAAUt2B,EAAKkG,QAAQowB,EAAQrnB,MAAO,QAAQ5I,UAAY,IAKhE,IAAIkwB,GAAeF,EAAQtqB,IAAI,QAC3BwqB,KACFxqB,EAAM/L,EAAKkG,QAAQqwB,EAAatnB,MAAO,QAAQ5I,UAEjD,IAAImwB,GAAaH,EAAQtqB,IAAI,MACzByqB,KAEAzqB,EADS,MAAPA,EACI/L,EAAKkG,QAAQswB,EAAWtnB,IAAK,QAAQ7I,UAGrC9B,KAAKwH,IAAIA,EAAK/L,EAAKkG,QAAQswB,EAAWtnB,IAAK,QAAQ7I,YAK/D,OACE8E,IAAa,MAAPA,EAAe,GAAIxH,MAAKwH,GAAO,KACrCY,IAAa,MAAPA,EAAe,GAAIpI,MAAKoI,GAAO,OAKzC7M,EAAOD,QAAU4B,GAKb,SAAS3B,EAAQD,EAASM,GAsB9B,QAASuB,GAAS8X,EAAWtX,EAAOkyB,EAAQ1lB,GAE1C,KAAMxI,MAAMC,QAAQiuB,IAAWA,YAAkBtzB,KAAYszB,YAAkB5tB,QAAQ,CACrF,GAAI6tB,GAAgB3lB,CACpBA,GAAU0lB,EACVA,EAASC,EAGX,GAAIjgB,GAAKnU,IACTA,MAAKq0B,gBACHzkB,MAAO,KACPC,IAAO,KAEPykB,YAAY,EAEZC,YAAa,SACbhiB,MAAO,KACPC,OAAQ,KACRgiB,UAAW,KACXC,UAAW,MAEbz0B,KAAKyO,QAAU9N,EAAK8F,cAAezG,KAAKq0B,gBAGxCr0B,KAAK00B,QAAQnb,GAGbvZ,KAAKgC,cAELhC,KAAK20B,MACH5E,IAAK/vB,KAAK+vB,IACV6E,SAAU50B,KAAKgG,MACf6uB,SACEthB,GAAIvT,KAAKuT,GAAGuhB,KAAK90B,MACjB0T,IAAK1T,KAAK0T,IAAIohB,KAAK90B,MACnB4tB,KAAM5tB,KAAK4tB,KAAKkH,KAAK90B,OAEvB+0B,eACAp0B,MACEq0B,KAAM,KACNC,SAAU9gB,EAAG+gB,UAAUJ,KAAK3gB,GAC5BghB,eAAgBhhB,EAAGihB,gBAAgBN,KAAK3gB,GACxCkhB,OAAQlhB,EAAGmhB,QAAQR,KAAK3gB,GACxBohB,aAAephB,EAAGqhB,cAAcV,KAAK3gB,KAKzCnU,KAAKy1B,MAAQ,GAAI5zB,GAAM7B,KAAK20B,MAC5B30B,KAAKgC,WAAWgG,KAAKhI,KAAKy1B,OAC1Bz1B,KAAK20B,KAAKc,MAAQz1B,KAAKy1B,MAGvBz1B,KAAK01B,SAAW,GAAIzyB,GAASjD,KAAK20B,MAClC30B,KAAKgC,WAAWgG,KAAKhI,KAAK01B,UAC1B11B,KAAK20B,KAAKh0B,KAAKq0B,KAAOh1B,KAAK01B,SAASV,KAAKF,KAAK90B,KAAK01B,UAGnD11B,KAAK21B,YAAc,GAAInzB,GAAYxC,KAAK20B,MACxC30B,KAAKgC,WAAWgG,KAAKhI,KAAK21B,aAI1B31B,KAAK41B,WAAa,GAAInzB,GAAWzC,KAAK20B,MACtC30B,KAAKgC,WAAWgG,KAAKhI,KAAK41B,YAG1B51B,KAAKo3B,UAAY,GAAIp0B,GAAUhD,KAAK20B,MACpC30B,KAAKgC,WAAWgG,KAAKhI,KAAKo3B,WAE1Bp3B,KAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGdtnB,GACFzO,KAAKkT,WAAWzE,GAId0lB,GACFn0B,KAAKg2B,UAAU7B,GAIblyB,EACFjC,KAAKi2B,SAASh0B,GAGdjC,KAAKyhB,SA5GT,GAEI9gB,IAFUT,EAAoB,IACrBA,EAAoB,IACtBA,EAAoB,IAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/B2B,EAAQ3B,EAAoB,IAC5Bg2B,EAAOh2B,EAAoB,IAC3B+C,EAAW/C,EAAoB,IAC/BsC,EAActC,EAAoB,IAClCuC,EAAavC,EAAoB,IACjC8C,EAAY9C,EAAoB,GAuGpCuB,GAAQ0R,UAAY,GAAI+iB,GAMxBz0B,EAAQ0R,UAAU8iB,SAAW,SAASh0B,GACpC,GAGIk0B,GAHAC,EAAiC,MAAlBp2B,KAAK81B,SAwBxB,IAhBEK,EAJGl0B,EAGIA,YAAiBpB,IAAWoB,YAAiBnB,GACvCmB,EAIA,GAAIpB,GAAQoB,GACvB6E,MACE8I,MAAO,OACPC,IAAK,UAVI,KAgBf7P,KAAK81B,UAAYK,EACjBn2B,KAAKo3B,WAAap3B,KAAKo3B,UAAUnB,SAASE,GAEtCC,EACF,GAA0B5vB,QAAtBxG,KAAKyO,QAAQmB,OAA0CpJ,QAApBxG,KAAKyO,QAAQoB,IAAkB,CACpE,GAAID,GAA8BpJ,QAAtBxG,KAAKyO,QAAQmB,MAAqB5P,KAAKyO,QAAQmB,MAAQ,KAC/DC,EAA4BrJ,QAApBxG,KAAKyO,QAAQoB,IAAqB7P,KAAKyO,QAAQoB,IAAM,IAEjE7P,MAAKu2B,UAAU3mB,EAAOC,GAAM2mB,SAAS,QAGrCx2B,MAAKy2B,KAAKD,SAAS,KASzB/0B,EAAQ0R,UAAU6iB,UAAY,SAAS7B,GAErC,GAAIgC,EAKFA,GAJGhC,EAGIA,YAAkBtzB,IAAWszB,YAAkBrzB,GACzCqzB,EAIA,GAAItzB,GAAQszB,GAPZ,KAUfn0B,KAAK+1B,WAAaI,EAClBn2B,KAAKo3B,UAAUpB,UAAUG,IAS3B10B,EAAQ0R,UAAUkkB,UAAY,SAASC,EAAS/kB,EAAOC,GAGrD,MAFehM,UAAX+L,IAAuBA,EAAS,IACrB/L,SAAXgM,IAAuBA,EAAS,IACGhM,SAAnCxG,KAAKo3B,UAAUjD,OAAOmD,GACjBt3B,KAAKo3B,UAAUjD,OAAOmD,GAASD,UAAU9kB,EAAMC,GAG/C,qBAAwB8kB,GASnC71B,EAAQ0R,UAAUokB,eAAiB,SAASD,GAC1C,MAAuC9wB,UAAnCxG,KAAKo3B,UAAUjD,OAAOmD,GAChBt3B,KAAKo3B,UAAUjD,OAAOmD,GAAS5O,UAAkEliB,SAAtDxG,KAAKo3B,UAAU3oB,QAAQ0lB,OAAOqD,WAAWF,IAA+E,GAArDt3B,KAAKo3B,UAAU3oB,QAAQ0lB,OAAOqD,WAAWF,KAGxJ,GAWX71B,EAAQ0R,UAAU4jB,aAAe,WAC/B,GAAIjrB,GAAM,KACNY,EAAM,IAGV,KAAK,GAAI4qB,KAAWt3B,MAAKo3B,UAAUjD,OACjC,GAAIn0B,KAAKo3B,UAAUjD,OAAOruB,eAAewxB,IACO,GAA1Ct3B,KAAKo3B,UAAUjD,OAAOmD,GAAS5O,QACjC,IAAK,GAAIljB,GAAI,EAAGA,EAAIxF,KAAKo3B,UAAUjD,OAAOmD,GAASxB,UAAUnwB,OAAQH,IAAK,CACxE,GAAI6J,GAAOrP,KAAKo3B,UAAUjD,OAAOmD,GAASxB,UAAUtwB,GAChD6B,EAAQ1G,EAAKkG,QAAQwI,EAAK0C,EAAG,QAAQ/K,SACzC8E,GAAa,MAAPA,EAAczE,EAAQyE,EAAMzE,EAAQA,EAAQyE,EAClDY,EAAa,MAAPA,EAAcrF,EAAcA,EAANqF,EAAcrF,EAAQqF,EAM1D,OACEZ,IAAa,MAAPA,EAAe,GAAIxH,MAAKwH,GAAO,KACrCY,IAAa,MAAPA,EAAe,GAAIpI,MAAKoI,GAAO,OAMzC7M,EAAOD,QAAU6B,GAKb,SAAS5B,EAAQD,EAASM,GAK9B,GAAI2D,GAAS3D,EAAoB,GAQjCN,GAAQ63B,qBAAuB,SAAS9C,EAAMI,GAE5C,GADAJ,EAAKI,eACDA,GACgC,GAA9B9uB,MAAMC,QAAQ6uB,GAAsB,CACtC,IAAK,GAAIvvB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,GAA8BgB,SAA1BuuB,EAAYvvB,GAAGkyB,OAAsB,CACvC,GAAIC,KACJA,GAAS/nB,MAAQ/L,EAAOkxB,EAAYvvB,GAAGoK,OAAO1I,SAASF,UACvD2wB,EAAS9nB,IAAMhM,EAAOkxB,EAAYvvB,GAAGqK,KAAK3I,SAASF,UACnD2tB,EAAKI,YAAY/sB,KAAK2vB,GAG1BhD,EAAKI,YAAY7e,KAAK,SAAU3Q,EAAGa,GACjC,MAAOb,GAAEqK,MAAQxJ,EAAEwJ,UAY3BhQ,EAAQg4B,kBAAoB,SAAUjD,EAAMI,GAC1C,GAAIA,GAAuDvuB,SAAxCmuB,EAAKC,SAASiD,gBAAgBtlB,MAAqB,CACpE3S,EAAQ63B,qBAAqB9C,EAAMI,EAQnC,KAAK,GANDnlB,GAAQ/L,EAAO8wB,EAAKc,MAAM7lB,OAC1BC,EAAMhM,EAAO8wB,EAAKc,MAAM5lB,KAExBioB,EAAcnD,EAAKc,MAAM5lB,IAAM8kB,EAAKc,MAAM7lB,MAC1CmoB,EAAYD,EAAanD,EAAKC,SAASiD,gBAAgBtlB,MAElD/M,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,GAA8BgB,SAA1BuuB,EAAYvvB,GAAGkyB,OAAsB,CACvC,GAAIM,GAAYn0B,EAAOkxB,EAAYvvB,GAAGoK,OAClCqoB,EAAUp0B,EAAOkxB,EAAYvvB,GAAGqK,IAEpC,IAAoB,gBAAhBmoB,EAAUE,GACZ,KAAM,IAAIt0B,OAAM,qCAAuCmxB,EAAYvvB,GAAGoK,MAExE,IAAkB,gBAAdqoB,EAAQC,GACV,KAAM,IAAIt0B,OAAM,mCAAqCmxB,EAAYvvB,GAAGqK,IAGtE,IAAIC,GAAWmoB,EAAUD,CACzB,IAAIloB,GAAY,EAAIioB,EAAW,CAE7B,GAAIpO,GAAS,EACTwO,EAAWtoB,EAAIuoB,OACnB,QAAQrD,EAAYvvB,GAAGkyB,QACrB,IAAK,QACCM,EAAUK,OAASJ,EAAQI,QAC7B1O,EAAS,GAEXqO,EAAUM,UAAU1oB,EAAM0oB,aAC1BN,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,QAErB4M,EAAQK,UAAU1oB,EAAM0oB,aACxBL,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAI1B,EAAO,QAE5BwO,EAASllB,IAAI,EAAG,QAChB,MACF,KAAK,SACH,GAAIulB,GAAYP,EAAQ5L,KAAK2L,EAAU,QACnCK,EAAML,EAAUK,KAGpBL,GAAUS,KAAK7oB,EAAM6oB,QACrBT,EAAUU,MAAM9oB,EAAM8oB,SACtBV,EAAUO,KAAK3oB,EAAM2oB,QACrBN,EAAUD,EAAUI,QAGpBJ,EAAUK,IAAIA,GACdJ,EAAQI,IAAIA,GACZJ,EAAQhlB,IAAIulB,EAAU,QAEtBR,EAAU3M,SAAS,EAAE,SACrB4M,EAAQ5M,SAAS,EAAE,SAEnB8M,EAASllB,IAAI,EAAG,QAChB,MACF,KAAK,UACC+kB,EAAUU,SAAWT,EAAQS,UAC/B/O,EAAS,GAEXqO,EAAUU,MAAM9oB,EAAM8oB,SACtBV,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,UAErB4M,EAAQS,MAAM9oB,EAAM8oB,SACpBT,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAE,UACnB4M,EAAQhlB,IAAI0W,EAAO,UAEnBwO,EAASllB,IAAI,EAAG,SAChB,MACF,KAAK,SACC+kB,EAAUO,QAAUN,EAAQM,SAC9B5O,EAAS,GAEXqO,EAAUO,KAAK3oB,EAAM2oB,QACrBP,EAAU3M,SAAS,EAAE,SACrB4M,EAAQM,KAAK3oB,EAAM2oB,QACnBN,EAAQ5M,SAAS,EAAE,SACnB4M,EAAQhlB,IAAI0W,EAAO,SAEnBwO,EAASllB,IAAI,EAAG,QAChB,MACF,SAEE,WADA0lB,SAAQhF,IAAI,2EAA4EoB,EAAYvvB,GAAGkyB,QAG3G,KAAmBS,EAAZH,GAEL,OADArD,EAAKI,YAAY/sB,MAAM4H,MAAOooB,EAAUhxB,UAAW6I,IAAKooB,EAAQjxB,YACxD+tB,EAAYvvB,GAAGkyB,QACrB,IAAK,QACHM,EAAU/kB,IAAI,EAAG,QACjBglB,EAAQhlB,IAAI,EAAG,OACf,MACF,KAAK,SACH+kB,EAAU/kB,IAAI,EAAG,SACjBglB,EAAQhlB,IAAI,EAAG,QACf,MACF,KAAK,UACH+kB,EAAU/kB,IAAI,EAAG,UACjBglB,EAAQhlB,IAAI,EAAG,SACf,MACF,KAAK,SACH+kB,EAAU/kB,IAAI,EAAG,KACjBglB,EAAQhlB,IAAI,EAAG,IACf,MACF,SAEE,WADA0lB,SAAQhF,IAAI,2EAA4EoB,EAAYvvB,GAAGkyB,QAI7G/C,EAAKI,YAAY/sB,MAAM4H,MAAOooB,EAAUhxB,UAAW6I,IAAKooB,EAAQjxB,aAKtEpH,EAAQg5B,iBAAiBjE,EAEzB,IAAIkE,GAAcj5B,EAAQk5B,SAASnE,EAAKc,MAAM7lB,MAAO+kB,EAAKI,aACtDgE,EAAYn5B,EAAQk5B,SAASnE,EAAKc,MAAM5lB,IAAI8kB,EAAKI,aACjDiE,EAAarE,EAAKc,MAAM7lB,MACxBqpB,EAAWtE,EAAKc,MAAM5lB,GACA,IAAtBgpB,EAAYK,SAAiBF,EAAwC,GAA3BrE,EAAKc,MAAM0D,aAAuBN,EAAYb,UAAY,EAAIa,EAAYZ,QAAU,GAC1G,GAApBc,EAAUG,SAAmBD,EAAsC,GAAzBtE,EAAKc,MAAM2D,WAAuBL,EAAUf,UAAY,EAAMe,EAAUd,QAAU,IACtG,GAAtBY,EAAYK,QAAsC,GAApBH,EAAUG,SAC1CvE,EAAKc,MAAM4D,YAAYL,EAAYC,KAYzCr5B,EAAQg5B,iBAAmB,SAASjE,GAGlC,IAAK,GAFDI,GAAcJ,EAAKI,YACnBuE,KACK9zB,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IACtC,IAAK,GAAIomB,GAAI,EAAGA,EAAImJ,EAAYpvB,OAAQimB,IAClCpmB,GAAKomB,GAA8B,GAAzBmJ,EAAYnJ,GAAGvV,QAA2C,GAAzB0e,EAAYvvB,GAAG6Q,SAExD0e,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGqK,IACvFklB,EAAYnJ,GAAGvV,QAAS,EAGjB0e,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAGhc,OAASmlB,EAAYvvB,GAAGqK,KAC9FklB,EAAYvvB,GAAGqK,IAAMklB,EAAYnJ,GAAG/b,IACpCklB,EAAYnJ,GAAGvV,QAAS,GAGjB0e,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGoK,OAASmlB,EAAYnJ,GAAG/b,KAAOklB,EAAYvvB,GAAGqK,MAC1FklB,EAAYvvB,GAAGoK,MAAQmlB,EAAYnJ,GAAGhc,MACtCmlB,EAAYnJ,GAAGvV,QAAS,GAMhC,KAAK,GAAI7Q,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAClCuvB,EAAYvvB,GAAG6Q,UAAW,GAC5BijB,EAAUtxB,KAAK+sB,EAAYvvB,GAI/BmvB,GAAKI,YAAcuE,EACnB3E,EAAKI,YAAY7e,KAAK,SAAU3Q,EAAGa,GACjC,MAAOb,GAAEqK,MAAQxJ,EAAEwJ,SAIvBhQ,EAAQ25B,WAAa,SAASC,GAC5B,IAAK,GAAIh0B,GAAG,EAAGA,EAAIg0B,EAAM7zB,OAAQH,IAC/BmzB,QAAQhF,IAAInuB,EAAG,GAAIlB,MAAKk1B,EAAMh0B,GAAGoK,OAAO,GAAItL,MAAKk1B,EAAMh0B,GAAGqK,KAAM2pB,EAAMh0B,GAAGoK,MAAO4pB,EAAMh0B,GAAGqK,IAAK2pB,EAAMh0B,GAAG6Q,SAS3GzW,EAAQ65B,oBAAsB,SAASC,EAAUC,GAG/C,IAAK,GAFDC,IAAe,EACfC,EAAeH,EAASI,QAAQ9yB,UAC3BxB,EAAI,EAAGA,EAAIk0B,EAAS3E,YAAYpvB,OAAQH,IAAK,CACpD,GAAIwyB,GAAY0B,EAAS3E,YAAYvvB,GAAGoK,MACpCqoB,EAAUyB,EAAS3E,YAAYvvB,GAAGqK,GACtC,IAAIgqB,GAAgB7B,GAA4BC,EAAf4B,EAAwB,CACvDD,GAAe,CACf,QAIJ,GAAoB,GAAhBA,GAAwBC,EAAeH,EAASvG,KAAKnsB,WAAa6yB,GAAgBF,EAAc,CAClG,GAAIlqB,GAAY5L,EAAO81B,GACnBI,EAAWl2B,EAAOo0B,EAElBxoB,GAAU8oB,QAAUwB,EAASxB,OAASmB,EAASM,cAAe,EACzDvqB,EAAUipB,SAAWqB,EAASrB,QAAUgB,EAASO,eAAgB,EACjExqB,EAAU6oB,aAAeyB,EAASzB,cAAcoB,EAASQ,aAAc,GAEhFR,EAASI,QAAUC,EAAS7yB,WAmChCtH,EAAQq1B,SAAW,SAASiB,EAAMiE,EAAM5nB,GACtC,GAAoC,GAAhC2jB,EAAKvB,KAAKI,YAAYpvB,OAAa,CACrC,GAAIy0B,GAAalE,EAAKT,MAAM2E,WAAW7nB,EACvC,QAAQ4nB,EAAKnzB,UAAYozB,EAAWzQ,QAAUyQ,EAAWnd,MAGzD,GAAIic,GAASt5B,EAAQk5B,SAASqB,EAAMjE,EAAKvB,KAAKI,YACzB,IAAjBmE,EAAOA,SACTiB,EAAOjB,EAAOlB,UAGhB,IAAIloB,GAAWlQ,EAAQy6B,yBAAyBnE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAM7lB,MAAOsmB,EAAKT,MAAM5lB,IACpGsqB,GAAOv6B,EAAQ06B,qBAAqBpE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAO0E,EAEvE,IAAIC,GAAalE,EAAKT,MAAM2E,WAAW7nB,EAAOzC,EAC9C,QAAQqqB,EAAKnzB,UAAYozB,EAAWzQ,QAAUyQ,EAAWnd,OAa7Drd,EAAQy1B,OAAS,SAASa,EAAMnkB,EAAGQ,GACjC,GAAoC,GAAhC2jB,EAAKvB,KAAKI,YAAYpvB,OAAa,CACrC,GAAIy0B,GAAalE,EAAKT,MAAM2E,WAAW7nB,EACvC,OAAO,IAAIjO,MAAKyN,EAAIqoB,EAAWnd,MAAQmd,EAAWzQ,QAGlD,GAAI4Q,GAAiB36B,EAAQy6B,yBAAyBnE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAM7lB,MAAOsmB,EAAKT,MAAM5lB,KACtG2qB,EAAgBtE,EAAKT,MAAM5lB,IAAMqmB,EAAKT,MAAM7lB,MAAQ2qB,EACpDE,EAAkBD,EAAgBzoB,EAAIQ,EACtCmoB,EAA4B96B,EAAQ+6B,6BAA6BzE,EAAKvB,KAAKI,YAAamB,EAAKT,MAAOgF,GAEpGG,EAAU,GAAIt2B,MAAKo2B,EAA4BD,EAAkBvE,EAAKT,MAAM7lB,MAChF,OAAOgrB,IAYXh7B,EAAQy6B,yBAA2B,SAAStF,EAAanlB,EAAOC,GAE9D,IAAK,GADDC,GAAW,EACNtK,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAEzBmoB,IAAapoB,GAAmBC,EAAVooB,IACxBnoB,GAAYmoB,EAAUD,GAG1B,MAAOloB,IAWTlQ,EAAQ06B,qBAAuB,SAASvF,EAAaU,EAAO0E,GAG1D,MAFAA,GAAOt2B,EAAOs2B,GAAMjzB,SAASF,UAC7BmzB,GAAQv6B,EAAQi7B,wBAAwB9F,EAAYU,EAAM0E,IAI5Dv6B,EAAQi7B,wBAA0B,SAAS9F,EAAaU,EAAO0E,GAC7D,GAAIW,GAAa,CACjBX,GAAOt2B,EAAOs2B,GAAMjzB,SAASF,SAE7B,KAAK,GAAIxB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAEzBmoB,IAAavC,EAAM7lB,OAASqoB,EAAUxC,EAAM5lB,KAC1CsqB,GAAQlC,IACV6C,GAAe7C,EAAUD,GAI/B,MAAO8C,IAWTl7B,EAAQ+6B,6BAA+B,SAAS5F,EAAaU,EAAOsF,GAKlE,IAAK,GAJDR,GAAiB,EACjBzqB,EAAW,EACXkrB,EAAgBvF,EAAM7lB,MAEjBpK,EAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAE7B,IAAImoB,GAAavC,EAAM7lB,OAASqoB,EAAUxC,EAAM5lB,IAAK,CAGnD,GAFAC,GAAYkoB,EAAYgD,EACxBA,EAAgB/C,EACZnoB,GAAYirB,EACd,KAGAR,IAAkBtC,EAAUD,GAKlC,MAAOuC,IAaT36B,EAAQq7B,mBAAqB,SAASlG,EAAaoF,EAAMe,EAAWC,GAClE,GAAIrC,GAAWl5B,EAAQk5B,SAASqB,EAAMpF,EACtC,OAAuB,IAAnB+D,EAASI,OACK,EAAZgC,EACuB,GAArBC,EACKrC,EAASd,WAAac,EAASb,QAAUkC,GAAQ,EAGjDrB,EAASd,UAAY,EAIL,GAArBmD,EACKrC,EAASb,SAAWkC,EAAOrB,EAASd,WAAa,EAGjDc,EAASb,QAAU,EAKvBkC,GAaXv6B,EAAQk5B,SAAW,SAASqB,EAAMpF,GAChC,IAAK,GAAIvvB,GAAI,EAAGA,EAAIuvB,EAAYpvB,OAAQH,IAAK,CAC3C,GAAIwyB,GAAYjD,EAAYvvB,GAAGoK,MAC3BqoB,EAAUlD,EAAYvvB,GAAGqK,GAE7B,IAAIsqB,GAAQnC,GAAoBC,EAAPkC,EACvB,OAAQjB,QAAQ,EAAMlB,UAAWA,EAAWC,QAASA,GAIzD,OAAQiB,QAAQ,EAAOlB,UAAWA,EAAWC,QAASA,KAKpD,SAASp4B,GA4Bb,QAAS+B,GAASgO,EAAOC,EAAKurB,EAAaC,EAAiBC,EAAaC,GAEvEv7B,KAAK85B,QAAU,EAEf95B,KAAKw7B,WAAY,EACjBx7B,KAAKy7B,UAAY,EACjBz7B,KAAKmoB,KAAO,EACZnoB,KAAKid,MAAQ,EAEbjd,KAAK07B,YACL17B,KAAK27B,UACL37B,KAAK47B,UAAY,EAEjB57B,KAAK67B,YAAc,EAAO,EAAM,EAAI,IACpC77B,KAAK87B,YAAc,IAAO,GAAM,EAAI,GAEpC97B,KAAKu7B,WAAaA,EAElBv7B,KAAKuzB,SAAS3jB,EAAOC,EAAKurB,EAAaC,EAAiBC,GAe1D15B,EAASuR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKurB,EAAaC,EAAiBC,GAC/Et7B,KAAKkzB,OAA6B1sB,SAApB80B,EAAYxvB,IAAoB8D,EAAQ0rB,EAAYxvB,IAClE9L,KAAKmzB,KAA2B3sB,SAApB80B,EAAY5uB,IAAoBmD,EAAMyrB,EAAY5uB,IAE1D1M,KAAKkzB,QAAUlzB,KAAKmzB,OACtBnzB,KAAKkzB,QAAU,IACflzB,KAAKmzB,MAAQ,GAGO,GAAlBnzB,KAAKw7B,WACPx7B,KAAK+7B,eAAeX,EAAaC,GAGnCr7B,KAAKg8B,SAASV,IAOhB15B,EAASuR,UAAU4oB,eAAiB,SAASX,EAAaC,GAExD,GAAIhpB,GAAOrS,KAAKmzB,KAAOnzB,KAAKkzB,OACxB+I,EAAkB,IAAP5pB,EACX6pB,EAAmBd,GAAea,EAAWZ,GAC7Cc,EAAmBj3B,KAAKwoB,MAAMxoB,KAAKyuB,IAAIsI,GAAU/2B,KAAK0uB,MAEtDwI,EAAe,GACfC,EAAkBn3B,KAAK4uB,IAAI,GAAGqI,GAE9BvsB,EAAQ,CACW,GAAnBusB,IACFvsB,EAAQusB,EAIV,KAAK,GADDG,IAAgB,EACX92B,EAAIoK,EAAO1K,KAAK2lB,IAAIrlB,IAAMN,KAAK2lB,IAAIsR,GAAmB32B,IAAK,CAClE62B,EAAkBn3B,KAAK4uB,IAAI,GAAGtuB,EAC9B,KAAK,GAAIomB,GAAI,EAAGA,EAAI5rB,KAAK87B,WAAWn2B,OAAQimB,IAAK,CAC/C,GAAI2Q,GAAWF,EAAkBr8B,KAAK87B,WAAWlQ,EACjD,IAAI2Q,GAAYL,EAAkB,CAChCI,GAAgB,EAChBF,EAAexQ,CACf,QAGJ,GAAqB,GAAjB0Q,EACF,MAGJt8B,KAAKy7B,UAAYW,EACjBp8B,KAAKid,MAAQof,EACbr8B,KAAKmoB,KAAOkU,EAAkBr8B,KAAK87B,WAAWM,IAShDx6B,EAASuR,UAAU6oB,SAAW,SAASV,GACjB90B,SAAhB80B,IACFA,KAGF,IAAIkB,GAAgCh2B,SAApB80B,EAAYxvB,IAAoB9L,KAAKkzB,OAAuB,EAAblzB,KAAKid,MAAYjd,KAAK87B,WAAW97B,KAAKy7B,WAAcH,EAAYxvB,IAC3H2wB,EAA8Bj2B,SAApB80B,EAAY5uB,IAAoB1M,KAAKmzB,KAAQnzB,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAAcH,EAAY5uB,GAEvH1M,MAAK27B,UAAgCn1B,SAApB80B,EAAY5uB,IAAoB1M,KAAK08B,aAAaD,GAAWnB,EAAY5uB,IAC1F1M,KAAK07B,YAAkCl1B,SAApB80B,EAAYxvB,IAAoB9L,KAAK08B,aAAaF,GAAalB,EAAYxvB,IAGvE,GAAnB9L,KAAKu7B,aAAuBv7B,KAAK27B,UAAY37B,KAAK07B,aAAe17B,KAAKmoB,MAAQ,IAChFnoB,KAAK27B,WAAa37B,KAAK27B,UAAY37B,KAAKmoB,MAG1CnoB,KAAK47B,UAAY57B,KAAK08B,aAAaD,GAAWA,EAAUz8B,KAAK08B,aAAaF,GAAaA,EACvFx8B,KAAK28B,YAAc38B,KAAK27B,UAAY37B,KAAK07B,YAGzC17B,KAAK85B,QAAU95B,KAAK27B,WAGtB/5B,EAASuR,UAAUupB,aAAe,SAASr1B,GACzC,GAAIu1B,GAAUv1B,EAASA,GAASrH,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAClE,OAAIp0B,IAASrH,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,YAAc,GAAOz7B,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAC7FmB,EAAW58B,KAAKid,MAAQjd,KAAK87B,WAAW97B,KAAKy7B,WAG7CmB,GASXh7B,EAASuR,UAAU0pB,QAAU,WAC3B,MAAQ78B,MAAK85B,SAAW95B,KAAK07B,aAM/B95B,EAASuR,UAAUkV,KAAO,WACxB,GAAIuJ,GAAO5xB,KAAK85B,OAChB95B,MAAK85B,SAAW95B,KAAKmoB,KAGjBnoB,KAAK85B,SAAWlI,IAClB5xB,KAAK85B,QAAU95B,KAAKmzB,OAOxBvxB,EAASuR,UAAU2pB,SAAW,WAC5B98B,KAAK85B,SAAW95B,KAAKmoB,KACrBnoB,KAAK27B,WAAa37B,KAAKmoB,KACvBnoB,KAAK28B,YAAc38B,KAAK27B,UAAY37B,KAAK07B,aAS3C95B,EAASuR,UAAUiV,WAAa,SAAS2U,GAEvC,GAAIjD,GAAW50B,KAAK2lB,IAAI7qB,KAAK85B,SAAW95B,KAAKmoB,KAAO,EAAK,EAAInoB,KAAK85B,QAC9D7F,EAAc,GAAK/vB,OAAO41B,GAAS7F,YAAY,EAGnD,IAAgBztB,SAAbu2B,GAA2Br4B,MAAMR,OAAO64B,KAqCzC,GAAgC,IAA5B9I,EAAYttB,QAAQ,MAA0C,IAA5BstB,EAAYttB,QAAQ,KAExD,IAAK,GAAInB,GAAIyuB,EAAYtuB,OAAS,EAAGH,EAAI,EAAGA,IAAK,CAC/C,GAAsB,KAAlByuB,EAAYzuB,GAGX,CAAA,GAAsB,KAAlByuB,EAAYzuB,IAA+B,KAAlByuB,EAAYzuB,GAAW,CACvDyuB,EAAcA,EAAYhpB,MAAM,EAAGzF,EACnC,OAGA,MAPAyuB,EAAcA,EAAYhpB,MAAM,EAAGzF,QAzCY,CAErD,GAAIw3B,GAAM,GACN70B,EAAQ8rB,EAAYttB,QAAQ,IAoBhC,IAnBY,IAATwB,IAED60B,EAAM/I,EAAYhpB,MAAM9C,GAExB8rB,EAAcA,EAAYhpB,MAAM,EAAG9C,IAErCA,EAAQjD,KAAKwH,IAAIunB,EAAYttB,QAAQ,KAAMstB,EAAYttB,QAAQ,MAClD,KAAVwB,GAEe,IAAb40B,IACD9I,GAAe,KAGjB9rB,EAAQ8rB,EAAYtuB,OAASo3B,GAEV,IAAbA,IAEN50B,GAAS40B,EAAW,GAEnB50B,EAAQ8rB,EAAYtuB,OAErB,IAAI,GAAIs3B,GAAM90B,EAAQ8rB,EAAYtuB,OAAQs3B,EAAM,EAAGA,IACjDhJ,GAAe,QAKjBA,GAAcA,EAAYhpB,MAAM,EAAG9C,EAGrC8rB,IAAe+I,EAoBjB,MAAO/I,IAWTryB,EAASuR,UAAU6hB,KAAO,aAS1BpzB,EAASuR,UAAU+pB,QAAU,WAC3B,MAAQl9B,MAAK85B,SAAW95B,KAAKid,MAAQjd,KAAK67B,WAAW77B,KAAKy7B,aAAe,GAG3E57B,EAAOD,QAAUgC,GAKb,SAAS/B,EAAQD,EAASM,GAgB9B,QAAS2B,GAAM8yB,EAAMlmB,GACnB,GAAI0uB,GAAMt5B,IAASu5B,MAAM,GAAGC,QAAQ,GAAGC,QAAQ,GAAGC,aAAa,EAC/Dv9B,MAAK4P,MAAQutB,EAAI/E,QAAQnlB,IAAI,GAAI,QAAQjM,UACzChH,KAAK6P,IAAMstB,EAAI/E,QAAQnlB,IAAI,EAAG,QAAQjM,UAEtChH,KAAK20B,KAAOA,EACZ30B,KAAKw9B,gBAAkB,EACvBx9B,KAAKy9B,YAAc,EACnBz9B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,EAGlBp5B,KAAKq0B,gBACHzkB,MAAO,KACPC,IAAK,KACLqrB,UAAW,aACXwC,UAAU,EACVC,UAAU,EACV7xB,IAAK,KACLY,IAAK,KACLkxB,QAAS,GACTC,QAAS,UAEX79B,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAKgG,OACH83B,UAEF99B,KAAK+9B,aAAe,KAGpB/9B,KAAK20B,KAAKE,QAAQthB,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OACxDA,KAAK20B,KAAKE,QAAQthB,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OACnDA,KAAK20B,KAAKE,QAAQthB,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAGtDA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKm+B,QAAQrJ,KAAK90B,OAGhDA,KAAK20B,KAAKE,QAAQthB,GAAG,aAAcvT,KAAKo+B,cAActJ,KAAK90B,OAG3DA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKq+B,SAASvJ,KAAK90B,OACjDA,KAAK20B,KAAKE,QAAQthB,GAAG,QAASvT,KAAKs+B,SAASxJ,KAAK90B,OAEjDA,KAAKkT,WAAWzE,GAsClB,QAAS8vB,GAAmBrD,GAC1B,GAAiB,cAAbA,GAA0C,YAAbA,EAC/B,KAAM,IAAI70B,WAAU,sBAAwB60B,EAAY,yCAof5D,QAASsD,GAAYV,EAAOl1B,GAC1B,OACEmJ,EAAG+rB,EAAM/rB,EAAIpR,EAAK2G,gBAAgBsB,GAClCoJ,EAAG8rB,EAAM9rB,EAAIrR,EAAK+G,eAAekB,IA1lBrC,GAAIjI,GAAOT,EAAoB,GAE3B2D,GADa3D,EAAoB,IACxBA,EAAoB,KAC7BqC,EAAYrC,EAAoB,IAChCyB,EAAWzB,EAAoB,GA0DnC2B,GAAMsR,UAAY,GAAI5Q,GAkBtBV,EAAMsR,UAAUD,WAAa,SAAUzE,GACrC,GAAIA,EAAS,CAEX,GAAIP,IAAU,YAAa,MAAO,MAAO,UAAW,UAAW,WAAY,WAAY,WAAY,cACnGvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,IAEvC,SAAWA,IAAW,OAASA,KAEjCzO,KAAKuzB,SAAS9kB,EAAQmB,MAAOnB,EAAQoB,OA4B3ChO,EAAMsR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAK2mB,EAASiI,GACnDA,KAAW,IACbA,GAAS,EAEX,IAAIvL,GAAkB1sB,QAAToJ,EAAqBjP,EAAKkG,QAAQ+I,EAAO,QAAQ5I,UAAY,KACtEmsB,EAAgB3sB,QAAPqJ,EAAqBlP,EAAKkG,QAAQgJ,EAAK,QAAQ7I,UAAc,IAG1E,IAFAhH,KAAK0+B,mBAEDlI,EAAS,CACX,GAAIriB,GAAKnU,KACL2+B,EAAY3+B,KAAK4P,MACjBgvB,EAAU5+B,KAAK6P,IACfC,EAA8B,gBAAZ0mB,GAAuBA,EAAU,IACnDqI,GAAW,GAAIv6B,OAAO0C,UACtB83B,GAAa,EAEbzW,EAAO,WACT,IAAKlU,EAAGnO,MAAM83B,MAAMiB,SAAU,CAC5B,GAAI5B,IAAM,GAAI74B,OAAO0C,UACjBmzB,EAAOgD,EAAM0B,EACbG,EAAO7E,EAAOrqB,EACdlE,EAAKozB,GAAmB,OAAX9L,EAAmBA,EAASvyB,EAAKgP,cAAcwqB,EAAMwE,EAAWzL,EAAQpjB,GACrFgnB,EAAKkI,GAAiB,OAAT7L,EAAmBA,EAASxyB,EAAKgP,cAAcwqB,EAAMyE,EAASzL,EAAMrjB,EAErFmvB,GAAU9qB,EAAGklB,YAAYztB,EAAGkrB,GAC5Bn1B,EAASi2B,kBAAkBzjB,EAAGwgB,KAAMxgB,EAAG1F,QAAQsmB,aAC/C+J,EAAaA,GAAcG,EACvBA,GACF9qB,EAAGwgB,KAAKE,QAAQjH,KAAK,eAAgBhe,MAAO,GAAItL,MAAK6P,EAAGvE,OAAQC,IAAK,GAAIvL,MAAK6P,EAAGtE,KAAM4uB,OAAOA,IAG5FO,EACEF,GACF3qB,EAAGwgB,KAAKE,QAAQjH,KAAK,gBAAiBhe,MAAO,GAAItL,MAAK6P,EAAGvE,OAAQC,IAAK,GAAIvL,MAAK6P,EAAGtE,KAAM4uB,OAAOA,IAMjGtqB,EAAG4pB,aAAezkB,WAAW+O,EAAM,KAKzC,OAAOA,KAGP,GAAI4W,GAAUj/B,KAAKq5B,YAAYnG,EAAQC,EAEvC,IADAxxB,EAASi2B,kBAAkB53B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAC/CkK,EAAS,CACX,GAAInrB,IAAUlE,MAAO,GAAItL,MAAKtE,KAAK4P,OAAQC,IAAK,GAAIvL,MAAKtE,KAAK6P,KAAM4uB,OAAOA,EAC3Ez+B,MAAK20B,KAAKE,QAAQjH,KAAK,cAAe9Z,GACtC9T,KAAK20B,KAAKE,QAAQjH,KAAK,eAAgB9Z,KAS7CjS,EAAMsR,UAAUurB,iBAAmB,WAC7B1+B,KAAK+9B,eACP1kB,aAAarZ,KAAK+9B,cAClB/9B,KAAK+9B,aAAe,OAaxBl8B,EAAMsR,UAAUkmB,YAAc,SAASzpB,EAAOC,GAC5C,GAIIwc,GAJA6S,EAAqB,MAATtvB,EAAiBjP,EAAKkG,QAAQ+I,EAAO,QAAQ5I,UAAYhH,KAAK4P,MAC1EuvB,EAAmB,MAAPtvB,EAAiBlP,EAAKkG,QAAQgJ,EAAK,QAAQ7I,UAAchH,KAAK6P,IAC1EnD,EAA2B,MAApB1M,KAAKyO,QAAQ/B,IAAe/L,EAAKkG,QAAQ7G,KAAKyO,QAAQ/B,IAAK,QAAQ1F,UAAY,KACtF8E,EAA2B,MAApB9L,KAAKyO,QAAQ3C,IAAenL,EAAKkG,QAAQ7G,KAAKyO,QAAQ3C,IAAK,QAAQ9E,UAAY,IAI1F,IAAItC,MAAMw6B,IAA0B,OAAbA,EACrB,KAAM,IAAIt7B,OAAM,kBAAoBgM,EAAQ,IAE9C,IAAIlL,MAAMy6B,IAAsB,OAAXA,EACnB,KAAM,IAAIv7B,OAAM,gBAAkBiM,EAAM,IAyC1C,IArCaqvB,EAATC,IACFA,EAASD,GAIC,OAARpzB,GACaA,EAAXozB,IACF7S,EAAQvgB,EAAMozB,EACdA,GAAY7S,EACZ8S,GAAU9S,EAGC,MAAP3f,GACEyyB,EAASzyB,IACXyyB,EAASzyB,IAOL,OAARA,GACEyyB,EAASzyB,IACX2f,EAAQ8S,EAASzyB,EACjBwyB,GAAY7S,EACZ8S,GAAU9S,EAGC,MAAPvgB,GACaA,EAAXozB,IACFA,EAAWpzB,IAOU,OAAzB9L,KAAKyO,QAAQmvB,QAAkB,CACjC,GAAIA,GAAUvY,WAAWrlB,KAAKyO,QAAQmvB,QACxB,GAAVA,IACFA,EAAU,GAEcA,EAArBuB,EAASD,IACPl/B,KAAK6P,IAAM7P,KAAK4P,QAAWguB,GAE9BsB,EAAWl/B,KAAK4P,MAChBuvB,EAASn/B,KAAK6P,MAIdwc,EAAQuR,GAAWuB,EAASD,GAC5BA,GAAY7S,EAAO,EACnB8S,GAAU9S,EAAO,IAMvB,GAA6B,OAAzBrsB,KAAKyO,QAAQovB,QAAkB,CACjC,GAAIA,GAAUxY,WAAWrlB,KAAKyO,QAAQovB,QACxB,GAAVA,IACFA,EAAU,GAEPsB,EAASD,EAAYrB,IACnB79B,KAAK6P,IAAM7P,KAAK4P,QAAWiuB,GAE9BqB,EAAWl/B,KAAK4P,MAChBuvB,EAASn/B,KAAK6P,MAIdwc,EAAS8S,EAASD,EAAYrB,EAC9BqB,GAAY7S,EAAO,EACnB8S,GAAU9S,EAAO,IAKvB,GAAI4S,GAAWj/B,KAAK4P,OAASsvB,GAAYl/B,KAAK6P,KAAOsvB,CAUrD,OAPOD,IAAYl/B,KAAK4P,OAASsvB,GAAcl/B,KAAK6P,KAASsvB,GAAYn/B,KAAK4P,OAASuvB,GAAYn/B,KAAK6P,KACjG7P,KAAK4P,OAASsvB,GAAYl/B,KAAK4P,OAASuvB,GAAcn/B,KAAK6P,KAAOqvB,GAAcl/B,KAAK6P,KAAOsvB,GACjGn/B,KAAK20B,KAAKE,QAAQjH,KAAK,oBAGzB5tB,KAAK4P,MAAQsvB,EACbl/B,KAAK6P,IAAMsvB,EACJF,GAOTp9B,EAAMsR,UAAUisB,SAAW,WACzB,OACExvB,MAAO5P,KAAK4P,MACZC,IAAK7P,KAAK6P,MAUdhO,EAAMsR,UAAUinB,WAAa,SAAU7nB,EAAO8sB,GAC5C,MAAOx9B,GAAMu4B,WAAWp6B,KAAK4P,MAAO5P,KAAK6P,IAAK0C,EAAO8sB,IAWvDx9B,EAAMu4B,WAAa,SAAUxqB,EAAOC,EAAK0C,EAAO8sB,GAI9C,MAHoB74B,UAAhB64B,IACFA,EAAc,GAEH,GAAT9sB,GAAe1C,EAAMD,GAAS,GAE9B+Z,OAAQ/Z,EACRqN,MAAO1K,GAAS1C,EAAMD,EAAQyvB,KAK9B1V,OAAQ,EACR1M,MAAO,IAUbpb,EAAMsR,UAAU6qB,aAAe,SAAS10B,GACtCtJ,KAAKw9B,gBAAkB,EACvBx9B,KAAKs/B,cAAgB,EAEhBt/B,KAAKyO,QAAQivB,UAIb19B,KAAKgG,MAAM83B,MAAMyB,gBAEtBv/B,KAAKgG,MAAM83B,MAAMluB,MAAQ5P,KAAK4P,MAC9B5P,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAK6P,IAC5B7P,KAAKgG,MAAM83B,MAAMiB,UAAW,EAExB/+B,KAAK20B,KAAK5E,IAAIrwB,OAChBM,KAAK20B,KAAK5E,IAAIrwB,KAAKuN,MAAM+f,OAAS,QAGpC1jB,EAAMD,mBAQRxH,EAAMsR,UAAU8qB,QAAU,SAAU30B,GAElC,GAAKtJ,KAAKyO,QAAQivB,UAKb19B,KAAKgG,MAAM83B,MAAMyB,cAAtB,CAEA,GAAIrE,GAAYl7B,KAAKyO,QAAQysB,SAC7BqD,GAAkBrD,EAClB,IAAIzM,GAAsB,cAAbyM,EAA6B5xB,EAAMk2B,OAASl2B,EAAMm2B,MAC/DhR,IAASzuB,KAAKw9B,eACd,IAAIhL,GAAYxyB,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAKgG,MAAM83B,MAAMluB,MAGpDE,EAAWnO,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,IACzF2iB,IAAY1iB,CAEZ,IAAIyC,GAAsB,cAAb2oB,EAA6Bl7B,KAAK20B,KAAKC,SAAS1I,OAAO3Z,MAAQvS,KAAK20B,KAAKC,SAAS1I,OAAO1Z,OAClGktB,GAAajR,EAAQlc,EAAQigB,EAC7B0M,EAAWl/B,KAAKgG,MAAM83B,MAAMluB,MAAQ8vB,EACpCP,EAASn/B,KAAKgG,MAAM83B,MAAMjuB,IAAM6vB,EAGhCC,EAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAUl/B,KAAKs/B,cAAc7Q,GAAO,GACnGmR,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,EAAQn/B,KAAKs/B,cAAc7Q,GAAO,EACnG,IAAIkR,GAAaT,GAAYU,GAAWT,EAKtC,MAJAn/B,MAAKw9B,iBAAmB/O,EACxBzuB,KAAKgG,MAAM83B,MAAMluB,MAAQ+vB,EACzB3/B,KAAKgG,MAAM83B,MAAMjuB,IAAM+vB,MACvB5/B,MAAKi+B,QAAQ30B,EAIftJ,MAAKs/B,cAAgB7Q,EACrBzuB,KAAKq5B,YAAY6F,EAAUC,GAG3Bn/B,KAAK20B,KAAKE,QAAQjH,KAAK,eACrBhe,MAAO,GAAItL,MAAKtE,KAAK4P,OACrBC,IAAO,GAAIvL,MAAKtE,KAAK6P,KACrB4uB,QAAQ,IAGVn1B,EAAMD,mBAQRxH,EAAMsR,UAAU+qB,WAAa,WAEtBl+B,KAAKyO,QAAQivB,UAKb19B,KAAKgG,MAAM83B,MAAMyB,gBAEtBv/B,KAAKgG,MAAM83B,MAAMiB,UAAW,EACxB/+B,KAAK20B,KAAK5E,IAAIrwB,OAChBM,KAAK20B,KAAK5E,IAAIrwB,KAAKuN,MAAM+f,OAAS,QAIpChtB,KAAK20B,KAAKE,QAAQjH,KAAK,gBACrBhe,MAAO,GAAItL,MAAKtE,KAAK4P,OACrBC,IAAO,GAAIvL,MAAKtE,KAAK6P,KACrB4uB,QAAQ,MAUZ58B,EAAMsR,UAAUirB,cAAgB,SAAS90B,GAEvC,GAAMtJ,KAAKyO,QAAQkvB,UAAY39B,KAAKyO,QAAQivB,SAA5C,CAGA,GAAIjP,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAa,IAClBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAS,GAMtBF,EAAO,CAKT,GAAIxR,EAEFA,GADU,EAARwR,EACM,EAAKA,EAAQ,EAGb,GAAK,EAAKA,EAAQ,EAI5B,IAAIoR,GAAUrB,GAAYzsB,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAAQ//B,KAAK20B,KAAK5E,IAAI7D,QACrE8T,EAAchgC,KAAKigC,eAAeJ,EAEtC7/B,MAAKkgC,KAAKjjB,EAAO+iB,EAAavR,GAKhCnlB,EAAMD,mBAORxH,EAAMsR,UAAUkrB,SAAW,WACzBr+B,KAAKgG,MAAM83B,MAAMluB,MAAQ5P,KAAK4P,MAC9B5P,KAAKgG,MAAM83B,MAAMjuB,IAAM7P,KAAK6P,IAC5B7P,KAAKgG,MAAM83B,MAAMyB,eAAgB,EACjCv/B,KAAKgG,MAAM83B,MAAM5R,OAAS,KAC1BlsB,KAAKy9B,YAAc,EACnBz9B,KAAKw9B,gBAAkB,GAOzB37B,EAAMsR,UAAUgrB,QAAU,WACxBn+B,KAAKgG,MAAM83B,MAAMyB,eAAgB,GAQnC19B,EAAMsR,UAAUmrB,SAAW,SAAUh1B,GAEnC,GAAMtJ,KAAKyO,QAAQkvB,UAAY39B,KAAKyO,QAAQivB,SAA5C,CAEA19B,KAAKgG,MAAM83B,MAAMyB,eAAgB,EAE5Bv/B,KAAKgG,MAAM83B,MAAM5R,SACpBlsB,KAAKgG,MAAM83B,MAAM5R,OAASsS,EAAWl1B,EAAM4iB,OAAQlsB,KAAK20B,KAAK5E,IAAI7D,QAGnE,IAAIjP,GAAQ,GAAK3T,EAAM2T,MAAQjd,KAAKy9B,aAChC0C,EAAangC,KAAKigC,eAAejgC,KAAKgG,MAAM83B,MAAM5R,QAElDqO,EAAiB54B,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,KAC3FuwB,EAAuBz+B,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAMmgC,GACrFE,EAAsB9F,EAAiB6F,EAGvClB,EAAYiB,EAAaC,GAAyBpgC,KAAKgG,MAAM83B,MAAMluB,OAASuwB,EAAaC,IAAyBnjB,EAClHkiB,EAAUgB,EAAaE,GAAwBrgC,KAAKgG,MAAM83B,MAAMjuB,KAAOswB,EAAaE,IAAwBpjB,CAGhHjd,MAAKm5B,aAA4B,GAAb,EAAIlc,EACxBjd,KAAKo5B,WAA0B,GAAbnc,EAAQ,CAE1B,IAAI0iB,GAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAU,EAAIjiB,GAAO,GACpF2iB,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,EAAQliB,EAAQ,GAAG,IAChF0iB,GAAaT,GAAYU,GAAWT,KACtCn/B,KAAKgG,MAAM83B,MAAMluB,MAAQ+vB,EACzB3/B,KAAKgG,MAAM83B,MAAMjuB,IAAM+vB,EACvB5/B,KAAKy9B,YAAc,EAAIn0B,EAAM2T,MAC7BiiB,EAAWS,EACXR,EAASS,GAGX5/B,KAAKuzB,SAAS2L,EAAUC,GAAQ,GAAO,GAEvCn/B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,EAElB9vB,EAAMD,mBASRxH,EAAMsR,UAAU8sB,eAAiB,SAAUJ,GACzC,GAAIzF,GACAc,EAAYl7B,KAAKyO,QAAQysB,SAI7B,IAFAqD,EAAkBrD,GAED,cAAbA,EACF,MAAOl7B,MAAK20B,KAAKh0B,KAAK00B,OAAOwK,EAAQ9tB,GAAG/K,SAGxC,IAAIwL,GAASxS,KAAK20B,KAAKC,SAAS1I,OAAO1Z,MAEvC,OADA4nB,GAAap6B,KAAKo6B,WAAW5nB,GACtBqtB,EAAQ7tB,EAAIooB,EAAWnd,MAAQmd,EAAWzQ,QA4BrD9nB,EAAMsR,UAAU+sB,KAAO,SAASjjB,EAAOiP,EAAQuC,GAE/B,MAAVvC,IACFA,GAAUlsB,KAAK4P,MAAQ5P,KAAK6P,KAAO,EAGrC,IAAI0qB,GAAiB54B,EAAS04B,yBAAyBr6B,KAAK20B,KAAKI,YAAa/0B,KAAK4P,MAAO5P,KAAK6P,KAC3FuwB,EAAuBz+B,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAMksB,GACrFmU,EAAsB9F,EAAiB6F,EAGvClB,EAAYhT,EAAOkU,GAAyBpgC,KAAK4P,OAASsc,EAAOkU,IAAyBnjB,EAC1FkiB,EAAYjT,EAAOmU,GAAwBrgC,KAAK6P,KAAOqc,EAAOmU,IAAwBpjB,CAG1Fjd,MAAKm5B,aAAe1K,EAAQ,GAAI,GAAQ,EACxCzuB,KAAKo5B,YAAc3K,EAAS,GAAI,GAAQ,CACxC,IAAIkR,GAAYh+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAamK,EAAUzQ,GAAO,GAChFmR,EAAUj+B,EAASs5B,mBAAmBj7B,KAAK20B,KAAKI,YAAaoK,GAAS1Q,GAAO,IAC7EkR,GAAaT,GAAYU,GAAWT,KACtCD,EAAWS,EACXR,EAASS,GAGX5/B,KAAKuzB,SAAS2L,EAAUC,GAAQ,GAAO,GAEvCn/B,KAAKm5B,cAAe,EACpBn5B,KAAKo5B,YAAa,GAWpBv3B,EAAMsR,UAAUmtB,KAAO,SAAS7R,GAE9B,GAAIpC,GAAQrsB,KAAK6P,IAAM7P,KAAK4P,MAGxBsvB,EAAWl/B,KAAK4P,MAAQyc,EAAOoC,EAC/B0Q,EAASn/B,KAAK6P,IAAMwc,EAAOoC,CAI/BzuB,MAAK4P,MAAQsvB,EACbl/B,KAAK6P,IAAMsvB,GAObt9B,EAAMsR,UAAU0U,OAAS,SAASA,GAChC,GAAIqE,IAAUlsB,KAAK4P,MAAQ5P,KAAK6P,KAAO,EAEnCwc,EAAOH,EAASrE,EAGhBqX,EAAWl/B,KAAK4P,MAAQyc,EACxB8S,EAASn/B,KAAK6P,IAAMwc,CAExBrsB,MAAKuzB,SAAS2L,EAAUC,IAG1Bt/B,EAAOD,QAAUiC,GAKb,SAAShC,EAAQD,GAGrB,GAAI2gC,GAAU,IAMd3gC,GAAQ4gC,aAAe,SAASv+B,GAC9BA,EAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,MAAOb,GAAEmN,KAAK9C,MAAQxJ,EAAEsM,KAAK9C,SASjChQ,EAAQ6gC,WAAa,SAASx+B,GAC5BA,EAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,GAAIs6B,GAAS,OAASn7B,GAAEmN,KAAQnN,EAAEmN,KAAK7C,IAAMtK,EAAEmN,KAAK9C,MAChD+wB,EAAS,OAASv6B,GAAEsM,KAAQtM,EAAEsM,KAAK7C,IAAMzJ,EAAEsM,KAAK9C,KAEpD,OAAO8wB,GAAQC,KAenB/gC,EAAQkC,MAAQ,SAASG,EAAOyX,EAAQknB,GACtC,GAAIp7B,GAAGq7B,CAEP,IAAID,EAEF,IAAKp7B,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IACzCvD,EAAMuD,GAAGmC,IAAM,IAKnB,KAAKnC,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAC9C,GAAI6J,GAAOpN,EAAMuD,EACjB,IAAI6J,EAAKvN,OAAsB,OAAbuN,EAAK1H,IAAc,CAEnC0H,EAAK1H,IAAM+R,EAAOonB,IAElB,GAAG,CAID,IAAK,GADDC,GAAgB,KACXnV,EAAI,EAAGoV,EAAK/+B,EAAM0D,OAAYq7B,EAAJpV,EAAQA,IAAK,CAC9C,GAAIhmB,GAAQ3D,EAAM2pB,EAClB,IAAkB,OAAdhmB,EAAM+B,KAAgB/B,IAAUyJ,GAAQzJ,EAAM9D,OAASlC,EAAQqhC,UAAU5xB,EAAMzJ,EAAO8T,EAAOrK,MAAO,CACtG0xB,EAAgBn7B,CAChB,QAIiB,MAAjBm7B,IAEF1xB,EAAK1H,IAAMo5B,EAAcp5B,IAAMo5B,EAAcvuB,OAASkH,EAAOrK,KAAKoW,gBAE7Dsb,MAafnhC,EAAQshC,QAAU,SAASj/B,EAAOyX,EAAQynB,GACxC,GAAI37B,GAAGq7B,EAAMO,CAGb,KAAK57B,EAAI,EAAGq7B,EAAO5+B,EAAM0D,OAAYk7B,EAAJr7B,EAAUA,IACzC,GAA+BgB,SAA3BvE,EAAMuD,GAAGkN,KAAK2uB,SAAwB,CACxCD,EAAS1nB,EAAOonB,IAChB,KAAK,GAAIO,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQg5B,EAAUl/B,EAAMuD,GAAGkN,KAAK2uB,UAAUl5B,QACvGi5B,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAIzDxjB,GAAMuD,GAAGmC,IAAMy5B,MAGfn/B,GAAMuD,GAAGmC,IAAM+R,EAAOonB,MAe5BlhC,EAAQqhC,UAAY,SAAS17B,EAAGa,EAAGsT,GACjC,MAASnU,GAAEkC,KAAOiS,EAAO8L,WAAa+a,EAAkBn6B,EAAEqB,KAAOrB,EAAEmM,OAC9DhN,EAAEkC,KAAOlC,EAAEgN,MAAQmH,EAAO8L,WAAa+a,EAAWn6B,EAAEqB,MACpDlC,EAAEoC,IAAM+R,EAAO+L,SAAW8a,EAAyBn6B,EAAEuB,IAAMvB,EAAEoM,QAC7DjN,EAAEoC,IAAMpC,EAAEiN,OAASkH,EAAO+L,SAAW8a,EAAan6B,EAAEuB,MAMvD,SAAS9H,EAAQD,EAASM,GAgC9B,QAAS6B,GAAS6N,EAAOC,EAAKurB,EAAarG,GAEzC/0B,KAAK85B,QAAU,GAAIx1B,MACnBtE,KAAKkzB,OAAS,GAAI5uB,MAClBtE,KAAKmzB,KAAO,GAAI7uB,MAEhBtE,KAAKw7B,WAAa,EAClBx7B,KAAKid,MAAQ,MACbjd,KAAKmoB,KAAO,EAGZnoB,KAAKuzB,SAAS3jB,EAAOC,EAAKurB,GAG1Bp7B,KAAKk6B,aAAc,EACnBl6B,KAAKi6B,eAAgB,EACrBj6B,KAAKg6B,cAAe,EACpBh6B,KAAK+0B,YAAcA,EACCvuB,SAAhBuuB,IACF/0B,KAAK+0B,gBAGP/0B,KAAKshC,OAASv/B,EAASw/B,OApDzB,GAAI19B,GAAS3D,EAAoB,IAC7ByB,EAAWzB,EAAoB,IAC/BS,EAAOT,EAAoB,EAsD/B6B,GAASw/B,QACPC,aACEC,YAAY,MACZC,OAAY,IACZC,OAAY,QACZC,KAAY,QACZC,QAAY,QACZxJ,IAAY,IACZK,MAAY,MACZH,KAAY,QAEduJ,aACEL,YAAY,WACZC,OAAY,eACZC,OAAY,aACZC,KAAY,aACZC,QAAY,YACZxJ,IAAY,YACZK,MAAY,OACZH,KAAY,KAUhBx2B,EAASoR,UAAU4uB,UAAY,SAAUT,GACvC,GAAIU,GAAgBrhC,EAAK8F,cAAe1E,EAASw/B,OACjDvhC,MAAKshC,OAAS3gC,EAAK8F,WAAWu7B,EAAeV,IAa/Cv/B,EAASoR,UAAUogB,SAAW,SAAS3jB,EAAOC,EAAKurB,GACjD,KAAMxrB,YAAiBtL,OAAWuL,YAAevL,OAC/C,KAAO,+CAGTtE,MAAKkzB,OAAmB1sB,QAAToJ,EAAsB,GAAItL,MAAKsL,EAAM5I,WAAa,GAAI1C,MACrEtE,KAAKmzB,KAAe3sB,QAAPqJ,EAAoB,GAAIvL,MAAKuL,EAAI7I,WAAa,GAAI1C,MAE3DtE,KAAKw7B,WACPx7B,KAAK+7B,eAAeX,IAOxBr5B,EAASoR,UAAU8uB,MAAQ,WACzBjiC,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAKkzB,OAAOlsB,WACpChH,KAAK08B,gBAOP36B,EAASoR,UAAUupB,aAAe,WAIhC,OAAQ18B,KAAKid,OACX,IAAK,OACHjd,KAAK85B,QAAQoI,YAAYliC,KAAKmoB,KAAOjjB,KAAKC,MAAMnF,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,OAClFnoB,KAAK85B,QAAQsI,SAAS,EACxB,KAAK,QAAgBpiC,KAAK85B,QAAQuI,QAAQ,EAC1C,KAAK,MACL,IAAK,UAAgBriC,KAAK85B,QAAQwI,SAAS,EAC3C,KAAK,OAAgBtiC,KAAK85B,QAAQyI,WAAW,EAC7C,KAAK,SAAgBviC,KAAK85B,QAAQ0I,WAAW,EAC7C,KAAK,SAAgBxiC,KAAK85B,QAAQ2I,gBAAgB,GAIpD,GAAiB,GAAbziC,KAAKmoB,KAEP,OAAQnoB,KAAKid,OACX,IAAK,cAAgBjd,KAAK85B,QAAQ2I,gBAAgBziC,KAAK85B,QAAQ4I,kBAAoB1iC,KAAK85B,QAAQ4I,kBAAoB1iC,KAAKmoB,KAAQ,MACjI,KAAK,SAAgBnoB,KAAK85B,QAAQ0I,WAAWxiC,KAAK85B,QAAQ6I,aAAe3iC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,KAAO,MACjH,KAAK,SAAgBnoB,KAAK85B,QAAQyI,WAAWviC,KAAK85B,QAAQ8I,aAAe5iC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,KAAO,MACjH,KAAK,OAAgBnoB,KAAK85B,QAAQwI,SAAStiC,KAAK85B,QAAQ+I,WAAa7iC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,KAAO,MAC3G,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAASriC,KAAK85B,QAAQgJ,UAAU,GAAM9iC,KAAK85B,QAAQgJ,UAAU,GAAK9iC,KAAKmoB,KAAO,EAAI,MACpH,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAQ,MAC5G,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,QAUnHpmB,EAASoR,UAAU0pB,QAAU,WAC3B,MAAQ78B,MAAK85B,QAAQ9yB,WAAahH,KAAKmzB,KAAKnsB,WAM9CjF,EAASoR,UAAUkV,KAAO,WACxB,GAAIuJ,GAAO5xB,KAAK85B,QAAQ9yB,SAIxB,IAAIhH,KAAK85B,QAAQiJ,WAAa,EAC5B,OAAQ/iC,KAAKid,OACX,IAAK,cAEHjd,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAYhH,KAAKmoB,KAAO,MAC/D,KAAK,SAAgBnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,MACzF,KAAK,SAAgBnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,GAAK,MAC9F,KAAK,OACHnoB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAwB,IAAZhH,KAAKmoB,KAAc,GAAK,GAEzE,IAAIxc,GAAI3L,KAAK85B,QAAQ+I,UACrB7iC,MAAK85B,QAAQwI,SAAS32B,EAAKA,EAAI3L,KAAKmoB,KACpC,MACF,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAAQriC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAO,MAC/E,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAO,MACjF,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,UAKlF,QAAQnoB,KAAKid,OACX,IAAK,cAAgBjd,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAK85B,QAAQ9yB,UAAYhH,KAAKmoB,KAAO,MAClF,KAAK,SAAgBnoB,KAAK85B,QAAQ0I,WAAWxiC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,KAAO,MACrF,KAAK,SAAgBnoB,KAAK85B,QAAQyI,WAAWviC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,KAAO;KACrF,KAAK,OAAgBnoB,KAAK85B,QAAQwI,SAAStiC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,KAAO,MACjF,KAAK,UACL,IAAK,MAAgBnoB,KAAK85B,QAAQuI,QAAQriC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAO,MAC/E,KAAK,QAAgBnoB,KAAK85B,QAAQsI,SAASpiC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,KAAO,MACjF,KAAK,OAAgBnoB,KAAK85B,QAAQoI,YAAYliC,KAAK85B,QAAQqI,cAAgBniC,KAAKmoB,MAKpF,GAAiB,GAAbnoB,KAAKmoB,KAEP,OAAQnoB,KAAKid,OACX,IAAK,cAAmBjd,KAAK85B,QAAQ4I,kBAAoB1iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQ2I,gBAAgB,EAAK,MACtG,KAAK,SAAmBziC,KAAK85B,QAAQ6I,aAAe3iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQ0I,WAAW,EAAK,MAC5F,KAAK,SAAmBxiC,KAAK85B,QAAQ8I,aAAe5iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQyI,WAAW,EAAK,MAC5F,KAAK,OAAmBviC,KAAK85B,QAAQ+I,WAAa7iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQwI,SAAS,EAAK,MACxF,KAAK,UACL,IAAK,MAAmBtiC,KAAK85B,QAAQgJ,UAAY9iC,KAAKmoB,KAAK,GAAGnoB,KAAK85B,QAAQuI,QAAQ,EAAI,MACvF,KAAK,QAAmBriC,KAAK85B,QAAQiJ,WAAa/iC,KAAKmoB,MAAMnoB,KAAK85B,QAAQsI,SAAS,EAAK,MACxF,KAAK,QAMLpiC,KAAK85B,QAAQ9yB,WAAa4qB,IAC5B5xB,KAAK85B,QAAU,GAAIx1B,MAAKtE,KAAKmzB,KAAKnsB,YAGpCrF,EAAS83B,oBAAoBz5B,KAAM4xB,IAQrC7vB,EAASoR,UAAUiV,WAAa,WAC9B,MAAOpoB,MAAK85B,SAcd/3B,EAASoR,UAAU6vB,SAAW,SAASC,EAAUC,GAC/CljC,KAAKid,MAAQgmB,EAETC,EAAU,IACZljC,KAAKmoB,KAAO+a,GAGdljC,KAAKw7B,WAAY,GAOnBz5B,EAASoR,UAAUgwB,aAAe,SAAUC,GAC1CpjC,KAAKw7B,UAAY4H,GAQnBrhC,EAASoR,UAAU4oB,eAAiB,SAASX,GAC3C,GAAmB50B,QAAf40B,EAAJ,CAMA,GAAIiI,GAAiB,QACjBC,EAAiB,OACjBC,EAAiB,MACjBC,EAAiB,KACjBC,EAAiB,IACjBC,EAAiB,IACjBC,EAAiB,CAGR,KAATN,EAAgBjI,IAAqBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,IAATkb,EAAejI,IAAsBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,IAATkb,EAAejI,IAAsBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,KACpE,GAATkb,EAAcjI,IAAuBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,IACpE,GAATkb,EAAcjI,IAAuBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,IACpE,EAATkb,EAAajI,IAAwBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAC7Ekb,EAAWjI,IAA0Bp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GACnE,EAAVmb,EAAclI,IAAuBp7B,KAAKid,MAAQ,QAAejd,KAAKmoB,KAAO,GAC7Emb,EAAYlI,IAAyBp7B,KAAKid,MAAQ,QAAejd,KAAKmoB,KAAO,GACrE,EAARob,EAAYnI,IAAyBp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GACrE,EAARob,EAAYnI,IAAyBp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GAC7Eob,EAAUnI,IAA2Bp7B,KAAKid,MAAQ,MAAejd,KAAKmoB,KAAO,GAC7Eob,EAAQ,EAAInI,IAAyBp7B,KAAKid,MAAQ,UAAejd,KAAKmoB,KAAO,GACpE,EAATqb,EAAapI,IAAwBp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAC7Eqb,EAAWpI,IAA0Bp7B,KAAKid,MAAQ,OAAejd,KAAKmoB,KAAO,GAClE,GAAXsb,EAAgBrI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,GAAXsb,EAAgBrI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,EAAXsb,EAAerI,IAAsBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7Esb,EAAarI,IAAwBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAClE,GAAXub,EAAgBtI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,GAAXub,EAAgBtI,IAAqBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,IAClE,EAAXub,EAAetI,IAAsBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7Eub,EAAatI,IAAwBp7B,KAAKid,MAAQ,SAAejd,KAAKmoB,KAAO,GAC7D,IAAhBwb,EAAsBvI,IAAep7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KAC7D,IAAhBwb,EAAsBvI,IAAep7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KAC7D,GAAhBwb,EAAqBvI,IAAgBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,IAC7D,GAAhBwb,EAAqBvI,IAAgBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,IAC7D,EAAhBwb,EAAoBvI,IAAiBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,GAC7Ewb,EAAkBvI,IAAmBp7B,KAAKid,MAAQ,cAAejd,KAAKmoB,KAAO,KASnFpmB,EAASoR,UAAU6hB,KAAO,SAASyD,GACjC,GAAIL,GAAQ,GAAI9zB,MAAKm0B,EAAKzxB,UAE1B,IAAkB,QAAdhH,KAAKid,MAAiB,CACxB,GAAIsb,GAAOH,EAAM+J,cAAgBj9B,KAAKwoB,MAAM0K,EAAM2K,WAAa,GAC/D3K,GAAM8J,YAAYh9B,KAAKwoB,MAAM6K,EAAOv4B,KAAKmoB,MAAQnoB,KAAKmoB,MACtDiQ,EAAMgK,SAAS,GACfhK,EAAMiK,QAAQ,GACdjK,EAAMkK,SAAS,GACflK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,SAAdziC,KAAKid,MACRmb,EAAM0K,UAAY,IACpB1K,EAAMiK,QAAQ,GACdjK,EAAMgK,SAAShK,EAAM2K,WAAa,IAIlC3K,EAAMiK,QAAQ,GAGhBjK,EAAMkK,SAAS,GACflK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,OAAdziC,KAAKid,MAAgB,CAE5B,OAAQjd,KAAKmoB,MACX,IAAK,GACL,IAAK,GACHiQ,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAAW,MAC1D,SACEzK,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,KAEjDzK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,WAAdziC,KAAKid,MAAoB,CAEhC,OAAQjd,KAAKmoB,MACX,IAAK,GACL,IAAK,GACHiQ,EAAMkK,SAA6C,GAApCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAAW,MAC1D,SACEzK,EAAMkK,SAA4C,EAAnCp9B,KAAKwoB,MAAM0K,EAAMyK,WAAa,IAEjDzK,EAAMmK,WAAW,GACjBnK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OAEnB,IAAkB,QAAdziC,KAAKid,MAAiB,CAC7B,OAAQjd,KAAKmoB,MACX,IAAK,GACHiQ,EAAMmK,WAAiD,GAAtCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,IAAW,MAC9D,SACExK,EAAMmK,WAAiD,GAAtCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,KAErDxK,EAAMoK,WAAW,GACjBpK,EAAMqK,gBAAgB,OACjB,IAAkB,UAAdziC,KAAKid,MAAmB,CAEjC,OAAQjd,KAAKmoB,MACX,IAAK,IACL,IAAK,IACHiQ,EAAMmK,WAAgD,EAArCr9B,KAAKwoB,MAAM0K,EAAMwK,aAAe,IACjDxK,EAAMoK,WAAW,EACjB,MACF,KAAK,GACHpK,EAAMoK,WAAiD,GAAtCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,IAAW,MAC9D,SACEvK,EAAMoK,WAAiD,GAAtCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,KAErDvK,EAAMqK,gBAAgB,OAEnB,IAAkB,UAAdziC,KAAKid,MAEZ,OAAQjd,KAAKmoB,MACX,IAAK,IACL,IAAK,IACHiQ,EAAMoK,WAAgD,EAArCt9B,KAAKwoB,MAAM0K,EAAMuK,aAAe,IACjDvK,EAAMqK,gBAAgB,EACtB,MACF,KAAK,GACHrK,EAAMqK,gBAA6D,IAA7Cv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoB,KAAe,MAC5E,SACEtK,EAAMqK,gBAA4D,IAA5Cv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoB,UAG5D,IAAkB,eAAd1iC,KAAKid,MAAwB,CACpC,GAAIkL,GAAOnoB,KAAKmoB,KAAO,EAAInoB,KAAKmoB,KAAO,EAAI,CAC3CiQ,GAAMqK,gBAAgBv9B,KAAKwoB,MAAM0K,EAAMsK,kBAAoBva,GAAQA,GAGrE,MAAOiQ,IAQTr2B,EAASoR,UAAU+pB,QAAU,WAC3B,GAAyB,GAArBl9B,KAAKg6B,aAEP,OADAh6B,KAAKg6B,cAAe,EACZh6B,KAAKid,OACX,IAAK,OACL,IAAK,QACL,IAAK,UACL,IAAK,MACL,IAAK,OACL,IAAK,SACL,IAAK,SACL,IAAK,cACH,OAAO,CACT,SACE,OAAO,MAGR,IAA0B,GAAtBjd,KAAKi6B,cAEZ,OADAj6B,KAAKi6B,eAAgB,EACbj6B,KAAKid,OACX,IAAK,UACL,IAAK,MACL,IAAK,OACL,IAAK,SACL,IAAK,SACL,IAAK,cACH,OAAO,CACT,SACE,OAAO,MAGR,IAAwB,GAApBjd,KAAKk6B,YAEZ,OADAl6B,KAAKk6B,aAAc,EACXl6B,KAAKid,OACX,IAAK,cACL,IAAK,SACL,IAAK,SACL,IAAK,OACH,OAAO,CACT,SACE,OAAO,EAIb,OAAQjd,KAAKid,OACX,IAAK,cACH,MAA0C,IAAlCjd,KAAK85B,QAAQ4I,iBACvB,KAAK,SACH,MAAqC,IAA7B1iC,KAAK85B,QAAQ6I,YACvB,KAAK,SACH,MAAmC,IAA3B3iC,KAAK85B,QAAQ+I,YAAkD,GAA7B7iC,KAAK85B,QAAQ8I,YACzD,KAAK,OACH,MAAmC,IAA3B5iC,KAAK85B,QAAQ+I,UACvB,KAAK,UACL,IAAK,MACH,MAAkC,IAA1B7iC,KAAK85B,QAAQgJ,SACvB,KAAK,QACH,MAAmC,IAA3B9iC,KAAK85B,QAAQiJ,UACvB,KAAK,OACH,OAAO,CACT,SACE,OAAO,IAWbhhC,EAASoR,UAAUywB,cAAgB,SAASnL,GAC9BjyB,QAARiyB,IACFA,EAAOz4B,KAAK85B,QAGd,IAAIwH,GAASthC,KAAKshC,OAAOE,YAAYxhC,KAAKid,MAC1C,OAAQqkB,IAAUA,EAAO37B,OAAS,EAAK9B,EAAO40B,GAAM6I,OAAOA,GAAU,IASvEv/B,EAASoR,UAAU0wB,cAAgB,SAASpL,GAC9BjyB,QAARiyB,IACFA,EAAOz4B,KAAK85B,QAGd,IAAIwH,GAASthC,KAAKshC,OAAOQ,YAAY9hC,KAAKid,MAC1C,OAAQqkB,IAAUA,EAAO37B,OAAS,EAAK9B,EAAO40B,GAAM6I,OAAOA,GAAU,IAGvEv/B,EAASoR,UAAU2wB,aAAe,WAKhC,QAASC,GAAK18B,GACZ,MAAQA,GAAQ8gB,EAAO,GAAK,EAAK,QAAU,OAG7C,QAAS6b,GAAMvL,GACb,MAAIA,GAAKwL,OAAO,GAAI3/B,MAAQ,OACnB,SAELm0B,EAAKwL,OAAOpgC,IAASoP,IAAI,EAAG,OAAQ,OAC/B,YAELwlB,EAAKwL,OAAOpgC,IAASoP,IAAI,GAAI,OAAQ,OAChC,aAEF,GAGT,QAASixB,GAAYzL,GACnB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,QAAU,gBAAkB,GAG7D,QAAS6/B,GAAa1L,GACpB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,SAAW,iBAAmB,GAG/D,QAAS8/B,GAAY3L,GACnB,MAAOA,GAAKwL,OAAO,GAAI3/B,MAAQ,QAAU,gBAAkB,GA9B7D,GAAI9D,GAAIqD,EAAO7D,KAAK85B,SAChBrB,EAAOj4B,EAAE6jC,OAAS7jC,EAAE6jC,OAAO,MAAQ7jC,EAAE8jC,KAAK,MAC1Cnc,EAAOnoB,KAAKmoB,IA+BhB,QAAQnoB,KAAKid,OACX,IAAK,cACH,MAAO8mB,GAAKtL,EAAK8E,gBAAgBrwB,MAEnC,KAAK,SACH,MAAO62B,GAAKtL,EAAK6E,WAAWpwB,MAE9B,KAAK,SACH,MAAO62B,GAAKtL,EAAK4E,WAAWnwB,MAE9B,KAAK,OACH,GAAIkwB,GAAQ3E,EAAK2E,OAIjB,OAHiB,IAAbp9B,KAAKmoB,OACPiV,EAAQA,EAAQ,KAAOA,EAAQ,IAE1BA,EAAQ,IAAM4G,EAAMvL,GAAQsL,EAAKtL,EAAK2E,QAE/C,KAAK,UACH,MAAO3E,GAAK6I,OAAO,QAAQiD,cACvBP,EAAMvL,GAAQyL,EAAYzL,GAAQsL,EAAKtL,EAAKA,OAElD,KAAK,MACH,GAAIJ,GAAMI,EAAKA,OACXC,EAAQD,EAAK6I,OAAO,QAAQiD,aAChC,OAAO,MAAQlM,EAAM,IAAMK,EAAQyL,EAAa1L,GAAQsL,EAAK1L,EAAM,EAErE,KAAK,QACH,MAAOI,GAAK6I,OAAO,QAAQiD,cACvBJ,EAAa1L,GAAQsL,EAAKtL,EAAKC,QAErC,KAAK,OACH,GAAIH,GAAOE,EAAKF,MAChB,OAAO,OAASA,EAAO6L,EAAY3L,GAAOsL,EAAKxL,EAEjD,SACE,MAAO,KAIb14B,EAAOD,QAAUmC,GAKb,SAASlC,GAOb,QAAS0C,KACPvC,KAAKyO,QAAU,KACfzO,KAAKgG,MAAQ,KAQfzD,EAAU4Q,UAAUD,WAAa,SAASzE,GACpCA,GACF9N,KAAK2E,OAAOtF,KAAKyO,QAASA,IAQ9BlM,EAAU4Q,UAAUsO,OAAS,WAE3B,OAAO,GAMTlf,EAAU4Q,UAAUG,QAAU,aAU9B/Q,EAAU4Q,UAAUqxB,WAAa,WAC/B,GAAIC,GAAWzkC,KAAKgG,MAAM0+B,iBAAmB1kC,KAAKgG,MAAMuM,OACpDvS,KAAKgG,MAAM2+B,kBAAoB3kC,KAAKgG,MAAMwM,MAK9C,OAHAxS,MAAKgG,MAAM0+B,eAAiB1kC,KAAKgG,MAAMuM,MACvCvS,KAAKgG,MAAM2+B,gBAAkB3kC,KAAKgG,MAAMwM,OAEjCiyB,GAGT5kC,EAAOD,QAAU2C,GAKb,SAAS1C,EAAQD,EAASM,GAe9B,QAASsC,GAAamyB,EAAMlmB,GAC1BzO,KAAK20B,KAAOA,EAGZ30B,KAAKq0B,gBACHuQ,iBAAiB,EAEjBC,QAASA,EACTR,OAAQ,MAEVrkC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK2pB,OAAS,EAEd3pB,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GA5BlB,GAAI9N,GAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC2D,EAAS3D,EAAoB,IAC7B2kC,EAAU3kC,EAAoB,GA4BlCsC,GAAY2Q,UAAY,GAAI5Q,GAM5BC,EAAY2Q,UAAUuhB,QAAU,WAC9B,GAAI7C,GAAMtgB,SAASM,cAAc,MACjCggB,GAAIhqB,UAAY,cAChBgqB,EAAI5kB,MAAM2W,SAAW,WACrBiO,EAAI5kB,MAAMtF,IAAM,MAChBkqB,EAAI5kB,MAAMuF,OAAS,OAEnBxS,KAAK6xB,IAAMA,GAMbrvB,EAAY2Q,UAAUG,QAAU,WAC9BtT,KAAKyO,QAAQm2B,iBAAkB,EAC/B5kC,KAAKyhB,SAELzhB,KAAK20B,KAAO,MAQdnyB,EAAY2Q,UAAUD,WAAa,SAASzE,GACtCA,GAEF9N,EAAKoF,iBAAiB,kBAAmB,SAAU,WAAY/F,KAAKyO,QAASA,IAQjFjM,EAAY2Q,UAAUsO,OAAS,WAC7B,GAAIzhB,KAAKyO,QAAQm2B,gBAAiB,CAChC,GAAIE,GAAS9kC,KAAK20B,KAAK5E,IAAIgV,kBACvB/kC,MAAK6xB,IAAIhoB,YAAci7B,IAErB9kC,KAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvCiT,EAAOrzB,YAAYzR,KAAK6xB,KAExB7xB,KAAK4P,QAGP,IAAIutB,GAAM,GAAI74B,OAAK,GAAIA,OAAO0C,UAAYhH,KAAK2pB,QAC3C5X,EAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASkI,GAE5BkH,EAASrkC,KAAKyO,QAAQo2B,QAAQ7kC,KAAKyO,QAAQ41B,QAC3CW,EAAQX,EAAOvK,QAAU,IAAMuK,EAAOlK,KAAO,KAAOt2B,EAAOs5B,GAAKmE,OAAO,8BAC3E0D,GAAQA,EAAM5f,OAAO,GAAG6f,cAAgBD,EAAME,UAAU,GAExDllC,KAAK6xB,IAAI5kB,MAAMxF,KAAOsK,EAAI,KAC1B/R,KAAK6xB,IAAImT,MAAQA,MAIbhlC,MAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvC7xB,KAAKklB,MAGP,QAAO,GAMT1iB,EAAY2Q,UAAUvD,MAAQ,WAG5B,QAASiF,KACPV,EAAG+Q,MAGH,IAAIjI,GAAQ9I,EAAGwgB,KAAKc,MAAM2E,WAAWjmB,EAAGwgB,KAAKC,SAAS1I,OAAO3Z,OAAO0K,MAChEuV,EAAW,EAAIvV,EAAQ,EACZ,IAAXuV,IAAiBA,EAAW,IAC5BA,EAAW,MAAMA,EAAW,KAEhCre,EAAGsN,SAGHtN,EAAGgxB,iBAAmB7rB,WAAWzE,EAAQ2d,GAd3C,GAAIre,GAAKnU,IAiBT6U,MAMFrS,EAAY2Q,UAAU+R,KAAO,WACG1e,SAA1BxG,KAAKmlC,mBACP9rB,aAAarZ,KAAKmlC,wBACXnlC,MAAKmlC,mBAUhB3iC,EAAY2Q,UAAUiyB,eAAiB,SAASjL,GAC9C,GAAIrsB,GAAInN,EAAKkG,QAAQszB,EAAM,QAAQnzB,UAC/Bm2B,GAAM,GAAI74B,OAAO0C,SACrBhH,MAAK2pB,OAAS7b,EAAIqvB,EAClBn9B,KAAKyhB,UAOPjf,EAAY2Q,UAAUkyB,eAAiB,WACrC,MAAO,IAAI/gC,OAAK,GAAIA,OAAO0C,UAAYhH,KAAK2pB,SAG9C9pB,EAAOD,QAAU4C,GAKb,SAAS3C,EAAQD,EAASM,GAiB9B,QAASuC,GAAYkyB,EAAMlmB,GACzBzO,KAAK20B,KAAOA,EAGZ30B,KAAKq0B,gBACHiR,gBAAgB,EAChBT,QAASA,EACTR,OAAQ,MAEVrkC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAK41B,WAAa,GAAItxB,MACtBtE,KAAKulC,eAGLvlC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAhClB,GAAI1K,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC2D,EAAS3D,EAAoB,IAC7B2kC,EAAU3kC,EAAoB,GA+BlCuC,GAAW0Q,UAAY,GAAI5Q,GAO3BE,EAAW0Q,UAAUD,WAAa,SAASzE,GACrCA,GAEF9N,EAAKoF,iBAAiB,iBAAkB,SAAU,WAAY/F,KAAKyO,QAASA,IAQhFhM,EAAW0Q,UAAUuhB,QAAU,WAC7B,GAAI7C,GAAMtgB,SAASM,cAAc,MACjCggB,GAAIhqB,UAAY,aAChBgqB,EAAI5kB,MAAM2W,SAAW,WACrBiO,EAAI5kB,MAAMtF,IAAM,MAChBkqB,EAAI5kB,MAAMuF,OAAS,OACnBxS,KAAK6xB,IAAMA,CAEX,IAAI2T,GAAOj0B,SAASM,cAAc,MAClC2zB,GAAKv4B,MAAM2W,SAAW,WACtB4hB,EAAKv4B,MAAMtF,IAAM,MACjB69B,EAAKv4B,MAAMxF,KAAO,QAClB+9B,EAAKv4B,MAAMuF,OAAS,OACpBgzB,EAAKv4B,MAAMsF,MAAQ,OACnBsf,EAAIpgB,YAAY+zB,GAGhBxlC,KAAK8D,OAAS,GAAIC,GAAOyhC,GACzBxlC,KAAK8D,OAAOyP,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OAClDA,KAAK8D,OAAOyP,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OAC7CA,KAAK8D,OAAOyP,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAChDA,KAAK8D,OAAOyP,GAAG,MAAS,SAAUjK,GAChCA,EAAMD,oBAOV5G,EAAW0Q,UAAUG,QAAU,WAC7BtT,KAAKyO,QAAQ62B,gBAAiB,EAC9BtlC,KAAKyhB,SAELzhB,KAAK8D,OAAOs/B,QAAO,GACnBpjC,KAAK8D,OAAS,KAEd9D,KAAK20B,KAAO,MAOdlyB,EAAW0Q,UAAUsO,OAAS,WAC5B,GAAIzhB,KAAKyO,QAAQ62B,eAAgB,CAC/B,GAAIR,GAAS9kC,KAAK20B,KAAK5E,IAAIgV,kBACvB/kC,MAAK6xB,IAAIhoB,YAAci7B,IAErB9kC,KAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,KAEvCiT,EAAOrzB,YAAYzR,KAAK6xB,KAG1B,IAAI9f,GAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASj1B,KAAK41B,YAEjCyO,EAASrkC,KAAKyO,QAAQo2B,QAAQ7kC,KAAKyO,QAAQ41B,QAC3CW,EAAQX,EAAOlK,KAAO,KAAOt2B,EAAO7D,KAAK41B,YAAY0L,OAAO,8BAChE0D,GAAQA,EAAM5f,OAAO,GAAG6f,cAAgBD,EAAME,UAAU,GAExDllC,KAAK6xB,IAAI5kB,MAAMxF,KAAOsK,EAAI,KAC1B/R,KAAK6xB,IAAImT,MAAQA,MAIbhlC,MAAK6xB,IAAIhoB,YACX7J,KAAK6xB,IAAIhoB,WAAWsH,YAAYnR,KAAK6xB,IAIzC,QAAO,GAOTpvB,EAAW0Q,UAAUsyB,cAAgB,SAAStL,GAC5Cn6B,KAAK41B,WAAaj1B,EAAKkG,QAAQszB,EAAM,QACrCn6B,KAAKyhB,UAOPhf,EAAW0Q,UAAUuyB,cAAgB,WACnC,MAAO,IAAIphC,MAAKtE,KAAK41B,WAAW5uB,YAQlCvE,EAAW0Q,UAAU6qB,aAAe,SAAS10B,GAC3CtJ,KAAKulC,YAAYxG,UAAW,EAC5B/+B,KAAKulC,YAAY3P,WAAa51B,KAAK41B,WAEnCtsB,EAAMq8B,kBACNr8B,EAAMD,kBAQR5G,EAAW0Q,UAAU8qB,QAAU,SAAU30B,GACvC,GAAKtJ,KAAKulC,YAAYxG,SAAtB,CAEA,GAAIhtB,GAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASj1B,KAAKulC,YAAY3P,YAActsB,EAAMk2B,OACjErF,EAAOn6B,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,EAEjC/R,MAAKylC,cAActL,GAGnBn6B,KAAK20B,KAAKE,QAAQjH,KAAK,cACrBuM,KAAM,GAAI71B,MAAKtE,KAAK41B,WAAW5uB,aAGjCsC,EAAMq8B,kBACNr8B,EAAMD,mBAQR5G,EAAW0Q,UAAU+qB,WAAa,SAAU50B,GACrCtJ,KAAKulC,YAAYxG,WAGtB/+B,KAAK20B,KAAKE,QAAQjH,KAAK,eACrBuM,KAAM,GAAI71B,MAAKtE,KAAK41B,WAAW5uB,aAGjCsC,EAAMq8B,kBACNr8B,EAAMD,mBAGRxJ,EAAOD,QAAU6C,GAKb,SAAS5C,EAAQD,EAASM,GAe9B,QAASwC,GAAUiyB,EAAMlmB,EAASm3B,EAAKC,GACrC7lC,KAAKK,GAAKM,EAAKqE,aACfhF,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHE,YAAa,OACbuR,iBAAiB,EACjBC,iBAAiB,EACjBC,OAAO,EACPC,iBAAkB,EAClBC,iBAAkB,EAClBC,aAAc,GACdC,aAAc,EACdC,UAAW,GACX9zB,MAAO,OACPmW,SAAS,EACT6S,YAAY,EACZD,aACE7zB,MAAOqE,IAAItF,OAAWkG,IAAIlG,QAC1B6gB,OAAQvb,IAAItF,OAAWkG,IAAIlG,SAE7Bw+B,OACEv9B,MAAO8hB,KAAK/iB,QACZ6gB,OAAQkC,KAAK/iB,SAEf86B,QACE75B,MAAOs1B,SAAUv2B,QACjB6gB,OAAQ0V,SAAUv2B,UAItBxG,KAAK6lC,iBAAmBA,EACxB7lC,KAAKsmC,aAAeV,EACpB5lC,KAAKgG,SACLhG,KAAKumC,aACHC,SACAC,UACAzB,UAGFhlC,KAAK+vB,OAEL/vB,KAAKy1B,OAAS7lB,MAAM,EAAGC,IAAI,GAE3B7P,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK0mC,iBAAmB,EAExB1mC,KAAKkT,WAAWzE,GAChBzO,KAAKuS,MAAQrO,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAC3DxK,KAAK2mC,SAAW3mC,KAAKuS,MACrBvS,KAAKwS,OAASxS,KAAKsmC,aAAahW,aAChCtwB,KAAKk5B,QAAS,EAEdl5B,KAAK4mC,WAAa,GAClB5mC,KAAK6mC,iBAAmB,GACxB7mC,KAAK8mC,aAAe,GAEpB9mC,KAAK+mC,WAAa,EAClB/mC,KAAKgnC,QAAS,EACdhnC,KAAKinC,eACLjnC,KAAKknC,cAAe,EAGpBlnC,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,EAGtBnnC,KAAK00B,SAEL,IAAIvgB,GAAKnU,IACTA,MAAK20B,KAAKE,QAAQthB,GAAG,eAAgB,WACnCY,EAAG4b,IAAIqX,cAAcn6B,MAAMtF,IAAMwM,EAAGwgB,KAAKC,SAASyS,UAAY,OApFlE,GAAI1mC,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BqC,EAAYrC,EAAoB,IAChC0B,EAAW1B,EAAoB,GAqFnCwC,GAASyQ,UAAY,GAAI5Q,GAGzBG,EAASyQ,UAAUm0B,SAAW,SAAS7e,EAAO8e,GACvCvnC,KAAKm0B,OAAOruB,eAAe2iB,KAC9BzoB,KAAKm0B,OAAO1L,GAAS8e,GAEvBvnC,KAAKmnC,gBAAkB,GAGzBzkC,EAASyQ,UAAUq0B,YAAc,SAAS/e,EAAO8e,GAC/CvnC,KAAKm0B,OAAO1L,GAAS8e,GAGvB7kC,EAASyQ,UAAUs0B,YAAc,SAAShf,GACpCzoB,KAAKm0B,OAAOruB,eAAe2iB,WACtBzoB,MAAKm0B,OAAO1L,GACnBzoB,KAAKmnC,gBAAkB,IAK3BzkC,EAASyQ,UAAUD,WAAa,SAAUzE,GACxC,GAAIA,EAAS,CACX,GAAIgT,IAAS,CACTzhB,MAAKyO,QAAQ8lB,aAAe9lB,EAAQ8lB,aAAuC/tB,SAAxBiI,EAAQ8lB,cAC7D9S,GAAS,EAEX,IAAIvT,IACF,cACA,kBACA,kBACA,QACA,mBACA,mBACA,eACA,eACA,YACA,QACA,UACA,cACA,QACA,SACA,aAEFvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAE3CzO,KAAK2mC,SAAWziC,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAEhD,GAAViX,GAAkBzhB,KAAK+vB,IAAIzQ,QAC7Btf,KAAK0nC,OACL1nC,KAAK2nC,UASXjlC,EAASyQ,UAAUuhB,QAAU,WAC3B10B,KAAK+vB,IAAIzQ,MAAQ/N,SAASM,cAAc,OACxC7R,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAKyO,QAAQ8D,MAC1CvS,KAAK+vB,IAAIzQ,MAAMrS,MAAMuF,OAASxS,KAAKwS,OAEnCxS,KAAK+vB,IAAIqX,cAAgB71B,SAASM,cAAc,OAChD7R,KAAK+vB,IAAIqX,cAAcn6B,MAAMsF,MAAQ,OACrCvS,KAAK+vB,IAAIqX,cAAcn6B,MAAMuF,OAASxS,KAAKwS,OAC3CxS,KAAK+vB,IAAIqX,cAAcn6B,MAAM2W,SAAW,WAGxC5jB,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMtF,IAAM,MACrB3H,KAAK4lC,IAAI34B,MAAMuF,OAAS,OACxBxS,KAAK4lC,IAAI34B,MAAMsF,MAAQ,OACvBvS,KAAK4lC,IAAI34B,MAAM26B,QAAU,QACzB5nC,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK4lC,MAGlCljC,EAASyQ,UAAU00B,kBAAoB,WACrCjnC,EAAQiQ,gBAAgB7Q,KAAKinC,YAE7B,IAAIl1B,GACAs0B,EAAYrmC,KAAKyO,QAAQ43B,UACzByB,EAAa,GACbC,EAAa,EACb/1B,EAAI+1B,EAAa,GAAMD,CAGzB/1B,GAD8B,QAA5B/R,KAAKyO,QAAQ8lB,YACXwT,EAGA/nC,KAAKuS,MAAQ8zB,EAAY0B,CAG/B,KAAK,GAAIzQ,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvIt3B,KAAKm0B,OAAOmD,GAAS0Q,SAASj2B,EAAGC,EAAGhS,KAAKinC,YAAajnC,KAAK4lC,IAAKS,EAAWyB,GAC3E91B,GAAK81B,EAAaC,GAKxBnnC,GAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKknC,cAAe,GAGtBxkC,EAASyQ,UAAU80B,cAAgB,WACR,GAArBjoC,KAAKknC,eACPtmC,EAAQiQ,gBAAgB7Q,KAAKinC,aAC7BrmC,EAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKknC,cAAe,IAOxBxkC,EAASyQ,UAAUw0B,KAAO,WACxB3nC,KAAKk5B,QAAS,EACTl5B,KAAK+vB,IAAIzQ,MAAMzV,aACc,QAA5B7J,KAAKyO,QAAQ8lB,YACfv0B,KAAK20B,KAAK5E,IAAItoB,KAAKgK,YAAYzR,KAAK+vB,IAAIzQ,OAGxCtf,KAAK20B,KAAK5E,IAAI1I,MAAM5V,YAAYzR,KAAK+vB,IAAIzQ,QAIxCtf,KAAK+vB,IAAIqX,cAAcv9B,YAC1B7J,KAAK20B,KAAK5E,IAAImY,qBAAqBz2B,YAAYzR,KAAK+vB,IAAIqX,gBAO5D1kC,EAASyQ,UAAUu0B,KAAO,WACxB1nC,KAAKk5B,QAAS,EACVl5B,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,OAG7Ctf,KAAK+vB,IAAIqX,cAAcv9B,YACzB7J,KAAK+vB,IAAIqX,cAAcv9B,WAAWsH,YAAYnR,KAAK+vB,IAAIqX,gBAU3D1kC,EAASyQ,UAAUogB,SAAW,SAAU3jB,EAAOC,GAC1B,GAAf7P,KAAKgnC,QAA8C,GAA3BhnC,KAAKyO,QAAQ8sB,YAA2C,IAArBv7B,KAAK8mC,cAC9Dl3B,EAAQ,IACVA,EAAQ,GAGZ5P,KAAKy1B,MAAM7lB,MAAQA,EACnB5P,KAAKy1B,MAAM5lB,IAAMA,GAOnBnN,EAASyQ,UAAUsO,OAAS,WAC1B,GAAIgjB,IAAU,EACV0D,EAAe,CAGnBnoC,MAAK+vB,IAAIqX,cAAcn6B,MAAMtF,IAAM3H,KAAK20B,KAAKC,SAASyS,UAAY,IAElE,KAAK,GAAI/P,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,IACvI6Q,IAIN,IAA2B,GAAvBnoC,KAAKmnC,gBAAuC,GAAhBgB,EAC9BnoC,KAAK0nC,WAEF,CACH1nC,KAAK2nC,OACL3nC,KAAKwS,OAAStO,OAAOlE,KAAKsmC,aAAar5B,MAAMuF,OAAOhI,QAAQ,KAAK,KAGjExK,KAAK+vB,IAAIqX,cAAcn6B,MAAMuF,OAASxS,KAAKwS,OAAS,KACpDxS,KAAKuS,MAAgC,GAAxBvS,KAAKyO,QAAQia,QAAkBxkB,QAAQ,GAAKlE,KAAKyO,QAAQ8D,OAAO/H,QAAQ,KAAK,KAAO,CAEjG,IAAIxE,GAAQhG,KAAKgG,MACbsZ,EAAQtf,KAAK+vB,IAAIzQ,KAGrBA,GAAMzX,UAAY,WAGlB7H,KAAKooC,oBAEL,IAAI7T,GAAcv0B,KAAKyO,QAAQ8lB,YAC3BuR,EAAkB9lC,KAAKyO,QAAQq3B,gBAC/BC,EAAkB/lC,KAAKyO,QAAQs3B,eAGnC//B,GAAMqiC,iBAAmBvC,EAAkB9/B,EAAMsiC,gBAAkB,EACnEtiC,EAAMuiC,iBAAmBxC,EAAkB//B,EAAMwiC,gBAAkB,EAEnExiC,EAAMyiC,eAAiBzoC,KAAK20B,KAAK5E,IAAImY,qBAAqB9X,YAAcpwB,KAAK+mC,WAAa/mC,KAAKuS,MAAQ,EAAIvS,KAAKyO,QAAQy3B,iBACxHlgC,EAAM0iC,gBAAkB,EACxB1iC,EAAM2iC,eAAiB3oC,KAAK20B,KAAK5E,IAAImY,qBAAqB9X,YAAcpwB,KAAK+mC,WAAa/mC,KAAKuS,MAAQ,EAAIvS,KAAKyO,QAAQw3B,iBACxHjgC,EAAM4iC,gBAAkB,EAGL,QAAfrU,GACFjV,EAAMrS,MAAMtF,IAAM,IAClB2X,EAAMrS,MAAMxF,KAAO,IACnB6X,EAAMrS,MAAMqW,OAAS,GACrBhE,EAAMrS,MAAMsF,MAAQvS,KAAKuS,MAAQ,KACjC+M,EAAMrS,MAAMuF,OAASxS,KAAKwS,OAAS,KACnCxS,KAAKgG,MAAMuM,MAAQvS,KAAK20B,KAAKC,SAASntB,KAAK8K,MAC3CvS,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASntB,KAAK+K,SAG5C8M,EAAMrS,MAAMtF,IAAM,GAClB2X,EAAMrS,MAAMqW,OAAS,IACrBhE,EAAMrS,MAAMxF,KAAO,IACnB6X,EAAMrS,MAAMsF,MAAQvS,KAAKuS,MAAQ,KACjC+M,EAAMrS,MAAMuF,OAASxS,KAAKwS,OAAS,KACnCxS,KAAKgG,MAAMuM,MAAQvS,KAAK20B,KAAKC,SAASvN,MAAM9U,MAC5CvS,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASvN,MAAM7U,QAG/CiyB,EAAUzkC,KAAK6oC,gBACfpE,EAAUzkC,KAAKwkC,cAAgBC,EAEL,GAAtBzkC,KAAKyO,QAAQu3B,MACfhmC,KAAK6nC,oBAGL7nC,KAAKioC,gBAGPjoC,KAAK8oC,aAAavU,GAEpB,MAAOkQ,IAOT/hC,EAASyQ,UAAU01B,cAAgB,WACjC,GAAIpE,IAAU,CACd7jC,GAAQiQ,gBAAgB7Q,KAAKumC,YAAYC,OACzC5lC,EAAQiQ,gBAAgB7Q,KAAKumC,YAAYE,OAEzC,IAAIlS,GAAcv0B,KAAKyO,QAAqB,YAGxC2sB,EAAcp7B,KAAKgnC,OAAShnC,KAAKgG,MAAMwiC,iBAAmB,GAAKxoC,KAAK6mC,iBAEpE1e,EAAO,GAAIvmB,GACb5B,KAAKy1B,MAAM7lB,MACX5P,KAAKy1B,MAAM5lB,IACXurB,EACAp7B,KAAK+vB,IAAIzQ,MAAMgR,aACftwB,KAAKyO,QAAQ6sB,YAAYt7B,KAAKyO,QAAQ8lB,aACvB,GAAfv0B,KAAKgnC,QAAmBhnC,KAAKyO,QAAQ8sB,WAGvCv7B,MAAKmoB,KAAOA,CAGZ,IAAIye,IAAc5mC,KAAK+vB,IAAIzQ,MAAMgR,aAAgBnI,EAAKyT,WAAa57B,KAAK+vB,IAAIzQ,MAAMgR,aAAenI,EAAKwU,gBAAoBxU,EAAKwU,YAAcxU,EAAKyT,WAAazT,EAAKA,KAEpKnoB,MAAK4mC,WAAaA,CAElB,IAAImC,GAAgB/oC,KAAKwS,OAASo0B,EAC9BoC,EAAiB,CAGrB,IAAmB,GAAfhpC,KAAKgnC,OAAiB,CACxBJ,EAAa5mC,KAAK6mC,iBAClBmC,EAAiB9jC,KAAKwoB,MAAO1tB,KAAK+vB,IAAIzQ,MAAMgR,aAAesW,EAAcmC,EACzE,KAAK,GAAIvjC,GAAI,EAAO,GAAMwjC,EAAVxjC,EAA0BA,IACxC2iB,EAAK2U,UAIP,IAFAiM,EAAgB/oC,KAAKwS,OAASo0B,EAEL,IAArB5mC,KAAK8mC,cAAiD,GAA3B9mC,KAAKyO,QAAQ8sB,WAAoB,CAC9D,GAAI0N,GAAsB9gB,EAAKwT,UAAYxT,EAAKA,KAAQnoB,KAAK8mC,YAC7D,IAAImC,EAAqB,EACvB,IAAK,GAAIzjC,GAAI,EAAOyjC,EAAJzjC,EAAwBA,IAAM2iB,EAAKE,WAEhD,IAAyB,EAArB4gB,EACP,IAAK,GAAIzjC,GAAI,GAAQyjC,EAALzjC,EAAyBA,IAAM2iB,EAAK2U,gBAKxDiM,IAAiB,GAInB/oC,MAAKkpC,YAAc/gB,EAAKwT,SACxB,IAMIoB,GANAoM,EAAiB,EAGjBz8B,EAAM,CAI8BlG,UAArCxG,KAAKyO,QAAQ6yB,OAAO/M,KACrBwI,EAAW/8B,KAAKyO,QAAQ6yB,OAAO/M,GAAawI,UAG9C/8B,KAAKopC,aAAe,CAEpB,KADA,GAAIp3B,GAAI,EACDtF,EAAMxH,KAAKwoB,MAAMqb,IAAgB,CACtC5gB,EAAKE,OACLrW,EAAI9M,KAAKwoB,MAAMhhB,EAAMk6B,GACrBuC,EAAiBz8B,EAAMk6B,CACvB,IAAI1J,GAAU/U,EAAK+U,WAEfl9B,KAAKyO,QAAyB,iBAAgB,GAAXyuB,GAAmC,GAAfl9B,KAAKgnC,QAAsD,GAAnChnC,KAAKyO,QAAyB,kBAC/GzO,KAAKqpC,aAAar3B,EAAI,EAAGmW,EAAKC,WAAW2U,GAAWxI,EAAa,cAAev0B,KAAKgG,MAAMsiC,iBAGzFpL,GAAWl9B,KAAKyO,QAAyB,iBAAoB,GAAfzO,KAAKgnC,QAChB,GAAnChnC,KAAKyO,QAAyB,iBAA6B,GAAfzO,KAAKgnC,QAA8B,GAAX9J,GAClElrB,GAAK,GACPhS,KAAKqpC,aAAar3B,EAAI,EAAGmW,EAAKC,WAAW2U,GAAWxI,EAAa,cAAev0B,KAAKgG,MAAMwiC,iBAE7FxoC,KAAKspC,YAAYt3B,EAAGuiB,EAAa,wBAAyBv0B,KAAKyO,QAAQw3B,iBAAkBjmC,KAAKgG,MAAM2iC,iBAGpG3oC,KAAKspC,YAAYt3B,EAAGuiB,EAAa,wBAAyBv0B,KAAKyO,QAAQy3B,iBAAkBlmC,KAAKgG,MAAMyiC,gBAGnF,GAAfzoC,KAAKgnC,QAAkC,GAAhB7e,EAAK2R,UAC9B95B,KAAK8mC,aAAep6B,GAGtBA,IAIA1M,KAAK0mC,iBADY,GAAf1mC,KAAKgnC,OACiBh1B,GAAKhS,KAAKkpC,YAAc/gB,EAAK2R,SAG7B95B,KAAK+vB,IAAIzQ,MAAMgR,aAAenI,EAAKwU,WAI7D,IAAI4M,GAAa,CACuB/iC,UAApCxG,KAAKyO,QAAQu2B,MAAMzQ,IAAuE/tB,SAAzCxG,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,OACnFggB,EAAavpC,KAAKgG,MAAMwjC,gBAE1B,IAAI7f,GAA+B,GAAtB3pB,KAAKyO,QAAQu3B,MAAgB9gC,KAAKwH,IAAI1M,KAAKyO,QAAQ43B,UAAWkD,GAAcvpC,KAAKyO,QAAQ03B,aAAe,GAAKoD,EAAavpC,KAAKyO,QAAQ03B,aAAe,EA0BnK,OAvBInmC,MAAKopC,aAAgBppC,KAAKuS,MAAQoX,GAAmC,GAAxB3pB,KAAKyO,QAAQia,SAC5D1oB,KAAKuS,MAAQvS,KAAKopC,aAAezf,EACjC3pB,KAAKyO,QAAQ8D,MAAQvS,KAAKuS,MAAQ,KAClC3R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzCzmC,KAAKyhB,SACLgjB,GAAU,GAGHzkC,KAAKopC,aAAgBppC,KAAKuS,MAAQoX,GAAmC,GAAxB3pB,KAAKyO,QAAQia,SAAmB1oB,KAAKuS,MAAQvS,KAAK2mC,UACtG3mC,KAAKuS,MAAQrN,KAAKwH,IAAI1M,KAAK2mC,SAAS3mC,KAAKopC,aAAezf,GACxD3pB,KAAKyO,QAAQ8D,MAAQvS,KAAKuS,MAAQ,KAClC3R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzCzmC,KAAKyhB,SACLgjB,GAAU,IAGV7jC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYC,OACzC5lC,EAAQsQ,gBAAgBlR,KAAKumC,YAAYE,QACzChC,GAAU,GAGLA,GAGT/hC,EAASyQ,UAAUs2B,aAAe,SAAUpiC,GAC1C,GAAIqiC,GAAgB1pC,KAAKkpC,YAAc7hC,EACnCsiC,EAAiBD,EAAgB1pC,KAAK0mC,gBAC1C,OAAOiD,IAYTjnC,EAASyQ,UAAUk2B,aAAe,SAAUr3B,EAAGuX,EAAMgL,EAAa1sB,EAAW+hC,GAE3E,GAAInhB,GAAQ7nB,EAAQ8Q,cAAc,MAAM1R,KAAKumC,YAAYE,OAAQzmC,KAAK+vB,IAAIzQ,MAC1EmJ,GAAM5gB,UAAYA,EAClB4gB,EAAMxE,UAAYsF,EACC,QAAfgL,GACF9L,EAAMxb,MAAMxF,KAAO,IAAMzH,KAAKyO,QAAQ03B,aAAe,KACrD1d,EAAMxb,MAAMqb,UAAY,UAGxBG,EAAMxb,MAAMoa,MAAQ,IAAMrnB,KAAKyO,QAAQ03B,aAAe,KACtD1d,EAAMxb,MAAMqb,UAAY,QAG1BG,EAAMxb,MAAMtF,IAAMqK,EAAI,GAAM43B,EAAkB5pC,KAAKyO,QAAQ23B,aAAe,KAE1E7c,GAAQ,EAER,IAAIsgB,GAAe3kC,KAAKwH,IAAI1M,KAAKgG,MAAM8jC,eAAe9pC,KAAKgG,MAAM+jC,eAC7D/pC,MAAKopC,aAAe7f,EAAK5jB,OAASkkC,IACpC7pC,KAAKopC,aAAe7f,EAAK5jB,OAASkkC,IAYtCnnC,EAASyQ,UAAUm2B,YAAc,SAAUt3B,EAAGuiB,EAAa1sB,EAAW8hB,EAAQpX,GAC5E,GAAmB,GAAfvS,KAAKgnC,OAAgB,CACvB,GAAInX,GAAOjvB,EAAQ8Q,cAAc,MAAM1R,KAAKumC,YAAYC,MAAOxmC,KAAK+vB,IAAIqX,cACxEvX,GAAKhoB,UAAYA,EACjBgoB,EAAK5L,UAAY,GAEE,QAAfsQ,EACF1E,EAAK5iB,MAAMxF,KAAQzH,KAAKuS,MAAQoX,EAAU,KAG1CkG,EAAK5iB,MAAMoa,MAASrnB,KAAKuS,MAAQoX,EAAU,KAG7CkG,EAAK5iB,MAAMsF,MAAQA,EAAQ,KAC3Bsd,EAAK5iB,MAAMtF,IAAMqK,EAAI,OASzBtP,EAASyQ,UAAU21B,aAAe,SAAUvU,GAI1C,GAHA3zB,EAAQiQ,gBAAgB7Q,KAAKumC,YAAYvB,OAGDx+B,SAApCxG,KAAKyO,QAAQu2B,MAAMzQ,IAAuE/tB,SAAzCxG,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,KAAoB,CACvG,GAAIyb,GAAQpkC,EAAQ8Q,cAAc,MAAO1R,KAAKumC,YAAYvB,MAAOhlC,KAAK+vB,IAAIzQ,MAC1E0lB,GAAMn9B,UAAY,eAAiB0sB,EACnCyQ,EAAM/gB,UAAYjkB,KAAKyO,QAAQu2B,MAAMzQ,GAAahL,KAGJ/iB,SAA1CxG,KAAKyO,QAAQu2B,MAAMzQ,GAAatnB,OAClCtM,EAAK2M,WAAW03B,EAAOhlC,KAAKyO,QAAQu2B,MAAMzQ,GAAatnB,OAGtC,QAAfsnB,EACFyQ,EAAM/3B,MAAMxF,KAAOzH,KAAKgG,MAAMwjC,gBAAkB,KAGhDxE,EAAM/3B,MAAMoa,MAAQrnB,KAAKgG,MAAMwjC,gBAAkB,KAGnDxE,EAAM/3B,MAAMsF,MAAQvS,KAAKwS,OAAS,KAIpC5R,EAAQsQ,gBAAgBlR,KAAKumC,YAAYvB,QAW3CtiC,EAASyQ,UAAUi1B,mBAAqB,WAEtC,KAAM,mBAAqBpoC,MAAKgG,OAAQ,CACtC,GAAIgkC,GAAYz4B,SAAS04B,eAAe,KACpCC,EAAmB34B,SAASM,cAAc,MAC9Cq4B,GAAiBriC,UAAY,sBAC7BqiC,EAAiBz4B,YAAYu4B,GAC7BhqC,KAAK+vB,IAAIzQ,MAAM7N,YAAYy4B,GAE3BlqC,KAAKgG,MAAMsiC,gBAAkB4B,EAAiBrlB,aAC9C7kB,KAAKgG,MAAM+jC,eAAiBG,EAAiB1qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAY+4B,GAG7B,KAAM,mBAAqBlqC,MAAKgG,OAAQ,CACtC,GAAImkC,GAAY54B,SAAS04B,eAAe,KACpCG,EAAmB74B,SAASM,cAAc,MAC9Cu4B,GAAiBviC,UAAY,sBAC7BuiC,EAAiB34B,YAAY04B,GAC7BnqC,KAAK+vB,IAAIzQ,MAAM7N,YAAY24B,GAE3BpqC,KAAKgG,MAAMwiC,gBAAkB4B,EAAiBvlB,aAC9C7kB,KAAKgG,MAAM8jC,eAAiBM,EAAiB5qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAYi5B,GAG7B,KAAM,mBAAqBpqC,MAAKgG,OAAQ,CACtC,GAAIqkC,GAAY94B,SAAS04B,eAAe,KACpCK,EAAmB/4B,SAASM,cAAc,MAC9Cy4B,GAAiBziC,UAAY,sBAC7ByiC,EAAiB74B,YAAY44B,GAC7BrqC,KAAK+vB,IAAIzQ,MAAM7N,YAAY64B,GAE3BtqC,KAAKgG,MAAMwjC,gBAAkBc,EAAiBzlB,aAC9C7kB,KAAKgG,MAAMukC,eAAiBD,EAAiB9qB,YAE7Cxf,KAAK+vB,IAAIzQ,MAAMnO,YAAYm5B,KAU/B5nC,EAASyQ,UAAU6hB,KAAO,SAASyD,GACjC,MAAOz4B,MAAKmoB,KAAK6M,KAAKyD,IAGxB54B,EAAOD,QAAU8C,GAKb,SAAS7C,EAAQD,EAASM,GAkB9B,QAASyC,GAAYsP,EAAOqlB,EAAS7oB,EAAS+7B,GAC5CxqC,KAAKK,GAAKi3B,CACV,IAAIppB,IAAU,WAAW,QAAQ,OAAO,mBAAmB,WAAW,aAAa,SAAS,aAC5FlO,MAAKyO,QAAU9N,EAAKsN,sBAAsBC,EAAOO,GACjDzO,KAAKyqC,kBAAwCjkC,SAApByL,EAAMpK,UAC/B7H,KAAKwqC,yBAA2BA,EAChCxqC,KAAK0qC,aAAe,EACpB1qC,KAAK6U,OAAO5C,GACkB,GAA1BjS,KAAKyqC,oBACPzqC,KAAKwqC,yBAAyB,IAAM,GAEtCxqC,KAAK81B,aACL91B,KAAK0oB,QAA4BliB,SAAlByL,EAAMyW,SAAwB,EAAOzW,EAAMyW,QA5B5D,GAAI/nB,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9ByqC,EAAOzqC,EAAoB,IAC3B0qC,EAAM1qC,EAAoB,IAC1B2qC,EAAS3qC,EAAoB,GAgCjCyC,GAAWwQ,UAAU8iB,SAAW,SAASh0B,GAC1B,MAATA,GACFjC,KAAK81B,UAAY7zB,EACQ,GAArBjC,KAAKyO,QAAQyH,MACflW,KAAK81B,UAAU5f,KAAK,SAAU3Q,EAAEa,GAAI,MAAOb,GAAEwM,EAAI3L,EAAE2L,KAIrD/R,KAAK81B,cASTnzB,EAAWwQ,UAAU23B,gBAAkB,SAASvlB,GAC9CvlB,KAAK0qC,aAAenlB,GAQtB5iB,EAAWwQ,UAAUD,WAAa,SAASzE,GACzC,GAAgBjI,SAAZiI,EAAuB,CACzB,GAAIP,IAAU,WAAW,QAAQ,OAAO,mBAAmB,WAC3DvN,GAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,GAE/C9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UAEpCA,EAAQs8B,YACuB,gBAAtBt8B,GAAQs8B,YACbt8B,EAAQs8B,WAAWC,kBACqB,WAAtCv8B,EAAQs8B,WAAWC,gBACrBhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,EAEa,WAAtCx8B,EAAQs8B,WAAWC,gBAC1BhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,GAGhCjrC,KAAKyO,QAAQs8B,WAAWC,gBAAkB,cAC1ChrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,KAOhB,QAAtBjrC,KAAKyO,QAAQxB,MACfjN,KAAK8G,KAAO,GAAI6jC,GAAK3qC,KAAKK,GAAIL,KAAKyO,SAEN,OAAtBzO,KAAKyO,QAAQxB,MACpBjN,KAAK8G,KAAO,GAAI8jC,GAAI5qC,KAAKK,GAAIL,KAAKyO,SAEL,UAAtBzO,KAAKyO,QAAQxB,QACpBjN,KAAK8G,KAAO,GAAI+jC,GAAO7qC,KAAKK,GAAIL,KAAKyO,WASzC9L,EAAWwQ,UAAU0B,OAAS,SAAS5C,GACrCjS,KAAKiS,MAAQA,EACbjS,KAAK4vB,QAAU3d,EAAM2d,SAAW,QAChC5vB,KAAK6H,UAAYoK,EAAMpK,WAAa7H,KAAK6H,WAAa,aAAe7H,KAAKwqC,yBAAyB,GAAK,GACxGxqC,KAAK0oB,QAA4BliB,SAAlByL,EAAMyW,SAAwB,EAAOzW,EAAMyW,QAC1D1oB,KAAKiN,MAAQgF,EAAMhF,MACnBjN,KAAKkT,WAAWjB,EAAMxD,UAcxB9L,EAAWwQ,UAAU60B,SAAW,SAASj2B,EAAGC,EAAGlB,EAAeo6B,EAAc7E,EAAWyB,GACrF,GACIqD,GAAMC,EADNC,EAA0B,GAAbvD,EAGbwD,EAAU1qC,EAAQwQ,cAAc,OAAQN,EAAeo6B,EAO3D,IANAI,EAAQl5B,eAAe,KAAM,IAAKL,GAClCu5B,EAAQl5B,eAAe,KAAM,IAAKJ,EAAIq5B,GACtCC,EAAQl5B,eAAe,KAAM,QAASi0B,GACtCiF,EAAQl5B,eAAe,KAAM,SAAU,EAAEi5B,GACzCC,EAAQl5B,eAAe,KAAM,QAAS,WAEZ,QAAtBpS,KAAKyO,QAAQxB,MACfk+B,EAAOvqC,EAAQwQ,cAAc,OAAQN,EAAeo6B,GACpDC,EAAK/4B,eAAe,KAAM,QAASpS,KAAK6H,WACtBrB,SAAfxG,KAAKiN,OACNk+B,EAAK/4B,eAAe,KAAM,QAASpS,KAAKiN,OAG1Ck+B,EAAK/4B,eAAe,KAAM,IAAK,IAAML,EAAI,IAAIC,EAAE,MAAQD,EAAIs0B,GAAa,IAAIr0B,GACzC,GAA/BhS,KAAKyO,QAAQ88B,OAAO78B,UACtB08B,EAAWxqC,EAAQwQ,cAAc,OAAQN,EAAeo6B,GACjB,OAAnClrC,KAAKyO,QAAQ88B,OAAOhX,YACtB6W,EAASh5B,eAAe,KAAM,IAAK,IAAIL,EAAE,MAAQC,EAAIq5B,GACnD,IAAIt5B,EAAE,IAAIC,EAAE,MAAOD,EAAIs0B,GAAa,IAAIr0B,EAAE,MAAOD,EAAIs0B,GAAa,KAAOr0B,EAAIq5B,IAG/ED,EAASh5B,eAAe,KAAM,IAAK,IAAIL,EAAE,IAAIC,EAAE,KACzCD,EAAE,KAAOC,EAAIq5B,GAAc,MACzBt5B,EAAIs0B,GAAa,KAAOr0B,EAAIq5B,GAClC,KAAMt5B,EAAIs0B,GAAa,IAAIr0B,GAE/Bo5B,EAASh5B,eAAe,KAAM,QAASpS,KAAK6H,UAAY,cAGnB,GAAnC7H,KAAKyO,QAAQ0D,WAAWzD,SAC1B9N,EAAQkR,UAAUC,EAAI,GAAMs0B,EAAUr0B,EAAGhS,KAAM8Q,EAAeo6B,OAG7D,CACH,GAAIM,GAAWtmC,KAAKwoB,MAAM,GAAM2Y,GAC5BoF,EAAavmC,KAAKwoB,MAAM,GAAMoa,GAC9B4D,EAAaxmC,KAAKwoB,MAAM,IAAOoa,GAE/Bne,EAASzkB,KAAKwoB,OAAO2Y,EAAa,EAAImF,GAAW,EAErD5qC,GAAQ0R,QAAQP,EAAI,GAAIy5B,EAAW7hB,EAAY3X,EAAIq5B,EAAaI,EAAa,EAAGD,EAAUC,EAAYzrC,KAAK6H,UAAY,OAAQiJ,EAAeo6B,GAC9ItqC,EAAQ0R,QAAQP,EAAI,IAAIy5B,EAAW7hB,EAAS,EAAG3X,EAAIq5B,EAAaK,EAAa,EAAGF,EAAUE,EAAY1rC,KAAK6H,UAAY,OAAQiJ,EAAeo6B,KAYlJvoC,EAAWwQ,UAAUkkB,UAAY,SAASgP,EAAWyB,GACnD,GAAIlC,GAAMr0B,SAASC,gBAAgB,6BAA6B,MAEhE,OADAxR,MAAKgoC,SAAS,EAAE,GAAIF,KAAclC,EAAIS,EAAUyB,IACxC6D,KAAM/F,EAAKnd,MAAOzoB,KAAK4vB,QAAS2E,YAAYv0B,KAAKyO,QAAQm9B,mBAGnEjpC,EAAWwQ,UAAU04B,UAAY,SAASC,GACxC,MAAO9rC,MAAK8G,KAAK+kC,UAAUC,IAG7BnpC,EAAWwQ,UAAU44B,KAAO,SAAS/U,EAAS/kB,EAAO+5B,GACnDhsC,KAAK8G,KAAKilC,KAAK/U,EAAS/kB,EAAO+5B,IAIjCnsC,EAAOD,QAAU+C,GAKb,SAAS9C,EAAQD,EAASM,GAY9B,QAAS0C,GAAO00B,EAAS5kB,EAAMmjB,GAC7B71B,KAAKs3B,QAAUA,EACft3B,KAAKmhC,aACLnhC,KAAKisC,cAAgB,EACrBjsC,KAAKksC,gBAAkBx5B,GAAQA,EAAKy5B,cACpCnsC,KAAK61B,QAAUA,EAEf71B,KAAK+vB,OACL/vB,KAAKgG,OACHyiB,OACElW,MAAO,EACPC,OAAQ,IAGZxS,KAAK6H,UAAY,KAEjB7H,KAAKiC,SACLjC,KAAKosC,gBACLpsC,KAAK4O,cACHy9B,WACAC,UAEFtsC,KAAKusC,kBAAmB,CACxB,IAAIp4B,GAAKnU,IACTA,MAAK61B,QAAQlB,KAAKE,QAAQthB,GAAG,mBAAoB,WAC/CY,EAAGo4B,kBAAmB,IAGxBvsC,KAAK00B,UAEL10B,KAAKgY,QAAQtF,GAxCf,CAAA,GAAI/R,GAAOT,EAAoB,GAC3B4B,EAAQ5B,EAAoB,GAChBA,GAAoB,IA6CpC0C,EAAMuQ,UAAUuhB,QAAU,WACxB,GAAIjM,GAAQlX,SAASM,cAAc,MACnC4W,GAAM5gB,UAAY,SAClB7H,KAAK+vB,IAAItH,MAAQA,CAEjB,IAAI+jB,GAAQj7B,SAASM,cAAc,MACnC26B,GAAM3kC,UAAY,QAClB4gB,EAAMhX,YAAY+6B,GAClBxsC,KAAK+vB,IAAIyc,MAAQA,CAEjB,IAAIC,GAAal7B,SAASM,cAAc,MACxC46B,GAAW5kC,UAAY,QACvB4kC,EAAW,kBAAoBzsC,KAC/BA,KAAK+vB,IAAI0c,WAAaA,EAEtBzsC,KAAK+vB,IAAI5jB,WAAaoF,SAASM,cAAc,OAC7C7R,KAAK+vB,IAAI5jB,WAAWtE,UAAY,QAEhC7H,KAAK+vB,IAAI+Q,KAAOvvB,SAASM,cAAc,OACvC7R,KAAK+vB,IAAI+Q,KAAKj5B,UAAY,QAK1B7H,KAAK+vB,IAAI2c,OAASn7B,SAASM,cAAc,OACzC7R,KAAK+vB,IAAI2c,OAAOz/B,MAAMuqB,WAAa,SACnCx3B,KAAK+vB,IAAI2c,OAAOzoB,UAAY,IAC5BjkB,KAAK+vB,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI2c,SAO3C9pC,EAAMuQ,UAAU6E,QAAU,SAAStF,GAEjC,GAAIkd,GAAUld,GAAQA,EAAKkd,OACvBA,aAAmB+c,SACrB3sC,KAAK+vB,IAAIyc,MAAM/6B,YAAYme,GAG3B5vB,KAAK+vB,IAAIyc,MAAMvoB,UADIzd,SAAZopB,GAAqC,OAAZA,EACLA,EAGA5vB,KAAKs3B,SAAW,GAI7Ct3B,KAAK+vB,IAAItH,MAAMuc,MAAQtyB,GAAQA,EAAKsyB,OAAS,GAExChlC,KAAK+vB,IAAIyc,MAAM7oB,WAIlBhjB,EAAKuH,gBAAgBlI,KAAK+vB,IAAIyc,MAAO,UAHrC7rC,EAAKiH,aAAa5H,KAAK+vB,IAAIyc,MAAO,SAOpC,IAAI3kC,GAAY6K,GAAQA,EAAK7K,WAAa,IACtCA,IAAa7H,KAAK6H,YAChB7H,KAAK6H,YACPlH,EAAKuH,gBAAgBlI,KAAK+vB,IAAItH,MAAOzoB,KAAK6H,WAC1ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI0c,WAAYzsC,KAAK6H,WAC/ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI5jB,WAAYnM,KAAK6H,WAC/ClH,EAAKuH,gBAAgBlI,KAAK+vB,IAAI+Q,KAAM9gC,KAAK6H,YAE3ClH,EAAKiH,aAAa5H,KAAK+vB,IAAItH,MAAO5gB,GAClClH,EAAKiH,aAAa5H,KAAK+vB,IAAI0c,WAAY5kC,GACvClH,EAAKiH,aAAa5H,KAAK+vB,IAAI5jB,WAAYtE,GACvClH,EAAKiH,aAAa5H,KAAK+vB,IAAI+Q,KAAMj5B,GACjC7H,KAAK6H,UAAYA,GAIf7H,KAAKiN,QACPtM,EAAK8M,cAAczN,KAAK+vB,IAAItH,MAAOzoB,KAAKiN,OACxCjN,KAAKiN,MAAQ,MAEXyF,GAAQA,EAAKzF,QACftM,EAAK2M,WAAWtN,KAAK+vB,IAAItH,MAAO/V,EAAKzF,OACrCjN,KAAKiN,MAAQyF,EAAKzF,QAQtBrK,EAAMuQ,UAAUy5B,cAAgB,WAC9B,MAAO5sC,MAAKgG,MAAMyiB,MAAMlW,OAW1B3P,EAAMuQ,UAAUsO,OAAS,SAASgU,EAAO/b,EAAQmzB,GAC/C,GAAIpI,IAAU,CAEdzkC,MAAKosC,aAAepsC,KAAK8sC,oBAAoB9sC,KAAK4O,aAAc5O,KAAKosC,aAAc3W,EAInF,IAAIsX,GAAe/sC,KAAK+vB,IAAI2c,OAAO7nB,YAC/BkoB,IAAgB/sC,KAAKgtC,mBACvBhtC,KAAKgtC,iBAAmBD,EAExBpsC,EAAK0H,QAAQrI,KAAKiC,MAAO,SAAUoN,GACjCA,EAAK49B,OAAQ,EACT59B,EAAK69B,WAAW79B,EAAKoS,WAG3BorB,GAAU,GAIR7sC,KAAK61B,QAAQpnB,QAAQ3M,MACvBA,EAAMA,MAAM9B,KAAKosC,aAAc1yB,EAAQmzB,GAGvC/qC,EAAMo/B,QAAQlhC,KAAKosC,aAAc1yB,EAAQ1Z,KAAKmhC,UAIhD,IAAI3uB,GAASxS,KAAKmtC,iBAAiBzzB,GAG/B+yB,EAAazsC,KAAK+vB,IAAI0c,UAC1BzsC,MAAK2H,IAAM8kC,EAAWW,UACtBptC,KAAKyH,KAAOglC,EAAWY,WACvBrtC,KAAKuS,MAAQk6B,EAAWrc,YACxBqU,EAAU9jC,EAAK8H,eAAezI,KAAM,SAAUwS,IAAWiyB,EAGzDA,EAAU9jC,EAAK8H,eAAezI,KAAKgG,MAAMyiB,MAAO,QAASzoB,KAAK+vB,IAAIyc,MAAMhtB,cAAgBilB,EACxFA,EAAU9jC,EAAK8H,eAAezI,KAAKgG,MAAMyiB,MAAO,SAAUzoB,KAAK+vB,IAAIyc,MAAM3nB,eAAiB4f,EAG1FzkC,KAAK+vB,IAAI5jB,WAAWc,MAAMuF,OAAUA,EAAS,KAC7CxS,KAAK+vB,IAAI0c,WAAWx/B,MAAMuF,OAAUA,EAAS,KAC7CxS,KAAK+vB,IAAItH,MAAMxb,MAAMuF,OAASA,EAAS,IAGvC,KAAK,GAAIhN,GAAI,EAAG8nC,EAAKttC,KAAKosC,aAAazmC,OAAY2nC,EAAJ9nC,EAAQA,IAAK,CAC1D,GAAI6J,GAAOrP,KAAKosC,aAAa5mC,EAC7B6J,GAAKk+B,YAAY7zB,GAGnB,MAAO+qB,IAST7hC,EAAMuQ,UAAUg6B,iBAAmB,SAAUzzB,GAE3C,GAAIlH,GACA45B,EAAepsC,KAAKosC,YAGxBpsC,MAAKwtC,gBACL,IAAIr5B,GAAKnU,IACT,IAAIosC,EAAazmC,OAAQ,CACvB,GAAImG,GAAMsgC,EAAa,GAAGzkC,IACtB+E,EAAM0/B,EAAa,GAAGzkC,IAAMykC,EAAa,GAAG55B,MAahD,IAZA7R,EAAK0H,QAAQ+jC,EAAc,SAAU/8B,GACnCvD,EAAM5G,KAAK4G,IAAIA,EAAKuD,EAAK1H,KACzB+E,EAAMxH,KAAKwH,IAAIA,EAAM2C,EAAK1H,IAAM0H,EAAKmD,QACVhM,SAAvB6I,EAAKqD,KAAK2uB,WACZltB,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU7uB,OAAStN,KAAKwH,IAAIyH,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU7uB,OAAOnD,EAAKmD,QAChG2B,EAAGgtB,UAAU9xB,EAAKqD,KAAK2uB,UAAU3Y,SAAU,KAO3C5c,EAAM4N,EAAOonB,KAAM,CAErB,GAAInX,GAAS7d,EAAM4N,EAAOonB,IAC1Bp0B,IAAOid,EACPhpB,EAAK0H,QAAQ+jC,EAAc,SAAU/8B,GACnCA,EAAK1H,KAAOgiB,IAGhBnX,EAAS9F,EAAMgN,EAAOrK,KAAKoW,SAAW,MAGtCjT,GAASkH,EAAOonB,KAAOpnB,EAAOrK,KAAKoW,QAIrC,OAFAjT,GAAStN,KAAKwH,IAAI8F,EAAQxS,KAAKgG,MAAMyiB,MAAMjW,SAQ7C5P,EAAMuQ,UAAUw0B,KAAO,WAChB3nC,KAAK+vB,IAAItH,MAAM5e,YAClB7J,KAAK61B,QAAQ9F,IAAI0d,SAASh8B,YAAYzR,KAAK+vB,IAAItH,OAG5CzoB,KAAK+vB,IAAI0c,WAAW5iC,YACvB7J,KAAK61B,QAAQ9F,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAI0c,YAG9CzsC,KAAK+vB,IAAI5jB,WAAWtC,YACvB7J,KAAK61B,QAAQ9F,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI5jB,YAG9CnM,KAAK+vB,IAAI+Q,KAAKj3B,YACjB7J,KAAK61B,QAAQ9F,IAAI+Q,KAAKrvB,YAAYzR,KAAK+vB,IAAI+Q,OAO/Cl+B,EAAMuQ,UAAUu0B,KAAO,WACrB,GAAIjf,GAAQzoB,KAAK+vB,IAAItH,KACjBA,GAAM5e,YACR4e,EAAM5e,WAAWsH,YAAYsX,EAG/B,IAAIgkB,GAAazsC,KAAK+vB,IAAI0c,UACtBA,GAAW5iC,YACb4iC,EAAW5iC,WAAWsH,YAAYs7B,EAGpC,IAAItgC,GAAanM,KAAK+vB,IAAI5jB,UACtBA,GAAWtC,YACbsC,EAAWtC,WAAWsH,YAAYhF,EAGpC,IAAI20B,GAAO9gC,KAAK+vB,IAAI+Q,IAChBA,GAAKj3B,YACPi3B,EAAKj3B,WAAWsH,YAAY2vB,IAQhCl+B,EAAMuQ,UAAUF,IAAM,SAAS5D,GAc7B,GAbArP,KAAKiC,MAAMoN,EAAKhP,IAAMgP,EACtBA,EAAKq+B,UAAU1tC,MAGYwG,SAAvB6I,EAAKqD,KAAK2uB,WAC+B76B,SAAvCxG,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,YAC3BrhC,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,WAAa7uB,OAAO,EAAGkW,SAAS,EAAOvgB,MAAMnI,KAAKisC,cAAehqC,UAC1FjC,KAAKisC,iBAEPjsC,KAAKmhC,UAAU9xB,EAAKqD,KAAK2uB,UAAUp/B,MAAM+F,KAAKqH,IAEhDrP,KAAK2tC,iBAEkC,IAAnC3tC,KAAKosC,aAAazlC,QAAQ0I,GAAa,CACzC,GAAIomB,GAAQz1B,KAAK61B,QAAQlB,KAAKc,KAC9Bz1B,MAAK4tC,gBAAgBv+B,EAAMrP,KAAKosC,aAAc3W,KAIlD7yB,EAAMuQ,UAAUw6B,eAAiB,WAC/B,GAA6BnnC,SAAzBxG,KAAKksC,gBAA+B,CACtC,GAAI2B,KACJ,IAAmC,gBAAxB7tC,MAAKksC,gBAA6B,CAC3C,IAAK,GAAI7K,KAAYrhC,MAAKmhC,UACxB0M,EAAU7lC,MAAMq5B,SAAUA,EAAUyM,UAAW9tC,KAAKmhC,UAAUE,GAAUp/B,MAAM,GAAGyQ,KAAK1S,KAAKksC,kBAE7F2B,GAAU33B,KAAK,SAAU3Q,EAAGa,GAC1B,MAAOb,GAAEuoC,UAAY1nC,EAAE0nC,gBAGtB,IAAmC,kBAAxB9tC,MAAKksC,gBAA+B,CAClD,IAAK,GAAI7K,KAAYrhC,MAAKmhC,UACxB0M,EAAU7lC,KAAKhI,KAAKmhC,UAAUE,GAAUp/B,MAAM,GAAGyQ,KAEnDm7B,GAAU33B,KAAKlW,KAAKksC,iBAGtB,GAAI2B,EAAUloC,OAAS,EACrB,IAAK,GAAIH,GAAI,EAAGA,EAAIqoC,EAAUloC,OAAQH,IACpCxF,KAAKmhC,UAAU0M,EAAUroC,GAAG67B,UAAUl5B,MAAQ3C,IAMtD5C,EAAMuQ,UAAUq6B,eAAiB,WAC/B,IAAK,GAAInM,KAAYrhC,MAAKmhC,UACpBnhC,KAAKmhC,UAAUr7B,eAAeu7B,KAChCrhC,KAAKmhC,UAAUE,GAAU3Y,SAAU,IASzC9lB,EAAMuQ,UAAUkD,OAAS,SAAShH,SACzBrP,MAAKiC,MAAMoN,EAAKhP,IACvBgP,EAAKq+B,UAAU,KAGf,IAAIvlC,GAAQnI,KAAKosC,aAAazlC,QAAQ0I,EACzB,KAATlH,GAAanI,KAAKosC,aAAahkC,OAAOD,EAAO,IAUnDvF,EAAMuQ,UAAU46B,kBAAoB,SAAS1+B,GAC3CrP,KAAK61B,QAAQmY,WAAW3+B,EAAKhP,KAO/BuC,EAAMuQ,UAAUsC,MAAQ,WAKtB,IAAK,GAJDjN,GAAQ7H,EAAK4H,QAAQvI,KAAKiC,OAC1BgsC,KACAC,KAEK1oC,EAAI,EAAGA,EAAIgD,EAAM7C,OAAQH,IACNgB,SAAtBgC,EAAMhD,GAAGkN,KAAK7C,KAChBq+B,EAASlmC,KAAKQ,EAAMhD,IAEtByoC,EAAWjmC,KAAKQ,EAAMhD,GAExBxF,MAAK4O,cACHy9B,QAAS4B,EACT3B,MAAO4B,GAGTpsC,EAAM0+B,aAAaxgC,KAAK4O,aAAay9B,SACrCvqC,EAAM2+B,WAAWzgC,KAAK4O,aAAa09B,QAYrC1pC,EAAMuQ,UAAU25B,oBAAsB,SAASl+B,EAAcu/B,EAAiB1Y,GAC5E,GAKIpmB,GAAM7J,EALN4mC,KACAgC,KACA5b,GAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,EACvCy+B,EAAa5Y,EAAM7lB,MAAQ4iB,EAC3B8b,EAAa7Y,EAAM5lB,IAAM2iB,EAIzB3jB,EAAiB,SAAUxH,GAC7B,MAAiBgnC,GAARhnC,EAA6B,GACpBinC,GAATjnC,EAA8B,EACA,EAMzC,IAAI8mC,EAAgBxoC,OAAS,EAC3B,IAAKH,EAAI,EAAGA,EAAI2oC,EAAgBxoC,OAAQH,IACtCxF,KAAKuuC,6BAA6BJ,EAAgB3oC,GAAI4mC,EAAcgC,EAAoB3Y,EAK5F,IAAI+Y,GAAoB7tC,EAAKgO,mBAAmBC,EAAay9B,QAASx9B,EAAgB,OAAO,QAS7F,IANA7O,KAAKyuC,cAAcD,EAAmB5/B,EAAay9B,QAASD,EAAcgC,EAAoB,SAAU/+B,GACtG,MAAQA,GAAKqD,KAAK9C,MAAQy+B,GAAch/B,EAAKqD,KAAK9C,MAAQ0+B,IAK/B,GAAzBtuC,KAAKusC,iBAEP,IADAvsC,KAAKusC,kBAAmB,EACnB/mC,EAAI,EAAGA,EAAIoJ,EAAa09B,MAAM3mC,OAAQH,IACzCxF,KAAKuuC,6BAA6B3/B,EAAa09B,MAAM9mC,GAAI4mC,EAAcgC,EAAoB3Y,OAG1F,CAEH,GAAIiZ,GAAkB/tC,EAAKgO,mBAAmBC,EAAa09B,MAAOz9B,EAAgB,OAAO,MAGzF7O,MAAKyuC,cAAcC,EAAiB9/B,EAAa09B,MAAOF,EAAcgC,EAAoB,SAAU/+B,GAClG,MAAQA,GAAKqD,KAAK7C,IAAMw+B,GAAch/B,EAAKqD,KAAK7C,IAAMy+B,IAM1D,IAAK9oC,EAAI,EAAGA,EAAI4mC,EAAazmC,OAAQH,IACnC6J,EAAO+8B,EAAa5mC,GACf6J,EAAK69B,WAAW79B,EAAKs4B,OAE1Bt4B,EAAKs/B,aAgBP,OAAOvC,IAGTxpC,EAAMuQ,UAAUs7B,cAAgB,SAAUG,EAAY3sC,EAAOmqC,EAAcgC,EAAoBS,GAC7F,GAAIx/B,GACA7J,CAEJ,IAAkB,IAAdopC,EAAkB,CACpB,IAAKppC,EAAIopC,EAAYppC,GAAK,IACxB6J,EAAOpN,EAAMuD,IACTqpC,EAAex/B,IAFQ7J,IAMWgB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,GAKxB,KAAK7J,EAAIopC,EAAa,EAAGppC,EAAIvD,EAAM0D,SACjC0J,EAAOpN,EAAMuD,IACTqpC,EAAex/B,IAFsB7J,IAMHgB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,MAmB5BzM,EAAMuQ,UAAUy6B,gBAAkB,SAASv+B,EAAM+8B,EAAc3W,GACvDpmB,EAAKy/B,UAAUrZ,IACZpmB,EAAK69B,WAAW79B,EAAKs4B,OAE1Bt4B,EAAKs/B,cACLvC,EAAapkC,KAAKqH,IAGdA,EAAK69B,WAAW79B,EAAKq4B,QAgB/B9kC,EAAMuQ,UAAUo7B,6BAA+B,SAASl/B,EAAM+8B,EAAcgC,EAAoB3Y,GAC1FpmB,EAAKy/B,UAAUrZ,GACmBjvB,SAAhC4nC,EAAmB/+B,EAAKhP,MAC1B+tC,EAAmB/+B,EAAKhP,KAAM,EAC9B+rC,EAAapkC,KAAKqH,IAIhBA,EAAK69B,WAAW79B,EAAKq4B;EAM7B7nC,EAAOD,QAAUgD,GAKb,SAAS/C,EAAQD,EAASM,GAW9B,QAAS2C,GAAiBy0B,EAAS5kB,EAAMmjB,GACvCjzB,EAAMrC,KAAKP,KAAMs3B,EAAS5kB,EAAMmjB,GAEhC71B,KAAKuS,MAAQ,EACbvS,KAAKwS,OAAS,EACdxS,KAAK2H,IAAM,EACX3H,KAAKyH,KAAO,EAfd,GACI7E,IADO1C,EAAoB,GACnBA,EAAoB,IAiBhC2C,GAAgBsQ,UAAY5M,OAAO8H,OAAOzL,EAAMuQ,WAShDtQ,EAAgBsQ,UAAUsO,OAAS,SAASgU,EAAO/b,GACjD,GAAI+qB,IAAU,CAEdzkC,MAAKosC,aAAepsC,KAAK8sC,oBAAoB9sC,KAAK4O,aAAc5O,KAAKosC,aAAc3W,GAGnFz1B,KAAKuS,MAAQvS,KAAK+vB,IAAI5jB,WAAWikB,YAGjCpwB,KAAK+vB,IAAI5jB,WAAWc,MAAMuF,OAAU,GAGpC,KAAK,GAAIhN,GAAI,EAAG8nC,EAAKttC,KAAKosC,aAAazmC,OAAY2nC,EAAJ9nC,EAAQA,IAAK,CAC1D,GAAI6J,GAAOrP,KAAKosC,aAAa5mC,EAC7B6J,GAAKk+B,YAAY7zB,GAGnB,MAAO+qB,IAMT5hC,EAAgBsQ,UAAUw0B,KAAO,WAC1B3nC,KAAK+vB,IAAI5jB,WAAWtC,YACvB7J,KAAK61B,QAAQ9F,IAAI5jB,WAAWsF,YAAYzR,KAAK+vB,IAAI5jB,aAIrDtM,EAAOD,QAAUiD,GAKb,SAAShD,EAAQD,EAASM,GA2B9B,QAAS4C,GAAQ6xB,EAAMlmB,GACrBzO,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHvtB,KAAM,KACNytB,YAAa,SACbwa,MAAO,OACPjtC,OAAO,EACPktC,WAAY,KAEZC,YAAY,EACZC,UACEC,YAAY,EACZ3H,aAAa,EACbv0B,KAAK,EACLoD,QAAQ,GAGV+4B,MAAO,SAAU//B,EAAM/G,GACrBA,EAAS+G,IAEXggC,SAAU,SAAUhgC,EAAM/G,GACxBA,EAAS+G,IAEXigC,OAAQ,SAAUjgC,EAAM/G,GACtBA,EAAS+G,IAEXkgC,SAAU,SAAUlgC,EAAM/G,GACxBA,EAAS+G,IAEXmgC,SAAU,SAAUngC,EAAM/G,GACxBA,EAAS+G,IAGXqK,QACErK,MACEmW,WAAY,GACZC,SAAU,IAEZqb,KAAM,IAER9c,QAAS,GAIXhkB,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAGpCr0B,KAAKyvC,aACH3oC,MAAO8I,MAAO,OAAQC,IAAK,SAG7B7P,KAAKo6B,YACHnF,SAAUN,EAAKh0B,KAAKs0B,SACpBI,OAAQV,EAAKh0B,KAAK00B,QAEpBr1B,KAAK+vB,OACL/vB,KAAKgG,SACLhG,KAAK8D,OAAS,IAEd,IAAIqQ,GAAKnU,IACTA,MAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGlB/1B,KAAK0vC,eACHz8B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAGw7B,OAAO77B,EAAO7R,QAEnB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAGy7B,UAAU97B,EAAO7R,QAEtBoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG07B,UAAU/7B,EAAO7R,SAKxBjC,KAAK8vC,gBACH78B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAG47B,aAAaj8B,EAAO7R,QAEzB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAG67B,gBAAgBl8B,EAAO7R,QAE5BoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG87B,gBAAgBn8B,EAAO7R,SAI9BjC,KAAKiC,SACLjC,KAAKm0B,UACLn0B,KAAKkwC,YAELlwC,KAAKmwC,aACLnwC,KAAKowC,YAAa,EAElBpwC,KAAKqwC,eAGLrwC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GA/HlB,GAAI1K,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAC3BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BqC,EAAYrC,EAAoB,IAChC0C,EAAQ1C,EAAoB,IAC5B2C,EAAkB3C,EAAoB,IACtCkC,EAAUlC,EAAoB,IAC9BmC,EAAYnC,EAAoB,IAChCoC,EAAYpC,EAAoB,IAChCiC,EAAiBjC,EAAoB,IAGrCowC,EAAY,gBACZC,EAAa,gBAoHjBztC,GAAQqQ,UAAY,GAAI5Q,GAGxBO,EAAQoU,OACN/K,WAAYhK,EACZquC,IAAKpuC,EACLqzB,MAAOnzB,EACP4P,MAAO7P,GAMTS,EAAQqQ,UAAUuhB,QAAU,WAC1B,GAAIpV,GAAQ/N,SAASM,cAAc,MACnCyN,GAAMzX,UAAY,UAClByX,EAAM,oBAAsBtf,KAC5BA,KAAK+vB,IAAIzQ,MAAQA,CAGjB,IAAInT,GAAaoF,SAASM,cAAc,MACxC1F,GAAWtE,UAAY,aACvByX,EAAM7N,YAAYtF,GAClBnM,KAAK+vB,IAAI5jB,WAAaA,CAGtB,IAAIsgC,GAAal7B,SAASM,cAAc,MACxC46B,GAAW5kC,UAAY,aACvByX,EAAM7N,YAAYg7B,GAClBzsC,KAAK+vB,IAAI0c,WAAaA,CAGtB,IAAI3L,GAAOvvB,SAASM,cAAc,MAClCivB,GAAKj5B,UAAY,OACjB7H,KAAK+vB,IAAI+Q,KAAOA,CAGhB,IAAI2M,GAAWl8B,SAASM,cAAc,MACtC47B,GAAS5lC,UAAY,WACrB7H,KAAK+vB,IAAI0d,SAAWA,EAGpBztC,KAAKywC,kBAGL,IAAIC,GAAkB,GAAI7tC,GAAgB0tC,EAAY,KAAMvwC,KAC5D0wC,GAAgB/I,OAChB3nC,KAAKm0B,OAAOoc,GAAcG,EAM1B1wC,KAAK8D,OAAS,GAAIC,GAAO/D,KAAK20B,KAAK5E,IAAI8H,iBAGvC73B,KAAK8D,OAAOyP,GAAG,eAAgB,SAAUjK,GACnCA,EAAMqnC,SACR3wC,KAAKq+B,SAAS/0B,IAEhBwrB,KAAK90B,OACPA,KAAK8D,OAAOyP,GAAG,WAAYvT,KAAKg+B,aAAalJ,KAAK90B,OAClDA,KAAK8D,OAAOyP,GAAG,UAAYvT,KAAKi+B,QAAQnJ,KAAK90B,OAC7CA,KAAK8D,OAAOyP,GAAG,SAAYvT,KAAKk+B,WAAWpJ,KAAK90B,OAGhDA,KAAK8D,OAAOyP,GAAG,MAAQvT,KAAK4wC,cAAc9b,KAAK90B,OAG/CA,KAAK8D,OAAOyP,GAAG,QAASvT,KAAK6wC,mBAAmB/b,KAAK90B,OAGrDA,KAAK8D,OAAOyP,GAAG,YAAavT,KAAK8wC,WAAWhc,KAAK90B,OAGjDA,KAAK2nC,QAmEP7kC,EAAQqQ,UAAUD,WAAa,SAASzE,GACtC,GAAIA,EAAS,CAEX,GAAIP,IAAU,OAAQ,QAAS,cAAe,UAAW,QAAS,aAAc,aAAc,iBAAkB,WAAW,OAC3HvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAEvC,UAAYA,KACgB,gBAAnBA,GAAQiL,QACjB1Z,KAAKyO,QAAQiL,OAAOonB,KAAOryB,EAAQiL,OACnC1Z,KAAKyO,QAAQiL,OAAOrK,KAAKmW,WAAa/W,EAAQiL,OAC9C1Z,KAAKyO,QAAQiL,OAAOrK,KAAKoW,SAAWhX,EAAQiL,QAEX,gBAAnBjL,GAAQiL,SACtB/Y,EAAKoF,iBAAiB,QAAS/F,KAAKyO,QAAQiL,OAAQjL,EAAQiL,QACxD,QAAUjL,GAAQiL,SACe,gBAAxBjL,GAAQiL,OAAOrK,MACxBrP,KAAKyO,QAAQiL,OAAOrK,KAAKmW,WAAa/W,EAAQiL,OAAOrK,KACrDrP,KAAKyO,QAAQiL,OAAOrK,KAAKoW,SAAWhX,EAAQiL,OAAOrK,MAEb,gBAAxBZ,GAAQiL,OAAOrK,MAC7B1O,EAAKoF,iBAAiB,aAAc,YAAa/F,KAAKyO,QAAQiL,OAAOrK,KAAMZ,EAAQiL,OAAOrK,SAM9F,YAAcZ,KACgB,iBAArBA,GAAQygC,UACjBlvC,KAAKyO,QAAQygC,SAASC,WAAc1gC,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAAS1H,YAAc/4B,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAASj8B,IAAcxE,EAAQygC,SAC5ClvC,KAAKyO,QAAQygC,SAAS74B,OAAc5H,EAAQygC,UAET,gBAArBzgC,GAAQygC,UACtBvuC,EAAKoF,iBAAiB,aAAc,cAAe,MAAO,UAAW/F,KAAKyO,QAAQygC,SAAUzgC,EAAQygC,UAKxG,IAAI6B,GAAc,SAAW96B,GAC3B,GAAIiD,GAAKzK,EAAQwH,EACjB,IAAIiD,EAAI,CACN,KAAMA,YAAc83B,WAClB,KAAM,IAAIptC,OAAM,UAAYqS,EAAO,uBAAyBA,EAAO,mBAErEjW,MAAKyO,QAAQwH,GAAQiD,IAEtB4b,KAAK90B,OACP,QAAS,WAAY,WAAY,SAAU,YAAYqI,QAAQ0oC,GAGhE/wC,KAAKixC,cAOTnuC,EAAQqQ,UAAU89B,UAAY,WAC5BjxC,KAAKkwC,YACLlwC,KAAKowC,YAAa,GAMpBttC,EAAQqQ,UAAUG,QAAU,WAC1BtT,KAAK0nC,OACL1nC,KAAKi2B,SAAS,MACdj2B,KAAKg2B,UAAU,MAEfh2B,KAAK8D,OAAS,KAEd9D,KAAK20B,KAAO,KACZ30B,KAAKo6B,WAAa,MAMpBt3B,EAAQqQ,UAAUu0B,KAAO,WAEnB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,OAI7Ctf,KAAK+vB,IAAI+Q,KAAKj3B,YAChB7J,KAAK+vB,IAAI+Q,KAAKj3B,WAAWsH,YAAYnR,KAAK+vB,IAAI+Q,MAI5C9gC,KAAK+vB,IAAI0d,SAAS5jC,YACpB7J,KAAK+vB,IAAI0d,SAAS5jC,WAAWsH,YAAYnR,KAAK+vB,IAAI0d,WAQtD3qC,EAAQqQ,UAAUw0B,KAAO,WAElB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,OAIvCtf,KAAK+vB,IAAI+Q,KAAKj3B,YACjB7J,KAAK20B,KAAK5E,IAAIgV,mBAAmBtzB,YAAYzR,KAAK+vB,IAAI+Q,MAInD9gC,KAAK+vB,IAAI0d,SAAS5jC,YACrB7J,KAAK20B,KAAK5E,IAAItoB,KAAKgK,YAAYzR,KAAK+vB,IAAI0d,WAW5C3qC,EAAQqQ,UAAUujB,aAAe,SAASvhB,GACxC,GAAI3P,GAAG8nC,EAAIjtC,EAAIgP,CAMf,KAJW7I,QAAP2O,IAAkBA,MACjBlP,MAAMC,QAAQiP,KAAMA,GAAOA,IAG3B3P,EAAI,EAAG8nC,EAAKttC,KAAKmwC,UAAUxqC,OAAY2nC,EAAJ9nC,EAAQA,IAC9CnF,EAAKL,KAAKmwC,UAAU3qC,GACpB6J,EAAOrP,KAAKiC,MAAM5B,GACdgP,GAAMA,EAAK6hC,UAKjB,KADAlxC,KAAKmwC,aACA3qC,EAAI,EAAG8nC,EAAKn4B,EAAIxP,OAAY2nC,EAAJ9nC,EAAQA,IACnCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKiC,MAAM5B,GACdgP,IACFrP,KAAKmwC,UAAUnoC,KAAK3H,GACpBgP,EAAK8hC,WASXruC,EAAQqQ,UAAUyjB,aAAe,WAC/B,MAAO52B,MAAKmwC,UAAUn8B,YAOxBlR,EAAQqQ,UAAUi+B,gBAAkB,WAClC,GAAI3b,GAAQz1B,KAAK20B,KAAKc,MAAM2J,WACxB33B,EAAQzH,KAAK20B,KAAKh0B,KAAKs0B,SAASQ,EAAM7lB,OACtCyX,EAAQrnB,KAAK20B,KAAKh0B,KAAKs0B,SAASQ,EAAM5lB,KAEtCsF,IACJ,KAAK,GAAImiB,KAAWt3B,MAAKm0B,OACvB,GAAIn0B,KAAKm0B,OAAOruB,eAAewxB,GAM7B,IAAK,GALDrlB,GAAQjS,KAAKm0B,OAAOmD,GACpB+Z,EAAkBp/B,EAAMm6B,aAInB5mC,EAAI,EAAGA,EAAI6rC,EAAgB1rC,OAAQH,IAAK,CAC/C,GAAI6J,GAAOgiC,EAAgB7rC,EAEtB6J,GAAK5H,KAAO4f,GAAWhY,EAAK5H,KAAO4H,EAAKkD,MAAQ9K,GACnD0N,EAAInN,KAAKqH,EAAKhP,IAMtB,MAAO8U,IAQTrS,EAAQqQ,UAAUm+B,UAAY,SAASjxC,GAErC,IAAK,GADD8vC,GAAYnwC,KAAKmwC,UACZ3qC,EAAI,EAAG8nC,EAAK6C,EAAUxqC,OAAY2nC,EAAJ9nC,EAAQA,IAC7C,GAAI2qC,EAAU3qC,IAAMnF,EAAI,CACtB8vC,EAAU/nC,OAAO5C,EAAG,EACpB,SASN1C,EAAQqQ,UAAUsO,OAAS,WACzB,GAAI/H,GAAS1Z,KAAKyO,QAAQiL,OACtB+b,EAAQz1B,KAAK20B,KAAKc,MAClBtrB,EAASxJ,EAAKmJ,OAAOK,OACrBsE,EAAUzO,KAAKyO,QACf8lB,EAAc9lB,EAAQ8lB,YACtBkQ,GAAU,EACVnlB,EAAQtf,KAAK+vB,IAAIzQ,MACjB4vB,EAAWzgC,EAAQygC,SAASC,YAAc1gC,EAAQygC,SAAS1H,WAG/DxnC,MAAKgG,MAAM2B,IAAM3H,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASxoB,OAAOzE,IAC3E3H,KAAKgG,MAAMyB,KAAOzH,KAAK20B,KAAKC,SAASntB,KAAK8K,MAAQvS,KAAK20B,KAAKC,SAASxoB,OAAO3E,KAG5E6X,EAAMzX,UAAY,WAAaqnC,EAAW,YAAc,IAGxDzK,EAAUzkC,KAAKuxC,gBAAkB9M,CAIjC,IAAI+M,GAAkB/b,EAAM5lB,IAAM4lB,EAAM7lB,MACpC6hC,EAAUD,GAAmBxxC,KAAK0xC,qBAAyB1xC,KAAKgG,MAAMuM,OAASvS,KAAKgG,MAAM2rC,SAC1FF,KAAQzxC,KAAKowC,YAAa,GAC9BpwC,KAAK0xC,oBAAsBF,EAC3BxxC,KAAKgG,MAAM2rC,UAAY3xC,KAAKgG,MAAMuM,KAElC,IAAIs6B,GAAU7sC,KAAKowC,WACfwB,EAAa5xC,KAAK6xC,cAClBC,GACFziC,KAAMqK,EAAOrK,KACbyxB,KAAMpnB,EAAOonB,MAEXiR,GACF1iC,KAAMqK,EAAOrK,KACbyxB,KAAMpnB,EAAOrK,KAAKoW,SAAW,GAE3BjT,EAAS,EACTiiB,EAAY/a,EAAOonB,KAAOpnB,EAAOrK,KAAKoW,QA+B1C,OA5BAzlB,MAAKm0B,OAAOoc,GAAY9uB,OAAOgU,EAAOsc,EAAgBlF,GAGtDlsC,EAAK0H,QAAQrI,KAAKm0B,OAAQ,SAAUliB,GAClC,GAAI+/B,GAAe//B,GAAS2/B,EAAcE,EAAcC,EACpDE,EAAehgC,EAAMwP,OAAOgU,EAAOuc,EAAanF,EACpDpI,GAAUwN,GAAgBxN,EAC1BjyB,GAAUP,EAAMO,SAElBA,EAAStN,KAAKwH,IAAI8F,EAAQiiB,GAC1Bz0B,KAAKowC,YAAa,EAGlB9wB,EAAMrS,MAAMuF,OAAUrI,EAAOqI,GAG7BxS,KAAKgG,MAAMuM,MAAQ+M,EAAM8Q,YACzBpwB,KAAKgG,MAAMwM,OAASA,EAGpBxS,KAAK+vB,IAAI+Q,KAAK7zB,MAAMtF,IAAMwC,EAAuB,OAAfoqB,EAC7Bv0B,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASxoB,OAAOzE,IAC1D3H,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QACxExS,KAAK+vB,IAAI+Q,KAAK7zB,MAAMxF,KAAO,IAG3Bg9B,EAAUzkC,KAAKwkC,cAAgBC,GAUjC3hC,EAAQqQ,UAAU0+B,YAAc,WAC9B,GAAIK,GAA+C,OAA5BlyC,KAAKyO,QAAQ8lB,YAAwB,EAAKv0B,KAAKkwC,SAASvqC,OAAS,EACpFwsC,EAAenyC,KAAKkwC,SAASgC,GAC7BN,EAAa5xC,KAAKm0B,OAAOge,IAAiBnyC,KAAKm0B,OAAOmc,EAE1D,OAAOsB,IAAc,MAQvB9uC,EAAQqQ,UAAUs9B,iBAAmB,WACnC,CAAA,GAEIphC,GAAMkG,EAFN68B,EAAYpyC,KAAKm0B,OAAOmc,EACXtwC,MAAKm0B,OAAOoc,GAG7B,GAAIvwC,KAAK+1B,YAEP,GAAIqc,EAAW,CACbA,EAAU1K,aACH1nC,MAAKm0B,OAAOmc,EAEnB,KAAK/6B,IAAUvV,MAAKiC,MAClB,GAAIjC,KAAKiC,MAAM6D,eAAeyP,GAAS,CACrClG,EAAOrP,KAAKiC,MAAMsT,GAClBlG,EAAKy1B,QAAUz1B,EAAKy1B,OAAOzuB,OAAOhH,EAClC,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACxBrlB,IAASA,EAAMgB,IAAI5D,IAASA,EAAKq4B,aAOvC,KAAK0K,EAAW,CACd,GAAI/xC,GAAK,KACLqS,EAAO,IACX0/B,GAAY,GAAIxvC,GAAMvC,EAAIqS,EAAM1S,MAChCA,KAAKm0B,OAAOmc,GAAa8B,CAEzB,KAAK78B,IAAUvV,MAAKiC,MACdjC,KAAKiC,MAAM6D,eAAeyP,KAC5BlG,EAAOrP,KAAKiC,MAAMsT,GAClB68B,EAAUn/B,IAAI5D,GAIlB+iC,GAAUzK,SAShB7kC,EAAQqQ,UAAUm/B,YAAc,WAC9B,MAAOtyC,MAAK+vB,IAAI0d,UAOlB3qC,EAAQqQ,UAAU8iB,SAAW,SAASh0B,GACpC,GACIkT,GADAhB,EAAKnU,KAELuyC,EAAevyC,KAAK81B,SAGxB,IAAK7zB,EAGA,CAAA,KAAIA,YAAiBpB,IAAWoB,YAAiBnB,IAIpD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK81B,UAAY7zB,MAHjBjC,MAAK81B,UAAY,IAoBnB,IAXIyc,IAEF5xC,EAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnDipC,EAAa7+B,IAAIpK,EAAOhB,KAI1B6M,EAAMo9B,EAAa18B,SACnB7V,KAAK6vC,UAAU16B,IAGbnV,KAAK81B,UAAW,CAElB,GAAIz1B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnD6K,EAAG2hB,UAAUviB,GAAGjK,EAAOhB,EAAUjI,KAInC8U,EAAMnV,KAAK81B,UAAUjgB,SACrB7V,KAAK2vC,OAAOx6B,GAGZnV,KAAKywC,qBAQT3tC,EAAQqQ,UAAUq/B,SAAW,WAC3B,MAAOxyC,MAAK81B,WAOdhzB,EAAQqQ,UAAU6iB,UAAY,SAAS7B,GACrC,GACIhf,GADAhB,EAAKnU,IAgBT,IAZIA,KAAK+1B,aACPp1B,EAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWniB,YAAYtK,EAAOhB,KAInC6M,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+1B,WAAa,KAClB/1B,KAAKiwC,gBAAgB96B,IAIlBgf,EAGA,CAAA,KAAIA,YAAkBtzB,IAAWszB,YAAkBrzB,IAItD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK+1B,WAAa5B,MAHlBn0B,MAAK+1B,WAAa,IASpB,IAAI/1B,KAAK+1B,WAAY,CAEnB,GAAI11B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWxiB,GAAGjK,EAAOhB,EAAUjI,KAIpC8U,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+vC,aAAa56B,GAIpBnV,KAAKywC,mBAGLzwC,KAAKyyC,SAELzyC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAO3CtQ,EAAQqQ,UAAUu/B,UAAY,WAC5B,MAAO1yC,MAAK+1B,YAOdjzB,EAAQqQ,UAAU66B,WAAa,SAAS3tC,GACtC,GAAIgP,GAAOrP,KAAK81B,UAAU5gB,IAAI7U,GAC1B22B,EAAUh3B,KAAK81B,UAAUhgB,YAEzBzG,IAEFrP,KAAKyO,QAAQ8gC,SAASlgC,EAAM,SAAUA,GAChCA,GAGF2nB,EAAQ3gB,OAAOhW,MAYvByC,EAAQqQ,UAAUw/B,SAAW,SAAU9b,GACrC,MAAOA,GAAS/vB,MAAQ9G,KAAKyO,QAAQ3H,OAAS+vB,EAAShnB,IAAM,QAAU,QAUzE/M,EAAQqQ,UAAUk/B,YAAc,SAAUxb,GACxC,GAAI/vB,GAAO9G,KAAK2yC,SAAS9b,EACzB,OAAY,cAAR/vB,GAA0CN,QAAlBqwB,EAAS5kB,MAC7Bs+B,EAGCvwC,KAAK+1B,WAAac,EAAS5kB,MAAQq+B,GAS9CxtC,EAAQqQ,UAAUy8B,UAAY,SAASz6B,GACrC,GAAIhB,GAAKnU,IAETmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIw2B,GAAW1iB,EAAG2hB,UAAU5gB,IAAI7U,EAAI8T,EAAGs7B,aACnCpgC,EAAO8E,EAAGlS,MAAM5B,GAChByG,EAAOqN,EAAGw+B,SAAS9b,GAEnBvwB,EAAcxD,EAAQoU,MAAMpQ,EAchC,IAZIuI,IAEG/I,GAAiB+I,YAAgB/I,GAMpC6N,EAAGc,YAAY5F,EAAMwnB,IAJrB1iB,EAAGy+B,YAAYvjC,GACfA,EAAO,QAONA,EAAM,CAET,IAAI/I,EAKC,KAEG,IAAID,WAFK,iBAARS,EAEa,4HAIA,sBAAwBA,EAAO,IAVnDuI,GAAO,GAAI/I,GAAYuwB,EAAU1iB,EAAGimB,WAAYjmB,EAAG1F,SACnDY,EAAKhP,GAAKA,EACV8T,EAAGC,SAAS/E,MAalBrP,KAAKyyC,SACLzyC,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAUw8B,OAAS7sC,EAAQqQ,UAAUy8B,UAO7C9sC,EAAQqQ,UAAU08B,UAAY,SAAS16B,GACrC,GAAI6B,GAAQ,EACR7C,EAAKnU,IACTmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIgP,GAAO8E,EAAGlS,MAAM5B,EAChBgP,KACF2H,IACA7C,EAAGy+B,YAAYvjC,MAIf2H,IAEFhX,KAAKyyC,SACLzyC,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,MAQ7CtQ,EAAQqQ,UAAUs/B,OAAS,WAGzB9xC,EAAK0H,QAAQrI,KAAKm0B,OAAQ,SAAUliB,GAClCA,EAAMwD,WASV3S,EAAQqQ,UAAU68B,gBAAkB,SAAS76B,GAC3CnV,KAAK+vC,aAAa56B,IAQpBrS,EAAQqQ,UAAU48B,aAAe,SAAS56B,GACxC,GAAIhB,GAAKnU,IAETmV,GAAI9M,QAAQ,SAAUhI,GACpB,GAAIyrC,GAAY33B,EAAG4hB,WAAW7gB,IAAI7U,GAC9B4R,EAAQkC,EAAGggB,OAAO9zB,EAEtB,IAAK4R,EA6BHA,EAAM+F,QAAQ8zB,OA7BJ,CAEV,GAAIzrC,GAAMiwC,GAAajwC,GAAMkwC,EAC3B,KAAM,IAAI3sC,OAAM,qBAAuBvD,EAAK,qBAG9C,IAAIwyC,GAAetsC,OAAO8H,OAAO8F,EAAG1F,QACpC9N,GAAK2E,OAAOutC,GACVrgC,OAAQ,OAGVP,EAAQ,GAAIrP,GAAMvC,EAAIyrC,EAAW33B,GACjCA,EAAGggB,OAAO9zB,GAAM4R,CAGhB,KAAK,GAAIsD,KAAUpB,GAAGlS,MACpB,GAAIkS,EAAGlS,MAAM6D,eAAeyP,GAAS,CACnC,GAAIlG,GAAO8E,EAAGlS,MAAMsT,EAChBlG,GAAKqD,KAAKT,OAAS5R,GACrB4R,EAAMgB,IAAI5D,GAKhB4C,EAAMwD,QACNxD,EAAM01B,UAQV3nC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAU88B,gBAAkB,SAAS96B,GAC3C,GAAIgf,GAASn0B,KAAKm0B,MAClBhf,GAAI9M,QAAQ,SAAUhI,GACpB,GAAI4R,GAAQkiB,EAAO9zB,EAEf4R,KACFA,EAAMy1B,aACCvT,GAAO9zB,MAIlBL,KAAKixC,YAELjxC,KAAK20B,KAAKE,QAAQjH,KAAK,UAAWxa,OAAO,KAQ3CtQ,EAAQqQ,UAAUo+B,aAAe,WAC/B,GAAIvxC,KAAK+1B,WAAY,CAEnB,GAAIma,GAAWlwC,KAAK+1B,WAAWlgB,QAC7BJ,MAAOzV,KAAKyO,QAAQugC,aAGlB/P,GAAWt+B,EAAKiG,WAAWspC,EAAUlwC,KAAKkwC,SAC9C,IAAIjR,EAAS,CAEX,GAAI9K,GAASn0B,KAAKm0B,MAClB+b,GAAS7nC,QAAQ,SAAUivB,GACzBnD,EAAOmD,GAASoQ,SAIlBwI,EAAS7nC,QAAQ,SAAUivB,GACzBnD,EAAOmD,GAASqQ,SAGlB3nC,KAAKkwC,SAAWA,EAGlB,MAAOjR,GAGP,OAAO,GASXn8B,EAAQqQ,UAAUiB,SAAW,SAAS/E,GACpCrP,KAAKiC,MAAMoN,EAAKhP,IAAMgP,CAGtB,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACpBrlB,IAAOA,EAAMgB,IAAI5D,IASvBvM,EAAQqQ,UAAU8B,YAAc,SAAS5F,EAAMwnB,GAC7C,GAAIic,GAAazjC,EAAKqD,KAAKT,KAM3B,IAHA5C,EAAK2I,QAAQ6e,GAGTic,GAAczjC,EAAKqD,KAAKT,MAAO,CACjC,GAAI8gC,GAAW/yC,KAAKm0B,OAAO2e,EACvBC,IAAUA,EAAS18B,OAAOhH,EAE9B,IAAIioB,GAAUt3B,KAAKqyC,YAAYhjC,EAAKqD,MAChCT,EAAQjS,KAAKm0B,OAAOmD,EACpBrlB,IAAOA,EAAMgB,IAAI5D,KAUzBvM,EAAQqQ,UAAUy/B,YAAc,SAASvjC,GAEvCA,EAAKq4B,aAGE1nC,MAAKiC,MAAMoN,EAAKhP,GAGvB,IAAI8H,GAAQnI,KAAKmwC,UAAUxpC,QAAQ0I,EAAKhP,GAC3B,KAAT8H,GAAanI,KAAKmwC,UAAU/nC,OAAOD,EAAO,GAG9CkH,EAAKy1B,QAAUz1B,EAAKy1B,OAAOzuB,OAAOhH,IASpCvM,EAAQqQ,UAAU6/B,qBAAuB,SAASxqC,GAGhD,IAAK,GAFD0lC,MAEK1oC,EAAI,EAAGA,EAAIgD,EAAM7C,OAAQH,IAC5BgD,EAAMhD,YAAclD,IACtB4rC,EAASlmC,KAAKQ,EAAMhD,GAGxB,OAAO0oC,IAaTprC,EAAQqQ,UAAUkrB,SAAW,SAAU/0B,GAErCtJ,KAAKqwC,YAAYhhC,KAAOvM,EAAQmwC,eAAe3pC,GAC/CtJ,KAAKqwC,YAAY6C,aAAe5pC,EAAMI,OAAOwpC,eAAgB,EAC7DlzC,KAAKqwC,YAAY8C,cAAgB7pC,EAAMI,OAAOypC,gBAAiB,EAC/DnzC,KAAKqwC,YAAY+C,UAAY,MAQ/BtwC,EAAQqQ,UAAU6qB,aAAe,SAAU10B,GACzC,GAAKtJ,KAAKyO,QAAQygC,SAASC,YAAenvC,KAAKyO,QAAQygC,SAAS1H,YAAhE,CAIA,GAEIxhC,GAFAqJ,EAAOrP,KAAKqwC,YAAYhhC,MAAQ,KAChC8E,EAAKnU,IAGT,IAAIqP,GAAQA,EAAKgkC,SAAU,CACzB,GAAIH,GAAelzC,KAAKqwC,YAAY6C,aAChCC,EAAgBnzC,KAAKqwC,YAAY8C,aAEjCD,IACFltC,GACEqJ,KAAM6jC,EACNI,SAAUhqC,EAAM4iB,OAAOna,GAGrBoC,EAAG1F,QAAQygC,SAASC,aACtBnpC,EAAM4J,MAAQP,EAAKqD,KAAK9C,MAAM5I,WAE5BmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAGpDjS,KAAKqwC,YAAY+C,WAAaptC,IAEvBmtC,GACPntC,GACEqJ,KAAM8jC,EACNG,SAAUhqC,EAAM4iB,OAAOna,GAGrBoC,EAAG1F,QAAQygC,SAASC,aACtBnpC,EAAM6J,IAAMR,EAAKqD,KAAK7C,IAAI7I,WAExBmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAGpDjS,KAAKqwC,YAAY+C,WAAaptC,IAG9BhG,KAAKqwC,YAAY+C,UAAYpzC,KAAK42B,eAAevpB,IAAI,SAAUhN,GAC7D,GAAIgP,GAAO8E,EAAGlS,MAAM5B,GAChB2F,GACFqJ,KAAMA,EACNikC,SAAUhqC,EAAM4iB,OAAOna,EAWzB,OARIoC,GAAG1F,QAAQygC,SAASC,aAClB,SAAW9/B,GAAKqD,OAAM1M,EAAM4J,MAAQP,EAAKqD,KAAK9C,MAAM5I,WACpD,OAASqI,GAAKqD,OAAQ1M,EAAM6J,IAAMR,EAAKqD,KAAK7C,IAAI7I,YAElDmN,EAAG1F,QAAQygC,SAAS1H,aAClB,SAAWn4B,GAAKqD,OAAM1M,EAAMiM,MAAQ5C,EAAKqD,KAAKT,OAG7CjM,IAIXsD,EAAMq8B,kBACNr8B,EAAMD,oBASVvG,EAAQqQ,UAAU8qB,QAAU,SAAU30B,GAGpC,GAFAA,EAAMD,iBAEFrJ,KAAKqwC,YAAY+C,UAAW,CAC9B,GAAIj/B,GAAKnU,KACLg1B,EAAOh1B,KAAK20B,KAAKh0B,KAAKq0B,MAAQ,KAC9BpL,EAAU5pB,KAAK20B,KAAK5E,IAAIrwB,KAAK2tC,WAAartC,KAAK20B,KAAKC,SAASntB,KAAK8K,KAGtEvS,MAAKqwC,YAAY+C,UAAU/qC,QAAQ,SAAUrC,GAC3C,GAAIutC,MACAzZ,EAAU3lB,EAAGwgB,KAAKh0B,KAAK00B,OAAO/rB,EAAM4iB,OAAOna,EAAI6X,GAC/C4pB,EAAUr/B,EAAGwgB,KAAKh0B,KAAK00B,OAAOrvB,EAAMstC,SAAW1pB,GAC/CD,EAASmQ,EAAU0Z,CAEvB,IAAI,SAAWxtC,GAAO,CACpB,GAAI4J,GAAQ,GAAItL,MAAK0B,EAAM4J,MAAQ+Z,EACnC4pB,GAAS3jC,MAAQolB,EAAOA,EAAKplB,GAASA,EAGxC,GAAI,OAAS5J,GAAO,CAClB,GAAI6J,GAAM,GAAIvL,MAAK0B,EAAM6J,IAAM8Z,EAC/B4pB,GAAS1jC,IAAMmlB,EAAOA,EAAKnlB,GAAOA,EAGpC,GAAI,SAAW7J,GAAO,CAEpB,GAAIiM,GAAQnP,EAAQ2wC,gBAAgBnqC,EACpCiqC,GAASthC,MAAQA,GAASA,EAAMqlB,QAIlC,GAAIT,GAAWl2B,EAAK2E,UAAWU,EAAMqJ,KAAKqD,KAAM6gC,EAChDp/B,GAAG1F,QAAQ+gC,SAAS3Y,EAAU,SAAUA,GAClCA,GACF1iB,EAAGu/B,iBAAiB1tC,EAAMqJ,KAAMwnB,OAKtC72B,KAAKowC,YAAa,EAClBpwC,KAAK20B,KAAKE,QAAQjH,KAAK,UAEvBtkB,EAAMq8B,oBAUV7iC,EAAQqQ,UAAUugC,iBAAmB,SAASrkC,EAAMrJ,GAE9C,SAAWA,KAAOqJ,EAAKqD,KAAK9C,MAAQ5J,EAAM4J,OAC1C,OAAS5J,KAASqJ,EAAKqD,KAAK7C,IAAQ7J,EAAM6J,KAC1C,SAAW7J,IAASqJ,EAAKqD,KAAKT,OAASjM,EAAMiM,OAC/CjS,KAAK2zC,aAAatkC,EAAMrJ,EAAMiM,QAUlCnP,EAAQqQ,UAAUwgC,aAAe,SAAStkC,EAAMioB,GAC9C,GAAIrlB,GAAQjS,KAAKm0B,OAAOmD,EACxB,IAAIrlB,GAASA,EAAMqlB,SAAWjoB,EAAKqD,KAAKT,MAAO,CAC7C,GAAI8gC,GAAW1jC,EAAKy1B,MACpBiO,GAAS18B,OAAOhH,GAChB0jC,EAASt9B,QACTxD,EAAMgB,IAAI5D,GACV4C,EAAMwD,QAENpG,EAAKqD,KAAKT,MAAQA,EAAMqlB,UAS5Bx0B,EAAQqQ,UAAU+qB,WAAa,SAAU50B,GACvC,GAAItJ,KAAKqwC,YAAY+C,UAAW,CAE9B,GAAIQ,MACAz/B,EAAKnU,KACLg3B,EAAUh3B,KAAK81B,UAAUhgB,aAEzBs9B,EAAYpzC,KAAKqwC,YAAY+C,SACjCpzC,MAAKqwC,YAAY+C,UAAY,KAC7BA,EAAU/qC,QAAQ,SAAUrC,GAC1B,GAAI3F,GAAK2F,EAAMqJ,KAAKhP,GAChBw2B,EAAW1iB,EAAG2hB,UAAU5gB,IAAI7U,EAAI8T,EAAGs7B,aAEnCxQ,GAAU,CACV,UAAWj5B,GAAMqJ,KAAKqD,OACxBusB,EAAWj5B,EAAM4J,OAAS5J,EAAMqJ,KAAKqD,KAAK9C,MAAM5I,UAChD6vB,EAASjnB,MAAQjP,EAAKkG,QAAQb,EAAMqJ,KAAKqD,KAAK9C,MACtConB,EAAQrkB,SAAS7L,MAAQkwB,EAAQrkB,SAAS7L,KAAK8I,OAAS,SAE9D,OAAS5J,GAAMqJ,KAAKqD,OACtBusB,EAAUA,GAAaj5B,EAAM6J,KAAO7J,EAAMqJ,KAAKqD,KAAK7C,IAAI7I,UACxD6vB,EAAShnB,IAAMlP,EAAKkG,QAAQb,EAAMqJ,KAAKqD,KAAK7C,IACpCmnB,EAAQrkB,SAAS7L,MAAQkwB,EAAQrkB,SAAS7L,KAAK+I,KAAO,SAE5D,SAAW7J,GAAMqJ,KAAKqD,OACxBusB,EAAUA,GAAaj5B,EAAMiM,OAASjM,EAAMqJ,KAAKqD,KAAKT,MACtD4kB,EAAS5kB,MAAQjM,EAAMqJ,KAAKqD,KAAKT,OAI/BgtB,GACF9qB,EAAG1F,QAAQ6gC,OAAOzY,EAAU,SAAUA,GAChCA,GAEFA,EAASG,EAAQnkB,UAAYxS,EAC7BuzC,EAAQ5rC,KAAK6uB,KAIb1iB,EAAGu/B,iBAAiB1tC,EAAMqJ,KAAMrJ,GAEhCmO,EAAGi8B,YAAa,EAChBj8B,EAAGwgB,KAAKE,QAAQjH,KAAK,eAOzBgmB,EAAQjuC,QACVqxB,EAAQniB,OAAO++B,GAGjBtqC,EAAMq8B,oBASV7iC,EAAQqQ,UAAUy9B,cAAgB,SAAUtnC,GAC1C,GAAKtJ,KAAKyO,QAAQwgC,WAAlB,CAEA,GAAI4E,GAAWvqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASD,QAC5CE,EAAWzqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASC,QAChD,IAAIF,GAAWE,EAEb,WADA/zC,MAAK6wC,mBAAmBvnC,EAI1B,IAAI0qC,GAAeh0C,KAAK42B,eAEpBvnB,EAAOvM,EAAQmwC,eAAe3pC,GAC9B6mC,EAAY9gC,GAAQA,EAAKhP,MAC7BL,MAAK02B,aAAayZ,EAElB,IAAI8D,GAAej0C,KAAK42B,gBAIpBqd,EAAatuC,OAAS,GAAKquC,EAAaruC,OAAS,IACnD3F,KAAK20B,KAAKE,QAAQjH,KAAK,UACrB3rB,MAAOgyC,MAUbnxC,EAAQqQ,UAAU29B,WAAa,SAAUxnC,GACvC,GAAKtJ,KAAKyO,QAAQwgC,YACbjvC,KAAKyO,QAAQygC,SAASj8B,IAA3B,CAEA,GAAIkB,GAAKnU,KACLg1B,EAAOh1B,KAAK20B,KAAKh0B,KAAKq0B,MAAQ,KAC9B3lB,EAAOvM,EAAQmwC,eAAe3pC,EAElC,IAAI+F,EAAM,CAIR,GAAIwnB,GAAW1iB,EAAG2hB,UAAU5gB,IAAI7F,EAAKhP,GACrCL,MAAKyO,QAAQ4gC,SAASxY,EAAU,SAAUA,GACpCA,GACF1iB,EAAG2hB,UAAUhgB,aAAajB,OAAOgiB,SAIlC,CAEH,GAAIqd,GAAOvzC,EAAK2G,gBAAgBtH,KAAK+vB,IAAIzQ,OACrCvN,EAAIzI,EAAM4iB,OAAOna,EAAImiC,EACrBtkC,EAAQ5P,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,GAC9BoiC,GACFvkC,MAAOolB,EAAOA,EAAKplB,GAASA,EAC5BggB,QAAS,WAIX,IAA0B,UAAtB5vB,KAAKyO,QAAQ3H,KAAkB,CACjC,GAAI+I,GAAM7P,KAAK20B,KAAKh0B,KAAK00B,OAAOtjB,EAAI/R,KAAKgG,MAAMuM,MAAQ,EACvD4hC,GAAQtkC,IAAMmlB,EAAOA,EAAKnlB,GAAOA,EAGnCskC,EAAQn0C,KAAK81B,UAAUjjB,UAAYlS,EAAKqE,YAExC,IAAIiN,GAAQnP,EAAQ2wC,gBAAgBnqC,EAChC2I,KACFkiC,EAAQliC,MAAQA,EAAMqlB,SAIxBt3B,KAAKyO,QAAQ2gC,MAAM+E,EAAS,SAAU9kC,GAChCA,GACF8E,EAAG2hB,UAAUhgB,aAAa7C,IAAI5D,QAYtCvM,EAAQqQ,UAAU09B,mBAAqB,SAAUvnC,GAC/C,GAAKtJ,KAAKyO,QAAQwgC,WAAlB,CAEA,GAAIkB,GACA9gC,EAAOvM,EAAQmwC,eAAe3pC,EAElC,IAAI+F,EAAM,CAER8gC,EAAYnwC,KAAK42B,cAEjB,IAAImd,GAAWzqC,EAAMwqC,UAAYxqC,EAAMwqC,SAASC,WAAY,CAC5D,IAAIA,EAAU,CAIZ5D,EAAUnoC,KAAKqH,EAAKhP,GACpB,IAAIo1B,GAAQ3yB,EAAQsxC,cAAcp0C,KAAK81B,UAAU5gB,IAAIi7B,EAAWnwC,KAAKyvC,aAGrEU,KACA,KAAK,GAAI9vC,KAAML,MAAKiC,MAClB,GAAIjC,KAAKiC,MAAM6D,eAAezF,GAAK,CACjC,GAAIg0C,GAAQr0C,KAAKiC,MAAM5B,GACnBuP,EAAQykC,EAAM3hC,KAAK9C,MACnBC,EAA0BrJ,SAAnB6tC,EAAM3hC,KAAK7C,IAAqBwkC,EAAM3hC,KAAK7C,IAAMD,CAExDA,IAAS6lB,EAAM3pB,KAAO+D,GAAO4lB,EAAM/oB,KACrCyjC,EAAUnoC,KAAKqsC,EAAMh0C,SAKxB,CAEH,GAAI8H,GAAQgoC,EAAUxpC,QAAQ0I,EAAKhP,GACtB,KAAT8H,EAEFgoC,EAAUnoC,KAAKqH,EAAKhP,IAIpB8vC,EAAU/nC,OAAOD,EAAO,GAI5BnI,KAAK02B,aAAayZ,GAElBnwC,KAAK20B,KAAKE,QAAQjH,KAAK,UACrB3rB,MAAOjC,KAAK42B,oBAWlB9zB,EAAQsxC,cAAgB,SAASte,GAC/B,GAAIppB,GAAM,KACNZ,EAAM,IAmBV,OAjBAgqB,GAAUztB,QAAQ,SAAUqK,IACf,MAAP5G,GAAe4G,EAAK9C,MAAQ9D,KAC9BA,EAAM4G,EAAK9C,OAGGpJ,QAAZkM,EAAK7C,KACI,MAAPnD,GAAegG,EAAK7C,IAAMnD,KAC5BA,EAAMgG,EAAK7C,MAIF,MAAPnD,GAAegG,EAAK9C,MAAQlD,KAC9BA,EAAMgG,EAAK9C,UAMf9D,IAAKA,EACLY,IAAKA,IAUT5J,EAAQmwC,eAAiB,SAAS3pC,GAEhC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,iBACxB,MAAO4D,GAAO,gBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAST/G,EAAQ2wC,gBAAkB,SAASnqC,GAEjC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,kBACxB,MAAO4D,GAAO,iBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAST/G,EAAQwxC,kBAAoB,SAAShrC,GAEnC,IADA,GAAII,GAASJ,EAAMI,OACZA,GAAQ,CACb,GAAIA,EAAO5D,eAAe,oBACxB,MAAO4D,GAAO,mBAEhBA,GAASA,EAAOG,WAGlB,MAAO,OAGThK,EAAOD,QAAUkD,GAKb,SAASjD,EAAQD,EAASM,GAS9B,QAAS6C,GAAO4xB,EAAMlmB,EAAS8lC,EAAM1O,GACnC7lC,KAAK20B,KAAOA,EACZ30B,KAAKq0B,gBACH3lB,SAAS,EACTs3B,OAAO,EACPwO,SAAU,GACVC,YAAa,EACbhtC,MACEihB,SAAS,EACT9E,SAAU,YAEZyD,OACEqB,SAAS,EACT9E,SAAU,aAGd5jB,KAAKu0C,KAAOA,EACZv0C,KAAKyO,QAAU9N,EAAK2E,UAAUtF,KAAKq0B,gBACnCr0B,KAAK6lC,iBAAmBA,EAExB7lC,KAAKinC,eACLjnC,KAAK+vB,OACL/vB,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,EACtBnnC,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAjClB,GAAI9N,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BqC,EAAYrC,EAAoB,GAkCpC6C,GAAOoQ,UAAY,GAAI5Q,GAEvBQ,EAAOoQ,UAAUsD,MAAQ,WACvBzW,KAAKm0B,UACLn0B,KAAKmnC,eAAiB,GAGxBpkC,EAAOoQ,UAAUm0B,SAAW,SAAS7e,EAAO8e,GAErCvnC,KAAKm0B,OAAOruB,eAAe2iB,KAC9BzoB,KAAKm0B,OAAO1L,GAAS8e,GAEvBvnC,KAAKmnC,gBAAkB,GAGzBpkC,EAAOoQ,UAAUq0B,YAAc,SAAS/e,EAAO8e,GAC7CvnC,KAAKm0B,OAAO1L,GAAS8e,GAGvBxkC,EAAOoQ,UAAUs0B,YAAc,SAAShf,GAClCzoB,KAAKm0B,OAAOruB,eAAe2iB,WACtBzoB,MAAKm0B,OAAO1L,GACnBzoB,KAAKmnC,gBAAkB,IAI3BpkC,EAAOoQ,UAAUuhB,QAAU,WACzB10B,KAAK+vB,IAAIzQ,MAAQ/N,SAASM,cAAc,OACxC7R,KAAK+vB,IAAIzQ,MAAMzX,UAAY,SAC3B7H,KAAK+vB,IAAIzQ,MAAMrS,MAAM2W,SAAW,WAChC5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,OAC3B3H,KAAK+vB,IAAIzQ,MAAMrS,MAAM26B,QAAU,QAE/B5nC,KAAK+vB,IAAI2kB,SAAWnjC,SAASM,cAAc,OAC3C7R,KAAK+vB,IAAI2kB,SAAS7sC,UAAY,aAC9B7H,KAAK+vB,IAAI2kB,SAASznC,MAAM2W,SAAW,WACnC5jB,KAAK+vB,IAAI2kB,SAASznC,MAAMtF,IAAM,MAE9B3H,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMtF,IAAM,MACrB3H,KAAK4lC,IAAI34B,MAAMsF,MAAQvS,KAAKyO,QAAQ+lC,SAAW,EAAI,KACnDx0C,KAAK4lC,IAAI34B,MAAMuF,OAAS,OAExBxS,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK4lC,KAChC5lC,KAAK+vB,IAAIzQ,MAAM7N,YAAYzR,KAAK+vB,IAAI2kB,WAMtC3xC,EAAOoQ,UAAUu0B,KAAO,WAElB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,QAQnDvc,EAAOoQ,UAAUw0B,KAAO,WAEjB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,QAI9Cvc,EAAOoQ,UAAUD,WAAa,SAASzE,GACrC,GAAIP,IAAU,UAAU,cAAc,QAAQ,OAAO,QACrDvN,GAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,IAGjD1L,EAAOoQ,UAAUsO,OAAS,WACxB,GAAI0mB,GAAe,CACnB,KAAK,GAAI7Q,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,IACvI6Q,IAKN,IAAuC,GAAnCnoC,KAAKyO,QAAQzO,KAAKu0C,MAAM7rB,SAA2C,GAAvB1oB,KAAKmnC,gBAA+C,GAAxBnnC,KAAKyO,QAAQC,SAAoC,GAAhBy5B,EAC3GnoC,KAAK0nC,WAEF,CAqBH,GApBA1nC,KAAK2nC,OACmC,YAApC3nC,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAA8D,eAApC5jB,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAC5E5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMxF,KAAO,MAC5BzH,KAAK+vB,IAAIzQ,MAAMrS,MAAMqb,UAAY,OACjCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMqb,UAAY,OACpCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAQzH,KAAKyO,QAAQ+lC,SAAW,GAAM,KAC9Dx0C,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAAQ,GAChCrnB,KAAK4lC,IAAI34B,MAAMxF,KAAO,MACtBzH,KAAK4lC,IAAI34B,MAAMoa,MAAQ,KAGvBrnB,KAAK+vB,IAAIzQ,MAAMrS,MAAMoa,MAAQ,MAC7BrnB,KAAK+vB,IAAIzQ,MAAMrS,MAAMqb,UAAY,QACjCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMqb,UAAY,QACpCtoB,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAASrnB,KAAKyO,QAAQ+lC,SAAW,GAAM,KAC/Dx0C,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAO,GAC/BzH,KAAK4lC,IAAI34B,MAAMoa,MAAQ,MACvBrnB,KAAK4lC,IAAI34B,MAAMxF,KAAO,IAGgB,YAApCzH,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,UAA8D,aAApC5jB,KAAKyO,QAAQzO,KAAKu0C,MAAM3wB,SAC5E5jB,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,EAAIzD,OAAOlE,KAAK20B,KAAK5E,IAAI7D,OAAOjf,MAAMtF,IAAI6C,QAAQ,KAAK,KAAO,KACzFxK,KAAK+vB,IAAIzQ,MAAMrS,MAAMqW,OAAS,OAE3B,CACH,GAAIqxB,GAAmB30C,KAAK20B,KAAKC,SAAS1I,OAAO1Z,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,MAC7FxS,MAAK+vB,IAAIzQ,MAAMrS,MAAMqW,OAAS,EAAIqxB,EAAmBzwC,OAAOlE,KAAK20B,KAAK5E,IAAI7D,OAAOjf,MAAMtF,IAAI6C,QAAQ,KAAK,KAAO,KAC/GxK,KAAK+vB,IAAIzQ,MAAMrS,MAAMtF,IAAM,GAGH,GAAtB3H,KAAKyO,QAAQu3B,OACfhmC,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAK+vB,IAAI2kB,SAAStkB,YAAc,GAAK,KAClEpwB,KAAK+vB,IAAI2kB,SAASznC,MAAMoa,MAAQ,GAChCrnB,KAAK+vB,IAAI2kB,SAASznC,MAAMxF,KAAO,GAC/BzH,KAAK4lC,IAAI34B,MAAMsF,MAAQ,QAGvBvS,KAAK+vB,IAAIzQ,MAAMrS,MAAMsF,MAAQvS,KAAKyO,QAAQ+lC,SAAW,GAAKx0C,KAAK+vB,IAAI2kB,SAAStkB,YAAc,GAAK,KAC/FpwB,KAAK40C,kBAGP,IAAIhlB,GAAU,EACd,KAAK,GAAI0H,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvI1H,GAAW5vB,KAAKm0B,OAAOmD,GAAS1H,QAAU,UAIhD5vB,MAAK+vB,IAAI2kB,SAASzwB,UAAY2L,EAC9B5vB,KAAK+vB,IAAI2kB,SAASznC,MAAMsjB,WAAe,IAAOvwB,KAAKyO,QAAQ+lC,SAAYx0C,KAAKyO,QAAQgmC,YAAe,OAIvG1xC,EAAOoQ,UAAUyhC,gBAAkB,WACjC,GAAI50C,KAAK+vB,IAAIzQ,MAAMzV,WAAY,CAC7BjJ,EAAQiQ,gBAAgB7Q,KAAKinC,YAC7B,IAAIjjB,GAAUza,OAAOsrC,iBAAiB70C,KAAK+vB,IAAIzQ,OAAOw1B,WAClD/M,EAAa7jC,OAAO8f,EAAQxZ,QAAQ,KAAK,KACzCuH,EAAIg2B,EACJ1B,EAAYrmC,KAAKyO,QAAQ+lC,SACzB1M,EAAa,IAAO9nC,KAAKyO,QAAQ+lC,SACjCxiC,EAAI+1B,EAAa,GAAMD,EAAa,CAExC9nC,MAAK4lC,IAAI34B,MAAMsF,MAAQ8zB,EAAY,EAAI0B,EAAa,IAEpD,KAAK,GAAIzQ,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KACO,GAAhCt3B,KAAKm0B,OAAOmD,GAAS5O,SAAkEliB,SAA9CxG,KAAK6lC,iBAAiBrO,WAAWF,IAAuE,GAA7Ct3B,KAAK6lC,iBAAiBrO,WAAWF,KACvIt3B,KAAKm0B,OAAOmD,GAAS0Q,SAASj2B,EAAGC,EAAGhS,KAAKinC,YAAajnC,KAAK4lC,IAAKS,EAAWyB,GAC3E91B,GAAK81B,EAAa9nC,KAAKyO,QAAQgmC,aAKrC7zC,GAAQsQ,gBAAgBlR,KAAKinC,eAIjCpnC,EAAOD,QAAUmD,GAKb,SAASlD,EAAQD,EAASM,GAqB9B,QAAS8C,GAAU2xB,EAAMlmB,GACvBzO,KAAKK,GAAKM,EAAKqE,aACfhF,KAAK20B,KAAOA,EAEZ30B,KAAKq0B,gBACHuX,iBAAkB,OAClBmJ,aAAc,UACd7+B,MAAM,EACN8+B,UAAU,EACVC,YAAa,QACb1J,QACE78B,SAAS,EACT6lB,YAAa,UAEftnB,MAAO,OACPioC,UACE3iC,MAAO,GACP4iC,cAAe,UACfpG,MAAO,UAEThE,YACEr8B,SAAS,EACTs8B,gBAAiB,cACjBC,MAAO,IAET94B,YACEzD,SAAS,EACT2D,KAAM,EACNpF,MAAO,UAETmoC,UACEtP,iBAAiB,EACjBC,iBAAiB,EACjBC,OAAO,EACPzzB,MAAO,OACPmW,SAAS,EACT6S,YAAY,EACZD,aACE7zB,MAAOqE,IAAItF,OAAWkG,IAAIlG,QAC1B6gB,OAAQvb,IAAItF,OAAWkG,IAAIlG,UAkB/B6uC,QACE3mC,SAAS,EACTs3B,OAAO,EACPv+B,MACEihB,SAAS,EACT9E,SAAU,YAEZyD,OACEqB,SAAS,EACT9E,SAAU,cAGduQ,QACEqD,gBAKJx3B,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBACpCr0B,KAAK+vB,OACL/vB,KAAKgG,SACLhG,KAAK8D,OAAS,KACd9D,KAAKm0B,UACLn0B,KAAKs1C,oBAAqB,EAC1Bt1C,KAAKu1C,iBAAkB,EACvBv1C,KAAKw1C,yBAA0B,CAE/B,IAAIrhC,GAAKnU,IACTA,MAAK81B,UAAY,KACjB91B,KAAK+1B,WAAa,KAGlB/1B,KAAK0vC,eACHz8B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAGw7B,OAAO77B,EAAO7R,QAEnB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAGy7B,UAAU97B,EAAO7R,QAEtBoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG07B,UAAU/7B,EAAO7R,SAKxBjC,KAAK8vC,gBACH78B,IAAO,SAAU3J,EAAOwK,GACtBK,EAAG47B,aAAaj8B,EAAO7R,QAEzB4S,OAAU,SAAUvL,EAAOwK,GACzBK,EAAG67B,gBAAgBl8B,EAAO7R,QAE5BoU,OAAU,SAAU/M,EAAOwK,GACzBK,EAAG87B,gBAAgBn8B,EAAO7R,SAI9BjC,KAAKiC,SACLjC,KAAKmwC,aACLnwC,KAAKy1C,UAAYz1C,KAAK20B,KAAKc,MAAM7lB,MACjC5P,KAAKqwC,eAELrwC,KAAKinC,eACLjnC,KAAKkT,WAAWzE,GAChBzO,KAAKwqC,0BAA4B,GACjCxqC,KAAK01C,QAAU,EACf11C,KAAK20B,KAAKE,QAAQthB,GAAG,eAAgB,WACnCY,EAAGshC,UAAYthC,EAAGwgB,KAAKc,MAAM7lB,MAC7BuE,EAAGyxB,IAAI34B,MAAMxF,KAAO9G,EAAKmJ,OAAOK,QAAQgK,EAAGnO,MAAMuM,OACjD4B,EAAGsN,OAAOlhB,KAAK4T,GAAG,KAIpBnU,KAAK00B,UACL10B,KAAKgsC,WAAapG,IAAK5lC,KAAK4lC,IAAKqB,YAAajnC,KAAKinC,YAAax4B,QAASzO,KAAKyO,QAAS0lB,OAAQn0B,KAAKm0B,QACpGn0B,KAAK20B,KAAKE,QAAQjH,KAAK,UAvJzB,GAAIjtB,GAAOT,EAAoB,GAC3BU,EAAUV,EAAoB,GAC9BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BqC,EAAYrC,EAAoB,IAChCwC,EAAWxC,EAAoB,IAC/ByC,EAAazC,EAAoB,IACjC6C,EAAS7C,EAAoB,IAC7By1C,EAAoBz1C,EAAoB,IAExCowC,EAAY,eAiJhBttC,GAAUmQ,UAAY,GAAI5Q,GAK1BS,EAAUmQ,UAAUuhB,QAAU,WAC5B,GAAIpV,GAAQ/N,SAASM,cAAc,MACnCyN,GAAMzX,UAAY,YAClB7H,KAAK+vB,IAAIzQ,MAAQA,EAGjBtf,KAAK4lC,IAAMr0B,SAASC,gBAAgB,6BAA6B,OACjExR,KAAK4lC,IAAI34B,MAAM2W,SAAW,WAC1B5jB,KAAK4lC,IAAI34B,MAAMuF,QAAU,GAAKxS,KAAKyO,QAAQwmC,aAAazqC,QAAQ,KAAK,IAAM,KAC3ExK,KAAK4lC,IAAI34B,MAAM26B,QAAU,QACzBtoB,EAAM7N,YAAYzR,KAAK4lC,KAGvB5lC,KAAKyO,QAAQ2mC,SAAS7gB,YAAc,OACpCv0B,KAAK41C,UAAY,GAAIlzC,GAAS1C,KAAK20B,KAAM30B,KAAKyO,QAAQ2mC,SAAUp1C,KAAK4lC,IAAK5lC,KAAKyO,QAAQ0lB,QAEvFn0B,KAAKyO,QAAQ2mC,SAAS7gB,YAAc,QACpCv0B,KAAK61C,WAAa,GAAInzC,GAAS1C,KAAK20B,KAAM30B,KAAKyO,QAAQ2mC,SAAUp1C,KAAK4lC,IAAK5lC,KAAKyO,QAAQ0lB,cACjFn0B,MAAKyO,QAAQ2mC,SAAS7gB,YAG7Bv0B,KAAK81C,WAAa,GAAI/yC,GAAO/C,KAAK20B,KAAM30B,KAAKyO,QAAQ4mC,OAAQ,OAAQr1C,KAAKyO,QAAQ0lB,QAClFn0B,KAAK+1C,YAAc,GAAIhzC,GAAO/C,KAAK20B,KAAM30B,KAAKyO,QAAQ4mC,OAAQ,QAASr1C,KAAKyO,QAAQ0lB,QAEpFn0B,KAAK2nC,QAOP3kC,EAAUmQ,UAAUD,WAAa,SAASzE,GACxC,GAAIA,EAAS,CACX,GAAIP,IAAU,WAAW,eAAe,SAAS,cAAc,mBAAmB,QAAQ,WAAW,WAAW,OAAO,SAC3F1H,UAAxBiI,EAAQwmC,aAAgDzuC,SAAnBiI,EAAQ+D,QAAsEhM,SAA9CxG,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QAC1GxS,KAAKu1C,iBAAkB,EACvBv1C,KAAKw1C,yBAA0B,GAEsBhvC,SAA9CxG,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,QAAgDhM,SAAxBiI,EAAQwmC,aACtErqC,UAAU6D,EAAQwmC,YAAc,IAAIzqC,QAAQ,KAAK,KAAOxK,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,SAC7FxS,KAAKu1C,iBAAkB,GAG3B50C,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASA,GAC/C9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,cACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UACxC9N,EAAK4N,aAAavO,KAAKyO,QAASA,EAAQ,UAEpCA,EAAQs8B,YACuB,gBAAtBt8B,GAAQs8B,YACbt8B,EAAQs8B,WAAWC,kBACqB,WAAtCv8B,EAAQs8B,WAAWC,gBACrBhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,EAEa,WAAtCx8B,EAAQs8B,WAAWC,gBAC1BhrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,GAGhCjrC,KAAKyO,QAAQs8B,WAAWC,gBAAkB,cAC1ChrC,KAAKyO,QAAQs8B,WAAWE,MAAQ,KAMpCjrC,KAAK41C,WACkBpvC,SAArBiI,EAAQ2mC,WACVp1C,KAAK41C,UAAU1iC,WAAWlT,KAAKyO,QAAQ2mC,UACvCp1C,KAAK61C,WAAW3iC,WAAWlT,KAAKyO,QAAQ2mC,WAIxCp1C,KAAK81C,YACgBtvC,SAAnBiI,EAAQ4mC,SACVr1C,KAAK81C,WAAW5iC,WAAWlT,KAAKyO,QAAQ4mC,QACxCr1C,KAAK+1C,YAAY7iC,WAAWlT,KAAKyO,QAAQ4mC,SAIzCr1C,KAAKm0B,OAAOruB,eAAewqC,IAC7BtwC,KAAKm0B,OAAOmc,GAAWp9B,WAAWzE,GAKlCzO,KAAK+vB,IAAIzQ,OACXtf,KAAKyhB,QAAO,IAOhBze,EAAUmQ,UAAUu0B,KAAO,WAErB1nC,KAAK+vB,IAAIzQ,MAAMzV,YACjB7J,KAAK+vB,IAAIzQ,MAAMzV,WAAWsH,YAAYnR,KAAK+vB,IAAIzQ,QASnDtc,EAAUmQ,UAAUw0B,KAAO,WAEpB3nC,KAAK+vB,IAAIzQ,MAAMzV,YAClB7J,KAAK20B,KAAK5E,IAAI7D,OAAOza,YAAYzR,KAAK+vB,IAAIzQ,QAS9Ctc,EAAUmQ,UAAU8iB,SAAW,SAASh0B,GACtC,GACEkT,GADEhB,EAAKnU,KAEPuyC,EAAevyC,KAAK81B,SAGtB,IAAK7zB,EAGA,CAAA,KAAIA,YAAiBpB,IAAWoB,YAAiBnB,IAIpD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK81B,UAAY7zB,MAHjBjC,MAAK81B,UAAY,IAoBnB,IAXIyc,IAEF5xC,EAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnDipC,EAAa7+B,IAAIpK,EAAOhB,KAI1B6M,EAAMo9B,EAAa18B,SACnB7V,KAAK6vC,UAAU16B,IAGbnV,KAAK81B,UAAW,CAElB,GAAIz1B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK0vC,cAAe,SAAUpnC,EAAUgB,GACnD6K,EAAG2hB,UAAUviB,GAAGjK,EAAOhB,EAAUjI,KAInC8U,EAAMnV,KAAK81B,UAAUjgB,SACrB7V,KAAK2vC,OAAOx6B,GAEdnV,KAAKywC,mBAELzwC,KAAKyhB,QAAO,IAQdze,EAAUmQ,UAAU6iB,UAAY,SAAS7B,GACvC,GACIhf,GADAhB,EAAKnU,IAgBT,IAZIA,KAAK+1B,aACPp1B,EAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWniB,YAAYtK,EAAOhB,KAInC6M,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+1B,WAAa,KAClB/1B,KAAKiwC,gBAAgB96B,IAIlBgf,EAGA,CAAA,KAAIA,YAAkBtzB,IAAWszB,YAAkBrzB,IAItD,KAAM,IAAIuF,WAAU,kDAHpBrG,MAAK+1B,WAAa5B,MAHlBn0B,MAAK+1B,WAAa,IASpB,IAAI/1B,KAAK+1B,WAAY,CAEnB,GAAI11B,GAAKL,KAAKK,EACdM,GAAK0H,QAAQrI,KAAK8vC,eAAgB,SAAUxnC,EAAUgB,GACpD6K,EAAG4hB,WAAWxiB,GAAGjK,EAAOhB,EAAUjI,KAIpC8U,EAAMnV,KAAK+1B,WAAWlgB,SACtB7V,KAAK+vC,aAAa56B,GAEpBnV,KAAK4vC,aASP5sC,EAAUmQ,UAAUy8B,UAAY,WAC9B5vC,KAAKywC,mBACLzwC,KAAKg2C,sBAELh2C,KAAKyhB,QAAO,IAEdze,EAAUmQ,UAAUw8B,OAAkB,SAAUx6B,GAAMnV,KAAK4vC,UAAUz6B,IACrEnS,EAAUmQ,UAAU08B,UAAkB,SAAU16B,GAAMnV,KAAK4vC,UAAUz6B,IACrEnS,EAAUmQ,UAAU68B,gBAAmB,SAAUE,GAC/C,IAAK,GAAI1qC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACxC,GAAIyM,GAAQjS,KAAK+1B,WAAW7gB,IAAIg7B,EAAS1qC,GACzCxF,MAAKi2C,aAAahkC,EAAOi+B,EAAS1qC,IAIpCxF,KAAKyhB,QAAO,IAEdze,EAAUmQ,UAAU48B,aAAe,SAAUG,GAAWlwC,KAAKgwC,gBAAgBE,IAQ7EltC,EAAUmQ,UAAU88B,gBAAkB,SAAUC,GAC9C,IAAK,GAAI1qC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BxF,KAAKm0B,OAAOruB,eAAeoqC,EAAS1qC,MACmB,SAArDxF,KAAKm0B,OAAO+b,EAAS1qC,IAAIiJ,QAAQm9B,kBACnC5rC,KAAK61C,WAAWpO,YAAYyI,EAAS1qC,IACrCxF,KAAK+1C,YAAYtO,YAAYyI,EAAS1qC,IACtCxF,KAAK+1C,YAAYt0B,WAGjBzhB,KAAK41C,UAAUnO,YAAYyI,EAAS1qC,IACpCxF,KAAK81C,WAAWrO,YAAYyI,EAAS1qC,IACrCxF,KAAK81C,WAAWr0B,gBAEXzhB,MAAKm0B,OAAO+b,EAAS1qC,IAGhCxF,MAAKywC,mBAELzwC,KAAKyhB,QAAO,IAWdze,EAAUmQ,UAAU8iC,aAAe,SAAUhkC,EAAOqlB,GAC7Ct3B,KAAKm0B,OAAOruB,eAAewxB,IAY9Bt3B,KAAKm0B,OAAOmD,GAASziB,OAAO5C,GACyB,SAAjDjS,KAAKm0B,OAAOmD,GAAS7oB,QAAQm9B,kBAC/B5rC,KAAK61C,WAAWrO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,IACjDt3B,KAAK+1C,YAAYvO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,MAGlDt3B,KAAK41C,UAAUpO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,IAChDt3B,KAAK81C,WAAWtO,YAAYlQ,EAASt3B,KAAKm0B,OAAOmD,OAlBnDt3B,KAAKm0B,OAAOmD,GAAW,GAAI30B,GAAWsP,EAAOqlB,EAASt3B,KAAKyO,QAASzO,KAAKwqC,0BACpB,SAAjDxqC,KAAKm0B,OAAOmD,GAAS7oB,QAAQm9B,kBAC/B5rC,KAAK61C,WAAWvO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,IAC9Ct3B,KAAK+1C,YAAYzO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,MAG/Ct3B,KAAK41C,UAAUtO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,IAC7Ct3B,KAAK81C,WAAWxO,SAAShQ,EAASt3B,KAAKm0B,OAAOmD,MAclDt3B,KAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,UASnBze,EAAUmQ,UAAU6iC,oBAAsB,WACxC,GAAsB,MAAlBh2C,KAAK81B,UAAmB,CAC1B,GACIwB,GADA4e,IAEJ,KAAK5e,IAAWt3B,MAAKm0B,OACfn0B,KAAKm0B,OAAOruB,eAAewxB,KAC7B4e,EAAc5e,MAGlB,KAAK,GAAI/hB,KAAUvV,MAAK81B,UAAUljB,MAChC,GAAI5S,KAAK81B,UAAUljB,MAAM9M,eAAeyP,GAAS,CAC/C,GAAIlG,GAAOrP,KAAK81B,UAAUljB,MAAM2C,EAChC,IAAkC/O,SAA9B0vC,EAAc7mC,EAAK4C,OACrB,KAAM,IAAIrO,OAAM,4IAElByL,GAAK0C,EAAIpR,EAAKkG,QAAQwI,EAAK0C,EAAE,QAC7BmkC,EAAc7mC,EAAK4C,OAAOjK,KAAKqH,GAGnC,IAAKioB,IAAWt3B,MAAKm0B,OACfn0B,KAAKm0B,OAAOruB,eAAewxB,IAC7Bt3B,KAAKm0B,OAAOmD,GAASrB,SAASigB,EAAc5e,MAYpDt0B,EAAUmQ,UAAUs9B,iBAAmB,WACrC,GAAIzwC,KAAK81B,WAA+B,MAAlB91B,KAAK81B,UAAmB,CAC5C,GAAIqgB,GAAmB,CACvB,KAAK,GAAI5gC,KAAUvV,MAAK81B,UAAUljB,MAChC,GAAI5S,KAAK81B,UAAUljB,MAAM9M,eAAeyP,GAAS,CAC/C,GAAIlG,GAAOrP,KAAK81B,UAAUljB,MAAM2C,EACpB/O,SAAR6I,IACEA,EAAKvJ,eAAe,SACHU,SAAf6I,EAAK4C,QACP5C,EAAK4C,MAAQq+B,GAIfjhC,EAAK4C,MAAQq+B,EAEf6F,EAAmB9mC,EAAK4C,OAASq+B,EAAY6F,EAAmB,EAAIA,GAK1E,GAAwB,GAApBA,QACKn2C,MAAKm0B,OAAOmc,GACnBtwC,KAAK81C,WAAWrO,YAAY6I,GAC5BtwC,KAAK+1C,YAAYtO,YAAY6I,GAC7BtwC,KAAK41C,UAAUnO,YAAY6I,GAC3BtwC,KAAK61C,WAAWpO,YAAY6I,OAEzB,CACH,GAAIr+B,IAAS5R,GAAIiwC,EAAW1gB,QAAS5vB,KAAKyO,QAAQsmC,aAClD/0C,MAAKi2C,aAAahkC,EAAOq+B,eAIpBtwC,MAAKm0B,OAAOmc,GACnBtwC,KAAK81C,WAAWrO,YAAY6I,GAC5BtwC,KAAK+1C,YAAYtO,YAAY6I,GAC7BtwC,KAAK41C,UAAUnO,YAAY6I,GAC3BtwC,KAAK61C,WAAWpO,YAAY6I,EAG9BtwC,MAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,UAQnBze,EAAUmQ,UAAUsO,OAAS,SAAS20B,GACpC,GAAI3R,IAAU,CAGdzkC,MAAKgG,MAAMuM,MAAQvS,KAAK+vB,IAAIzQ,MAAM8Q,YAClCpwB,KAAKgG,MAAMwM,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAGhChM,SAAnBxG,KAAK2xC,WAA2B3xC,KAAKgG,MAAMuM,QAC7C6jC,GAAmB,GAIrB3R,EAAUzkC,KAAKwkC,cAAgBC,CAG/B,IAAI+M,GAAkBxxC,KAAK20B,KAAKc,MAAM5lB,IAAM7P,KAAK20B,KAAKc,MAAM7lB,MACxD6hC,EAAUD,GAAmBxxC,KAAK0xC,mBA6BtC,IA5BA1xC,KAAK0xC,oBAAsBF,EAKZ,GAAX/M,IACFzkC,KAAK4lC,IAAI34B,MAAMsF,MAAQ5R,EAAKmJ,OAAOK,OAAO,EAAEnK,KAAKgG,MAAMuM,OACvDvS,KAAK4lC,IAAI34B,MAAMxF,KAAO9G,EAAKmJ,OAAOK,QAAQnK,KAAKgG,MAAMuM,QAGN,KAA1CvS,KAAKyO,QAAQ+D,OAAS,IAAI7L,QAAQ,MAA8C,GAAhC3G,KAAKw1C,2BACxDx1C,KAAKu1C,iBAAkB,IAKC,GAAxBv1C,KAAKu1C,iBACHv1C,KAAKyO,QAAQwmC,aAAej1C,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,OAC1ExS,KAAKyO,QAAQwmC,YAAcj1C,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,KACvExS,KAAK4lC,IAAI34B,MAAMuF,OAASxS,KAAK20B,KAAKC,SAASiD,gBAAgBrlB,OAAS,MAEtExS,KAAKu1C,iBAAkB,GAGvBv1C,KAAK4lC,IAAI34B,MAAMuF,QAAU,GAAKxS,KAAKyO,QAAQwmC,aAAazqC,QAAQ,KAAK,IAAM,KAI9D,GAAXi6B,GAA6B,GAAVgN,GAA6C,GAA3BzxC,KAAKs1C,oBAAkD,GAApBc,EAC1E3R,EAAUzkC,KAAKq2C,gBAAkB5R,MAIjC,IAAsB,GAAlBzkC,KAAKy1C,UAAgB,CACvB,GAAI9rB,GAAS3pB,KAAK20B,KAAKc,MAAM7lB,MAAQ5P,KAAKy1C,UACtChgB,EAAQz1B,KAAK20B,KAAKc,MAAM5lB,IAAM7P,KAAK20B,KAAKc,MAAM7lB,KAClD,IAAwB,GAApB5P,KAAKgG,MAAMuM,MAAY,CACzB,GAAI+jC,GAAmBt2C,KAAKgG,MAAMuM,MAAMkjB,EACpC7L,EAAUD,EAAS2sB,CACvBt2C,MAAK4lC,IAAI34B,MAAMxF,MAASzH,KAAKgG,MAAMuM,MAAQqX,EAAW,MAO5D,MAFA5pB,MAAK81C,WAAWr0B,SAChBzhB,KAAK+1C,YAAYt0B,SACVgjB,GAQTzhC,EAAUmQ,UAAUkjC,aAAe,WAGjC,GADAz1C,EAAQiQ,gBAAgB7Q,KAAKinC,aACL,GAApBjnC,KAAKgG,MAAMuM,OAAgC,MAAlBvS,KAAK81B,UAAmB,CACnD,GAAI7jB,GAAOzM,EACP+wC,KACAC,KACAC,KACAC,GAAe,EAGfxG,IACJ,KAAK,GAAI5Y,KAAWt3B,MAAKm0B,OACnBn0B,KAAKm0B,OAAOruB,eAAewxB,KAC7BrlB,EAAQjS,KAAKm0B,OAAOmD,GACC,GAAjBrlB,EAAMyW,SAAgEliB,SAA5CxG,KAAKyO,QAAQ0lB,OAAOqD,WAAWF,IAAqE,GAA3Ct3B,KAAKyO,QAAQ0lB,OAAOqD,WAAWF,IACpH4Y,EAASloC,KAAKsvB,GAIpB,IAAI4Y,EAASvqC,OAAS,EAAG,CAEvB,GAAIgxC,GAAU32C,KAAK20B,KAAKh0B,KAAK40B,cAAcv1B,KAAK20B,KAAKC,SAASl1B,KAAK6S,OAC/DqkC,EAAU52C,KAAK20B,KAAKh0B,KAAK40B,aAAa,EAAIv1B,KAAK20B,KAAKC,SAASl1B,KAAK6S,OAClEwjB,IAQJ,KANA/1B,KAAK62C,iBAAiB3G,EAAUna,EAAY4gB,EAASC,GAGrD52C,KAAK82C,eAAe5G,EAAUna,GAGzBvwB,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/B+wC,EAAsBrG,EAAS1qC,IAAMxF,KAAK+2C,qBAAqBhhB,EAAWma,EAAS1qC,IAIrFxF,MAAKg3C,YAAY9G,EAAUqG,EAAuBE,GAIlDC,EAAe12C,KAAKi3C,aAAa/G,EAAUuG,EAC3C,IAAIS,GAAa,CACjB,IAAoB,GAAhBR,GAAwB12C,KAAK01C,QAAUwB,EAKzC,MAJAt2C,GAAQsQ,gBAAgBlR,KAAKinC,aAC7BjnC,KAAKs1C,oBAAqB,EAC1Bt1C,KAAK01C,UACL11C,KAAK20B,KAAKE,QAAQjH,KAAK,WAChB,CAUP,KAPI5tB,KAAK01C,QAAUwB,GACjBve,QAAQhF,IAAI,6EAEd3zB,KAAK01C,QAAU,EACf11C,KAAKs1C,oBAAqB,EAGrB9vC,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/ByM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAC7BgxC,EAAmBtG,EAAS1qC,IAAMxF,KAAKm3C,qBAAqBphB,EAAWma,EAAS1qC,IAAKyM,EAIvF,KAAKzM,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/ByM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IACF,OAAvByM,EAAMxD,QAAQxB,OAChBgF,EAAM85B,KAAKyK,EAAmBtG,EAAS1qC,IAAKyM,EAAOjS,KAAKgsC,UAG5D2J,GAAkB5J,KAAKmE,EAAUsG,EAAoBx2C,KAAKgsC,YAOhE,MADAprC,GAAQsQ,gBAAgBlR,KAAKinC,cACtB,GAiBTjkC,EAAUmQ,UAAU0jC,iBAAmB,SAAU3G,EAAUna,EAAY4gB,EAASC,GAC9E,GAAI3kC,GAAOzM,EAAGomB,EAAGvc,CACjB,IAAI6gC,EAASvqC,OAAS,EACpB,IAAKH,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACpCyM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAC7BuwB,EAAWma,EAAS1qC,MACpB,IAAI4xC,GAAgBrhB,EAAWma,EAAS1qC,GAExC;GAA0B,GAAtByM,EAAMxD,QAAQyH,KAAc,CAC9B,GAAImhC,GAAQnyC,KAAKwH,IAAI,EAAG/L,EAAK4O,kBAAkB0C,EAAM6jB,UAAW6gB,EAAS,IAAK,UAC9E,KAAK/qB,EAAIyrB,EAAOzrB,EAAI3Z,EAAM6jB,UAAUnwB,OAAQimB,IAE1C,GADAvc,EAAO4C,EAAM6jB,UAAUlK,GACVplB,SAAT6I,EAAoB,CACtB,GAAIA,EAAK0C,EAAI6kC,EAAS,CACpBQ,EAAcpvC,KAAKqH,EACnB,OAGA+nC,EAAcpvC,KAAKqH,QAMzB,KAAKuc,EAAI,EAAGA,EAAI3Z,EAAM6jB,UAAUnwB,OAAQimB,IACtCvc,EAAO4C,EAAM6jB,UAAUlK,GACVplB,SAAT6I,GACEA,EAAK0C,EAAI4kC,GAAWtnC,EAAK0C,EAAI6kC,GAC/BQ,EAAcpvC,KAAKqH,KAgBjCrM,EAAUmQ,UAAU2jC,eAAiB,SAAU5G,EAAUna,GACvD,GAAI9jB,EACJ,IAAIi+B,EAASvqC,OAAS,EACpB,IAAK,GAAIH,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAEnC,GADAyM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IACC,GAA1ByM,EAAMxD,QAAQumC,SAAkB,CAClC,GAAIoC,GAAgBrhB,EAAWma,EAAS1qC,GACxC,IAAI4xC,EAAczxC,OAAS,EAAG,CAC5B,GAAI2xC,GAAY,EACZC,EAAiBH,EAAczxC,OAI/B6xC,EAAYx3C,KAAK20B,KAAKh0B,KAAKw0B,eAAeiiB,EAAcA,EAAczxC,OAAS,GAAGoM,GAAK/R,KAAK20B,KAAKh0B,KAAKw0B,eAAeiiB,EAAc,GAAGrlC,GACtI0lC,EAAiBF,EAAiBC,CACtCF,GAAYpyC,KAAK4G,IAAI5G,KAAKwyC,KAAK,GAAMH,GAAiBryC,KAAKwH,IAAI,EAAGxH,KAAKwoB,MAAM+pB,IAG7E,KAAK,GADDE,MACK/rB,EAAI,EAAO2rB,EAAJ3rB,EAAoBA,GAAK0rB,EACvCK,EAAY3vC,KAAKovC,EAAcxrB,GAGjCmK,GAAWma,EAAS1qC,IAAMmyC,KAgBpC30C,EAAUmQ,UAAU6jC,YAAc,SAAU9G,EAAUna,EAAY0gB,GAChE,GAAI3K,GAAW75B,EAAOzM,EAGlBiJ,EAFAmpC,KACAC,IAEJ,IAAI3H,EAASvqC,OAAS,EAAG,CACvB,IAAKH,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BsmC,EAAY/V,EAAWma,EAAS1qC,IAChCiJ,EAAUzO,KAAKm0B,OAAO+b,EAAS1qC,IAAIiJ,QAC/Bq9B,EAAUnmC,OAAS,IACrBsM,EAAQjS,KAAKm0B,OAAO+b,EAAS1qC,IAES,SAAlCiJ,EAAQymC,SAASC,eAA6C,OAAjB1mC,EAAQxB,MACvB,QAA5BwB,EAAQm9B,iBAA6BgM,EAAuBA,EAAoB5jC,OAAO/B,EAAM45B,UAAUC,IAClE+L,EAAuBA,EAAqB7jC,OAAO/B,EAAM45B,UAAUC,IAG5G2K,EAAYvG,EAAS1qC,IAAMyM,EAAM45B,UAAUC,EAAUoE,EAAS1qC,IAMpEmwC,GAAkBmC,oBAAoBF,EAAsBnB,EAAavG,EAAU,iBAAmB,QACtGyF,EAAkBmC,oBAAoBD,EAAsBpB,EAAavG,EAAU,kBAAmB,WAW1GltC,EAAUmQ,UAAU8jC,aAAe,SAAU/G,EAAUuG,GACrD,GAGoEsB,GAAQC,EAHxEvT,GAAU,EACVwT,GAAgB,EAChBC,GAAiB,EACjBC,EAAU,IAAKC,EAAW,IAAKC,EAAU,KAAMC,EAAW,IAE9D,IAAIpI,EAASvqC,OAAS,EAAG,CAEvB,IAAK,GAAIH,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAAK,CACxC,GAAIyM,GAAQjS,KAAKm0B,OAAO+b,EAAS1qC,GAC7ByM,IAA2C,SAAlCA,EAAMxD,QAAQm9B,kBACzBqM,GAAgB,EAChBE,EAAU,EACVE,EAAU,GAEHpmC,GAASA,EAAMxD,QAAQm9B,mBAC9BsM,GAAiB,EACjBE,EAAW,EACXE,EAAW,GAKf,IAAK,GAAI9yC,GAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAC/BixC,EAAY3wC,eAAeoqC,EAAS1qC,KAClCixC,EAAYvG,EAAS1qC,IAAI+yC,UAAW,IACtCR,EAAStB,EAAYvG,EAAS1qC,IAAIsG,IAClCksC,EAASvB,EAAYvG,EAAS1qC,IAAIkH,IAEe,SAA7C+pC,EAAYvG,EAAS1qC,IAAIomC,kBAC3BqM,GAAgB,EAChBE,EAAUA,EAAUJ,EAASA,EAASI,EACtCE,EAAoBL,EAAVK,EAAmBL,EAASK,IAGtCH,GAAiB,EACjBE,EAAWA,EAAWL,EAASA,EAASK,EACxCE,EAAsBN,EAAXM,EAAoBN,EAASM,GAM3B,IAAjBL,GACFj4C,KAAK41C,UAAUriB,SAAS4kB,EAASE,GAEb,GAAlBH,GACFl4C,KAAK61C,WAAWtiB,SAAS6kB,EAAUE,GAoCvC,MAjCA7T,GAAUzkC,KAAKw4C,qBAAqBP,EAAgBj4C,KAAK41C,YAAenR,EACxEA,EAAUzkC,KAAKw4C,qBAAqBN,EAAgBl4C,KAAK61C,aAAepR,EAElD,GAAlByT,GAA2C,GAAjBD,GAC5Bj4C,KAAK41C,UAAU6C,WAAY,EAC3Bz4C,KAAK61C,WAAW4C,WAAY,IAG5Bz4C,KAAK41C,UAAU6C,WAAY,EAC3Bz4C,KAAK61C,WAAW4C,WAAY,GAE9Bz4C,KAAK61C,WAAW7O,QAAUiR,EACI,GAA1Bj4C,KAAK61C,WAAW7O,QACWhnC,KAAK41C,UAAU7O,WAAtB,GAAlBmR,EAAqDl4C,KAAK61C,WAAWtjC,MAChB,EAEzDkyB,EAAUzkC,KAAK41C,UAAUn0B,UAAYgjB,EACrCzkC,KAAK61C,WAAWhP,iBAAmB7mC,KAAK41C,UAAUhP,WAClD5mC,KAAK61C,WAAW/O,aAAe9mC,KAAK41C,UAAU9O,aAC9CrC,EAAUzkC,KAAK61C,WAAWp0B,UAAYgjB,GAGtCA,EAAUzkC,KAAK61C,WAAWp0B,UAAYgjB,EAIE,IAAtCyL,EAASvpC,QAAQ,mBACnBupC,EAAS9nC,OAAO8nC,EAASvpC,QAAQ,kBAAkB,GAEV,IAAvCupC,EAASvpC,QAAQ,oBACnBupC,EAAS9nC,OAAO8nC,EAASvpC,QAAQ,mBAAmB,GAG/C89B,GAYTzhC,EAAUmQ,UAAUqlC,qBAAuB,SAAUE,EAAU5X,GAC7D,GAAI7B,IAAU,CAad,OAZgB,IAAZyZ,EACE5X,EAAK/Q,IAAIzQ,MAAMzV,YAA6B,GAAfi3B,EAAK5H,SACpC4H,EAAK4G,OACLzI,GAAU,GAIP6B,EAAK/Q,IAAIzQ,MAAMzV,YAA6B,GAAfi3B,EAAK5H,SACrC4H,EAAK6G,OACL1I,GAAU,GAGPA,GAaTj8B,EAAUmQ,UAAU4jC,qBAAuB,SAAU4B,GAKnD,IAAK,GAHDC,GAAQC,EADRC,KAEA7jB,EAAWj1B,KAAK20B,KAAKh0B,KAAKs0B,SAErBzvB,EAAI,EAAGA,EAAImzC,EAAWhzC,OAAQH,IACrCozC,EAAS3jB,EAAS0jB,EAAWnzC,GAAGuM,GAAK/R,KAAKgG,MAAMuM,MAChDsmC,EAASF,EAAWnzC,GAAGwM,EACvB8mC,EAAc9wC,MAAM+J,EAAG6mC,EAAQ5mC,EAAG6mC,GAGpC,OAAOC,IAcT91C,EAAUmQ,UAAUgkC,qBAAuB,SAAUwB,EAAY1mC,GAC/D,GACI2mC,GAAQC,EADRC,KAEA7jB,EAAWj1B,KAAK20B,KAAKh0B,KAAKs0B,SAC1B6L,EAAO9gC,KAAK41C,UACZmD,EAAY70C,OAAOlE,KAAK4lC,IAAI34B,MAAMuF,OAAOhI,QAAQ,KAAK,IACpB,UAAlCyH,EAAMxD,QAAQm9B,mBAChB9K,EAAO9gC,KAAK61C,WAGd,KAAK,GAAIrwC,GAAI,EAAGA,EAAImzC,EAAWhzC,OAAQH,IACrCozC,EAAS3jB,EAAS0jB,EAAWnzC,GAAGuM,GAAK/R,KAAKgG,MAAMuM,MAChDsmC,EAAS3zC,KAAKwoB,MAAMoT,EAAK2I,aAAakP,EAAWnzC,GAAGwM,IACpD8mC,EAAc9wC,MAAM+J,EAAG6mC,EAAQ5mC,EAAG6mC,GAKpC,OAFA5mC,GAAM64B,gBAAgB5lC,KAAK4G,IAAIitC,EAAWjY,EAAK2I,aAAa,KAErDqP,GAITj5C,EAAOD,QAAUoD,GAKb,SAASnD,EAAQD,EAASM,GAgB9B,QAAS+C,GAAU0xB,EAAMlmB,GACvBzO,KAAK+vB,KACH0c,WAAY,KACZjG,SACAwS,cACAC,cACAjoC,WACEw1B,SACAwS,cACAC,gBAGJj5C,KAAKgG,OACHyvB,OACE7lB,MAAO,EACPC,IAAK,EACLurB,YAAa,GAEf8d,QAAS,GAGXl5C,KAAKq0B,gBACHE,YAAa,SAEbuR,iBAAiB,EACjBC,iBAAiB,EACjBzE,OAAQ,MAEVthC,KAAKyO,QAAU9N,EAAK2E,UAAWtF,KAAKq0B,gBAEpCr0B,KAAK20B,KAAOA,EAGZ30B,KAAK00B,UAEL10B,KAAKkT,WAAWzE,GAjDlB,GAAI9N,GAAOT,EAAoB,GAC3BqC,EAAYrC,EAAoB,IAChC6B,EAAW7B,EAAoB,IAC/ByB,EAAWzB,EAAoB,IAC/B2D,EAAS3D,EAAoB,GAgDjC+C,GAASkQ,UAAY,GAAI5Q,GAUzBU,EAASkQ,UAAUD,WAAa,SAASzE,GACnCA,IAEF9N,EAAKoF,iBACH,cACA,kBACA,kBACA,cACA,UACC/F,KAAKyO,QAASA,GAIb,UAAYA,KACe,kBAAlB5K,GAAOwgC,OAEhBxgC,EAAOwgC,OAAO51B,EAAQ41B,QAGtBxgC,EAAOygC,KAAK71B,EAAQ41B,WAS5BphC,EAASkQ,UAAUuhB,QAAU,WAC3B10B,KAAK+vB,IAAI0c,WAAal7B,SAASM,cAAc,OAC7C7R,KAAK+vB,IAAI5jB,WAAaoF,SAASM,cAAc,OAE7C7R,KAAK+vB,IAAI0c,WAAW5kC,UAAY,sBAChC7H,KAAK+vB,IAAI5jB,WAAWtE,UAAY,uBAMlC5E,EAASkQ,UAAUG,QAAU,WAEvBtT,KAAK+vB,IAAI0c,WAAW5iC,YACtB7J,KAAK+vB,IAAI0c,WAAW5iC,WAAWsH,YAAYnR,KAAK+vB,IAAI0c,YAElDzsC,KAAK+vB,IAAI5jB,WAAWtC,YACtB7J,KAAK+vB,IAAI5jB,WAAWtC,WAAWsH,YAAYnR,KAAK+vB,IAAI5jB,YAGtDnM,KAAK20B,KAAO,MAOd1xB,EAASkQ,UAAUsO,OAAS,WAC1B,GAAIhT,GAAUzO,KAAKyO,QACfzI,EAAQhG,KAAKgG,MACbymC,EAAazsC,KAAK+vB,IAAI0c,WACtBtgC,EAAanM,KAAK+vB,IAAI5jB,WAGtB24B,EAAiC,OAAvBr2B,EAAQ8lB,YAAwBv0B,KAAK20B,KAAK5E,IAAIpoB,IAAM3H,KAAK20B,KAAK5E,IAAIzM,OAC5E61B,EAAiB1M,EAAW5iC,aAAei7B,CAG/C9kC,MAAKooC,oBAGL,IACItC,IADc9lC,KAAKyO,QAAQ8lB,YACTv0B,KAAKyO,QAAQq3B,iBAC/BC,EAAkB/lC,KAAKyO,QAAQs3B,eAGnC//B,GAAMqiC,iBAAmBvC,EAAkB9/B,EAAMsiC,gBAAkB,EACnEtiC,EAAMuiC,iBAAmBxC,EAAkB//B,EAAMwiC,gBAAkB,EACnExiC,EAAMwM,OAASxM,EAAMqiC,iBAAmBriC,EAAMuiC,iBAC9CviC,EAAMuM,MAAQk6B,EAAWrc,YAEzBpqB,EAAM0iC,gBAAkB1oC,KAAK20B,KAAKC,SAASl1B,KAAK8S,OAASxM,EAAMuiC,kBACnC,OAAvB95B,EAAQ8lB,YAAuBv0B,KAAK20B,KAAKC,SAAStR,OAAO9Q,OAASxS,KAAK20B,KAAKC,SAASjtB,IAAI6K,QAC9FxM,EAAMyiC,eAAiB,EACvBziC,EAAM4iC,gBAAkB5iC,EAAM0iC,gBAAkB1iC,EAAMuiC,iBACtDviC,EAAM2iC,eAAiB,CAGvB,IAAIyQ,GAAwB3M,EAAW4M,YACnCC,EAAwBntC,EAAWktC,WAsBvC,OArBA5M,GAAW5iC,YAAc4iC,EAAW5iC,WAAWsH,YAAYs7B,GAC3DtgC,EAAWtC,YAAcsC,EAAWtC,WAAWsH,YAAYhF,GAE3DsgC,EAAWx/B,MAAMuF,OAASxS,KAAKgG,MAAMwM,OAAS,KAE9CxS,KAAKu5C,iBAGDH,EACFtU,EAAOlzB,aAAa66B,EAAY2M,GAGhCtU,EAAOrzB,YAAYg7B,GAEjB6M,EACFt5C,KAAK20B,KAAK5E,IAAIgV,mBAAmBnzB,aAAazF,EAAYmtC,GAG1Dt5C,KAAK20B,KAAK5E,IAAIgV,mBAAmBtzB,YAAYtF,GAGxCnM,KAAKwkC,cAAgB2U,GAO9Bl2C,EAASkQ,UAAUomC,eAAiB,WAClC,GAAIhlB,GAAcv0B,KAAKyO,QAAQ8lB,YAG3B3kB,EAAQjP,EAAKkG,QAAQ7G,KAAK20B,KAAKc,MAAM7lB,MAAO,UAC5CC,EAAMlP,EAAKkG,QAAQ7G,KAAK20B,KAAKc,MAAM5lB,IAAK,UACxC2pC,EAAgBx5C,KAAK20B,KAAKh0B,KAAK00B,OAA2C,GAAnCr1B,KAAKgG,MAAM+jC,gBAAkB,KAAS/iC,UAC7Eo0B,EAAcoe,EAAgB73C,EAASk5B,wBAAwB76B,KAAK20B,KAAKI,YAAa/0B,KAAK20B,KAAKc,MAAO+jB,EAC3Gpe,IAAep7B,KAAK20B,KAAKh0B,KAAK00B,OAAO,GAAGruB,SAExC,IAAImhB,GAAO,GAAIpmB,GAAS,GAAIuC,MAAKsL,GAAQ,GAAItL,MAAKuL,GAAMurB,EAAap7B,KAAK20B,KAAKI,YAC3E/0B,MAAKyO,QAAQ6yB,QACfnZ,EAAK4Z,UAAU/hC,KAAKyO,QAAQ6yB,QAE9BthC,KAAKmoB,KAAOA,CAKZ,IAAI4H,GAAM/vB,KAAK+vB,GACfA,GAAI/e,UAAUw1B,MAAQzW,EAAIyW,MAC1BzW,EAAI/e,UAAUgoC,WAAajpB,EAAIipB,WAC/BjpB,EAAI/e,UAAUioC,WAAalpB,EAAIkpB,WAC/BlpB,EAAIyW,SACJzW,EAAIipB,cACJjpB,EAAIkpB,aAEJ,IAAIQ,GAEAvc,EAGAwc,EAGA7xC,EAPAkK,EAAI,EAEJ4nC,EAAQ,EACRpnC,EAAQ,EAERqnC,EAAmBpzC,OACnBkG,EAAM,CAIV,KADAyb,EAAK8Z,QACE9Z,EAAK0U,WAAmB,IAANnwB,GACvBA,IAEA+sC,EAAMtxB,EAAKC,aACX8U,EAAU/U,EAAK+U,UACfr1B,EAAYsgB,EAAK2b,eAEjB6V,EAAQ5nC,EACRA,EAAI/R,KAAK20B,KAAKh0B,KAAKs0B,SAASwkB,GAC5BlnC,EAAQR,EAAI4nC,EACRD,IACFA,EAASzsC,MAAMsF,MAAQA,EAAQ,MAG7BvS,KAAKyO,QAAQq3B,iBACf9lC,KAAK65C,kBAAkB9nC,EAAGoW,EAAKyb,gBAAiBrP,EAAa1sB,GAG3Dq1B,GAAWl9B,KAAKyO,QAAQs3B,iBACtBh0B,EAAI,IACkBvL,QAApBozC,IACFA,EAAmB7nC,GAErB/R,KAAK85C,kBAAkB/nC,EAAGoW,EAAK0b,gBAAiBtP,EAAa1sB,IAE/D6xC,EAAW15C,KAAK+5C,kBAAkBhoC,EAAGwiB,EAAa1sB,IAGlD6xC,EAAW15C,KAAKg6C,kBAAkBjoC,EAAGwiB,EAAa1sB,GAGpDsgB,EAAKE,MAIP,IAAIroB,KAAKyO,QAAQs3B,gBAAiB,CAChC,GAAIkU,GAAWj6C,KAAK20B,KAAKh0B,KAAK00B,OAAO,GACjC6kB,EAAW/xB,EAAK0b,cAAcoW,GAC9BE,EAAYD,EAASv0C,QAAU3F,KAAKgG,MAAM8jC,gBAAkB,IAAM,IAE9CtjC,QAApBozC,GAA6CA,EAAZO,IACnCn6C,KAAK85C,kBAAkB,EAAGI,EAAU3lB,EAAa1sB,GAKrDlH,EAAK0H,QAAQrI,KAAK+vB,IAAI/e,UAAW,SAAUopC,GACzC,KAAOA,EAAIz0C,QAAQ,CACjB,GAAI4B,GAAO6yC,EAAIC,KACX9yC,IAAQA,EAAKsC,YACftC,EAAKsC,WAAWsH,YAAY5J,OAcpCtE,EAASkQ,UAAU0mC,kBAAoB,SAAU9nC,EAAGwX,EAAMgL,EAAa1sB,GAErE,GAAI4gB,GAAQzoB,KAAK+vB,IAAI/e,UAAUioC,WAAW3nC,OAE1C,KAAKmX,EAAO,CAEV,GAAImH,GAAUre,SAAS04B,eAAe,GACtCxhB,GAAQlX,SAASM,cAAc,OAC/B4W,EAAMhX,YAAYme,GAClB5vB,KAAK+vB,IAAI0c,WAAWh7B,YAAYgX,GAElCzoB,KAAK+vB,IAAIkpB,WAAWjxC,KAAKygB,GAEzBA,EAAM6xB,WAAW,GAAGC,UAAYhxB,EAEhCd,EAAMxb,MAAMtF,IAAsB,OAAf4sB,EAAyBv0B,KAAKgG,MAAMuiC,iBAAmB,KAAQ,IAClF9f,EAAMxb,MAAMxF,KAAOsK,EAAI,KACvB0W,EAAM5gB,UAAY,cAAgBA,GAYpC5E,EAASkQ,UAAU2mC,kBAAoB,SAAU/nC,EAAGwX,EAAMgL,EAAa1sB,GAErE,GAAI4gB,GAAQzoB,KAAK+vB,IAAI/e,UAAUgoC,WAAW1nC,OAE1C,KAAKmX,EAAO,CAEV,GAAImH,GAAUre,SAAS04B,eAAe1gB,EACtCd,GAAQlX,SAASM,cAAc,OAC/B4W,EAAMhX,YAAYme,GAClB5vB,KAAK+vB,IAAI0c,WAAWh7B,YAAYgX,GAElCzoB,KAAK+vB,IAAIipB,WAAWhxC,KAAKygB,GAEzBA,EAAM6xB,WAAW,GAAGC,UAAYhxB,EAChCd,EAAM5gB,UAAY,cAAgBA,EAGlC4gB,EAAMxb,MAAMtF,IAAsB,OAAf4sB,EAAwB,IAAOv0B,KAAKgG,MAAMqiC,iBAAoB,KACjF5f,EAAMxb,MAAMxF,KAAOsK,EAAI,MAWzB9O,EAASkQ,UAAU6mC,kBAAoB,SAAUjoC,EAAGwiB,EAAa1sB,GAE/D,GAAIgoB,GAAO7vB,KAAK+vB,IAAI/e,UAAUw1B,MAAMl1B,OAC/Bue,KAEHA,EAAOte,SAASM,cAAc,OAC9B7R,KAAK+vB,IAAI5jB,WAAWsF,YAAYoe,IAElC7vB,KAAK+vB,IAAIyW,MAAMx+B,KAAK6nB,EAEpB,IAAI7pB,GAAQhG,KAAKgG,KAYjB,OAVE6pB,GAAK5iB,MAAMtF,IADM,OAAf4sB,EACevuB,EAAMuiC,iBAAmB,KAGzBvoC,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAAS,KAEnDqd,EAAK5iB,MAAMuF,OAASxM,EAAM0iC,gBAAkB,KAC5C7Y,EAAK5iB,MAAMxF,KAAQsK,EAAI/L,EAAMyiC,eAAiB,EAAK,KAEnD5Y,EAAKhoB,UAAY,uBAAyBA,EAEnCgoB,GAWT5sB,EAASkQ,UAAU4mC,kBAAoB,SAAUhoC,EAAGwiB,EAAa1sB,GAE/D,GAAIgoB,GAAO7vB,KAAK+vB,IAAI/e,UAAUw1B,MAAMl1B,OAC/Bue,KAEHA,EAAOte,SAASM,cAAc,OAC9B7R,KAAK+vB,IAAI5jB,WAAWsF,YAAYoe,IAElC7vB,KAAK+vB,IAAIyW,MAAMx+B,KAAK6nB,EAEpB,IAAI7pB,GAAQhG,KAAKgG,KAYjB,OAVE6pB,GAAK5iB,MAAMtF,IADM,OAAf4sB,EACe,IAGAv0B,KAAK20B,KAAKC,SAASjtB,IAAI6K,OAAS,KAEnDqd,EAAK5iB,MAAMxF,KAAQsK,EAAI/L,EAAM2iC,eAAiB,EAAK,KACnD9Y,EAAK5iB,MAAMuF,OAASxM,EAAM4iC,gBAAkB,KAE5C/Y,EAAKhoB,UAAY,uBAAyBA,EAEnCgoB,GAQT5sB,EAASkQ,UAAUi1B,mBAAqB,WAKjCpoC,KAAK+vB,IAAIma,mBACZlqC,KAAK+vB,IAAIma,iBAAmB34B,SAASM,cAAc,OACnD7R,KAAK+vB,IAAIma,iBAAiBriC,UAAY,qBACtC7H,KAAK+vB,IAAIma,iBAAiBj9B,MAAM2W,SAAW,WAE3C5jB,KAAK+vB,IAAIma,iBAAiBz4B,YAAYF,SAAS04B,eAAe,MAC9DjqC,KAAK+vB,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAIma,mBAE3ClqC,KAAKgG,MAAMsiC,gBAAkBtoC,KAAK+vB,IAAIma,iBAAiBrlB,aACvD7kB,KAAKgG,MAAM+jC,eAAiB/pC,KAAK+vB,IAAIma,iBAAiB1qB,YAGjDxf,KAAK+vB,IAAIqa,mBACZpqC,KAAK+vB,IAAIqa,iBAAmB74B,SAASM,cAAc,OACnD7R,KAAK+vB,IAAIqa,iBAAiBviC,UAAY,qBACtC7H,KAAK+vB,IAAIqa,iBAAiBn9B,MAAM2W,SAAW,WAE3C5jB,KAAK+vB,IAAIqa,iBAAiB34B,YAAYF,SAAS04B,eAAe,MAC9DjqC,KAAK+vB,IAAI0c,WAAWh7B,YAAYzR,KAAK+vB,IAAIqa,mBAE3CpqC,KAAKgG,MAAMwiC,gBAAkBxoC,KAAK+vB,IAAIqa,iBAAiBvlB,aACvD7kB,KAAKgG,MAAM8jC,eAAiB9pC,KAAK+vB,IAAIqa,iBAAiB5qB,aASxDvc,EAASkQ,UAAU6hB,KAAO,SAASyD,GACjC,MAAOz4B,MAAKmoB,KAAK6M,KAAKyD,IAGxB54B,EAAOD,QAAUqD,GAKb,SAASpD,EAAQD,EAASM,GAc9B,QAASgC,GAAMwQ,EAAM0nB,EAAY3rB,GAC/BzO,KAAKK,GAAK,KACVL,KAAK8kC,OAAS,KACd9kC,KAAK0S,KAAOA,EACZ1S,KAAK+vB,IAAM,KACX/vB,KAAKo6B,WAAaA,MAClBp6B,KAAKyO,QAAUA,MAEfzO,KAAKqzC,UAAW,EAChBrzC,KAAKktC,WAAY,EACjBltC,KAAKitC,OAAQ,EAEbjtC,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KACZzH,KAAKuS,MAAQ,KACbvS,KAAKwS,OAAS,KA3BhB,GAAIzO,GAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,EA6B/BgC,GAAKiR,UAAUrR,OAAQ,EAKvBI,EAAKiR,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,EAChBrzC,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAM3Bvf,EAAKiR,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,EAChBrzC,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAQ3Bvf,EAAKiR,UAAU6E,QAAU,SAAStF,GAChC1S,KAAK0S,KAAOA,EACZ1S,KAAKitC,OAAQ,EACTjtC,KAAKktC,WAAWltC,KAAKyhB,UAO3Bvf,EAAKiR,UAAUu6B,UAAY,SAAS5I,GAC9B9kC,KAAKktC,WACPltC,KAAK0nC,OACL1nC,KAAK8kC,OAASA,EACV9kC,KAAK8kC,QACP9kC,KAAK2nC,QAIP3nC,KAAK8kC,OAASA,GASlB5iC,EAAKiR,UAAU27B,UAAY,WAEzB,OAAO,GAOT5sC,EAAKiR,UAAUw0B,KAAO,WACpB,OAAO,GAOTzlC,EAAKiR,UAAUu0B,KAAO,WACpB,OAAO,GAMTxlC,EAAKiR,UAAUsO,OAAS,aAOxBvf,EAAKiR,UAAUw7B,YAAc,aAO7BzsC,EAAKiR,UAAUo6B,YAAc,aAS7BrrC,EAAKiR,UAAUqnC,qBAAuB,SAAUC,GAC9C,GAAIz6C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAAS74B,SAAWrW,KAAK+vB,IAAI2qB,aAAc,CAE3E,GAAIvmC,GAAKnU,KAEL06C,EAAenpC,SAASM,cAAc,MAC1C6oC,GAAa7yC,UAAY,SACzB6yC,EAAa1V,MAAQ,mBAGrB,GAAIjhC,GAAO22C,GAAcnnC,GAAG,MAAO,SAAUjK,GAC3C6K,EAAG2wB,OAAOiJ,kBAAkB55B,GAC5B7K,EAAMq8B,kBACNr8B,EAAMD,mBAGRoxC,EAAOhpC,YAAYipC,GACnB16C,KAAK+vB,IAAI2qB,aAAeA,OAEhB16C,KAAKqzC,UAAYrzC,KAAK+vB,IAAI2qB,eAE9B16C,KAAK+vB,IAAI2qB,aAAa7wC,YACxB7J,KAAK+vB,IAAI2qB,aAAa7wC,WAAWsH,YAAYnR,KAAK+vB,IAAI2qB,cAExD16C,KAAK+vB,IAAI2qB,aAAe,OAS5Bx4C,EAAKiR,UAAUwnC,gBAAkB,SAAU/xC,GACzC,GAAIgnB,EACJ,IAAI5vB,KAAKyO,QAAQmsC,SAAU,CACzB,GAAI/jB,GAAW72B,KAAK8kC,OAAOjP,QAAQC,UAAU5gB,IAAIlV,KAAKK,GACtDuvB,GAAU5vB,KAAKyO,QAAQmsC,SAAS/jB,OAGhCjH,GAAU5vB,KAAK0S,KAAKkd,OAGtB,IAAGA,IAAY5vB,KAAK4vB,QAAS,CAE3B,GAAIA,YAAmB+c,SACrB/jC,EAAQqb,UAAY,GACpBrb,EAAQ6I,YAAYme,OAEjB,IAAeppB,QAAXopB,EACPhnB,EAAQqb,UAAY2L,MAGpB,IAAwB,cAAlB5vB,KAAK0S,KAAK5L,MAA8CN,SAAtBxG,KAAK0S,KAAKkd,QAChD,KAAM,IAAIhsB,OAAM,sCAAwC5D,KAAKK,GAIjEL,MAAK4vB,QAAUA,IASnB1tB,EAAKiR,UAAU0nC,aAAe,SAAUjyC,GACf,MAAnB5I,KAAK0S,KAAKsyB,MACZp8B,EAAQo8B,MAAQhlC,KAAK0S,KAAKsyB,OAAS,GAGnCp8B,EAAQkyC,gBAAgB,UAS3B54C,EAAKiR,UAAU4nC,sBAAwB,SAASnyC,GAC/C,GAAI5I,KAAKyO,QAAQusC,gBAAkBh7C,KAAKyO,QAAQusC,eAAer1C,OAAS,EAAG,CACzE,GAAIs1C,KAEJ,IAAIh1C,MAAMC,QAAQlG,KAAKyO,QAAQusC,gBAC7BC,EAAaj7C,KAAKyO,QAAQusC,mBAEvB,CAAA,GAAmC,OAA/Bh7C,KAAKyO,QAAQusC,eAIpB,MAHAC,GAAa10C,OAAO6G,KAAKpN,KAAK0S,MAMhC,IAAK,GAAIlN,GAAI,EAAGA,EAAIy1C,EAAWt1C,OAAQH,IAAK,CAC1C,GAAIyQ,GAAOglC,EAAWz1C,GAClB6B,EAAQrH,KAAK0S,KAAKuD,EAET,OAAT5O,EACFuB,EAAQsyC,aAAa,QAAUjlC,EAAM5O,GAGrCuB,EAAQkyC,gBAAgB,QAAU7kC,MAW1C/T,EAAKiR,UAAUgoC,aAAe,SAASvyC,GAEjC5I,KAAKiN,QACPtM,EAAK8M,cAAc7E,EAAS5I,KAAKiN,OACjCjN,KAAKiN,MAAQ,MAIXjN,KAAK0S,KAAKzF,QACZtM,EAAK2M,WAAW1E,EAAS5I,KAAK0S,KAAKzF,OACnCjN,KAAKiN,MAAQjN,KAAK0S,KAAKzF,QAI3BpN,EAAOD,QAAUsC,GAKb,SAASrC,EAAQD,EAASM,GAkB9B,QAASiC,GAAgBuQ,EAAM0nB,EAAY3rB,GASzC,GARAzO,KAAKgG,OACH4pB,SACErd,MAAO,IAGXvS,KAAK6jB,UAAW,EAGZnR,EAAM,CACR,GAAkBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAAKrS,GAE7D,IAAgBmG,QAAZkM,EAAK7C,IACP,KAAM,IAAIjM,OAAM,kCAAoC8O,EAAKrS,IAI7D6B,EAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAElCzO,KAAKo7C,cAAe,EApCtB,GACIl5C,IADShC,EAAoB,IACtBA,EAAoB,KAC3B2C,EAAkB3C,EAAoB,IACtCoC,EAAYpC,EAAoB,GAoCpCiC,GAAegR,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAEjDC,EAAegR,UAAUkoC,cAAgB,kBACzCl5C,EAAegR,UAAUrR,OAAQ,EAOjCK,EAAegR,UAAU27B,UAAY,SAASrZ,GAE5C,MAAQz1B,MAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,KAAS7P,KAAK0S,KAAK7C,IAAM4lB,EAAM7lB,OAMjEzN,EAAegR,UAAUsO,OAAS,WAChC,GAAIsO,GAAM/vB,KAAK+vB,GAuBf,IAtBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAIjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAMxB5vB,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAIsC,GAAanM,KAAK8kC,OAAO/U,IAAI5jB,UACjC,KAAKA,EACH,KAAM,IAAIvI,OAAM,iEAElBuI,GAAWsF,YAAYse,EAAIygB,KAQ7B,GANAxwC,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIH,SAC3B5vB,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIH,SACpC5vB,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAa,IAAM7H,KAAK0S,KAAK7K,UAAa,KAChE7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY7H,KAAKq7C,cAAgBxzC,EAGzC7H,KAAK6jB,SAA6D,WAAlDta,OAAOsrC,iBAAiB9kB,EAAIH,SAAS/L,SAGrD7jB,KAAKgG,MAAM4pB,QAAQrd,MAAQvS,KAAK+vB,IAAIH,QAAQQ,YAC5CpwB,KAAKwS,OAAS,EAEdxS,KAAKitC,OAAQ,IAQjB9qC,EAAegR,UAAUw0B,KAAOrlC,EAAU6Q,UAAUw0B,KAMpDxlC,EAAegR,UAAUu0B,KAAOplC,EAAU6Q,UAAUu0B,KAMpDvlC,EAAegR,UAAUw7B,YAAcrsC,EAAU6Q,UAAUw7B,YAM3DxsC,EAAegR,UAAUo6B,YAAc,SAAS7zB,GAC9C,GAAI4hC,GAAqC,QAA7Bt7C,KAAKyO,QAAQ8lB,WACzBv0B,MAAK+vB,IAAIH,QAAQ3iB,MAAMtF,IAAM2zC,EAAQ,GAAK,IAC1Ct7C,KAAK+vB,IAAIH,QAAQ3iB,MAAMqW,OAASg4B,EAAQ,IAAM,EAC9C,IAAI9oC,EAGJ,IAA2BhM,SAAvBxG,KAAK0S,KAAK2uB,SAAwB,CACpC,GAAIka,GAAev7C,KAAK0S,KAAK2uB,SACzBF,EAAYnhC,KAAK8kC,OAAO3D,UACxB8K,EAAgB9K,EAAUoa,GAAcpzC,KAE5C,IAAa,GAATmzC,EAAe,CAEjB9oC,EAASxS,KAAK8kC,OAAO3D,UAAUoa,GAAc/oC,OAASkH,EAAOrK,KAAKoW,SAClEjT,GAA2B,GAAjBy5B,EAAqBvyB,EAAOonB,KAAO,GAAIpnB,EAAOrK,KAAKoW,SAAW,CACxE,IAAI2b,GAASphC,KAAK8kC,OAAOn9B,GACzB,KAAK,GAAI05B,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQ8jC,IACrE7K,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAMzD2b,IAA2B,GAAjB6K,EAAqBvyB,EAAOonB,KAAO,GAAMpnB,EAAOrK,KAAKoW,SAAW,EAC1EzlB,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAMy5B,EAAS,KAClCphC,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,OAGzB,CACH,GAAI8d,GAASphC,KAAK8kC,OAAOn9B,GACzB,KAAK,GAAI05B,KAAYF,GACfA,EAAUr7B,eAAeu7B,IACQ,GAA/BF,EAAUE,GAAU3Y,SAAmByY,EAAUE,GAAUl5B,MAAQ8jC,IACrE7K,GAAUD,EAAUE,GAAU7uB,OAASkH,EAAOrK,KAAKoW,SAIzDjT,GAASxS,KAAK8kC,OAAO3D,UAAUoa,GAAc/oC,OAASkH,EAAOrK,KAAKoW,SAClEzlB,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAMy5B,EAAS,KAClCphC,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,QAM1BtjB,MAAK8kC,iBAAkBjiC,IAEzB2P,EAAStN,KAAKwH,IAAI1M,KAAK8kC,OAAOtyB,OAC1BxS,KAAK8kC,OAAOjP,QAAQlB,KAAKC,SAAS1I,OAAO1Z,OACzCxS,KAAK8kC,OAAOjP,QAAQlB,KAAKC,SAASiD,gBAAgBrlB,QACtDxS,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAM2zC,EAAQ,IAAM,GACvCt7C,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAASg4B,EAAQ,GAAK,MAGzC9oC,EAASxS,KAAK8kC,OAAOtyB,OAErBxS,KAAK+vB,IAAIygB,IAAIvjC,MAAMtF,IAAM3H,KAAK8kC,OAAOn9B,IAAM,KAC3C3H,KAAK+vB,IAAIygB,IAAIvjC,MAAMqW,OAAS,GAGhCtjB,MAAK+vB,IAAIygB,IAAIvjC,MAAMuF,OAASA,EAAS,MAGvC3S,EAAOD,QAAUuC,GAKb,SAAStC,EAAQD,EAASM,GAe9B,QAASkC,GAASsQ,EAAM0nB,EAAY3rB,GAalC,GAZAzO,KAAKgG,OACH8pB,KACEvd,MAAO,EACPC,OAAQ,GAEVqd,MACEtd,MAAO,EACPC,OAAQ,IAKRE,GACgBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAI1DxQ,GAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAhCpC,CAAA,GAAIvM,GAAOhC,EAAoB,GACpBA,GAAoB,GAkC/BkC,EAAQ+Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAO1CE,EAAQ+Q,UAAU27B,UAAY,SAASrZ,GAGrC,GAAIjD,IAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,CAC3C,OAAQ5P,MAAK0S,KAAK9C,MAAQ6lB,EAAM7lB,MAAQ4iB,GAAcxyB,KAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,IAAM2iB,GAMtFpwB,EAAQ+Q,UAAUsO,OAAS,WACzB,GAAIsO,GAAM/vB,KAAK+vB,GA6Bf,IA5BKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAGjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAGxBG,EAAIF,KAAOte,SAASM,cAAc,OAClCke,EAAIF,KAAKhoB,UAAY,OAGrBkoB,EAAID,IAAMve,SAASM,cAAc,OACjCke,EAAID,IAAIjoB,UAAY,MAGpBkoB,EAAIygB,IAAI,iBAAmBxwC,KAE3BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EAAY,KAAM,IAAI7oC,OAAM,iEACjC6oC,GAAWh7B,YAAYse,EAAIygB,KAE7B,IAAKzgB,EAAIF,KAAKhmB,WAAY,CACxB,GAAIsC,GAAanM,KAAK8kC,OAAO/U,IAAI5jB,UACjC,KAAKA,EAAY,KAAM,IAAIvI,OAAM,iEACjCuI,GAAWsF,YAAYse,EAAIF,MAE7B,IAAKE,EAAID,IAAIjmB,WAAY,CACvB,GAAIi3B,GAAO9gC,KAAK8kC,OAAO/U,IAAI+Q,IAC3B,KAAK30B,EAAY,KAAM,IAAIvI,OAAM,2DACjCk9B,GAAKrvB,YAAYse,EAAID,KAQvB,GANA9vB,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIygB,KAC3BxwC,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIygB,KACpCxwC,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAW,IAAM7H,KAAK0S,KAAK7K,UAAY,KAC7D7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY,WAAaA,EACjCkoB,EAAIF,KAAKhoB,UAAY,YAAcA,EACnCkoB,EAAID,IAAIjoB,UAAa,WAAaA,EAGlC7H,KAAKgG,MAAM8pB,IAAItd,OAASud,EAAID,IAAIQ,aAChCtwB,KAAKgG,MAAM8pB,IAAIvd,MAAQwd,EAAID,IAAIM,YAC/BpwB,KAAKgG,MAAM6pB,KAAKtd,MAAQwd,EAAIF,KAAKO,YACjCpwB,KAAKuS,MAAQwd,EAAIygB,IAAIpgB,YACrBpwB,KAAKwS,OAASud,EAAIygB,IAAIlgB,aAEtBtwB,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAIygB,MAOhCpuC,EAAQ+Q,UAAUw0B,KAAO,WAClB3nC,KAAKktC,WACRltC,KAAKyhB,UAOTrf,EAAQ+Q,UAAUu0B,KAAO,WACvB,GAAI1nC,KAAKktC,UAAW,CAClB,GAAInd,GAAM/vB,KAAK+vB,GAEXA,GAAIygB,IAAI3mC,YAAckmB,EAAIygB,IAAI3mC,WAAWsH,YAAY4e,EAAIygB,KACzDzgB,EAAIF,KAAKhmB,YAAakmB,EAAIF,KAAKhmB,WAAWsH,YAAY4e,EAAIF,MAC1DE,EAAID,IAAIjmB,YAAckmB,EAAID,IAAIjmB,WAAWsH,YAAY4e,EAAID,KAE7D9vB,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB9qC,EAAQ+Q,UAAUw7B,YAAc,WAC9B,GAAI/+B,GAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,OAC3Cm/B,EAAQ/uC,KAAKyO,QAAQsgC,MAErByB,EAAMxwC,KAAK+vB,IAAIygB,IACf3gB,EAAO7vB,KAAK+vB,IAAIF,KAChBC,EAAM9vB,KAAK+vB,IAAID,GAIjB9vB,MAAKyH,KADM,SAATsnC,EACUn/B,EAAQ5P,KAAKuS,MAET,QAATw8B,EACKn/B,EAIAA,EAAQ5P,KAAKuS,MAAQ,EAInCi+B,EAAIvjC,MAAMxF,KAAOzH,KAAKyH,KAAO,KAG7BooB,EAAK5iB,MAAMxF,KAAQmI,EAAQ5P,KAAKgG,MAAM6pB,KAAKtd,MAAQ,EAAK,KAGxDud,EAAI7iB,MAAMxF,KAAQmI,EAAQ5P,KAAKgG,MAAM8pB,IAAIvd,MAAQ,EAAK,MAOxDnQ,EAAQ+Q,UAAUo6B,YAAc,WAC9B,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3Bic,EAAMxwC,KAAK+vB,IAAIygB,IACf3gB,EAAO7vB,KAAK+vB,IAAIF,KAChBC,EAAM9vB,KAAK+vB,IAAID,GAEnB,IAAmB,OAAfyE,EACFic,EAAIvjC,MAAMtF,KAAW3H,KAAK2H,KAAO,GAAK,KAEtCkoB,EAAK5iB,MAAMtF,IAAS,IACpBkoB,EAAK5iB,MAAMuF,OAAUxS,KAAK8kC,OAAOn9B,IAAM3H,KAAK2H,IAAM,EAAK,KACvDkoB,EAAK5iB,MAAMqW,OAAS,OAEjB,CACH,GAAIk4B,GAAgBx7C,KAAK8kC,OAAOjP,QAAQ7vB,MAAMwM,OAC1C+d,EAAairB,EAAgBx7C,KAAK8kC,OAAOn9B,IAAM3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,GAE7E6oC,GAAIvjC,MAAMtF,KAAW3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,QAAU,GAAK,KACzEqd,EAAK5iB,MAAMtF,IAAU6zC,EAAgBjrB,EAAc,KACnDV,EAAK5iB,MAAMqW,OAAS,IAGtBwM,EAAI7iB,MAAMtF,KAAQ3H,KAAKgG,MAAM8pB,IAAItd,OAAS,EAAK,MAGjD3S,EAAOD,QAAUwC,GAKb,SAASvC,EAAQD,EAASM,GAc9B,QAASmC,GAAWqQ,EAAM0nB,EAAY3rB,GAcpC,GAbAzO,KAAKgG,OACH8pB,KACEnoB,IAAK,EACL4K,MAAO,EACPC,OAAQ,GAEVod,SACEpd,OAAQ,EACRipC,WAAY,IAKZ/oC,GACgBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAI1DxQ,GAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GAhCpC,GAAIvM,GAAOhC,EAAoB,GAmC/BmC,GAAU8Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAO5CG,EAAU8Q,UAAU27B,UAAY,SAASrZ,GAGvC,GAAIjD,IAAYiD,EAAM5lB,IAAM4lB,EAAM7lB,OAAS,CAC3C,OAAQ5P,MAAK0S,KAAK9C,MAAQ6lB,EAAM7lB,MAAQ4iB,GAAcxyB,KAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,IAAM2iB,GAMtFnwB,EAAU8Q,UAAUsO,OAAS,WAC3B,GAAIsO,GAAM/vB,KAAK+vB,GA0Bf,IAzBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAI7d,MAAQX,SAASM,cAAc,OAInCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAI7d,MAAMT,YAAYse,EAAIH,SAG1BG,EAAID,IAAMve,SAASM,cAAc,OACjCke,EAAI7d,MAAMT,YAAYse,EAAID,KAG1BC,EAAI7d,MAAM,iBAAmBlS,KAE7BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAI7d,MAAMrI,WAAY,CACzB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EACH,KAAM,IAAI7oC,OAAM,iEAElB6oC,GAAWh7B,YAAYse,EAAI7d,OAQ7B,GANAlS,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAI7d,OAC3BlS,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAI7d,OACpClS,KAAKm7C,aAAan7C,KAAK+vB,IAAI7d,MAG3B,IAAIrK,IAAa7H,KAAK0S,KAAK7K,UAAW,IAAM7H,KAAK0S,KAAK7K,UAAY,KAC7D7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAI7d,MAAMrK,UAAa,aAAeA,EACtCkoB,EAAID,IAAIjoB,UAAa,WAAaA,EAGlC7H,KAAKuS,MAAQwd,EAAI7d,MAAMke,YACvBpwB,KAAKwS,OAASud,EAAI7d,MAAMoe,aACxBtwB,KAAKgG,MAAM8pB,IAAIvd,MAAQwd,EAAID,IAAIM,YAC/BpwB,KAAKgG,MAAM8pB,IAAItd,OAASud,EAAID,IAAIQ,aAChCtwB,KAAKgG,MAAM4pB,QAAQpd,OAASud,EAAIH,QAAQU,aAGxCP,EAAIH,QAAQ3iB,MAAMwuC,WAAa,EAAIz7C,KAAKgG,MAAM8pB,IAAIvd,MAAQ,KAG1Dwd,EAAID,IAAI7iB,MAAMtF,KAAQ3H,KAAKwS,OAASxS,KAAKgG,MAAM8pB,IAAItd,QAAU,EAAK,KAClEud,EAAID,IAAI7iB,MAAMxF,KAAQzH,KAAKgG,MAAM8pB,IAAIvd,MAAQ,EAAK,KAElDvS,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAI7d,QAOhC7P,EAAU8Q,UAAUw0B,KAAO,WACpB3nC,KAAKktC,WACRltC,KAAKyhB,UAOTpf,EAAU8Q,UAAUu0B,KAAO,WACrB1nC,KAAKktC,YACHltC,KAAK+vB,IAAI7d,MAAMrI,YACjB7J,KAAK+vB,IAAI7d,MAAMrI,WAAWsH,YAAYnR,KAAK+vB,IAAI7d,OAGjDlS,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB7qC,EAAU8Q,UAAUw7B,YAAc,WAChC,GAAI/+B,GAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,MAE/C5P,MAAKyH,KAAOmI,EAAQ5P,KAAKgG,MAAM8pB,IAAIvd,MAGnCvS,KAAK+vB,IAAI7d,MAAMjF,MAAMxF,KAAOzH,KAAKyH,KAAO,MAO1CpF,EAAU8Q,UAAUo6B,YAAc,WAChC,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3BriB,EAAQlS,KAAK+vB,IAAI7d,KAGnBA,GAAMjF,MAAMtF,IADK,OAAf4sB,EACgBv0B,KAAK2H,IAAM,KAGV3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,OAAU,MAItE3S,EAAOD,QAAUyC,GAKb,SAASxC,EAAQD,EAASM,GAe9B,QAASoC,GAAWoQ,EAAM0nB,EAAY3rB,GASpC,GARAzO,KAAKgG,OACH4pB,SACErd,MAAO,IAGXvS,KAAK6jB,UAAW,EAGZnR,EAAM,CACR,GAAkBlM,QAAdkM,EAAK9C,MACP,KAAM,IAAIhM,OAAM,oCAAsC8O,EAAKrS,GAE7D,IAAgBmG,QAAZkM,EAAK7C,IACP,KAAM,IAAIjM,OAAM,kCAAoC8O,EAAKrS,IAI7D6B,EAAK3B,KAAKP,KAAM0S,EAAM0nB,EAAY3rB,GA/BpC,GACIvM,IADShC,EAAoB,IACtBA,EAAoB,IAiC/BoC,GAAU6Q,UAAY,GAAIjR,GAAM,KAAM,KAAM,MAE5CI,EAAU6Q,UAAUkoC,cAAgB,aAOpC/4C,EAAU6Q,UAAU27B,UAAY,SAASrZ,GAEvC,MAAQz1B,MAAK0S,KAAK9C,MAAQ6lB,EAAM5lB,KAAS7P,KAAK0S,KAAK7C,IAAM4lB,EAAM7lB,OAMjEtN,EAAU6Q,UAAUsO,OAAS,WAC3B,GAAIsO,GAAM/vB,KAAK+vB,GAsBf,IArBKA,IAEH/vB,KAAK+vB,OACLA,EAAM/vB,KAAK+vB,IAGXA,EAAIygB,IAAMj/B,SAASM,cAAc,OAIjCke,EAAIH,QAAUre,SAASM,cAAc,OACrCke,EAAIH,QAAQ/nB,UAAY,UACxBkoB,EAAIygB,IAAI/+B,YAAYse,EAAIH,SAGxBG,EAAIygB,IAAI,iBAAmBxwC,KAE3BA,KAAKitC,OAAQ,IAIVjtC,KAAK8kC,OACR,KAAM,IAAIlhC,OAAM,yCAElB,KAAKmsB,EAAIygB,IAAI3mC,WAAY,CACvB,GAAI4iC,GAAazsC,KAAK8kC,OAAO/U,IAAI0c,UACjC,KAAKA,EACH,KAAM,IAAI7oC,OAAM,iEAElB6oC,GAAWh7B,YAAYse,EAAIygB,KAQ7B,GANAxwC,KAAKktC,WAAY,EAMbltC,KAAKitC,MAAO,CACdjtC,KAAK26C,gBAAgB36C,KAAK+vB,IAAIH,SAC9B5vB,KAAK66C,aAAa76C,KAAK+vB,IAAIygB,KAC3BxwC,KAAK+6C,sBAAsB/6C,KAAK+vB,IAAIygB,KACpCxwC,KAAKm7C,aAAan7C,KAAK+vB,IAAIygB,IAG3B,IAAI3oC,IAAa7H,KAAK0S,KAAK7K,UAAa,IAAM7H,KAAK0S,KAAK7K,UAAa,KAChE7H,KAAKqzC,SAAW,YAAc,GACnCtjB,GAAIygB,IAAI3oC,UAAY7H,KAAKq7C,cAAgBxzC,EAGzC7H,KAAK6jB,SAA6D,WAAlDta,OAAOsrC,iBAAiB9kB,EAAIH,SAAS/L,SAKrD7jB,KAAK+vB,IAAIH,QAAQ3iB,MAAMyuC,SAAW,OAClC17C,KAAKgG,MAAM4pB,QAAQrd,MAAQvS,KAAK+vB,IAAIH,QAAQQ,YAC5CpwB,KAAKwS,OAASxS,KAAK+vB,IAAIygB,IAAIlgB,aAC3BtwB,KAAK+vB,IAAIH,QAAQ3iB,MAAMyuC,SAAW,GAElC17C,KAAKitC,OAAQ,EAGfjtC,KAAKw6C,qBAAqBzqB,EAAIygB,KAC9BxwC,KAAK27C,mBACL37C,KAAK47C,qBAOPt5C,EAAU6Q,UAAUw0B,KAAO,WACpB3nC,KAAKktC,WACRltC,KAAKyhB,UAQTnf,EAAU6Q,UAAUu0B,KAAO,WACzB,GAAI1nC,KAAKktC,UAAW,CAClB,GAAIsD,GAAMxwC,KAAK+vB,IAAIygB,GAEfA,GAAI3mC,YACN2mC,EAAI3mC,WAAWsH,YAAYq/B,GAG7BxwC,KAAK2H,IAAM,KACX3H,KAAKyH,KAAO,KAEZzH,KAAKktC,WAAY,IAQrB5qC,EAAU6Q,UAAUw7B,YAAc,WAChC,GAGIkN,GACA1rB,EAJA2rB,EAAc97C,KAAK8kC,OAAOvyB,MAC1B3C,EAAQ5P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK9C,OAC3CC,EAAM7P,KAAKo6B,WAAWnF,SAASj1B,KAAK0S,KAAK7C,MAKhCisC,EAATlsC,IACFA,GAASksC,GAEPjsC,EAAM,EAAIisC,IACZjsC,EAAM,EAAIisC,EAEZ,IAAIC,GAAW72C,KAAKwH,IAAImD,EAAMD,EAAO,EAoBrC,QAlBI5P,KAAK6jB,UACP7jB,KAAKyH,KAAOmI,EACZ5P,KAAKuS,MAAQwpC,EAAW/7C,KAAKgG,MAAM4pB,QAAQrd,MAC3C4d,EAAenwB,KAAKgG,MAAM4pB,QAAQrd,QAOlCvS,KAAKyH,KAAOmI,EACZ5P,KAAKuS,MAAQwpC,EACb5rB,EAAejrB,KAAK4G,IAAI+D,EAAMD,EAAQ,EAAI5P,KAAKyO,QAAQuV,QAAShkB,KAAKgG,MAAM4pB,QAAQrd,QAGrFvS,KAAK+vB,IAAIygB,IAAIvjC,MAAMxF,KAAOzH,KAAKyH,KAAO,KACtCzH,KAAK+vB,IAAIygB,IAAIvjC,MAAMsF,MAAQwpC,EAAW,KAE9B/7C,KAAKyO,QAAQsgC,OACnB,IAAK,OACH/uC,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAO,GAC9B,MAEF,KAAK,QACHzH,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOvC,KAAKwH,IAAKqvC,EAAW5rB,EAAe,EAAInwB,KAAKyO,QAAQuV,QAAU,GAAK,IAClG,MAEF,KAAK,SACHhkB,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOvC,KAAKwH,KAAKqvC,EAAW5rB,EAAe,EAAInwB,KAAKyO,QAAQuV,SAAW,EAAG,GAAK,IACtG,MAEF,SAIM63B,EAFA77C,KAAK6jB,SACHhU,EAAM,EACM3K,KAAKwH,KAAKkD,EAAO,IAGhBugB,EAIL,EAARvgB,EACY1K,KAAK4G,KAAK8D,EACnBC,EAAMD,EAAQugB,EAAe,EAAInwB,KAAKyO,QAAQuV,SAIrC,EAGlBhkB,KAAK+vB,IAAIH,QAAQ3iB,MAAMxF,KAAOo0C,EAAc,OAQlDv5C,EAAU6Q,UAAUo6B,YAAc,WAChC,GAAIhZ,GAAcv0B,KAAKyO,QAAQ8lB,YAC3Bic,EAAMxwC,KAAK+vB,IAAIygB,GAGjBA,GAAIvjC,MAAMtF,IADO,OAAf4sB,EACcv0B,KAAK2H,IAAM,KAGV3H,KAAK8kC,OAAOtyB,OAASxS,KAAK2H,IAAM3H,KAAKwS,OAAU,MAQpElQ,EAAU6Q,UAAUwoC,iBAAmB,WACrC,GAAI37C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAASC,aAAenvC,KAAK+vB,IAAIisB,SAAU,CAE3E,GAAIA,GAAWzqC,SAASM,cAAc,MACtCmqC,GAASn0C,UAAY,YACrBm0C,EAAS9I,aAAelzC,KASxBA,KAAK+vB,IAAIygB,IAAI/+B,YAAYuqC,GACzBh8C,KAAK+vB,IAAIisB,SAAWA,OAEZh8C,KAAKqzC,UAAYrzC,KAAK+vB,IAAIisB,WAE9Bh8C,KAAK+vB,IAAIisB,SAASnyC,YACpB7J,KAAK+vB,IAAIisB,SAASnyC,WAAWsH,YAAYnR,KAAK+vB,IAAIisB,UAEpDh8C,KAAK+vB,IAAIisB,SAAW,OAQxB15C,EAAU6Q,UAAUyoC,kBAAoB,WACtC,GAAI57C,KAAKqzC,UAAYrzC,KAAKyO,QAAQygC,SAASC,aAAenvC,KAAK+vB,IAAIksB,UAAW,CAE5E,GAAIA,GAAY1qC,SAASM,cAAc,MACvCoqC,GAAUp0C,UAAY,aACtBo0C,EAAU9I,cAAgBnzC,KAS1BA,KAAK+vB,IAAIygB,IAAI/+B,YAAYwqC,GACzBj8C,KAAK+vB,IAAIksB,UAAYA,OAEbj8C,KAAKqzC,UAAYrzC,KAAK+vB,IAAIksB,YAE9Bj8C,KAAK+vB,IAAIksB,UAAUpyC,YACrB7J,KAAK+vB,IAAIksB,UAAUpyC,WAAWsH,YAAYnR,KAAK+vB,IAAIksB,WAErDj8C,KAAK+vB,IAAIksB,UAAY,OAIzBp8C,EAAOD,QAAU0C,GAKb,SAASzC,EAAQD,EAASM,GAkC9B,QAASgD,GAASqW,EAAW7G,EAAMjE,GACjC,KAAMzO,eAAgBkD,IACpB,KAAM,IAAIsW,aAAY,mDAGxBxZ,MAAKk8C,0BACLl8C,KAAKm8C,0BAGLn8C,KAAKyZ,iBAAmBF,EAGxBvZ,KAAKo8C,kBAAoB,GACzBp8C,KAAKq8C,eAAiB,IAAOr8C,KAAKo8C,kBAClCp8C,KAAKs8C,WAAa,EAClBt8C,KAAKu8C,YAAc,EACnBv8C,KAAKw8C,gBAAiB,EACtBx8C,KAAKy8C,wBAA0B,GAE/Bz8C,KAAK08C,cAAe,EAEpB18C,KAAK28C,kBAAoB1pC,IAAI,KAAK2pC,KAAK,KAAKC,SAAS,KAAKC,QAAQ,KAAKC,IAAI,MAG3E/8C,KAAKq0B,gBACH2oB,OACEC,KAAM,EACNC,UAAW,GACXC,UAAW,GACX1xB,OAAQ,GACR2xB,MAAO,UACPC,MAAO72C,OACP0gB,SAAU,GACVC,SAAU,GACVm2B,UAAW,QACXC,SAAU,GACVC,SAAU,UACVC,SAAUj3C,OACVk3C,gBAAiB,EACjBC,gBAAiB,QACjBC,MAAO,GACPzyC,OACIiB,OAAQ,UACRD,WAAY,UACdE,WACED,OAAQ,UACRD,WAAY,WAEdG,OACEF,OAAQ,UACRD,WAAY,YAGhB8F,MAAOzL,OACPwZ,YAAa,EACb69B,oBAAqBr3C,QAEvBs3C,OACE52B,SAAU,EACVC,SAAU,GACV5U,MAAO,EACPwrC,yBAA0B,EAC1BC,WAAY,IACZ/wC,MAAO,OACP9B,OACEA,MAAM,UACNkB,UAAU,UACVC,MAAO,WAETgxC,UAAW,UACXC,SAAU,GACVC,SAAU,QACVC,SAAU,QACVC,gBAAiB,EACjBC,gBAAiB,QACjBM,eAAe,aACfC,iBAAkB,EAClBC,MACEx4C,OAAQ,GACRy4C,IAAK,EACLC,UAAW73C,QAEb83C,aAAc,QAEhBC,kBAAiB,EACjBC,SACEC,WACE/vC,SAAS,EACTgwC,cAAe,EACfC,sBAAuB,KACvBC,eAAgB,GAChBC,aAAc,GACdC,eAAgB,IAChBC,QAAS,KAEXC,WACEJ,eAAgB,EAChBC,aAAc,IACdC,eAAgB,IAChBG,aAAc,IACdF,QAAS,KAEXG,uBACExwC,SAAS,EACTkwC,eAAgB,EAChBC,aAAc,IACdC,eAAgB,IAChBG,aAAc,IACdF,QAAS,KAEXA,QAAS,KACTH,eAAgB,KAChBC,aAAc,KACdC,eAAgB,MAElBK,YACEzwC,SAAS,EACT0wC,gBAAiB,IACjBC,iBAAiB,IACjBC,cAAc,IACdC,eAAgB,GAChBC,qBAAsB,GACtBC,gBAAiB,IACjBC,oBAAqB,GACrBC,mBAAoB,EACpBC,YAAa,IACbC,mBAAoB,GACpBC,sBAAuB,GACvBC,WAAY,GACZC,aAAcztC,MAAQ,EACRC,OAAQ,EACRiZ,OAAQ,GACtBw0B,sBAAuB,IACvBC,kBAAmB,GACnBC,uBAAwB,GAE1BC,YACE1xC,SAAS,GAEX2xC,UACE3xC,SAAS,EACT4xC,OAAQvuC,EAAG,GAAIC,EAAG,GAAIkuB,KAAM,MAE9BqgB,kBACE7xC,SAAS,EACT8xC,kBAAkB,GAEpBC,oBACE/xC,SAAQ,EACRgyC,gBAAiB,IACjBC,YAAa,IACbzlB,UAAW,KACX0lB,OAAQ,WAEVC,wBAAwB,EACxBC,cACEpyC,SAAS,EACTqyC,SAAS,EACTj6C,KAAM,aACNk6C,UAAW,IAEbC,YAAc,GACdC,YAAc,GACdC,WAAW,EACXC,wBAAyB,IACzBC,uBAAuB,EACvBhd,OAAQ,KACRQ,QAASA,EACTze,SACE5N,MAAO,IACP8kC,UAAW,QACXC,SAAU,GACVC,SAAU,UACVryC,OACEiB,OAAQ,OACRD,WAAY,YAGhBm1C,aAAa,EACbC,WAAW,EACX5jB,UAAU,EACVrxB,OAAO,EACPk1C,iBAAiB,EACjBC,iBAAiB,EACjBlvC,MAAQ,OACRC,OAAS,OACTy8B,YAAY,GAEdjvC,KAAK0hD,UAAY/gD,EAAK2E,UAAWtF,KAAKq0B,gBACtCr0B,KAAK2hD,WAAa,EAGlB3hD,KAAK4hD,UAAY5E,SAASc,UAC1B99C,KAAK6hD,oBAAqB,EAC1B7hD,KAAK8hD,mBAAqBC,YAAaC,SAGvChiD,KAAKiiD,eAAiB,EAAEjiD,KAAKo8C,kBAC7Bp8C,KAAKkiD,wBAA0B,iBAC/BliD,KAAKmiD,WAAa,EAClBniD,KAAKoiD,YAAc,EACnBpiD,KAAKqiD,YAAc,EACnBriD,KAAKsiD,kBAAoB,EACzBtiD,KAAKuiD,kBAAoB,EACzBviD,KAAKwiD,eAAiB,KACtBxiD,KAAKyiD,mBAAqB,KAC1BziD,KAAK0iD,UAAY,CAGjB,IAAIv/C,GAAUnD,IACdA,MAAKm0B,OAAS,GAAI9wB,GAClBrD,KAAK2iD,OAAS,GAAIr/C,GAClBtD,KAAK2iD,OAAOC,kBAAkB,WAC5Bz/C,EAAQ0/C,YAIV7iD,KAAK8iD,WAAa,EAClB9iD,KAAK+iD,WAAa,EAClB/iD,KAAKgjD,cAAgB,EAIrBhjD,KAAKijD,qBAELjjD,KAAK00B,UAEL10B,KAAKkjD,oBAELljD,KAAKmjD,qBAELnjD,KAAKojD,uBAELpjD,KAAKqjD,uBAILrjD,KAAKsjD,gBAAgBtjD,KAAKsf,MAAME,YAAc,EAAGxf,KAAKsf,MAAMuF,aAAe,GAC3E7kB,KAAKgd,UAAU,GACfhd,KAAKkT,WAAWzE,GAGhBzO,KAAKujD,kBAAmB,EACxBvjD,KAAKwjD,mBACLxjD,KAAKyjD,sBAAuB,EAC5BzjD,KAAK0jD,YAAa,EAClB1jD,KAAKohD,wBAA0B,KAC/BphD,KAAK2jD,eAAgB,EAGrB3jD,KAAK4jD,oBACL5jD,KAAK6jD,0BACL7jD,KAAK8jD,eACL9jD,KAAKg9C,SACLh9C,KAAK89C,SAGL99C,KAAK+jD,eAAqBhyC,EAAK,EAAEC,EAAK,GACtChS,KAAKgkD,mBAAqBjyC,EAAK,EAAEC,EAAK,GACtChS,KAAKikD,iBAAmBlyC,EAAK,EAAEC,EAAK,GACpChS,KAAKkkD,cACLlkD,KAAKid,MAAQ,EACbjd,KAAKmkD,cAAgBnkD,KAAKid,MAG1Bjd,KAAKokD,UAAY,KACjBpkD,KAAKqkD,UAAY,KAGjBrkD,KAAKskD,gBACHrxC,IAAO,SAAU3J,EAAOwK,GACtB3Q,EAAQohD,UAAUzwC,EAAO7R,OACzBkB,EAAQyM,SAEViF,OAAU,SAAUvL,EAAOwK,GACzB3Q,EAAQqhD,aAAa1wC,EAAO7R,MAAO6R,EAAOpB,MAC1CvP,EAAQyM,SAEVyG,OAAU,SAAU/M,EAAOwK,GACzB3Q,EAAQshD,aAAa3wC,EAAO7R,OAC5BkB,EAAQyM,UAGZ5P,KAAK0kD,gBACHzxC,IAAO,SAAU3J,EAAOwK,GACtB3Q,EAAQwhD,UAAU7wC,EAAO7R,OACzBkB,EAAQyM,SAEViF,OAAU,SAAUvL,EAAOwK,GACzB3Q,EAAQyhD,aAAa9wC,EAAO7R,OAC5BkB,EAAQyM,SAEVyG,OAAU,SAAU/M,EAAOwK,GACzB3Q,EAAQ0hD,aAAa/wC,EAAO7R,OAC5BkB,EAAQyM,UAKZ5P,KAAK8kD,QAAS,EACd9kD,KAAK+kD,MAAQv+C,OAGbxG,KAAKgY,QAAQtF,EAAK1S,KAAK0hD,UAAUvC,WAAWzwC,SAAW1O,KAAK0hD,UAAUjB,mBAAmB/xC,SAGzF1O,KAAK08C,cAAe,EAC6B,GAA7C18C,KAAK0hD,UAAUjB,mBAAmB/xC,QACpC1O,KAAKglD,2BAI2B,GAA5BhlD,KAAK0hD,UAAUP,WACjBnhD,KAAKilD,WAAWz+C,QAAW,EAAKxG,KAAK0hD,UAAUvC,WAAWzwC,SAK1D1O,KAAK0hD,UAAUvC,WAAWzwC,SAC5B1O,KAAKklD,sBA/VT,GAAInoC,GAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BilD,EAAWjlD,EAAoB,IAC/BS,EAAOT,EAAoB,GAC3BklD,EAAallD,EAAoB,IACjCW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BuD,EAAYvD,EAAoB,IAChCwD,EAAcxD,EAAoB,IAClCmD,EAASnD,EAAoB,IAC7BoD,EAASpD,EAAoB,IAC7BqD,EAAOrD,EAAoB,IAC3BkD,EAAOlD,EAAoB,IAC3BsD,EAAQtD,EAAoB,IAC5BmlD,EAAcnlD,EAAoB,IAClColD,EAAYplD,EAAoB,IAChC2kC,EAAU3kC,EAAoB,GAGlCA,GAAoB,IAiVpB6c,EAAQ7Z,EAAQiQ,WAOhBjQ,EAAQiQ,UAAU+oC,wBAA0B,WAC1C,GAAIqJ,GAAcv8C,UAAUC,UAAUs7B,aACtCvkC,MAAKwlD,iBAAkB,EACgB,IAAnCD,EAAY5+C,QAAQ,YACtB3G,KAAKwlD,iBAAkB,EAEiB,IAAjCD,EAAY5+C,QAAQ,WACvB4+C,EAAY5+C,QAAQ,WAAa,KACnC3G,KAAKwlD,iBAAkB,IAa7BtiD,EAAQiQ,UAAUsyC,eAAiB,WAIjC,IAAK,GAHDC,GAAUn0C,SAASo0C,qBAAsB,UAGpCngD,EAAI,EAAGA,EAAIkgD,EAAQ//C,OAAQH,IAAK,CACvC,GAAIogD,GAAMF,EAAQlgD,GAAGogD,IACjBrhD,EAAQqhD,GAAO,qBAAqBnhD,KAAKmhD,EAC7C,IAAIrhD,EAEF,MAAOqhD,GAAI1gB,UAAU,EAAG0gB,EAAIjgD,OAASpB,EAAM,GAAGoB,QAIlD,MAAO,OAQTzC,EAAQiQ,UAAU0yC,UAAY,WAC5B,GAAsDC,GAAlDC,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAChD,KAAK,GAAIC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACdF,EAAQH,EAAKM,YAAgB,OAAIH,EAAOH,EAAKM,YAAY3+C,MACzDy+C,EAAQJ,EAAKM,YAAiB,QAAIF,EAAOJ,EAAKM,YAAY/+B,OAC1D0+B,EAAQD,EAAKM,YAAkB,SAAIL,EAAOD,EAAKM,YAAY9iC,QAC3D0iC,EAAQF,EAAKM,YAAe,MAAIJ,EAAOF,EAAKM,YAAYz+C,KAMhE,OAHY,MAARs+C,GAAuB,MAARC,GAAwB,KAARH,GAAuB,MAARC,IAChDD,EAAO,EAAGC,EAAO,EAAGC,EAAO,EAAGC,EAAO,IAE/BD,KAAMA,EAAMC,KAAMA,EAAMH,KAAMA,EAAMC,KAAMA,IASpD9iD,EAAQiQ,UAAUkzC,YAAc,SAAS5wB,GACvC,OAAQ1jB,EAAI,IAAO0jB,EAAMywB,KAAOzwB,EAAMwwB,MAC9Bj0C,EAAI,IAAOyjB,EAAMuwB,KAAOvwB,EAAMswB,QAUxC7iD,EAAQiQ,UAAU8xC,WAAa,SAASqB,EAAkBC,EAAaC,GACrExmD,KAAK6iD,SAAQ,GAEOr8C,SAAhB+/C,IACFA,GAAc,GAEK//C,SAAjBggD,IACFA,GAAe,GAEQhgD,SAArB8/C,IACFA,GAAmB,EAGrB,IACIG,GADAhxB,EAAQz1B,KAAK6lD,WAGjB,IAAmB,GAAfU,EAAqB,CACvB,GAAIG,GAAgB1mD,KAAK8jD,YAAYn+C,MAIjC8gD,GAH+B,GAA/BzmD,KAAK0hD,UAAUZ,aACwB,GAArC9gD,KAAK0hD,UAAUvC,WAAWzwC,SAC5Bg4C,GAAiB1mD,KAAK0hD,UAAUvC,WAAWC,gBAC/B,UAAYsH,EAAgB,WAAa,SAGzC,QAAUA,EAAgB,QAAU,SAIT,GAArC1mD,KAAK0hD,UAAUvC,WAAWzwC,SAC1Bg4C,GAAiB1mD,KAAK0hD,UAAUvC,WAAWC,gBACjC,YAAcsH,EAAgB,YAAc,cAG5C,YAAcA,EAAgB,aAAe,SAK7D,IAAIC,GAASzhD,KAAK4G,IAAI9L,KAAKsf,MAAMC,OAAOC,YAAc,IAAKxf,KAAKsf,MAAMC,OAAOsF,aAAe,IAC5F4hC,IAAaE,MAEV,CACH,GAAInP,GAAgD,IAApCtyC,KAAK2lB,IAAI4K,EAAMywB,KAAOzwB,EAAMwwB,MACxCW,EAAgD,IAApC1hD,KAAK2lB,IAAI4K,EAAMuwB,KAAOvwB,EAAMswB,MAExCc,EAAa7mD,KAAKsf,MAAMC,OAAOC,YAAeg4B,EAC9CsP,EAAa9mD,KAAKsf,MAAMC,OAAOsF,aAAe+hC,CAElDH,GAA2BK,GAAdD,EAA4BA,EAAaC,EAGpDL,EAAY,IACdA,EAAY,EAId,IAAIv6B,GAASlsB,KAAKqmD,YAAY5wB,EAC9B,IAAoB,GAAhB+wB,EAAuB,CACzB,GAAI/3C,IAAWmV,SAAUsI,EAAQjP,MAAOwpC,EAAWM,UAAWT,EAC9DtmD,MAAK6nB,OAAOpZ,GACZzO,KAAK8kD,QAAS,EACd9kD,KAAK4P,YAGLsc,GAAOna,GAAK00C,EACZv6B,EAAOla,GAAKy0C,EACZv6B,EAAOna,GAAK,GAAM/R,KAAKsf,MAAMC,OAAOC,YACpC0M,EAAOla,GAAK,GAAMhS,KAAKsf,MAAMC,OAAOsF,aACpC7kB,KAAKgd,UAAUypC,GACfzmD,KAAKsjD,iBAAiBp3B,EAAOna,GAAGma,EAAOla,IAS3C9O,EAAQiQ,UAAU6zC,qBAAuB,WACvChnD,KAAKinD,qBACL,KAAK,GAAIC,KAAOlnD,MAAKg9C,MACfh9C,KAAKg9C,MAAMl3C,eAAeohD,IAC5BlnD,KAAK8jD,YAAY97C,KAAKk/C,IAiB5BhkD,EAAQiQ,UAAU6E,QAAU,SAAStF,EAAM8zC,GAOzC,GANqBhgD,SAAjBggD,IACFA,GAAe,GAGjBxmD,KAAK08C,cAAe,EAEhBhqC,GAAQA,EAAKod,MAAQpd,EAAKsqC,OAAStqC,EAAKorC,OAC1C,KAAM,IAAItkC,aAAY,iGAYxB,IAP+C,GAA3CxZ,KAAK0hD,UAAUnB,iBAAiB7xC,SAClC1O,KAAKmnD,wBAIPnnD,KAAKkT,WAAWR,GAAQA,EAAKjE,SAEzBiE,GAAQA,EAAKod,KAEf,GAAGpd,GAAQA,EAAKod,IAAK,CACnB,GAAIs3B,GAAU3jD,EAAU4jD,WAAW30C,EAAKod,IAExC,YADA9vB,MAAKgY,QAAQovC,QAIZ,IAAI10C,GAAQA,EAAK40C,OAEpB,GAAG50C,GAAQA,EAAK40C,MAAO,CACrB,GAAIC,GAAY7jD,EAAY8jD,WAAW90C,EAAK40C,MAE5C,YADAtnD,MAAKgY,QAAQuvC,QAKfvnD,MAAKynD,UAAU/0C,GAAQA,EAAKsqC,OAC5Bh9C,KAAK0nD,UAAUh1C,GAAQA,EAAKorC,MAE9B99C,MAAK2nD,mBACe,GAAhBnB,IAC+C,GAA7CxmD,KAAK0hD,UAAUjB,mBAAmB/xC,SACpC1O,KAAK4nD,eACL5nD,KAAKglD,4BAIDhlD,KAAK0hD,UAAUP,WACjBnhD,KAAK6nD,aAGT7nD,KAAK4P,SAEP5P,KAAK08C,cAAe,GAOtBx5C,EAAQiQ,UAAUD,WAAa,SAAUzE,GACvC,GAAIA,EAAS,CACX,GAAI5I,GACAqI,GAAU,QAAQ,QAAQ,eAAe,qBAAqB,aAAa,aAC7E,WAAW,mBAAmB,QAAQ,SAAS,aAAa,YAAY,WAAW,aAOrF,IAJAvN,EAAK+F,uBAAuBwH,EAAOlO,KAAK0hD,UAAWjzC,GACnD9N,EAAK+F,wBAAwB,SAAS1G,KAAK0hD,UAAU1E,MAAOvuC,EAAQuuC,OACpEr8C,EAAK+F,wBAAwB,QAAQ,UAAU1G,KAAK0hD,UAAU5D,MAAOrvC,EAAQqvC,OAEzErvC,EAAQ+vC,UACV79C,EAAK4N,aAAavO,KAAK0hD,UAAUlD,QAAS/vC,EAAQ+vC,QAAQ,aAC1D79C,EAAK4N,aAAavO,KAAK0hD,UAAUlD,QAAS/vC,EAAQ+vC,QAAQ,aAEtD/vC,EAAQ+vC,QAAQU,uBAAuB,CACzCl/C,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,CAC3C,KAAK7I,IAAQ4I,GAAQ+vC,QAAQU,sBACvBzwC,EAAQ+vC,QAAQU,sBAAsBp5C,eAAeD,KACvD7F,KAAK0hD,UAAUlD,QAAQU,sBAAsBr5C,GAAQ4I,EAAQ+vC,QAAQU,sBAAsBr5C;CAkDnG,GA5CI4I,EAAQ2gC,QAAQpvC,KAAK28C,iBAAiB1pC,IAAMxE,EAAQ2gC,OACpD3gC,EAAQq5C,SAAS9nD,KAAK28C,iBAAiBC,KAAOnuC,EAAQq5C,QACtDr5C,EAAQs5C,aAAa/nD,KAAK28C,iBAAiBE,SAAWpuC,EAAQs5C,YAC9Dt5C,EAAQu5C,YAAYhoD,KAAK28C,iBAAiBG,QAAUruC,EAAQu5C,WAC5Dv5C,EAAQw5C,WAAWjoD,KAAK28C,iBAAiBI,IAAMtuC,EAAQw5C,UAE3DtnD,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,gBAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,sBAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,cAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,cAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,YAC1C9N,EAAK4N,aAAavO,KAAK0hD,UAAWjzC,EAAQ,oBAGtCA,EAAQ8xC,mBACVvgD,KAAKkoD,SAAWloD,KAAK0hD,UAAUnB,iBAAiBC,kBAK9C/xC,EAAQqvC,QACkBt3C,SAAxBiI,EAAQqvC,MAAM3yC,QACZxK,EAAKwD,SAASsK,EAAQqvC,MAAM3yC,QAC9BnL,KAAK0hD,UAAU5D,MAAM3yC,SACrBnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMA,MAAQsD,EAAQqvC,MAAM3yC,MACjDnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMkB,UAAYoC,EAAQqvC,MAAM3yC,MACrDnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMmB,MAAQmC,EAAQqvC,MAAM3yC,QAGf3E,SAA9BiI,EAAQqvC,MAAM3yC,MAAMA,QAA0BnL,KAAK0hD,UAAU5D,MAAM3yC,MAAMA,MAAQsD,EAAQqvC,MAAM3yC,MAAMA,OACnE3E,SAAlCiI,EAAQqvC,MAAM3yC,MAAMkB,YAA0BrM,KAAK0hD,UAAU5D,MAAM3yC,MAAMkB,UAAYoC,EAAQqvC,MAAM3yC,MAAMkB,WAC3E7F,SAA9BiI,EAAQqvC,MAAM3yC,MAAMmB,QAA0BtM,KAAK0hD,UAAU5D,MAAM3yC,MAAMmB,MAAQmC,EAAQqvC,MAAM3yC,MAAMmB,QAE3GtM,KAAK0hD,UAAU5D,MAAMQ,cAAe,GAGjC7vC,EAAQqvC,MAAMR,WACW92C,SAAxBiI,EAAQqvC,MAAM3yC,QACZxK,EAAKwD,SAASsK,EAAQqvC,MAAM3yC,OAAmBnL,KAAK0hD,UAAU5D,MAAMR,UAAY7uC,EAAQqvC,MAAM3yC,MAC3D3E,SAA9BiI,EAAQqvC,MAAM3yC,MAAMA,QAAsBnL,KAAK0hD,UAAU5D,MAAMR,UAAY7uC,EAAQqvC,MAAM3yC,MAAMA,SAK1GsD,EAAQuuC,OACNvuC,EAAQuuC,MAAM7xC,MAAO,CACvB,GAAIg9C,GAAcxnD,EAAKuK,WAAWuD,EAAQuuC,MAAM7xC,MAChDnL,MAAK0hD,UAAU1E,MAAM7xC,MAAMgB,WAAag8C,EAAYh8C,WACpDnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMiB,OAAS+7C,EAAY/7C,OAChDpM,KAAK0hD,UAAU1E,MAAM7xC,MAAMkB,UAAUF,WAAag8C,EAAY97C,UAAUF,WACxEnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMkB,UAAUD,OAAS+7C,EAAY97C,UAAUD,OACpEpM,KAAK0hD,UAAU1E,MAAM7xC,MAAMmB,MAAMH,WAAag8C,EAAY77C,MAAMH,WAChEnM,KAAK0hD,UAAU1E,MAAM7xC,MAAMmB,MAAMF,OAAS+7C,EAAY77C,MAAMF,OAGhE,GAAIqC,EAAQ0lB,OACV,IAAK,GAAIi0B,KAAa35C,GAAQ0lB,OAC5B,GAAI1lB,EAAQ0lB,OAAOruB,eAAesiD,GAAY,CAC5C,GAAIn2C,GAAQxD,EAAQ0lB,OAAOi0B,EAC3BpoD,MAAKm0B,OAAOlhB,IAAIm1C,EAAWn2C,GAKjC,GAAIxD,EAAQ2X,QAAS,CACnB,IAAKvgB,IAAQ4I,GAAQ2X,QACf3X,EAAQ2X,QAAQtgB,eAAeD,KACjC7F,KAAK0hD,UAAUt7B,QAAQvgB,GAAQ4I,EAAQ2X,QAAQvgB,GAG/C4I,GAAQ2X,QAAQjb,QAClBnL,KAAK0hD,UAAUt7B,QAAQjb,MAAQxK,EAAKuK,WAAWuD,EAAQ2X,QAAQjb,QAmBnE,GAfI,cAAgBsD,KACdA,EAAQ45C,WACLroD,KAAKsoD,YACRtoD,KAAKsoD,UAAY,GAAIhD,GAAUtlD,KAAKsf,OACpCtf,KAAKsoD,UAAU/0C,GAAG,SAAUvT,KAAKuoD,gBAAgBzzB,KAAK90B,QAIpDA,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,YAKd75C,EAAQg4B,OACV,KAAM,IAAI7iC,OAAM,6EAMlB5D,MAAKijD,qBAELjjD,KAAKwoD,0BAELxoD,KAAKyoD,0BAELzoD,KAAK0oD,yBAIL1oD,KAAKuoD,kBAELvoD,KAAK2kB,QAAQ3kB,KAAK0hD,UAAUnvC,MAAOvS,KAAK0hD,UAAUlvC,QAClDxS,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAaT1M,EAAQiQ,UAAUuhB,QAAU,WAE1B,KAAO10B,KAAKyZ,iBAAiBiK,iBAC3B1jB,KAAKyZ,iBAAiBtI,YAAYnR,KAAKyZ,iBAAiBkK,WAe1D,IAZA3jB,KAAKsf,MAAQ/N,SAASM,cAAc,OACpC7R,KAAKsf,MAAMzX,UAAY,oBACvB7H,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKsf,MAAMrS,MAAM4W,SAAW,SAK5B7jB,KAAKsf,MAAMC,OAAShO,SAASM,cAAc,UAC3C7R,KAAKsf,MAAMC,OAAOtS,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMC,QAE7Bvf,KAAKsf,MAAMC,OAAOyH,WAQlB,CACH,GAAID,GAAM/mB,KAAKsf,MAAMC,OAAOyH,WAAW,KACvChnB,MAAK2hD,YAAcp4C,OAAOo/C,kBAAoB,IAAM5hC,EAAI6hC,8BAC9C7hC,EAAI8hC,2BACJ9hC,EAAI+hC,0BACJ/hC,EAAIgiC,yBACJhiC,EAAIiiC,wBAA0B,GAExChpD,KAAKsf,MAAMC,OAAOyH,WAAW,MAAMiiC,aAAajpD,KAAK2hD,WAAY,EAAG,EAAG3hD,KAAK2hD,WAAY,EAAG,OAhB1D,CACjC,GAAI79B,GAAWvS,SAASM,cAAe,MACvCiS,GAAS7W,MAAM9B,MAAQ,MACvB2Y,EAAS7W,MAAM8W,WAAc,OAC7BD,EAAS7W,MAAM+W,QAAW,OAC1BF,EAASG,UAAa,mDACtBjkB,KAAKsf,MAAMC,OAAO9N,YAAYqS,GAgBhC,GAAI3P,GAAKnU,IACTA,MAAKwlC,QACLxlC,KAAKkpD,SACLlpD,KAAK8D,OAAS,GAAIC,GAAO/D,KAAKsf,MAAMC,QACpCvf,KAAK8D,OAAOoR,IAAI,SAASi0C,KAAK/lB,QAAQ,IAEtCpjC,KAAK8D,OAAOyP,GAAG,MAAaY,EAAGi1C,OAAOt0B,KAAK3gB,IAC3CnU,KAAK8D,OAAOyP,GAAG,YAAaY,EAAGk1C,aAAav0B,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,QAAaY,EAAGgqB,QAAQrJ,KAAK3gB,IAC5CnU,KAAK8D,OAAOyP,GAAG,QAAaY,EAAGmqB,SAASxJ,KAAK3gB,IAC7CixC,EAAWkE,QAAQtpD,KAAK8D,OAAQqQ,EAAGkqB,SAASvJ,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,WAAaY,EAAG6pB,aAAalJ,KAAK3gB,IACjDnU,KAAK8D,OAAOyP,GAAG,UAAaY,EAAG8pB,QAAQnJ,KAAK3gB,IAC5CnU,KAAK8D,OAAOyP,GAAG,SAAaY,EAAG+pB,WAAWpJ,KAAK3gB,IAE/CnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,YAAawL,EAAGo1C,kBAAkBz0B,KAAK3gB,IAC1EnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,aAAcwL,EAAGiqB,cAActJ,KAAK3gB,IACvEnU,KAAKsf,MAAMC,OAAO5W,iBAAiB,iBAAkBwL,EAAGiqB,cAActJ,KAAK3gB,IAG3EnU,KAAKyZ,iBAAiBhI,YAAYzR,KAAKsf,QAQzCpc,EAAQiQ,UAAUo1C,gBAAkB,WAClC,GAAIp0C,GAAKnU,IACawG,UAAlBxG,KAAKmlD,UACPnlD,KAAKmlD,SAAS7xC,UAEhBtT,KAAKmlD,SAAWA,IAEhBnlD,KAAKmlD,SAASqE,QAEVxpD,KAAK0hD,UAAUrB,SAAS3xC,SAAW1O,KAAKypD,aAC1CzpD,KAAKmlD,SAASrwB,KAAK,KAAQ90B,KAAK0pD,QAAQ50B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,KAAQ90B,KAAK2pD,aAAa70B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK4pD,UAAU90B,KAAK3gB,GAAM,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK2pD,aAAa70B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK6pD,UAAU/0B,KAAK3gB,GAAM,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAK8pD,aAAah1B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,QAAQ90B,KAAK+pD,WAAWj1B,KAAK3gB,GAAK,WACrDnU,KAAKmlD,SAASrwB,KAAK,QAAQ90B,KAAK8pD,aAAah1B,KAAK3gB,GAAK,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,OAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAQ,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKkqD,SAASp1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,IAAQ90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAQ,SACvDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKgqD,QAAQl1B,KAAK3gB,GAAO,WACrDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAO,SACvDnU,KAAKmlD,SAASrwB,KAAK,WAAW90B,KAAKkqD,SAASp1B,KAAK3gB,GAAI,WACrDnU,KAAKmlD,SAASrwB,KAAK,WAAW90B,KAAKiqD,UAAUn1B,KAAK3gB,GAAK,UAGV,GAA3CnU,KAAK0hD,UAAUnB,iBAAiB7xC,UAClC1O,KAAKmlD,SAASrwB,KAAK,MAAM90B,KAAKmnD,sBAAsBryB,KAAK3gB,IACzDnU,KAAKmlD,SAASrwB,KAAK,SAAS90B,KAAKmqD,gBAAgBr1B,KAAK3gB,MAU1DjR,EAAQiQ,UAAUG,QAAU,WAC1BtT,KAAK4P,MAAQ,aACb5P,KAAKyhB,OAAS,aACdzhB,KAAK+kD,OAAQ,EAGb/kD,KAAKoqD,+BAGLpqD,KAAKmlD,SAASqE,QAGdxpD,KAAK8D,OAAOwP,UAGZtT,KAAK0T,MAEL1T,KAAKqqD,oBAAoBrqD,KAAKyZ,mBAGhCvW,EAAQiQ,UAAUk3C,oBAAsB,SAASC,GAC/C,KAAoC,GAA7BA,EAAU5mC,iBACf1jB,KAAKqqD,oBAAoBC,EAAU3mC,YACnC2mC,EAAUn5C,YAAYm5C,EAAU3mC,aAUpCzgB,EAAQiQ,UAAUo3C,YAAc,SAAUr+B,GACxC,OACEna,EAAGma,EAAOna,EAAIpR,EAAK2G,gBAAgBtH,KAAKsf,MAAMC,QAC9CvN,EAAGka,EAAOla,EAAIrR,EAAK+G,eAAe1H,KAAKsf,MAAMC,UASjDrc,EAAQiQ,UAAUkrB,SAAW,SAAU/0B,IACjC,GAAIhF,OAAO0C,UAAYhH,KAAK0iD,UAAY,MAC1C1iD,KAAKwlC,KAAK3F,QAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QAC3ClsB,KAAKwlC,KAAKglB,SAAU,EACpBxqD,KAAKkpD,MAAMjsC,MAAQjd,KAAKyqD,YAGxBzqD,KAAK0iD,WAAY,GAAIp+C,OAAO0C,UAE5BhH,KAAK0qD,aAAa1qD,KAAKwlC,KAAK3F,WAQhC38B,EAAQiQ,UAAU6qB,aAAe,SAAU10B,GACzCtJ,KAAK2qD,iBAAiBrhD,IAUxBpG,EAAQiQ,UAAUw3C,iBAAmB,SAASrhD,GAElB9C,SAAtBxG,KAAKwlC,KAAK3F,SACZ7/B,KAAKq+B,SAAS/0B,EAGhB,IAAIw8C,GAAO9lD,KAAK4qD,WAAW5qD,KAAKwlC,KAAK3F,QASrC,IANA7/B,KAAKwlC,KAAKzG,UAAW,EACrB/+B,KAAKwlC,KAAK2K,aACVnwC,KAAKwlC,KAAK/nB,YAAczd,KAAK6qD,kBAC7B7qD,KAAKwlC,KAAK2gB,OAAS,KACnBnmD,KAAK2jD,eAAgB,EAET,MAARmC,GAA4C,GAA5B9lD,KAAK0hD,UAAUH,UAAmB,CACpDvhD,KAAK2jD,eAAgB,EACrB3jD,KAAKwlC,KAAK2gB,OAASL,EAAKzlD,GAEnBylD,EAAKgF,cACR9qD,KAAK+qD,cAAcjF,GAAK,GAG1B9lD,KAAK4tB,KAAK,aAAao9B,QAAQhrD,KAAK42B,eAAeomB,OAGnD,KAAK,GAAIiO,KAAYjrD,MAAKkrD,aAAalO,MACrC,GAAIh9C,KAAKkrD,aAAalO,MAAMl3C,eAAemlD,GAAW,CACpD,GAAIhnD,GAASjE,KAAKkrD,aAAalO,MAAMiO,GACjCr/C,GACFvL,GAAI4D,EAAO5D,GACXylD,KAAM7hD,EAGN8N,EAAG9N,EAAO8N,EACVC,EAAG/N,EAAO+N,EACVm5C,OAAQlnD,EAAOknD,OACfC,OAAQnnD,EAAOmnD,OAGjBnnD,GAAOknD,QAAS,EAChBlnD,EAAOmnD,QAAS,EAEhBprD,KAAKwlC,KAAK2K,UAAUnoC,KAAK4D,IAK/BtC,EAAMD,kBAQRnG,EAAQiQ,UAAU8qB,QAAU,SAAU30B,GACpCtJ,KAAKqrD,cAAc/hD,IAUrBpG,EAAQiQ,UAAUk4C,cAAgB,SAAS/hD,GACzC,IAAItJ,KAAKwlC,KAAKglB,QAAd,CAKAxqD,KAAKsrD,aAEL,IAAIzrB,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QACjC/X,EAAKnU,KACLwlC,EAAOxlC,KAAKwlC,KACZ2K,EAAY3K,EAAK2K,SACrB,IAAIA,GAAaA,EAAUxqC,QAAsC,GAA5B3F,KAAK0hD,UAAUH,UAAmB,CAErE,GAAI/hB,GAASK,EAAQ9tB,EAAIyzB,EAAK3F,QAAQ9tB,EAClC0tB,EAASI,EAAQ7tB,EAAIwzB,EAAK3F,QAAQ7tB,CAGtCm+B,GAAU9nC,QAAQ,SAAUuD,GAC1B,GAAIk6C,GAAOl6C,EAAEk6C,IAERl6C,GAAEu/C,SACLrF,EAAK/zC,EAAIoC,EAAGo3C,qBAAqBp3C,EAAGq3C,qBAAqB5/C,EAAEmG,GAAKytB,IAG7D5zB,EAAEw/C,SACLtF,EAAK9zC,EAAImC,EAAGs3C,qBAAqBt3C,EAAGu3C,qBAAqB9/C,EAAEoG,GAAKytB,MAM/Dz/B,KAAK8kD,SACR9kD,KAAK8kD,QAAS,EACd9kD,KAAK4P,aAKP,IAAkC,GAA9B5P,KAAK0hD,UAAUJ,YAAqB,CAEtC,GAA0B96C,SAAtBxG,KAAKwlC,KAAK3F,QAEZ,WADA7/B,MAAK2qD,iBAAiBrhD,EAGxB,IAAI8jB,GAAQyS,EAAQ9tB,EAAI/R,KAAKwlC,KAAK3F,QAAQ9tB,EACtCsb,EAAQwS,EAAQ7tB,EAAIhS,KAAKwlC,KAAK3F,QAAQ7tB,CAE1ChS,MAAKsjD,gBACHtjD,KAAKwlC,KAAK/nB,YAAY1L,EAAIqb,EAC1BptB,KAAKwlC,KAAK/nB,YAAYzL,EAAIqb,GAE5BrtB,KAAK6iD,UAITv5C,EAAMD,mBAORnG,EAAQiQ,UAAU+qB,WAAa,SAAU50B,GACvCtJ,KAAK2rD,eAAeriD,IAItBpG,EAAQiQ,UAAUw4C,eAAiB,SAASriD,GAC1CtJ,KAAKwlC,KAAKzG,UAAW,CACrB,IAAIoR,GAAYnwC,KAAKwlC,KAAK2K,SACtBA,IAAaA,EAAUxqC,QACzBwqC,EAAU9nC,QAAQ,SAAUuD,GAE1BA,EAAEk6C,KAAKqF,OAASv/C,EAAEu/C,OAClBv/C,EAAEk6C,KAAKsF,OAASx/C,EAAEw/C,SAEpBprD,KAAK8kD,QAAS,EACd9kD,KAAK4P,SAGL5P,KAAK6iD,UAEmB,GAAtB7iD,KAAK2jD,cACP3jD,KAAK4tB,KAAK,WAAWo9B,aAGrBhrD,KAAK4tB,KAAK,WAAWo9B,QAAQhrD,KAAK42B,eAAeomB,QAGnD1zC,EAAMD,kBAORnG,EAAQiQ,UAAUi2C,OAAS,SAAU9/C,GACnC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKikD,gBAAkBpkB,EACvB7/B,KAAK4rD,WAAW/rB,IASlB38B,EAAQiQ,UAAUk2C,aAAe,SAAU//C,GACzC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAK6rD,iBAAiBhsB,IAQxB38B,EAAQiQ,UAAUgrB,QAAU,SAAU70B,GACpC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKikD,gBAAkBpkB,EACvB7/B,KAAK8rD,cAAcjsB,IAQrB38B,EAAQiQ,UAAU44C,WAAa,SAAUziD,GACvC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACrClsB,MAAKgsD,iBAAiBnsB,IAQxB38B,EAAQiQ,UAAUmrB,SAAW,SAAUh1B,GACrC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OAErClsB,MAAKwlC,KAAKglB,SAAU,EACd,SAAWxqD,MAAKkpD,QACpBlpD,KAAKkpD,MAAMjsC,MAAQ,EAIrB,IAAIA,GAAQjd,KAAKkpD,MAAMjsC,MAAQ3T,EAAM2T,KACrCjd,MAAKisD,MAAMhvC,EAAO4iB,IAUpB38B,EAAQiQ,UAAU84C,MAAQ,SAAShvC,EAAO4iB,GACxC,GAA+B,GAA3B7/B,KAAK0hD,UAAU/jB,SAAkB,CACnC,GAAIuuB,GAAWlsD,KAAKyqD,WACR,MAARxtC,IACFA,EAAQ,MAENA,EAAQ,KACVA,EAAQ,GAGV,IAAIkvC,GAAsB,IACR3lD,UAAdxG,KAAKwlC,MACmB,GAAtBxlC,KAAKwlC,KAAKzG,WACZotB,EAAsBnsD,KAAKosD,YAAYpsD,KAAKwlC,KAAK3F,SAIrD,IAAIpiB,GAAczd,KAAK6qD,kBAEnBwB,EAAYpvC,EAAQivC,EACpBI,GAAM,EAAID,GAAaxsB,EAAQ9tB,EAAI0L,EAAY1L,EAAIs6C,EACnDE,GAAM,EAAIF,GAAaxsB,EAAQ7tB,EAAIyL,EAAYzL,EAAIq6C,CASvD,IAPArsD,KAAKkkD,YAAcnyC,EAAM/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxCC,EAAMhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAE3DhS,KAAKgd,UAAUC,GACfjd,KAAKsjD,gBAAgBgJ,EAAIC,GACzBvsD,KAAKwsD,wBAEsB,MAAvBL,EAA6B,CAC/B,GAAIM,GAAuBzsD,KAAK0sD,YAAYP,EAC5CnsD,MAAKwlC,KAAK3F,QAAQ9tB,EAAI06C,EAAqB16C,EAC3C/R,KAAKwlC,KAAK3F,QAAQ7tB,EAAIy6C,EAAqBz6C,EAY7C,MATAhS,MAAK6iD,UAEU5lC,EAAXivC,EACFlsD,KAAK4tB,KAAK,QAASsN,UAAU,MAG7Bl7B,KAAK4tB,KAAK,QAASsN,UAAU,MAGxBje,IAYX/Z,EAAQiQ,UAAUirB,cAAgB,SAAS90B,GAEzC,GAAImlB,GAAQ,CAYZ,IAXInlB,EAAMolB,WACRD,EAAQnlB,EAAMolB,WAAW,IAChBplB,EAAMqlB,SAGfF,GAASnlB,EAAMqlB,OAAO,GAMpBF,EAAO,CAET,GAAIxR,GAAQjd,KAAKyqD,YACbvqB,EAAOzR,EAAQ,EACP,GAARA,IACFyR,GAAe,EAAIA,GAErBjjB,GAAU,EAAIijB,CAGd,IAAIL,GAAU7/B,KAAKuqD,aAAax4C,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAGzD//B,MAAKisD,MAAMhvC,EAAO4iB,GAIpBv2B,EAAMD,kBASRnG,EAAQiQ,UAAUo2C,kBAAoB,SAAUjgD,GAC9C,GAAIu2B,GAAU7/B,KAAKuqD,aAAax4C,EAAGzI,EAAMw2B,MAAO9tB,EAAG1I,EAAMy2B,OAGrD//B,MAAK2sD,UACP3sD,KAAK4sD,gBAAgB/sB,EAKvB,IAAI1rB,GAAKnU,KACL6sD,EAAY,WACd14C,EAAG24C,gBAAgBjtB,GAarB,IAXI7/B,KAAK+sD,YACPt6B,cAAczyB,KAAK+sD,YAEhB/sD,KAAKwlC,KAAKzG,WACb/+B,KAAK+sD,WAAazzC,WAAWuzC,EAAW7sD,KAAK0hD,UAAUt7B,QAAQ5N,QAOrC,GAAxBxY,KAAK0hD,UAAUp1C,MAAe,CAEhC,IAAK,GAAI0gD,KAAUhtD,MAAK4hD,SAAS9D,MAC3B99C,KAAK4hD,SAAS9D,MAAMh4C,eAAeknD,KACrChtD,KAAK4hD,SAAS9D,MAAMkP,GAAQ1gD,OAAQ,QAC7BtM,MAAK4hD,SAAS9D,MAAMkP,GAK/B,IAAIjqC,GAAM/iB,KAAK4qD,WAAW/qB,EACf,OAAP9c,IACFA,EAAM/iB,KAAKitD,WAAWptB,IAEb,MAAP9c,GACF/iB,KAAKktD,aAAanqC,EAIpB,KAAK,GAAIojC,KAAUnmD,MAAK4hD,SAAS5E,MAC3Bh9C,KAAK4hD,SAAS5E,MAAMl3C,eAAeqgD,KACjCpjC,YAAexf,IAAQwf,EAAI1iB,IAAM8lD,GAAUpjC,YAAe3f,IAAe,MAAP2f,KACpE/iB,KAAKmtD,YAAYntD,KAAK4hD,SAAS5E,MAAMmJ,UAC9BnmD,MAAK4hD,SAAS5E,MAAMmJ,GAIjCnmD,MAAKyhB,WAYTve,EAAQiQ,UAAU25C,gBAAkB,SAAUjtB,GAC5C,GAOIx/B,GAPA0iB,GACFtb,KAAQzH,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC1CpK,IAAQ3H,KAAKyrD,qBAAqB5rB,EAAQ7tB,GAC1CqV,MAAQrnB,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC1CuR,OAAQtjB,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAIxCo7C,EAAgBptD,KAAK2sD,SACrBU,GAAkB,CAEtB,IAAqB7mD,QAAjBxG,KAAK2sD,SAAuB,CAE9B,GAAI3P,GAAQh9C,KAAKg9C,MACbsQ,IACJ,KAAKjtD,IAAM28C,GACT,GAAIA,EAAMl3C,eAAezF,GAAK,CAC5B,GAAIylD,GAAO9I,EAAM38C,EACbylD,GAAKyH,kBAAkBxqC,IACDvc,SAApBs/C,EAAK0H,YACPF,EAAiBtlD,KAAK3H,GAM1BitD,EAAiB3nD,OAAS,IAG5B3F,KAAK2sD,SAAW3sD,KAAKg9C,MAAMsQ,EAAiBA,EAAiB3nD,OAAS,IAEtE0nD,GAAkB,GAItB,GAAsB7mD,SAAlBxG,KAAK2sD,UAA6C,GAAnBU,EAA0B,CAE3D,GAAIvP,GAAQ99C,KAAK89C,MACb2P,IACJ,KAAKptD,IAAMy9C,GACT,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACbqtD,GAAKC,WAAkCnnD,SAApBknD,EAAKF,YACxBE,EAAKH,kBAAkBxqC,IACzB0qC,EAAiBzlD,KAAK3H,GAKxBotD,EAAiB9nD,OAAS,IAC5B3F,KAAK2sD,SAAW3sD,KAAK89C,MAAM2P,EAAiBA,EAAiB9nD,OAAS,KAI1E,GAAI3F,KAAK2sD,UAEP,GAAI3sD,KAAK2sD,UAAYS,EAAe,CAClC,GAAIj5C,GAAKnU,IACJmU,GAAGy5C,QACNz5C,EAAGy5C,MAAQ,GAAIpqD,GAAM2Q,EAAGmL,MAAOnL,EAAGutC,UAAUt7B,UAM9CjS,EAAGy5C,MAAMC,YAAYhuB,EAAQ9tB,EAAI,EAAG8tB,EAAQ7tB,EAAI,GAChDmC,EAAGy5C,MAAME,QAAQ35C,EAAGw4C,SAASa,YAC7Br5C,EAAGy5C,MAAMjmB,YAIP3nC,MAAK4tD,OACP5tD,KAAK4tD,MAAMlmB,QAYjBxkC,EAAQiQ,UAAUy5C,gBAAkB,SAAU/sB,GACvC7/B,KAAK2sD,UAAa3sD,KAAK4qD,WAAW/qB,KACrC7/B,KAAK2sD,SAAWnmD,OACZxG,KAAK4tD,OACP5tD,KAAK4tD,MAAMlmB,SAajBxkC,EAAQiQ,UAAUwR,QAAU,SAASpS,EAAOC,GAC1C,GAAIu7C,IAAY,EACZC,EAAWhuD,KAAKsf,MAAMC,OAAOhN,MAC7B07C,EAAYjuD,KAAKsf,MAAMC,OAAO/M,MAC9BD,IAASvS,KAAK0hD,UAAUnvC,OAASC,GAAUxS,KAAK0hD,UAAUlvC,QAAUxS,KAAKsf,MAAMrS,MAAMsF,OAASA,GAASvS,KAAKsf,MAAMrS,MAAMuF,QAAUA,GACpIxS,KAAKsf,MAAMrS,MAAMsF,MAAQA,EACzBvS,KAAKsf,MAAMrS,MAAMuF,OAASA,EAE1BxS,KAAKsf,MAAMC,OAAOtS,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAMC,OAAOtS,MAAMuF,OAAS,OAEjCxS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,WAC/D3hD,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,WAEjE3hD,KAAK0hD,UAAUnvC,MAAQA,EACvBvS,KAAK0hD,UAAUlvC,OAASA,EAExBu7C,GAAY,IAMR/tD,KAAKsf,MAAMC,OAAOhN,OAASvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,aAClE3hD,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,WAC/DoM,GAAY,GAEV/tD,KAAKsf,MAAMC,OAAO/M,QAAUxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,aACpE3hD,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,WACjEoM,GAAY,IAIC,GAAbA,GACF/tD,KAAK4tB,KAAK,UAAWrb,MAAMvS,KAAKsf,MAAMC,OAAOhN,MAAQvS,KAAK2hD,WAAWnvC,OAAOxS,KAAKsf,MAAMC,OAAO/M,OAASxS,KAAK2hD,WAAYqM,SAAUA,EAAWhuD,KAAK2hD,WAAYsM,UAAWA,EAAYjuD,KAAK2hD,cAS9Lz+C,EAAQiQ,UAAUs0C,UAAY,SAASzK,GACrC,GAAIkR,GAAeluD,KAAKokD,SAExB,IAAIpH,YAAiBn8C,IAAWm8C,YAAiBl8C,GAC/Cd,KAAKokD,UAAYpH,MAEd,IAAI/2C,MAAMC,QAAQ82C,GACrBh9C,KAAKokD,UAAY,GAAIvjD,GACrBb,KAAKokD,UAAUnxC,IAAI+pC,OAEhB,CAAA,GAAKA,EAIR,KAAM,IAAI32C,WAAU,4BAHpBrG,MAAKokD,UAAY,GAAIvjD,GAgBvB,GAVIqtD,GAEFvtD,EAAK0H,QAAQrI,KAAKskD,eAAgB,SAAUh8C,EAAUgB,GACpD4kD,EAAax6C,IAAIpK,EAAOhB,KAK5BtI,KAAKg9C,SAEDh9C,KAAKokD,UAAW,CAElB,GAAIjwC,GAAKnU,IACTW,GAAK0H,QAAQrI,KAAKskD,eAAgB,SAAUh8C,EAAUgB,GACpD6K,EAAGiwC,UAAU7wC,GAAGjK,EAAOhB,IAIzB,IAAI6M,GAAMnV,KAAKokD,UAAUvuC,QACzB7V,MAAKukD,UAAUpvC,GAEjBnV,KAAKmuD,oBAQPjrD,EAAQiQ,UAAUoxC,UAAY,SAASpvC,GAErC,IAAK,GADD9U,GACKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9CnF,EAAK8U,EAAI3P,EACT,IAAIkN,GAAO1S,KAAKokD,UAAUlvC,IAAI7U,GAC1BylD,EAAO,GAAIviD,GAAKmP,EAAM1S,KAAK2iD,OAAQ3iD,KAAKm0B,OAAQn0B,KAAK0hD,UAEzD,IADA1hD,KAAKg9C,MAAM38C,GAAMylD,IACG,GAAfA,EAAKqF,QAAkC,GAAfrF,EAAKsF,QAAgC,OAAXtF,EAAK/zC,GAAyB,OAAX+zC,EAAK9zC,GAAa,CAC1F,GAAIyZ,GAAS,EAAStW,EAAIxP,OAAS,GAC/ByoD,EAAQ,EAAIlpD,KAAKymB,GAAKzmB,KAAKE,QACZ,IAAf0gD,EAAKqF,SAAkBrF,EAAK/zC,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,IACnC,GAAftI,EAAKsF,SAAkBtF,EAAK9zC,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,IAExDpuD,KAAK8kD,QAAS,EAGhB9kD,KAAKgnD,uBAC4C,GAA7ChnD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,0BACLruD,KAAKsuD,kBACLtuD,KAAKuuD,kBAAkBvuD,KAAKg9C,OAC5Bh9C,KAAKwuD,gBAQPtrD,EAAQiQ,UAAUqxC,aAAe,SAASrvC,EAAIs5C,GAE5C,IAAK,GADDzR,GAAQh9C,KAAKg9C,MACRx3C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GACTsgD,EAAO9I,EAAM38C,GACbqS,EAAO+7C,EAAYjpD,EACnBsgD,GAEFA,EAAK4I,cAAch8C,EAAM1S,KAAK0hD,YAI9BoE,EAAO,GAAIviD,GAAKorD,WAAY3uD,KAAK2iD,OAAQ3iD,KAAKm0B,OAAQn0B,KAAK0hD,WAC3D1E,EAAM38C,GAAMylD,GAGhB9lD,KAAK8kD,QAAS,EACmC,GAA7C9kD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKgnD,uBACLhnD,KAAKuuD,kBAAkBvR,IAQzB95C,EAAQiQ,UAAUsxC,aAAe,SAAStvC,GAExC,IAAK,GADD6nC,GAAQh9C,KAAKg9C,MACRx3C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,SACNw3C,GAAM38C,GAEfL,KAAKgnD,uBAC4C,GAA7ChnD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,0BACLruD,KAAKsuD,kBACLtuD,KAAKmuD,mBACLnuD,KAAKuuD,kBAAkBvR,IASzB95C,EAAQiQ,UAAUu0C,UAAY,SAAS5J,GACrC,GAAI8Q,GAAe5uD,KAAKqkD,SAExB,IAAIvG,YAAiBj9C,IAAWi9C,YAAiBh9C,GAC/Cd,KAAKqkD,UAAYvG,MAEd,IAAI73C,MAAMC,QAAQ43C,GACrB99C,KAAKqkD,UAAY,GAAIxjD,GACrBb,KAAKqkD,UAAUpxC,IAAI6qC,OAEhB,CAAA,GAAKA,EAIR,KAAM,IAAIz3C,WAAU,4BAHpBrG,MAAKqkD,UAAY,GAAIxjD,GAgBvB,GAVI+tD,GAEFjuD,EAAK0H,QAAQrI,KAAK0kD,eAAgB,SAAUp8C,EAAUgB,GACpDslD,EAAal7C,IAAIpK,EAAOhB,KAK5BtI,KAAK89C,SAED99C,KAAKqkD,UAAW,CAElB,GAAIlwC,GAAKnU,IACTW,GAAK0H,QAAQrI,KAAK0kD,eAAgB,SAAUp8C,EAAUgB,GACpD6K,EAAGkwC,UAAU9wC,GAAGjK,EAAOhB,IAIzB,IAAI6M,GAAMnV,KAAKqkD,UAAUxuC,QACzB7V,MAAK2kD,UAAUxvC,GAGjBnV,KAAKsuD,mBAQPprD,EAAQiQ,UAAUwxC,UAAY,SAAUxvC,GAItC,IAAK,GAHD2oC,GAAQ99C,KAAK89C,MACbuG,EAAYrkD,KAAKqkD,UAEZ7+C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GAETqpD,EAAU/Q,EAAMz9C,EAChBwuD,IACFA,EAAQC,YAGV,IAAIp8C,GAAO2xC,EAAUnvC,IAAI7U,GAAK0uD,iBAAoB,GAClDjR,GAAMz9C,GAAM,GAAI+C,GAAKsP,EAAM1S,KAAMA,KAAK0hD,WAExC1hD,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,GACvB99C,KAAKgvD,qBACLhvD,KAAKquD,0BAC4C,GAA7CruD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,6BAST9hD,EAAQiQ,UAAUyxC,aAAe,SAAUzvC,GAGzC,IAAK,GAFD2oC,GAAQ99C,KAAK89C,MACbuG,EAAYrkD,KAAKqkD,UACZ7+C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GAETkN,EAAO2xC,EAAUnvC,IAAI7U,GACrBqtD,EAAO5P,EAAMz9C,EACbqtD,IAEFA,EAAKoB,aACLpB,EAAKgB,cAAch8C,EAAM1S,KAAK0hD,WAC9BgM,EAAK5Q,YAIL4Q,EAAO,GAAItqD,GAAKsP,EAAM1S,KAAMA,KAAK0hD,WACjC1hD,KAAK89C,MAAMz9C,GAAMqtD,GAIrB1tD,KAAKgvD,qBAC4C,GAA7ChvD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,IAQzB56C,EAAQiQ,UAAU0xC,aAAe,SAAU1vC,GAEzC,IAAK,GADD2oC,GAAQ99C,KAAK89C,MACRt4C,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IAAK,CAC9C,GAAInF,GAAK8U,EAAI3P,GACTkoD,EAAO5P,EAAMz9C,EACbqtD,KACc,MAAZA,EAAKuB,WACAjvD,MAAKkvD,QAAiB,QAAS,MAAExB,EAAKuB,IAAI5uD,IAEnDqtD,EAAKoB,mBACEhR,GAAMz9C,IAIjBL,KAAK8kD,QAAS,EACd9kD,KAAKuuD,kBAAkBzQ,GAC0B,GAA7C99C,KAAK0hD,UAAUjB,mBAAmB/xC,SAAwC,GAArB1O,KAAK08C,eAC5D18C,KAAK4nD,eACL5nD,KAAKglD,4BAEPhlD,KAAKquD,2BAOPnrD,EAAQiQ,UAAUm7C,gBAAkB,WAClC,GAAIjuD,GACA28C,EAAQh9C,KAAKg9C,MACbc,EAAQ99C,KAAK89C,KACjB,KAAKz9C,IAAM28C,GACLA,EAAMl3C,eAAezF,KACvB28C,EAAM38C,GAAIy9C,SACVd,EAAM38C,GAAI8uD,gBAId,KAAK9uD,IAAMy9C,GACT,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACjBqtD,GAAKtkC,KAAO,KACZskC,EAAKrkC,GAAK,KACVqkC,EAAK5Q,YAaX55C,EAAQiQ,UAAUo7C,kBAAoB,SAASxrC,GAC7C,GAAI1iB,GAGA6b,EAAW1V,OACX2V,EAAW3V,MACf,KAAKnG,IAAM0iB,GACT,GAAIA,EAAIjd,eAAezF,GAAK,CAC1B,GAAIgH,GAAQ0b,EAAI1iB,GAAIuU,UACNpO,UAAVa,IACF6U,EAAyB1V,SAAb0V,EAA0B7U,EAAQnC,KAAK4G,IAAIzE,EAAO6U,GAC9DC,EAAyB3V,SAAb2V,EAA0B9U,EAAQnC,KAAKwH,IAAIrF,EAAO8U,IAMpE,GAAiB3V,SAAb0V,GAAuC1V,SAAb2V,EAC5B,IAAK9b,IAAM0iB,GACLA,EAAIjd,eAAezF,IACrB0iB,EAAI1iB,GAAI+uD,cAAclzC,EAAUC,IAUxCjZ,EAAQiQ,UAAUsO,OAAS,WACzBzhB,KAAK2kB,QAAQ3kB,KAAK0hD,UAAUnvC,MAAOvS,KAAK0hD,UAAUlvC,QAClDxS,KAAK6iD,WAQP3/C,EAAQiQ,UAAU0vC,QAAU,SAAS3pB,GACnC,GAAInS,GAAM/mB,KAAKsf,MAAMC,OAAOyH,WAAW,KAEvCD,GAAIkiC,aAAajpD,KAAK2hD,WAAY,EAAG,EAAG3hD,KAAK2hD,WAAY,EAAG,EAG5D,IAAI0N,GAAIrvD,KAAKsf,MAAMC,OAAOhN,MAASvS,KAAK2hD,WACpCh2C,EAAI3L,KAAKsf,MAAMC,OAAO/M,OAAUxS,KAAK2hD,UACzC56B,GAAIE,UAAU,EAAG,EAAGooC,EAAG1jD,GAGvBob,EAAIuoC,OACJvoC,EAAIwoC,UAAUvvD,KAAKyd,YAAY1L,EAAG/R,KAAKyd,YAAYzL,GACnD+U,EAAI9J,MAAMjd,KAAKid,MAAOjd,KAAKid,OAE3Bjd,KAAK+jD,eACHhyC,EAAK/R,KAAKurD,qBAAqB,GAC/Bv5C,EAAKhS,KAAKyrD,qBAAqB,IAEjCzrD,KAAKgkD,mBACHjyC,EAAK/R,KAAKurD,qBAAqBvrD,KAAKsf,MAAMC,OAAOC,YAAcxf,KAAK2hD,YACpE3vC,EAAKhS,KAAKyrD,qBAAqBzrD,KAAKsf,MAAMC,OAAOsF,aAAe7kB,KAAK2hD,aAGvD,GAAVzoB,IACJl5B,KAAKwvD,gBAAgB,sBAAuBzoC,IAClB,GAAtB/mB,KAAKwlC,KAAKzG,UAA4Cv4B,SAAvBxG,KAAKwlC,KAAKzG,UAA4D,GAAlC/+B,KAAK0hD,UAAUF,kBACpFxhD,KAAKwvD,gBAAgB,aAAczoC,KAIb,GAAtB/mB,KAAKwlC,KAAKzG,UAA4Cv4B,SAAvBxG,KAAKwlC,KAAKzG,UAA4D,GAAlC/+B,KAAK0hD,UAAUD,kBACpFzhD,KAAKwvD,gBAAgB,aAAazoC,GAAI,GAGxB,GAAVmS,GAC2B,GAA3Bl5B,KAAK6hD,oBACP7hD,KAAKwvD,gBAAgB,oBAAqBzoC,GAQ9CA,EAAI0oC,UAEU,GAAVv2B,GACFnS,EAAIE,UAAU,EAAG,EAAGooC,EAAG1jD,IAU3BzI,EAAQiQ,UAAUmwC,gBAAkB,SAASoM,EAASC,GAC3BnpD,SAArBxG,KAAKyd,cACPzd,KAAKyd,aACH1L,EAAG,EACHC,EAAG,IAISxL,SAAZkpD,IACF1vD,KAAKyd,YAAY1L,EAAI29C,GAEPlpD,SAAZmpD,IACF3vD,KAAKyd,YAAYzL,EAAI29C,GAGvB3vD,KAAK4tB,KAAK,gBAQZ1qB,EAAQiQ,UAAU03C,gBAAkB,WAClC,OACE94C,EAAG/R,KAAKyd,YAAY1L,EACpBC,EAAGhS,KAAKyd,YAAYzL,IASxB9O,EAAQiQ,UAAU6J,UAAY,SAASC,GACrCjd,KAAKid,MAAQA,GAQf/Z,EAAQiQ,UAAUs3C,UAAY,WAC5B,MAAOzqD,MAAKid,OAUd/Z,EAAQiQ,UAAUo4C,qBAAuB,SAASx5C,GAChD,OAAQA,EAAI/R,KAAKyd,YAAY1L,GAAK/R,KAAKid,OAUzC/Z,EAAQiQ,UAAUq4C,qBAAuB,SAASz5C,GAChD,MAAOA,GAAI/R,KAAKid,MAAQjd,KAAKyd,YAAY1L,GAU3C7O,EAAQiQ,UAAUs4C,qBAAuB,SAASz5C,GAChD,OAAQA,EAAIhS,KAAKyd,YAAYzL,GAAKhS,KAAKid,OAUzC/Z,EAAQiQ,UAAUu4C,qBAAuB,SAAS15C,GAChD,MAAOA,GAAIhS,KAAKid,MAAQjd,KAAKyd,YAAYzL,GAU3C9O,EAAQiQ,UAAUu5C,YAAc,SAAUnnC,GACxC,OAAQxT,EAAG/R,KAAKwrD,qBAAqBjmC,EAAIxT,GAAIC,EAAGhS,KAAK0rD,qBAAqBnmC,EAAIvT,KAShF9O,EAAQiQ,UAAUi5C,YAAc,SAAU7mC,GACxC,OAAQxT,EAAG/R,KAAKurD,qBAAqBhmC,EAAIxT,GAAIC,EAAGhS,KAAKyrD,qBAAqBlmC,EAAIvT,KAUhF9O,EAAQiQ,UAAUy8C,WAAa,SAAS7oC,EAAI8oC,GACvBrpD,SAAfqpD,IACFA,GAAa,EAIf,IAAI7S,GAAQh9C,KAAKg9C,MACb3J,IAEJ,KAAK,GAAIhzC,KAAM28C,GACTA,EAAMl3C,eAAezF,KACvB28C,EAAM38C,GAAIyvD,eAAe9vD,KAAKid,MAAMjd,KAAK+jD,cAAc/jD,KAAKgkD,mBACxDhH,EAAM38C,GAAIyqD,aACZzX,EAASrrC,KAAK3H,IAGV28C,EAAM38C,GAAI0vD,UAAYF,IACxB7S,EAAM38C,GAAI0rC,KAAKhlB,GAOvB,KAAK,GAAInb,GAAI,EAAGokD,EAAO3c,EAAS1tC,OAAYqqD,EAAJpkD,EAAUA,KAC5CoxC,EAAM3J,EAASznC,IAAImkD,UAAYF,IACjC7S,EAAM3J,EAASznC,IAAImgC,KAAKhlB,IAW9B7jB,EAAQiQ,UAAU88C,WAAa,SAASlpC,GACtC,GAAI+2B,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIz9C,KAAMy9C,GACb,GAAIA,EAAMh4C,eAAezF,GAAK,CAC5B,GAAIqtD,GAAO5P,EAAMz9C,EACjBqtD,GAAK1qB,SAAShjC,KAAKid,OACfywC,EAAKC,WACP7P,EAAMz9C,GAAI0rC,KAAKhlB,KAYvB7jB,EAAQiQ,UAAU+8C,kBAAoB,SAASnpC,GAC7C,GAAI+2B,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIz9C,KAAMy9C,GACTA,EAAMh4C,eAAezF,IACvBy9C,EAAMz9C,GAAI6vD,kBAAkBnpC,IASlC7jB,EAAQiQ,UAAU00C,WAAa,WACgB,GAAzC7nD,KAAK0hD,UAAUb,wBACjB7gD,KAAKmwD,qBAKP,KADA,GAAIn5C,GAAQ,EACLhX,KAAK8kD,QAAU9tC,EAAQhX,KAAK0hD,UAAUN,yBAC3CphD,KAAKowD,eACLp5C,GAG0C,IAAxChX,KAAK0hD,UAAUL,uBACjBrhD,KAAKilD,WAAWz+C,QAAW,GAAO,GAGS,GAAzCxG,KAAK0hD,UAAUb,wBACjB7gD,KAAKqwD,uBAUTntD,EAAQiQ,UAAUg9C,oBAAsB,WACtC,GAAInT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACTA,EAAMl3C,eAAezF,IACJ,MAAf28C,EAAM38C,GAAI0R,GAA4B,MAAfirC,EAAM38C,GAAI2R,IACnCgrC,EAAM38C,GAAIiwD,UAAUv+C,EAAIirC,EAAM38C,GAAI8qD,OAClCnO,EAAM38C,GAAIiwD,UAAUt+C,EAAIgrC,EAAM38C,GAAI+qD,OAClCpO,EAAM38C,GAAI8qD,QAAS,EACnBnO,EAAM38C,GAAI+qD,QAAS,IAW3BloD,EAAQiQ,UAAUk9C,oBAAsB,WACtC,GAAIrT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACTA,EAAMl3C,eAAezF,IACM,MAAzB28C,EAAM38C,GAAIiwD,UAAUv+C,IACtBirC,EAAM38C,GAAI8qD,OAASnO,EAAM38C,GAAIiwD,UAAUv+C,EACvCirC,EAAM38C,GAAI+qD,OAASpO,EAAM38C,GAAIiwD,UAAUt+C,IAa/C9O,EAAQiQ,UAAUo9C,UAAY,SAASC,GACrC,GAAIxT,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAI38C,KAAM28C,GACb,GAAIA,EAAMl3C,eAAezF,IAAO28C,EAAM38C,GAAIowD,SAASD,GACjD,OAAO,CAGX,QAAO,GAUTttD,EAAQiQ,UAAUu9C,mBAAqB,WACrC,GAEIvK,GAFA3zB,EAAWxyB,KAAKy8C,wBAChBO,EAAQh9C,KAAKg9C,MAEb2T,GAAe,CAEnB,IAAI3wD,KAAK0hD,UAAUT,YAAc,EAC/B,IAAKkF,IAAUnJ,GACTA,EAAMl3C,eAAeqgD,KACvBnJ,EAAMmJ,GAAQyK,oBAAoBp+B,EAAUxyB,KAAK0hD,UAAUT,aAC3D0P,GAAe,OAKnB,KAAKxK,IAAUnJ,GACTA,EAAMl3C,eAAeqgD,KACvBnJ,EAAMmJ,GAAQ0K,aAAar+B,GAC3Bm+B,GAAe,EAKrB,IAAoB,GAAhBA,EAAsB,CACxB,GAAIG,GAAgB9wD,KAAK0hD,UAAUR,YAAch8C,KAAKwH,IAAI1M,KAAKid,MAAM,IACrE,OAAI6zC,GAAgB,GAAI9wD,KAAK0hD,UAAUT,aAC9B,EAGAjhD,KAAKuwD,UAAUO,GAG1B,OAAO,GAIT5tD,EAAQiQ,UAAU49C,oBAAsB,WACtC,GAAI/T,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAImJ,KAAUnJ,GACbA,EAAMl3C,eAAeqgD,IACvBnJ,EAAMmJ,GAAQ6K,kBAKpB9tD,EAAQiQ,UAAU89C,mBAAqB,WACrCjxD,KAAKkxD,sBAAsB,uBACgB,GAAvClxD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,SAC7E/gD,KAAKmxD,mBAAmB,wBAS5BjuD,EAAQiQ,UAAUi9C,aAAe,WAC/B,IAAKpwD,KAAKujD,kBACW,GAAfvjD,KAAK8kD,OAAgB,CACvB,GAAIsM,IAAmB,EACnBC,GAAsB,CAE1BrxD,MAAKkxD,sBAAsB,8BAC3B,IAAII,GAAatxD,KAAKkxD,sBAAsB,qBACD,IAAvClxD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,UAC7EsQ,EAAsBrxD,KAAKmxD,mBAAmB,sBAIhD,KAAK,GAAI3rD,GAAI,EAAGA,EAAI8rD,EAAW3rD,OAAQH,IAAM4rD,EAAmBE,EAAW,IAAMF,CAGjFpxD,MAAK8kD,OAASsM,GAAoBC,EAEf,GAAfrxD,KAAK8kD,OACP9kD,KAAKixD,qBAI4B,GAA7BjxD,KAAKyjD,uBACPzjD,KAAK4tB,KAAK,sBACV5tB,KAAKyjD,sBAAuB,GAIhCzjD,KAAKohD,4BAYXl+C,EAAQiQ,UAAUo+C,eAAiB,WAEjCvxD,KAAK+kD,MAAQv+C,OAGbxG,KAAKwxD,mBAEL,IAAIC,GAAYntD,KAAK64B,KACrBn9B,MAAKowD,cACL,IAAI7T,GAAcj4C,KAAK64B,MAAQs0B,GAG1BzxD,KAAKq8C,eAAiBr8C,KAAKs8C,WAAa,EAAIC,GAAsC,GAAvBv8C,KAAKw8C,iBAA2C,GAAfx8C,KAAK8kD,SACpG9kD,KAAKowD,eAGkB,GAAnBpwD,KAAKs8C,aACPt8C,KAAKw8C,gBAAiB,GAI1B,IAAIkV,GAAkBptD,KAAK64B,KAC3Bn9B,MAAK6iD,UACL7iD,KAAKs8C,WAAah4C,KAAK64B,MAAQu0B,EAG/B1xD,KAAK4P,SAGe,mBAAXrG,UACTA,OAAOooD,sBAAwBpoD,OAAOooD,uBAAyBpoD,OAAOqoD,0BACvCroD,OAAOsoD,6BAA+BtoD,OAAOuoD,yBAM9E5uD,EAAQiQ,UAAUvD,MAAQ,WACxB,GAAmB,GAAf5P,KAAK8kD,QAAqC,GAAnB9kD,KAAK8iD,YAAsC,GAAnB9iD,KAAK+iD,YAAyC,GAAtB/iD,KAAKgjD,cACzEhjD,KAAK+kD,QAEN/kD,KAAK+kD,MADqB,GAAxB/kD,KAAKwlD,gBACMj8C,OAAO+P,WAAWtZ,KAAKuxD,eAAez8B,KAAK90B,MAAOA,KAAKq8C,gBAGvD9yC,OAAOooD,sBAAsB3xD,KAAKuxD,eAAez8B,KAAK90B,YAOvE,IAFAA,KAAK6iD,UAED7iD,KAAKohD,wBAA0B,EAAG,CAKpC,GAAIjtC,GAAKnU,KACL8T,GACFi+C,WAAY59C,EAAGitC,wBAEjBphD,MAAKohD,wBAA0B,EAC/BphD,KAAKyjD,sBAAuB,EAC5BnqC,WAAW,WACTnF,EAAGyZ,KAAK,aAAc9Z,IACrB,OAGH9T,MAAKohD,wBAA0B,GAWrCl+C,EAAQiQ,UAAUq+C,kBAAoB,WACpC,GAAuB,GAAnBxxD,KAAK8iD,YAAsC,GAAnB9iD,KAAK+iD,WAAiB,CAChD,GAAItlC,GAAczd,KAAK6qD,iBACvB7qD,MAAKsjD,gBAAgB7lC,EAAY1L,EAAE/R,KAAK8iD,WAAYrlC,EAAYzL,EAAEhS,KAAK+iD,YAEzE,GAA0B,GAAtB/iD,KAAKgjD,cAAoB,CAC3B,GAAI92B,IACFna,EAAG/R,KAAKsf,MAAMC,OAAOC,YAAc,EACnCxN,EAAGhS,KAAKsf,MAAMC,OAAOsF,aAAe,EAEtC7kB,MAAKisD,MAAMjsD,KAAKid,OAAO,EAAIjd,KAAKgjD,eAAgB92B,KAQpDhpB,EAAQiQ,UAAU6+C,aAAe,WACF,GAAzBhyD,KAAKujD,iBACPvjD,KAAKujD,kBAAmB,GAGxBvjD,KAAKujD,kBAAmB,EACxBvjD,KAAK4P,UAWT1M,EAAQiQ,UAAUu1C,uBAAyB,SAASlC,GAIlD,GAHqBhgD,SAAjBggD,IACFA,GAAe,GAE0B,GAAvCxmD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAAiB,CAC9F/gD,KAAKgvD,oBAEL,KAAK,GAAI7I,KAAUnmD,MAAKkvD,QAAiB,QAAS,MAC5ClvD,KAAKkvD,QAAiB,QAAS,MAAEppD,eAAeqgD,IACwB3/C,SAAtExG,KAAK89C,MAAM99C,KAAKkvD,QAAiB,QAAS,MAAE/I,GAAQ8L,qBAC/CjyD,MAAKkvD,QAAiB,QAAS,MAAE/I,OAK3C,CAEHnmD,KAAKkvD,QAAiB,QAAS,QAC/B,KAAK,GAAIlC,KAAUhtD,MAAK89C,MAClB99C,KAAK89C,MAAMh4C,eAAeknD,KAC5BhtD,KAAK89C,MAAMkP,GAAQiC,IAAM,MAM/BjvD,KAAKquD,0BACA7H,IACHxmD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAWT1M,EAAQiQ,UAAU67C,mBAAqB,WACrC,GAA2C,GAAvChvD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAC7E,IAAK,GAAIiM,KAAUhtD,MAAK89C,MACtB,GAAI99C,KAAK89C,MAAMh4C,eAAeknD,GAAS,CACrC,GAAIU,GAAO1tD,KAAK89C,MAAMkP,EACtB,IAAgB,MAAZU,EAAKuB,IAAa,CACpB,GAAI9I,GAAS,UAAUnyC,OAAO05C,EAAKrtD,GACnCL,MAAKkvD,QAAiB,QAAS,MAAE/I,GAAU,GAAI5iD,IACtClD,GAAG8lD,EACFlJ,KAAK,EACLG,MAAM,SACNC,MAAM,GACN6U,mBAAmB,SACblyD,KAAK0hD,WACrBgM,EAAKuB,IAAMjvD,KAAKkvD,QAAiB,QAAS,MAAE/I,GAC5CuH,EAAKuB,IAAIgD,aAAevE,EAAKrtD,GAC7BqtD,EAAKyE,wBAYfjvD,EAAQiQ,UAAUgpC,wBAA0B,WAC1C,IAAK,GAAIiW,KAAS/M,GACZA,EAAYv/C,eAAessD,KAC7BlvD,EAAQiQ,UAAUi/C,GAAS/M,EAAY+M,KAQ7ClvD,EAAQiQ,UAAUk/C,cAAgB,WAChC15B,QAAQhF,IAAI,mEACZ3zB,KAAKsyD,kBAMPpvD,EAAQiQ,UAAUm/C,eAAiB,WACjC,GAAIC,KACJ,KAAK,GAAIpM,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,GAClBqM,GAAkBxyD,KAAKg9C,MAAMmO,OAC7BsH,GAAkBzyD,KAAKg9C,MAAMoO,QAC7BprD,KAAKokD,UAAUxxC,MAAMuzC,GAAQp0C,GAAK7M,KAAKwoB,MAAMo4B,EAAK/zC,IAAM/R,KAAKokD,UAAUxxC,MAAMuzC,GAAQn0C,GAAK9M,KAAKwoB,MAAMo4B,EAAK9zC,KAC5GugD,EAAUvqD,MAAM3H,GAAG8lD,EAAOp0C,EAAE7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAGC,EAAE9M,KAAKwoB,MAAMo4B,EAAK9zC,GAAGwgD,eAAeA,EAAeC,eAAeA,IAIvHzyD,KAAKokD,UAAUvvC,OAAO09C,IAMxBrvD,EAAQiQ,UAAUu/C,aAAe,SAASv9C,GACxC,GAAIo9C,KACJ,IAAY/rD,SAAR2O,GACF,GAA0B,GAAtBlP,MAAMC,QAAQiP,IAChB,IAAK,GAAI3P,GAAI,EAAGA,EAAI2P,EAAIxP,OAAQH,IAC9B,GAA2BgB,SAAvBxG,KAAKg9C,MAAM7nC,EAAI3P,IAAmB,CACpC,GAAIsgD,GAAO9lD,KAAKg9C,MAAM7nC,EAAI3P,GAC1B+sD,GAAUp9C,EAAI3P,KAAOuM,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,SAKnE,IAAwBxL,SAApBxG,KAAKg9C,MAAM7nC,GAAoB,CACjC,GAAI2wC,GAAO9lD,KAAKg9C,MAAM7nC,EACtBo9C,GAAUp9C,IAAQpD,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,SAKhE,KAAK,GAAIm0C,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACtBoM,GAAUpM,IAAWp0C,EAAG7M,KAAKwoB,MAAMo4B,EAAK/zC,GAAIC,EAAG9M,KAAKwoB,MAAMo4B,EAAK9zC,IAIrE,MAAOugD,IAWTrvD,EAAQiQ,UAAUw/C,YAAc,SAAUxM,EAAQ13C,GAChD,GAAIzO,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrB3/C,SAAZiI,IACFA,KAEF,IAAImkD,IAAgB7gD,EAAG/R,KAAKg9C,MAAMmJ,GAAQp0C,EAAGC,EAAGhS,KAAKg9C,MAAMmJ,GAAQn0C,EACnEvD,GAAQmV,SAAWgvC,EACnBnkD,EAAQokD,aAAe1M,EAEvBnmD,KAAK6nB,OAAOpZ,OAGZkqB,SAAQhF,IAAI,iCAWhBzwB,EAAQiQ,UAAU0U,OAAS,SAAUpZ,GACnC,MAAgBjI,UAAZiI,OACFA,OAGwBjI,SAAtBiI,EAAQkb,SAAoClb,EAAQkb,QAAa5X,EAAG,EAAGC,EAAG,IACpDxL,SAAtBiI,EAAQkb,OAAO5X,IAA6BtD,EAAQkb,OAAO5X,EAAK,GAC1CvL,SAAtBiI,EAAQkb,OAAO3X,IAA6BvD,EAAQkb,OAAO3X,EAAK,GAC1CxL,SAAtBiI,EAAQwO,QAAoCxO,EAAQwO,MAAYjd,KAAKyqD,aAC/CjkD,SAAtBiI,EAAQmV,WAAoCnV,EAAQmV,SAAY5jB,KAAK6qD,mBAC/CrkD,SAAtBiI,EAAQs4C,YAAoCt4C,EAAQs4C,WAAaj3C,SAAS,IAC1ErB,EAAQs4C,aAAc,IAAsBt4C,EAAQs4C,WAAaj3C,SAAS,IAC1ErB,EAAQs4C,aAAc,IAAsBt4C,EAAQs4C,cACrBvgD,SAA/BiI,EAAQs4C,UAAUj3C,WAA0BrB,EAAQs4C,UAAUj3C,SAAW,KACpCtJ,SAArCiI,EAAQs4C,UAAU+L,iBAAgCrkD,EAAQs4C,UAAU+L,eAAiB,qBAEzF9yD,MAAK+yD,YAAYtkD,KAcnBvL,EAAQiQ,UAAU4/C,YAAc,SAAUtkD,GACxC,GAAgBjI,SAAZiI,EAEF,YADAA,KAKFzO,MAAKsrD,cACiB,GAAlB78C,EAAQukD,SACVhzD,KAAKwiD,eAAiB/zC,EAAQokD,aAC9B7yD,KAAKyiD,mBAAqBh0C,EAAQkb,QAIb,GAAnB3pB,KAAKmiD,YACPniD,KAAKizD,kBAAkB,GAGzBjzD,KAAKoiD,YAAcpiD,KAAKyqD,YACxBzqD,KAAKsiD,kBAAoBtiD,KAAK6qD,kBAC9B7qD,KAAKqiD,YAAc5zC,EAAQwO,MAI3Bjd,KAAKgd,UAAUhd,KAAKqiD,YACpB,IAAI6Q,GAAalzD,KAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,eAClGsuC,GACFphD,EAAGmhD,EAAWnhD,EAAItD,EAAQmV,SAAS7R,EACnCC,EAAGkhD,EAAWlhD,EAAIvD,EAAQmV,SAAS5R,EAErChS,MAAKuiD,mBACHxwC,EAAG/R,KAAKsiD,kBAAkBvwC,EAAIohD,EAAmBphD,EAAI/R,KAAKqiD,YAAc5zC,EAAQkb,OAAO5X,EACvFC,EAAGhS,KAAKsiD,kBAAkBtwC,EAAImhD,EAAmBnhD,EAAIhS,KAAKqiD,YAAc5zC,EAAQkb,OAAO3X,GAIvD,GAA9BvD,EAAQs4C,UAAUj3C,SACO,MAAvB9P,KAAKwiD,gBACPxiD,KAAKozD,eAAiBpzD,KAAK6iD,QAC3B7iD,KAAK6iD,QAAU7iD,KAAKqzD,gBAGpBrzD,KAAKgd,UAAUhd,KAAKqiD,aACpBriD,KAAKsjD,gBAAgBtjD,KAAKuiD,kBAAkBxwC,EAAG/R,KAAKuiD,kBAAkBvwC,GACtEhS,KAAK6iD,YAIP7iD,KAAKiiD,eAAiB,GAAKjiD,KAAKo8C,kBAAoB3tC,EAAQs4C,UAAUj3C,SAAW,OAAU,EAAI9P,KAAKo8C,kBACpGp8C,KAAKkiD,wBAA0BzzC,EAAQs4C,UAAU+L,eACjD9yD,KAAKozD,eAAiBpzD,KAAK6iD,QAC3B7iD,KAAK6iD,QAAU7iD,KAAKizD,kBACpBjzD,KAAK6iD,UACL7iD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAQT1M,EAAQiQ,UAAUkgD,cAAgB,WAChC,GAAIT,IAAgB7gD,EAAG/R,KAAKg9C,MAAMh9C,KAAKwiD,gBAAgBzwC,EAAGC,EAAGhS,KAAKg9C,MAAMh9C,KAAKwiD,gBAAgBxwC,GACzFkhD,EAAalzD,KAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,eAClGsuC,GACFphD,EAAGmhD,EAAWnhD,EAAI6gD,EAAa7gD,EAC/BC,EAAGkhD,EAAWlhD,EAAI4gD,EAAa5gD,GAE7BswC,EAAoBtiD,KAAK6qD,kBACzBtI,GACFxwC,EAAGuwC,EAAkBvwC,EAAIohD,EAAmBphD,EAAI/R,KAAKid,MAAQjd,KAAKyiD,mBAAmB1wC,EACrFC,EAAGswC,EAAkBtwC,EAAImhD,EAAmBnhD,EAAIhS,KAAKid,MAAQjd,KAAKyiD,mBAAmBzwC,EAGvFhS,MAAKsjD,gBAAgBf,EAAkBxwC,EAAEwwC,EAAkBvwC,GAC3DhS,KAAKozD,kBAGPlwD,EAAQiQ,UAAUm4C,YAAc,WACH,MAAvBtrD,KAAKwiD,iBACPxiD,KAAK6iD,QAAU7iD,KAAKozD,eACpBpzD,KAAKwiD,eAAiB,KACtBxiD,KAAKyiD,mBAAqB,OAS9Bv/C,EAAQiQ,UAAU8/C,kBAAoB,SAAU9Q,GAC9CniD,KAAKmiD,WAAaA,GAAcniD,KAAKmiD,WAAaniD,KAAKiiD,eACvDjiD,KAAKmiD,YAAcniD,KAAKiiD,cAExB,IAAIxwB,GAAW9wB,EAAKqP,gBAAgBhQ,KAAKkiD,yBAAyBliD,KAAKmiD,WAEvEniD,MAAKgd,UAAUhd,KAAKoiD,aAAepiD,KAAKqiD,YAAcriD,KAAKoiD,aAAe3wB,GAC1EzxB,KAAKsjD,gBACHtjD,KAAKsiD,kBAAkBvwC,GAAK/R,KAAKuiD,kBAAkBxwC,EAAI/R,KAAKsiD,kBAAkBvwC,GAAK0f,EACnFzxB,KAAKsiD,kBAAkBtwC,GAAKhS,KAAKuiD,kBAAkBvwC,EAAIhS,KAAKsiD,kBAAkBtwC,GAAKyf,GAGrFzxB,KAAKozD,iBACLpzD,KAAK8kD,QAAS,EAGV9kD,KAAKmiD,YAAc,IACrBniD,KAAKmiD,WAAa,EAEhBniD,KAAK6iD,QADoB,MAAvB7iD,KAAKwiD,eACQxiD,KAAKqzD,cAGLrzD,KAAKozD,eAEtBpzD,KAAK4tB,KAAK,uBAId1qB,EAAQiQ,UAAUigD,eAAiB,aAQnClwD,EAAQiQ,UAAUs2C,SAAW,WAC3B,OAAQzpD,KAAKsoD,WAAatoD,KAAKsoD,UAAUgL,QAQ3CpwD,EAAQiQ,UAAU6vB,SAAW,WAC3B,MAAOhjC,MAAKgd,aAQd9Z,EAAQiQ,UAAUogD,SAAW,WAC3B,MAAOvzD,MAAKyqD,aAQdvnD,EAAQiQ,UAAUqgD,qBAAuB,WACvC,MAAOxzD,MAAKosD,aAAar6C,EAAG,GAAM/R,KAAKsf,MAAMC,OAAOC,YAAaxN,EAAG,GAAMhS,KAAKsf,MAAMC,OAAOsF,gBAI9F3hB,EAAQiQ,UAAUsgD,eAAiB,SAAStN,GAC1C,MAA2B3/C,UAAvBxG,KAAKg9C,MAAMmJ,GACNnmD,KAAKg9C,MAAMmJ,GAAQC,YAD5B,QAKFvmD,EAAOD,QAAUsD,GAKb,SAASrD,EAAQD,EAASM,GAoB9B,QAASkD,GAAMurD,EAAYxrD,EAASuwD,GAClC,IAAKvwD,EACH,KAAM,qBAER,IAAI+K,IAAU,QAAQ,WAClBwzC,EAAY/gD,EAAKsN,sBAAsBC,EAAOwlD,EAClD1zD,MAAKyO,QAAUizC,EAAU5D,MACzB99C,KAAKw+C,QAAUkD,EAAUlD,QACzBx+C,KAAKyO,QAAsB,aAAIilD,EAA+B,aAG9D1zD,KAAKmD,QAAUA,EAGfnD,KAAKK,GAASmG,OACdxG,KAAK2zD,OAASntD,OACdxG,KAAK4zD,KAASptD,OACdxG,KAAKglC,MAASx+B,OACdxG,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAQvS,KAAKyO,QAAQsvC,yBACvD/9C,KAAKqH,MAASb,OACdxG,KAAKqzC,UAAW,EAChBrzC,KAAKsM,OAAQ,EACbtM,KAAK8zD,iBAAmBnsD,IAAI,EAAEF,KAAK,EAAE8K,MAAM,EAAEC,OAAO,EAAEuhD,MAAM,GAC5D/zD,KAAKg0D,YAAa,EAElBh0D,KAAKopB,KAAO,KACZppB,KAAKqpB,GAAK,KACVrpB,KAAKivD,IAAM,KAEXjvD,KAAKi0D,WAAa,KAClBj0D,KAAKk0D,SAAW,KAIhBl0D,KAAKm0D,kBACLn0D,KAAKo0D,gBAELp0D,KAAK2tD,WAAY,EAEjB3tD,KAAKq0D,YAAc,EACnBr0D,KAAKs0D,aAAc,EAEnBt0D,KAAK0uD,cAAcC,GAEnB3uD,KAAKu0D,qBAAsB,EAC3Bv0D,KAAKw0D,cAAgBprC,KAAK,KAAMC,GAAG,KAAMorC,cACzCz0D,KAAK00D,cAAgB,KAhEvB,GAAI/zD,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,GAuE/BkD,GAAK+P,UAAUu7C,cAAgB,SAASC,GACtC,GAAKA,EAAL,CAIA,GAAIzgD,IAAU,QAAQ,WAAW,WAAW,YAAY,WAAW,kBAAkB,kBAAkB,QACrG,2BAA2B,aAAa,mBAAmB,OAAO,eAAe,iBAoCnF,QAlCAvN,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASkgD,GAEvBnoD,SAApBmoD,EAAWvlC,OAA+BppB,KAAK2zD,OAAShF,EAAWvlC,MACjD5iB,SAAlBmoD,EAAWtlC,KAA+BrpB,KAAK4zD,KAAOjF,EAAWtlC,IAE/C7iB,SAAlBmoD,EAAWtuD,KAA+BL,KAAKK,GAAKsuD,EAAWtuD,IAC1CmG,SAArBmoD,EAAWlmC,QAA+BzoB,KAAKyoB,MAAQkmC,EAAWlmC,MAAOzoB,KAAKg0D,YAAa,GAEtExtD,SAArBmoD,EAAW3pB,QAA6BhlC,KAAKglC,MAAQ2pB,EAAW3pB,OAC3Cx+B,SAArBmoD,EAAWtnD,QAA6BrH,KAAKqH,MAAQsnD,EAAWtnD,OAC1Cb,SAAtBmoD,EAAWhpD,SAA6B3F,KAAKw+C,QAAQK,aAAe8P,EAAWhpD,QAE1Da,SAArBmoD,EAAWxjD,QACbnL,KAAKyO,QAAQ6vC,cAAe,EACxB39C,EAAKwD,SAASwqD,EAAWxjD,QAC3BnL,KAAKyO,QAAQtD,MAAMA,MAAQwjD,EAAWxjD,MACtCnL,KAAKyO,QAAQtD,MAAMkB,UAAYsiD,EAAWxjD,QAGX3E,SAA3BmoD,EAAWxjD,MAAMA,QAA0BnL,KAAKyO,QAAQtD,MAAMA,MAAQwjD,EAAWxjD,MAAMA,OACxD3E,SAA/BmoD,EAAWxjD,MAAMkB,YAA0BrM,KAAKyO,QAAQtD,MAAMkB,UAAYsiD,EAAWxjD,MAAMkB,WAChE7F,SAA3BmoD,EAAWxjD,MAAMmB,QAA0BtM,KAAKyO,QAAQtD,MAAMmB,MAAQqiD,EAAWxjD,MAAMmB,SAK/FtM,KAAK88C,UAEL98C,KAAKq0D,WAAar0D,KAAKq0D,YAAoC7tD,SAArBmoD,EAAWp8C,MACjDvS,KAAKs0D,YAAct0D,KAAKs0D,aAAsC9tD,SAAtBmoD,EAAWhpD,OAEnD3F,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAOvS,KAAKyO,QAAQsvC,yBAG9C/9C,KAAKyO,QAAQxB,OACnB,IAAK,OAAiBjN,KAAK+rC,KAAO/rC,KAAK20D,SAAW,MAClD,KAAK,QAAiB30D,KAAK+rC,KAAO/rC,KAAK40D,UAAY,MACnD,KAAK,eAAiB50D,KAAK+rC,KAAO/rC,KAAK60D,gBAAkB,MACzD,KAAK,YAAiB70D,KAAK+rC,KAAO/rC,KAAK80D,aAAe,MACtD,SAAsB90D,KAAK+rC,KAAO/rC,KAAK20D,aAQ3CvxD,EAAK+P,UAAU2pC,QAAU,WACvB98C,KAAK8uD,aAEL9uD,KAAKopB,KAAOppB,KAAKmD,QAAQ65C,MAAMh9C,KAAK2zD,SAAW,KAC/C3zD,KAAKqpB,GAAKrpB,KAAKmD,QAAQ65C,MAAMh9C,KAAK4zD,OAAS,KAC3C5zD,KAAK2tD,UAAa3tD,KAAKopB,MAAQppB,KAAKqpB,GAEhCrpB,KAAK2tD,WACP3tD,KAAKopB,KAAK2rC,WAAW/0D,MACrBA,KAAKqpB,GAAG0rC,WAAW/0D,QAGfA,KAAKopB,MACPppB,KAAKopB,KAAK4rC,WAAWh1D,MAEnBA,KAAKqpB,IACPrpB,KAAKqpB,GAAG2rC,WAAWh1D,QAQzBoD,EAAK+P,UAAU27C,WAAa,WACtB9uD,KAAKopB,OACPppB,KAAKopB,KAAK4rC,WAAWh1D,MACrBA,KAAKopB,KAAO,MAEVppB,KAAKqpB,KACPrpB,KAAKqpB,GAAG2rC,WAAWh1D,MACnBA,KAAKqpB,GAAK,MAGZrpB,KAAK2tD,WAAY,GAQnBvqD,EAAK+P,UAAUq6C,SAAW,WACxB,MAA6B,kBAAfxtD,MAAKglC,MAAuBhlC,KAAKglC,QAAUhlC,KAAKglC;EAQhE5hC,EAAK+P,UAAUyB,SAAW,WACxB,MAAO5U,MAAKqH,OASdjE,EAAK+P,UAAUi8C,cAAgB,SAAStjD,EAAKY,GAC3C,IAAK1M,KAAKq0D,YAA6B7tD,SAAfxG,KAAKqH,MAAqB,CAChD,GAAI4V,IAASjd,KAAKyO,QAAQ0Y,SAAWnnB,KAAKyO,QAAQyY,WAAaxa,EAAMZ,EACrE9L,MAAKyO,QAAQ8D,OAAQvS,KAAKqH,MAAQyE,GAAOmR,EAAQjd,KAAKyO,QAAQyY,SAC9DlnB,KAAK6zD,cAAgB7zD,KAAKyO,QAAQ8D,MAAOvS,KAAKyO,QAAQsvC,2BAU1D36C,EAAK+P,UAAU44B,KAAO,WACpB,KAAM,uCAQR3oC,EAAK+P,UAAUo6C,kBAAoB,SAASxqC,GAC1C,GAAI/iB,KAAK2tD,UAAW,CAClB,GAAIv+B,GAAU,GACV6lC,EAAQj1D,KAAKopB,KAAKrX,EAClBmjD,EAAQl1D,KAAKopB,KAAKpX,EAClBmjD,EAAMn1D,KAAKqpB,GAAGtX,EACdqjD,EAAMp1D,KAAKqpB,GAAGrX,EACdqjD,EAAOtyC,EAAItb,KACX6tD,EAAOvyC,EAAIpb,IAEXujB,EAAOlrB,KAAKu1D,mBAAmBN,EAAOC,EAAOC,EAAKC,EAAKC,EAAMC,EAEjE,OAAelmC,GAAPlE,EAGR,OAAO,GAIX9nB,EAAK+P,UAAUqiD,UAAY,WACzB,GAAIC,GAAWz1D,KAAKyO,QAAQtD,KAgB5B,OAfiC,MAA7BnL,KAAKyO,QAAQ6vC,aACfmX,GACEppD,UAAWrM,KAAKqpB,GAAG5a,QAAQtD,MAAMkB,UAAUD,OAC3CE,MAAOtM,KAAKqpB,GAAG5a,QAAQtD,MAAMmB,MAAMF,OACnCjB,MAAOnL,KAAKqpB,GAAG5a,QAAQtD,MAAMiB,SAGK,QAA7BpM,KAAKyO,QAAQ6vC,cAAuD,GAA7Bt+C,KAAKyO,QAAQ6vC,gBAC3DmX,GACEppD,UAAWrM,KAAKopB,KAAK3a,QAAQtD,MAAMkB,UAAUD,OAC7CE,MAAOtM,KAAKopB,KAAK3a,QAAQtD,MAAMmB,MAAMF,OACrCjB,MAAOnL,KAAKopB,KAAK3a,QAAQtD,MAAMiB,SAId,GAAjBpM,KAAKqzC,SAA4BoiB,EAASppD,UACvB,GAAdrM,KAAKsM,MAAuBmpD,EAASnpD,MACTmpD,EAAStqD,OAWhD/H,EAAK+P,UAAUwhD,UAAY,SAAS5tC,GAKlC,GAHAA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIO,UAActnB,KAAK01D,gBAEnB11D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAExB,GAGInX,GAHA+8C,EAAMjvD,KAAK21D,MAAM5uC,EAIrB,IAAI/mB,KAAKyoB,MAAO,CACd,GAAyC,GAArCzoB,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAE5B91D,MAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,QAG3C,CACH,GAAID,GAAGC,EACHyZ,EAASzrB,KAAKw+C,QAAQK,aAAe,EACrCiH,EAAO9lD,KAAKopB,IACX08B,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAI+zC,EAAKvzC,MAAQ,EAC1BP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAI8zC,EAAKtzC,OAAS,GAE7BxS,KAAKi2D,QAAQlvC,EAAKhV,EAAGC,EAAGyZ,GACxBvZ,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,KAUhD5O,EAAK+P,UAAUuiD,cAAgB,WAC7B,MAAqB,IAAjB11D,KAAKqzC,SACCnuC,KAAKwH,IAAIxH,KAAK4G,IAAI9L,KAAK6zD,cAAe7zD,KAAKyO,QAAQ0Y,UAAW,GAAInnB,KAAKm2D,iBAG7D,GAAdn2D,KAAKsM,MACApH,KAAKwH,IAAIxH,KAAK4G,IAAI9L,KAAKyO,QAAQuvC,WAAYh+C,KAAKyO,QAAQ0Y,UAAW,GAAInnB,KAAKm2D,iBAG5EjxD,KAAKwH,IAAI1M,KAAKyO,QAAQ8D,MAAO,GAAIvS,KAAKm2D,kBAKnD/yD,EAAK+P,UAAUijD,mBAAqB,WAClC,GAAyC,GAArCp2D,KAAKyO,QAAQqyC,aAAaC,SAAwD,GAArC/gD,KAAKyO,QAAQqyC,aAAapyC,QACzE,MAAO1O,MAAKivD,GAET,IAAyC,GAArCjvD,KAAKyO,QAAQqyC,aAAapyC,QACjC,OAAQqD,EAAE,EAAEC,EAAE,EAGd,IAAIqkD,GAAO,KACPC,EAAO,KACP3P,EAAS3mD,KAAKyO,QAAQqyC,aAAaE,UACnCl6C,EAAO9G,KAAKyO,QAAQqyC,aAAah6C,KAEjC8X,EAAK1Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACpC8M,EAAK3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EA2JxC,OA1JY,YAARlL,GAA8B,iBAARA,EACpB5B,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACjEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAEvB7e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAGzB7e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,GAEvB7e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,IAGtB,YAAR/X,IACFuvD,EAAY1P,EAAS9nC,EAAdD,EAAmB5e,KAAKopB,KAAKrX,EAAIskD,IAGnCnxD,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KACtEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAEvB5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAGzB5e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GACxBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,GAEvB5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAC7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,IAGtB,YAAR9X,IACFwvD,EAAY3P,EAAS/nC,EAAdC,EAAmB7e,KAAKopB,KAAKpX,EAAIskD,IAI7B,iBAARxvD,EACH5B,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACrEqkD,EAAOr2D,KAAKopB,KAAKrX,EAEfukD,EADEt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACjBhS,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,EAG3B7e,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,GAG7B3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KAExEqkD,EADEr2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EACjB/R,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAG3B5e,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAEpC03C,EAAOt2D,KAAKopB,KAAKpX,GAGJ,cAARlL,GAELuvD,EADEr2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EACjB/R,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAG3B5e,KAAKqpB,GAAGtX,GAAK,EAAI40C,GAAU/nC,EAEpC03C,EAAOt2D,KAAKopB,KAAKpX,GAEF,YAARlL,GACPuvD,EAAOr2D,KAAKopB,KAAKrX,EAEfukD,EADEt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACjBhS,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,EAG3B7e,KAAKqpB,GAAGrX,GAAK,EAAI20C,GAAU9nC,GAIhC3Z,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,GACjEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAE/Br2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAGjCr2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,GAE/Br2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS9nC,EAC9By3C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS9nC,EAC9Bw3C,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,EAAOr2D,KAAKqpB,GAAGtX,EAAIskD,IAInCnxD,KAAK2lB,IAAI7qB,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAAK7M,KAAK2lB,IAAI7qB,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KACtEhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EACpBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAE/Bt2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAGjCt2D,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,IACzBhS,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAExBskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,GAE/Bt2D,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,IAE7BskD,EAAOr2D,KAAKopB,KAAKrX,EAAI40C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKopB,KAAKpX,EAAI20C,EAAS/nC,EAC9B03C,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,EAAOt2D,KAAKqpB,GAAGrX,EAAIskD,MAOtCvkD,EAAGskD,EAAMrkD,EAAGskD,IASxBlzD,EAAK+P,UAAUwiD,MAAQ,SAAU5uC,GAI/B,GAFAA,EAAIa,YACJb,EAAIc,OAAO7nB,KAAKopB,KAAKrX,EAAG/R,KAAKopB,KAAKpX,GACO,GAArChS,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAyC,GAArC1O,KAAKyO,QAAQqyC,aAAaC,QAAkB,CAC9C,GAAIkO,GAAMjvD,KAAKo2D,oBACf,OAAa,OAATnH,EAAIl9C,GACNgV,EAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9B+U,EAAIlH,SACG,OAKPkH,EAAIwvC,iBAAiBtH,EAAIl9C,EAAEk9C,EAAIj9C,EAAEhS,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GACpD+U,EAAIlH,SACGovC,GAMT,MAFAloC,GAAIwvC,iBAAiBv2D,KAAKivD,IAAIl9C,EAAE/R,KAAKivD,IAAIj9C,EAAEhS,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9D+U,EAAIlH,SACG7f,KAAKivD,IAMd,MAFAloC,GAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,GAC9B+U,EAAIlH,SACG,MAYXzc,EAAK+P,UAAU8iD,QAAU,SAAUlvC,EAAKhV,EAAGC,EAAGyZ,GAE5C1E,EAAIa,YACJb,EAAI2E,IAAI3Z,EAAGC,EAAGyZ,EAAQ,EAAG,EAAIvmB,KAAKymB,IAAI,GACtC5E,EAAIlH,UAWNzc,EAAK+P,UAAU4iD,OAAS,SAAUhvC,EAAKwC,EAAMxX,EAAGC,GAC9C,GAAIuX,EAAM,CACRxC,EAAIQ,MAASvnB,KAAKopB,KAAKiqB,UAAYrzC,KAAKqpB,GAAGgqB,SAAY,QAAU,IACjErzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAC7C,IAAIuW,EAEJ,IAAuB,GAAnB/zD,KAAKg0D,WAAoB,CAC3B,GAAIxtB,GAAQpiC,OAAOmlB,GAAMxhB,MAAM,MAC3ByuD,EAAYhwB,EAAM7gC,OAClB43C,EAAWr5C,OAAOlE,KAAKyO,QAAQ8uC,SACnCwW,GAAQ/hD,GAAK,EAAIwkD,GAAa,EAAIjZ,CAGlC,KAAK,GADDhrC,GAAQwU,EAAI0vC,YAAYjwB,EAAM,IAAIj0B,MAC7B/M,EAAI,EAAOgxD,EAAJhxD,EAAeA,IAAK,CAClC,GAAI8hB,GAAYP,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,KAC1CA,GAAQ+U,EAAY/U,EAAQ+U,EAAY/U,EAE1C,GAAIC,GAASxS,KAAKyO,QAAQ8uC,SAAWiZ,EACjC/uD,EAAOsK,EAAIQ,EAAQ,EACnB5K,EAAMqK,EAAIQ,EAAS,CAGvBxS,MAAK8zD,iBAAmBnsD,IAAIA,EAAIF,KAAKA,EAAK8K,MAAMA,EAAMC,OAAOA,EAAOuhD,MAAMA,GAG/E,GAAIA,GAAQ/zD,KAAK8zD,gBAAgBC,KAEjChtC,GAAIuoC,OAE+B,cAA/BtvD,KAAKyO,QAAQwvC,iBAChBl3B,EAAIwoC,UAAUx9C,EAAGgiD,GACjB/zD,KAAK02D,yBAAyB3vC,GAC9BhV,EAAI,EACJgiD,EAAQ,GAIT/zD,KAAK22D,eAAe5vC,GACpB/mB,KAAK42D,eAAe7vC,EAAIhV,EAAEgiD,EAAOvtB,EAAOgwB,EAAWjZ,GAEnDx2B,EAAI0oC,YASLrsD,EAAK+P,UAAUujD,yBAA2B,SAAS3vC,GAClD,GAAIlI,GAAK7e,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,EAC3B4M,EAAK5e,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,EAC3B8kD,EAAiB3xD,KAAK4xD,MAAMj4C,EAAID,IAGf,GAAjBi4C,GAA4B,EAALj4C,GAAYi4C,EAAiB,GAAU,EAALj4C,KAC5Di4C,GAAkC3xD,KAAKymB,IAGxC5E,EAAIgwC,OAAOF,IASZzzD,EAAK+P,UAAUwjD,eAAiB,SAAS5vC,GACxC,GAA8BvgB,SAA1BxG,KAAKyO,QAAQgvC,UAAoD,OAA1Bz9C,KAAKyO,QAAQgvC,UAA+C,SAA1Bz9C,KAAKyO,QAAQgvC,SAAqB,CAC9G12B,EAAIiB,UAAYhoB,KAAKyO,QAAQgvC,QAE7B,IAAIuZ,GAAa,CAEoB,gBAA/Bh3D,KAAKyO,QAAQwvC,eACfl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,MAA4C,IAA9BvS,KAAK8zD,gBAAgBthD,OAAcxS,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAE/F,cAA/BxS,KAAKyO,QAAQwvC,eACpBl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,QAAevS,KAAK8zD,gBAAgBthD,OAASwkD,GAAah3D,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAExG,cAA/BxS,KAAKyO,QAAQwvC,eACpBl3B,EAAIkwC,SAAuC,IAA7Bj3D,KAAK8zD,gBAAgBvhD,MAAaykD,EAAYh3D,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,QAG7GuU,EAAIkwC,SAASj3D,KAAK8zD,gBAAgBrsD,KAAMzH,KAAK8zD,gBAAgBnsD,IAAK3H,KAAK8zD,gBAAgBvhD,MAAOvS,KAAK8zD,gBAAgBthD,UAezHpP,EAAK+P,UAAUyjD,eAAiB,SAAS7vC,EAAKhV,EAAGgiD,EAAOvtB,EAAOgwB,EAAWjZ,GAMxE,GAJDx2B,EAAIiB,UAAYhoB,KAAKyO,QAAQ6uC,WAAa,QAC1Cv2B,EAAIuB,UAAY,SAGoB,cAA/BtoB,KAAKyO,QAAQwvC,eAAgC,CAC/C,GAAI+Y,GAAa,CACkB,eAA/Bh3D,KAAKyO,QAAQwvC,gBACfl3B,EAAIwB,aAAe,aACnBwrC,GAAS,EAAIiD,GAEyB,cAA/Bh3D,KAAKyO,QAAQwvC,gBACpBl3B,EAAIwB,aAAe,UACnBwrC,GAAS,EAAIiD,GAGbjwC,EAAIwB,aAAe,aAIrBxB,GAAIwB,aAAe,QAIjBvoB,MAAKyO,QAAQivC,gBAAkB,IACjC32B,EAAIO,UAActnB,KAAKyO,QAAQivC,gBAC/B32B,EAAIY,YAAc3nB,KAAKyO,QAAQkvC,gBAC/B52B,EAAImwC,SAAc,QAErB,KAAK,GAAI1xD,GAAI,EAAOgxD,EAAJhxD,EAAeA,IACzBxF,KAAKyO,QAAQivC,gBAAkB,GAChC32B,EAAIowC,WAAW3wB,EAAMhhC,GAAIuM,EAAGgiD,GAEhChtC,EAAIyB,SAASge,EAAMhhC,GAAIuM,EAAGgiD,GAC1BA,GAASxW,GAaXn6C,EAAK+P,UAAU2hD,cAAgB,SAAS/tC,GAEtCA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIO,UAAYtnB,KAAK01D,eAErB,IAAIzG,GAAM,IAEV,IAAwBzoD,SAApBugB,EAAIqwC,YAA2B,CACjCrwC,EAAIuoC,MAEJ,IAAI+H,IAAW,EAEbA,GAD+B7wD,SAA7BxG,KAAKyO,QAAQ0vC,KAAKx4C,QAAkDa,SAA1BxG,KAAKyO,QAAQ0vC,KAAKC,KACnDp+C,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,MAG3C,EAAE,GAIfr3B,EAAIqwC,YAAYC,GAChBtwC,EAAIuwC,eAAiB,EAGrBrI,EAAMjvD,KAAK21D,MAAM5uC,GAGjBA,EAAIqwC,aAAa,IACjBrwC,EAAIuwC,eAAiB,EACrBvwC,EAAI0oC,cAIJ1oC,GAAIa,YACJb,EAAIwwC,QAAU,QACsB/wD,SAAhCxG,KAAKyO,QAAQ0vC,KAAKE,UAEpBt3B,EAAIywC,WAAWx3D,KAAKopB,KAAKrX,EAAE/R,KAAKopB,KAAKpX,EAAEhS,KAAKqpB,GAAGtX,EAAE/R,KAAKqpB,GAAGrX,GACpDhS,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,IAAIp+C,KAAKyO,QAAQ0vC,KAAKE,UAAUr+C,KAAKyO,QAAQ0vC,KAAKC,MAE9D53C,SAA7BxG,KAAKyO,QAAQ0vC,KAAKx4C,QAAkDa,SAA1BxG,KAAKyO,QAAQ0vC,KAAKC,IAEnEr3B,EAAIywC,WAAWx3D,KAAKopB,KAAKrX,EAAE/R,KAAKopB,KAAKpX,EAAEhS,KAAKqpB,GAAGtX,EAAE/R,KAAKqpB,GAAGrX,GACpDhS,KAAKyO,QAAQ0vC,KAAKx4C,OAAO3F,KAAKyO,QAAQ0vC,KAAKC,OAIhDr3B,EAAIc,OAAO7nB,KAAKopB,KAAKrX,EAAG/R,KAAKopB,KAAKpX,GAClC+U,EAAIe,OAAO9nB,KAAKqpB,GAAGtX,EAAG/R,KAAKqpB,GAAGrX,IAEhC+U,EAAIlH,QAIN,IAAI7f,KAAKyoB,MAAO,CACd,GAAIvW,EACJ,IAAyC,GAArClS,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAE5B91D,MAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,KAUhD5O,EAAK+P,UAAU2iD,aAAe,SAAU2B,GACtC,OACE1lD,GAAI,EAAI0lD,GAAcz3D,KAAKopB,KAAKrX,EAAI0lD,EAAaz3D,KAAKqpB,GAAGtX,EACzDC,GAAI,EAAIylD,GAAcz3D,KAAKopB,KAAKpX,EAAIylD,EAAaz3D,KAAKqpB,GAAGrX,IAa7D5O,EAAK+P,UAAU+iD,eAAiB,SAAUnkD,EAAGC,EAAGyZ,EAAQgsC,GACtD,GAAIrJ,GAA6B,GAApBqJ,EAAa,EAAE,GAASvyD,KAAKymB,EAC1C,QACE5Z,EAAGA,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,GACzBp8C,EAAGA,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,KAW7BhrD,EAAK+P,UAAU0hD,iBAAmB,SAAS9tC,GACzC,GAAI7U,EAMJ,IAJA6U,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIiB,UAAYjB,EAAIY,YACpBZ,EAAIO,UAAYtnB,KAAK01D,gBAEjB11D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAExB,GAAI4lC,GAAMjvD,KAAK21D,MAAM5uC,GAEjBqnC,EAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrEpM,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAE1D,IAAyC,GAArCl+C,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EAAa,CAC5D,GAAI2G,GAAY,IAAK,IAAK51D,KAAKopB,KAAKrX,EAAIk9C,EAAIl9C,GAAK,IAAK/R,KAAKqpB,GAAGtX,EAAIk9C,EAAIl9C,IAClE8jD,EAAY,IAAK,IAAK71D,KAAKopB,KAAKpX,EAAIi9C,EAAIj9C,GAAK,IAAKhS,KAAKqpB,GAAGrX,EAAIi9C,EAAIj9C,GACtEE,IAASH,EAAE6jD,EAAW5jD,EAAE6jD,OAGxB3jD,GAAQlS,KAAK81D,aAAa,GAG5B/uC,GAAI2wC,MAAMxlD,EAAMH,EAAGG,EAAMF,EAAGo8C,EAAOzoD,GACnCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,OACPzoB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,OAG3C,CAEH,GAAID,GAAGC,EACHyZ,EAAS,IAAOvmB,KAAKwH,IAAI,IAAI1M,KAAKw+C,QAAQK,cAC1CiH,EAAO9lD,KAAKopB,IACX08B,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAiB,GAAb+zC,EAAKvzC,MAClBP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAkB,GAAd8zC,EAAKtzC,QAEpBxS,KAAKi2D,QAAQlvC,EAAKhV,EAAGC,EAAGyZ,EAGxB,IAAI2iC,GAAQ,GAAMlpD,KAAKymB,GACnBhmB,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAC1DhsC,GAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1C1E,EAAI2wC,MAAMxlD,EAAMH,EAAGG,EAAMF,EAAGo8C,EAAOzoD,GACnCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,QACPvW,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,MAKlD5O,EAAK+P,UAAUwkD,eAAiB,SAAS7pD,GACvC,GAAImhD,GAAMjvD,KAAKo2D,qBAEXrkD,EAAI7M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAG9N,KAAKopB,KAAKrX,EAAK,EAAEjE,GAAG,EAAIA,GAAImhD,EAAIl9C,EAAI7M,KAAK4uB,IAAIhmB,EAAE,GAAG9N,KAAKqpB,GAAGtX,EAC9EC,EAAI9M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAG9N,KAAKopB,KAAKpX,EAAK,EAAElE,GAAG,EAAIA,GAAImhD,EAAIj9C,EAAI9M,KAAK4uB,IAAIhmB,EAAE,GAAG9N,KAAKqpB,GAAGrX,CAElF,QAAQD,EAAEA,EAAEC,EAAEA,IAWhB5O,EAAK+P,UAAUykD,oBAAsB,SAASxuC,EAAKrC,GACjD,GAIIxB,GAAI6oC,EAAMyJ,EAAkBC,EAAiBC,EAJ7C/oD,EAAgB,GAChBC,EAAY,EACZC,EAAM,EACNC,EAAO,EAEP6oD,EAAY,GACZlS,EAAO9lD,KAAKqpB,EAKhB,KAJY,GAARD,IACF08B,EAAO9lD,KAAKopB,MAGAja,GAAPD,GAA2BF,EAAZC,GAA2B,CAC/C,GAAIG,GAAwB,IAAdF,EAAMC,EAOpB,IALAoW,EAAMvlB,KAAK23D,eAAevoD,GAC1Bg/C,EAAQlpD,KAAK4xD,MAAOhR,EAAK9zC,EAAIuT,EAAIvT,EAAK8zC,EAAK/zC,EAAIwT,EAAIxT,GACnD8lD,EAAmB/R,EAAK+R,iBAAiB9wC,EAAIqnC,GAC7C0J,EAAkB5yD,KAAKyqB,KAAKzqB,KAAK4uB,IAAIvO,EAAIxT,EAAE+zC,EAAK/zC,EAAE,GAAK7M,KAAK4uB,IAAIvO,EAAIvT,EAAE8zC,EAAK9zC,EAAE,IAC7E+lD,EAAaF,EAAmBC,EAC5B5yD,KAAK2lB,IAAIktC,GAAcC,EACzB,KAEoB,GAAbD,EACK,GAAR3uC,EACFla,EAAME,EAGND,EAAOC,EAIG,GAARga,EACFja,EAAOC,EAGPF,EAAME,EAIVH,IAIF,MAFAsW,GAAIzX,EAAIsB,EAEDmW,GAUTniB,EAAK+P,UAAUyhD,WAAa,SAAS7tC,GAEnCA,EAAIY,YAAc3nB,KAAKw1D,YACvBzuC,EAAIiB,UAAYjB,EAAIY,YACpBZ,EAAIO,UAAYtnB,KAAK01D,eAGrB,IAAItH,GAAOzoD,EAAQsyD,CAGnB,IAAIj4D,KAAKopB,MAAQppB,KAAKqpB,GAAI,CAKxB,GAHArpB,KAAK21D,MAAM5uC,GAG8B,GAArC/mB,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAIugD,GAAMjvD,KAAKo2D,oBACf6B,GAAWj4D,KAAK43D,qBAAoB,EAAO7wC,EAC3C,IAAImxC,GAAWl4D,KAAK23D,eAAezyD,KAAKwH,IAAI,EAAKurD,EAASnqD,EAAI,IAC9DsgD,GAAQlpD,KAAK4xD,MAAOmB,EAASjmD,EAAIkmD,EAASlmD,EAAKimD,EAASlmD,EAAImmD,EAASnmD,OAElE,CACHq8C,EAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EACrE,IAAI6M,GAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAC7Cu5C,EAAep4D,KAAKqpB,GAAGwuC,iBAAiB9wC,EAAKqnC,GAC7CiK,GAAiBF,EAAoBC,GAAgBD,CAEzDF,MACAA,EAASlmD,GAAK,EAAIsmD,GAAiBr4D,KAAKopB,KAAKrX,EAAIsmD,EAAgBr4D,KAAKqpB,GAAGtX,EACzEkmD,EAASjmD,GAAK,EAAIqmD,GAAiBr4D,KAAKopB,KAAKpX,EAAIqmD,EAAgBr4D,KAAKqpB,GAAGrX,EAU3E,GANArM,GAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,iBACtDn3B,EAAI2wC,MAAMO,EAASlmD,EAAEkmD,EAASjmD,EAAGo8C,EAAOzoD,GACxCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,MAAO,CACd,GAAIvW,EAEFA,GADuC,GAArClS,KAAKyO,QAAQqyC,aAAapyC,SAA0B,MAAPugD,EACvCjvD,KAAK23D,eAAe,IAGpB33D,KAAK81D,aAAa,IAE5B91D,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,QAG3C,CAEH,GACID,GAAGC,EAAG0lD,EADN5R,EAAO9lD,KAAKopB,KAEZqC,EAAS,IAAOvmB,KAAKwH,IAAI,IAAI1M,KAAKw+C,QAAQK,aACzCiH,GAAKvzC,OACRuzC,EAAKkQ,OAAOjvC,GAEV++B,EAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAiB,GAAb+zC,EAAKvzC,MAClBP,EAAI8zC,EAAK9zC,EAAIyZ,EACbisC,GACE3lD,EAAGA,EACHC,EAAG8zC,EAAK9zC,EACRo8C,MAAO,GAAMlpD,KAAKymB,MAIpB5Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAkB,GAAd8zC,EAAKtzC,OAClBklD,GACE3lD,EAAG+zC,EAAK/zC,EACRC,EAAGA,EACHo8C,MAAO,GAAMlpD,KAAKymB,KAGtB5E,EAAIa,YAEJb,EAAI2E,IAAI3Z,EAAGC,EAAGyZ,EAAQ,EAAG,EAAIvmB,KAAKymB,IAAI,GACtC5E,EAAIlH,QAGJ,IAAIla,IAAU,GAAK,EAAI3F,KAAKyO,QAAQ8D,OAASvS,KAAKyO,QAAQyvC,gBAC1Dn3B,GAAI2wC,MAAMA,EAAM3lD,EAAG2lD,EAAM1lD,EAAG0lD,EAAMtJ,MAAOzoD,GACzCohB,EAAInH,OACJmH,EAAIlH,SAGA7f,KAAKyoB,QACPvW,EAAQlS,KAAKk2D,eAAenkD,EAAGC,EAAGyZ,EAAQ,IAC1CzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOvW,EAAMH,EAAGG,EAAMF,MAiBlD5O,EAAK+P,UAAUoiD,mBAAqB,SAAU+C,EAAGC,EAAIC,EAAGC,EAAIC,EAAGC,GAC7D,GAAInvD,GAAc,CAClB,IAAIxJ,KAAKopB,MAAQppB,KAAKqpB,GACpB,GAAyC,GAArCrpB,KAAKyO,QAAQqyC,aAAapyC,QAAiB,CAC7C,GAAI2nD,GAAMC,CACV,IAAyC,GAArCt2D,KAAKyO,QAAQqyC,aAAapyC,SAAwD,GAArC1O,KAAKyO,QAAQqyC,aAAaC,QACzEsV,EAAOr2D,KAAKivD,IAAIl9C,EAChBukD,EAAOt2D,KAAKivD,IAAIj9C,MAEb,CACH,GAAIi9C,GAAMjvD,KAAKo2D,oBACfC,GAAOpH,EAAIl9C,EACXukD,EAAOrH,EAAIj9C,EAEb,GACI2T,GACAngB,EAAEsI,EAAEiE,EAAEC,EAAG4mD,EAAOC,EAFhBC,EAAc,GAGlB,KAAKtzD,EAAI,EAAO,GAAJA,EAAQA,IAClBsI,EAAI,GAAItI,EACRuM,EAAI7M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAGwqD,EAAM,EAAExqD,GAAG,EAAIA,GAAIuoD,EAAOnxD,KAAK4uB,IAAIhmB,EAAE,GAAG0qD,EAC5DxmD,EAAI9M,KAAK4uB,IAAI,EAAEhmB,EAAE,GAAGyqD,EAAM,EAAEzqD,GAAG,EAAIA,GAAIwoD,EAAOpxD,KAAK4uB,IAAIhmB,EAAE,GAAG2qD,EACxDjzD,EAAI,IACNmgB,EAAW3lB,KAAK+4D,mBAAmBH,EAAMC,EAAM9mD,EAAEC,EAAG0mD,EAAGC,GACvDG,EAAyBA,EAAXnzC,EAAyBA,EAAWmzC,GAEpDF,EAAQ7mD,EAAG8mD,EAAQ7mD,CAErBxI,GAAcsvD,MAGdtvD,GAAcxJ,KAAK+4D,mBAAmBT,EAAGC,EAAGC,EAAGC,EAAGC,EAAGC,OAGpD,CACH,GAAI5mD,GAAGC,EAAG4M,EAAIC,EACV4M,EAAS,IAAOzrB,KAAKw+C,QAAQK,aAC7BiH,EAAO9lD,KAAKopB,IACZ08B,GAAKvzC,MAAQuzC,EAAKtzC,QACpBT,EAAI+zC,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,MACxBP,EAAI8zC,EAAK9zC,EAAIyZ,IAGb1Z,EAAI+zC,EAAK/zC,EAAI0Z,EACbzZ,EAAI8zC,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAE1BoM,EAAK7M,EAAI2mD,EACT75C,EAAK7M,EAAI2mD,EACTnvD,EAActE,KAAK2lB,IAAI3lB,KAAKyqB,KAAK/Q,EAAGA,EAAKC,EAAGA,GAAM4M,GAGpD,MAAIzrB,MAAK8zD,gBAAgBrsD,KAAOixD,GAC9B14D,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,MAAQmmD,GACzD14D,KAAK8zD,gBAAgBnsD,IAAMgxD,GAC3B34D,KAAK8zD,gBAAgBnsD,IAAM3H,KAAK8zD,gBAAgBthD,OAASmmD,EAClD,EAGAnvD,GAIXpG,EAAK+P,UAAU4lD,mBAAqB,SAAST,EAAGC,EAAGC,EAAGC,EAAGC,EAAGC,GAC1D,GAAIK,GAAKR,EAAGF,EACVW,EAAKR,EAAGF,EACRW,EAAYF,EAAGA,EAAKC,EAAGA,EACvBE,IAAOT,EAAKJ,GAAMU,GAAML,EAAKJ,GAAMU,GAAMC,CAEvCC,GAAI,EACNA,EAAI,EAEO,EAAJA,IACPA,EAAI,EAGN,IAAIpnD,GAAIumD,EAAKa,EAAIH,EACfhnD,EAAIumD,EAAKY,EAAIF,EACbr6C,EAAK7M,EAAI2mD,EACT75C,EAAK7M,EAAI2mD,CAQX,OAAOzzD,MAAKyqB,KAAK/Q,EAAGA,EAAKC,EAAGA,IAQ9Bzb,EAAK+P,UAAU6vB,SAAW,SAAS/lB,GACjCjd,KAAKm2D,gBAAkB,EAAIl5C,GAI7B7Z,EAAK+P,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,GAGlBjwC,EAAK+P,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,GAGlBjwC,EAAK+P,UAAUg/C,mBAAqB,WACjB,OAAbnyD,KAAKivD,KAA8B,OAAdjvD,KAAKopB,MAA6B,OAAZppB,KAAKqpB,IAClDrpB,KAAKivD,IAAIl9C,EAAI,IAAO/R,KAAKopB,KAAKrX,EAAI/R,KAAKqpB,GAAGtX,GAC1C/R,KAAKivD,IAAIj9C,EAAI,IAAOhS,KAAKopB,KAAKpX,EAAIhS,KAAKqpB,GAAGrX,KAG1ChS,KAAKivD,IAAIl9C,EAAI,EACb/R,KAAKivD,IAAIj9C,EAAI,IASjB5O,EAAK+P,UAAU+8C,kBAAoB,SAASnpC,GAC1C,GAAgC,GAA5B/mB,KAAKu0D,oBAA6B,CACpC,GAA+B,OAA3Bv0D,KAAKw0D,aAAaprC,MAA0C,OAAzBppB,KAAKw0D,aAAanrC,GAAa,CACpE,GAAI+vC,GAAa,cAAcplD,OAAOhU,KAAKK,IACvCg5D,EAAW,YAAYrlD,OAAOhU,KAAKK,IACnCqhD,GACY1E,OAAO/qC,MAAM,GAAIwZ,OAAO,EAAGzL,YAAY,EAAG69B,oBAAqB,GAC/DW,SAASO,QAAQ,GACjBI,YAAac,sBAAuB,EAAGD,aAAcztC,MAAM,EAAGC,OAAQ,EAAGiZ,OAAO,IAEhGzrB,MAAKw0D,aAAaprC,KAAO,GAAI7lB,IAC1BlD,GAAG+4D,EACFhc,MAAM,MACJjyC,OAAOgB,WAAW,UAAWC,OAAO,UAAWC,WAAYF,WAAW,mBAClEu1C,GACV1hD,KAAKw0D,aAAanrC,GAAK,GAAI9lB,IACxBlD,GAAGg5D,EACFjc,MAAM,MACNjyC,OAAOgB,WAAW,UAAWC,OAAO,UAAWC,WAAYF,WAAW,mBAChEu1C,GAGZ1hD,KAAKw0D,aAAaC,aACqB,GAAnCz0D,KAAKw0D,aAAaprC,KAAKiqB,WACzBrzC,KAAKw0D,aAAaC,UAAUrrC,KAAOppB,KAAKs5D,2BAA2BvyC,GACnE/mB,KAAKw0D,aAAaprC,KAAKrX,EAAI/R,KAAKw0D,aAAaC,UAAUrrC,KAAKrX,EAC5D/R,KAAKw0D,aAAaprC,KAAKpX,EAAIhS,KAAKw0D,aAAaC,UAAUrrC,KAAKpX,GAEzB,GAAjChS,KAAKw0D,aAAanrC,GAAGgqB,WACvBrzC,KAAKw0D,aAAaC,UAAUprC,GAAKrpB,KAAKu5D,yBAAyBxyC,GAC/D/mB,KAAKw0D,aAAanrC,GAAGtX,EAAI/R,KAAKw0D,aAAaC,UAAUprC,GAAGtX,EACxD/R,KAAKw0D,aAAanrC,GAAGrX,EAAIhS,KAAKw0D,aAAaC,UAAUprC,GAAGrX,GAG1DhS,KAAKw0D,aAAaprC,KAAK2iB,KAAKhlB,GAC5B/mB,KAAKw0D,aAAanrC,GAAG0iB,KAAKhlB,OAG1B/mB,MAAKw0D,cAAgBprC,KAAK,KAAMC,GAAG,KAAMorC,eAQ7CrxD,EAAK+P,UAAUqmD,oBAAsB,WACnCx5D,KAAKi0D,WAAaj0D,KAAKopB,KACvBppB,KAAKk0D,SAAWl0D,KAAKqpB,GACrBrpB,KAAKu0D,qBAAsB,GAO7BnxD,EAAK+P,UAAUsmD,qBAAuB,WACpCz5D,KAAK2zD,OAAS3zD,KAAKopB,KAAK/oB,GACxBL,KAAK4zD,KAAO5zD,KAAKqpB,GAAGhpB,GAChBL,KAAK2zD,QAAU3zD,KAAKi0D,WAAW5zD,GACjCL,KAAKi0D,WAAWe,WAAWh1D,MAEpBA,KAAK4zD,MAAQ5zD,KAAKk0D,SAAS7zD,IAClCL,KAAKk0D,SAASc,WAAWh1D,MAG3BA,KAAKi0D,WAAa,KAClBj0D,KAAKk0D,SAAW,KAChBl0D,KAAKu0D,qBAAsB,GAW7BnxD,EAAK+P,UAAUumD,wBAA0B,SAAS3nD,EAAEC,GAClD,GAAIyiD,GAAYz0D,KAAKw0D,aAAaC,UAC9BkF,EAAez0D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI/hB,EAAI0iD,EAAUrrC,KAAKrX,EAAE,GAAK7M,KAAK4uB,IAAI9hB,EAAIyiD,EAAUrrC,KAAKpX,EAAE,IAC1F4nD,EAAe10D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI/hB,EAAI0iD,EAAUprC,GAAGtX,EAAI,GAAK7M,KAAK4uB,IAAI9hB,EAAIyiD,EAAUprC,GAAGrX,EAAI,GAE9F,OAAmB,IAAf2nD,GACF35D,KAAK00D,cAAgB10D,KAAKopB,KAC1BppB,KAAKopB,KAAOppB,KAAKw0D,aAAaprC,KACvBppB,KAAKw0D,aAAaprC,MAEL,GAAbwwC,GACP55D,KAAK00D,cAAgB10D,KAAKqpB,GAC1BrpB,KAAKqpB,GAAKrpB,KAAKw0D,aAAanrC,GACrBrpB,KAAKw0D,aAAanrC,IAGlB,MASXjmB,EAAK+P,UAAU0mD,qBAAuB,WACG,GAAnC75D,KAAKw0D,aAAaprC,KAAKiqB,UACzBrzC,KAAKopB,KAAOppB,KAAK00D,cACjB10D,KAAK00D,cAAgB,KACrB10D,KAAKw0D,aAAaprC,KAAK8nB,YAEiB,GAAjClxC,KAAKw0D,aAAanrC,GAAGgqB,WAC5BrzC,KAAKqpB,GAAKrpB,KAAK00D,cACf10D,KAAK00D,cAAgB,KACrB10D,KAAKw0D,aAAanrC,GAAG6nB,aAUzB9tC,EAAK+P,UAAUmmD,2BAA6B,SAASvyC,GAEnD,GAAI+yC,EACJ,IAAyC,GAArC95D,KAAKyO,QAAQqyC,aAAapyC,QAC5BorD,EAAqB95D,KAAK43D,qBAAoB,EAAM7wC,OAEjD,CACH,GAAIqnC,GAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrE6M,EAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAE7Ck7C,EAAiB/5D,KAAKopB,KAAKyuC,iBAAiB9wC,EAAKqnC,EAAQlpD,KAAKymB,IAC9DquC,GAAmB7B,EAAoB4B,GAAkB5B,CAC7D2B,MACAA,EAAmB/nD,EAAI,EAAoB/R,KAAKopB,KAAKrX,GAAK,EAAIioD,GAAmBh6D,KAAKqpB,GAAGtX,EACzF+nD,EAAmB9nD,EAAI,EAAoBhS,KAAKopB,KAAKpX,GAAK,EAAIgoD,GAAmBh6D,KAAKqpB,GAAGrX,EAG3F,MAAO8nD,IAST12D,EAAK+P,UAAUomD,yBAA2B,SAASxyC,GAEjD,GAAuBkzC,EACvB,IAAyC,GAArCj6D,KAAKyO,QAAQqyC,aAAapyC,QAC5BurD,EAAmBj6D,KAAK43D,qBAAoB,EAAO7wC,OAEhD,CACH,GAAIqnC,GAAQlpD,KAAK4xD,MAAO92D,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAAKhS,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,GACrE6M,EAAM5e,KAAKqpB,GAAGtX,EAAI/R,KAAKopB,KAAKrX,EAC5B8M,EAAM7e,KAAKqpB,GAAGrX,EAAIhS,KAAKopB,KAAKpX,EAC5BmmD,EAAoBjzD,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAC7Cu5C,EAAep4D,KAAKqpB,GAAGwuC,iBAAiB9wC,EAAKqnC,GAC7CiK,GAAiBF,EAAoBC,GAAgBD,CAEzD8B,MACAA,EAAiBloD,GAAK,EAAIsmD,GAAiBr4D,KAAKopB,KAAKrX,EAAIsmD,EAAgBr4D,KAAKqpB,GAAGtX,EACjFkoD,EAAiBjoD,GAAK,EAAIqmD,GAAiBr4D,KAAKopB,KAAKpX,EAAIqmD,EAAgBr4D,KAAKqpB,GAAGrX,EAGnF,MAAOioD,IAGTp6D,EAAOD,QAAUwD,GAIb,SAASvD,EAAQD,EAASM,GAQ9B,QAASmD,KACPrD,KAAKyW,QACLzW,KAAKk6D,aAAe,EARXh6D,EAAoB,EAe/BmD,GAAO82D,UACJ/tD,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aACxIC,OAAQ,UAAWD,WAAY,UAAWE,WAAYD,OAAQ,UAAWD,WAAY,WAAYG,OAAQF,OAAQ,UAAWD,WAAY,aAO3I9I,EAAO8P,UAAUsD,MAAQ,WACvBzW,KAAKm0B,UACLn0B,KAAKm0B,OAAOxuB,OAAS,WAEnB,GAAIH,GAAI,CACR,KAAM,GAAI9E,KAAKV,MACTA,KAAK8F,eAAepF,IACtB8E,GAGJ,OAAOA,KAWXnC,EAAO8P,UAAU+B,IAAM,SAAUkzC,GAC/B,GAAIn2C,GAAQjS,KAAKm0B,OAAOi0B,EACxB,IAAa5hD,QAATyL,EAAoB,CAEtB,GAAI9J,GAAQnI,KAAKk6D,aAAe72D,EAAO82D,QAAQx0D,MAC/C3F,MAAKk6D,eACLjoD,KACAA,EAAM9G,MAAQ9H,EAAO82D,QAAQhyD,GAC7BnI,KAAKm0B,OAAOi0B,GAAan2C,EAG3B,MAAOA,IAUT5O,EAAO8P,UAAUF,IAAM,SAAUm1C,EAAWn7C,GAE1C,MADAjN,MAAKm0B,OAAOi0B,GAAan7C,EAClBA,GAGTpN,EAAOD,QAAUyD,GAKb,SAASxD,GAMb,QAASyD,KACPtD,KAAK2iD,UACL3iD,KAAKo6D,eACLp6D,KAAKsI,SAAW9B,OAQlBlD,EAAO6P,UAAUyvC,kBAAoB,SAASt6C,GAC5CtI,KAAKsI,SAAWA,GASlBhF,EAAO6P,UAAUknD,KAAO,SAASC,EAAKC,GACpC,GAAIC,GAAMx6D,KAAK2iD,OAAO2X,EACtB,IAAY9zD,SAARg0D,EAAmB,CAErB,GAAIrmD,GAAKnU,IACTw6D,GAAM,GAAIC,OACVD,EAAIE,OAAS,WAEO,GAAd16D,KAAKuS,QACPhB,SAASojB,KAAKljB,YAAYzR,MAC1BA,KAAKuS,MAAQvS,KAAKowB,YAClBpwB,KAAKwS,OAASxS,KAAKswB,aACnB/e,SAASojB,KAAKxjB,YAAYnR,OAGxBmU,EAAG7L,WACL6L,EAAGwuC,OAAO2X,GAAOE,EACjBrmD,EAAG7L,SAAStI,QAIhBw6D,EAAIG,QAAU,WACMn0D,SAAd+zD,GACF5hC,QAAQiiC,MAAM,wBAAyBN,SAChCt6D,MAAK4lD,IACRzxC,EAAG7L,UACL6L,EAAG7L,SAAStI,OAGPmU,EAAGimD,YAAYE,MAAS,GAC/B3hC,QAAQiiC,MAAM,8BAA+BL,SACtCv6D,MAAK4lD,IACRzxC,EAAG7L,UACL6L,EAAG7L,SAAStI,QAIdA,KAAK4lD,IAAM2U,EACXpmD,EAAGimD,YAAYE,IAAO,IAI1BE,EAAI5U,IAAM0U,EAGZ,MAAOE,IAGT36D,EAAOD,QAAU0D,GAKb,SAASzD,EAAQD,EAASM,GA6B9B,QAASqD,GAAKorD,EAAYkM,EAAWC,EAAWpH,GAC9C,GAAIhS,GAAY/gD,EAAKsN,uBAAuB,SAASylD,EACrD1zD,MAAKyO,QAAUizC,EAAU1E,MAEzBh9C,KAAKqzC,UAAW,EAChBrzC,KAAKsM,OAAQ,EAEbtM,KAAK89C,SACL99C,KAAKmvD,gBACLnvD,KAAK+6D,iBAEL/6D,KAAKg7D,kBAAoB,EAGzBh7D,KAAKK,GAAKmG,OACVxG,KAAKwyD,gBAAiB,EACtBxyD,KAAKyyD,gBAAiB,EACtBzyD,KAAKmrD,QAAS,EACdnrD,KAAKorD,QAAS,EACdprD,KAAKi7D,qBAAsB,EAC3Bj7D,KAAKk7D,kBAAsB,EAC3Bl7D,KAAKm7D,gBAAkBzH,EAAiB1W,MAAMvxB,OAC9CzrB,KAAKo7D,aAAc,EACnBp7D,KAAK49C,MAAQ,GACb59C,KAAKq7D,kBAAmB,EACxBr7D,KAAKs7D,qBAAsB,EAC3Bt7D,KAAK8zD,iBAAmBnsD,IAAI,EAAGF,KAAK,EAAG8K,MAAM,EAAGC,OAAO,EAAGuhD,MAAM,GAChE/zD,KAAKomD,aAAez+C,IAAI,EAAGF,KAAK,EAAG4f,MAAM,EAAG/D,OAAO,GAEnDtjB,KAAK66D,UAAYA,EACjB76D,KAAK86D,UAAYA,EAGjB96D,KAAKu7D,GAAK,EACVv7D,KAAKw7D,GAAK,EACVx7D,KAAKy7D,GAAK,EACVz7D,KAAK07D,GAAK,EACV17D,KAAK+R,EAAI,KACT/R,KAAKgS,EAAI,KAGThS,KAAK27D,eAAiBF,GAAG,EAAEC,GAAG,EAAE3pD,EAAE,EAAEC,EAAE,GAEtChS,KAAK++C,QAAU2U,EAAiBlV,QAAQO,QACxC/+C,KAAKswD,WAAav+C,EAAE,KAAKC,EAAE,MAE3BhS,KAAK0uD,cAAcC,EAAYjN,GAG/B1hD,KAAK47D,eACL57D,KAAK67D,mBAAqB,EAC1B77D,KAAK87D,eAAiB,EACtB97D,KAAK+7D,uBAA0BrI,EAAiBvU,WAAWa,YAAYztC,MACvEvS,KAAKg8D,wBAA0BtI,EAAiBvU,WAAWa,YAAYxtC,OACvExS,KAAKi8D,wBAA0BvI,EAAiBvU,WAAWa,YAAYv0B,OACvEzrB,KAAKigD,sBAAwByT,EAAiBvU,WAAWc,sBACzDjgD,KAAKk8D,gBAAkB,EAGvBl8D,KAAKm2D,gBAAkB,EACvBn2D,KAAKm8D,aAAe,EACpBn8D,KAAK+jD,eAAiBhyC,EAAK,KAAMC,EAAK,MACtChS,KAAKgkD,mBAAqBjyC,EAAM,IAAKC,EAAM,KAC3ChS,KAAKiyD,aAAe,KA1FtB,GAAItxD,GAAOT,EAAoB,EAiG/BqD,GAAK4P,UAAU69C,eAAiB,WAC9BhxD,KAAK+R,EAAI/R,KAAK27D,cAAc5pD,EAC5B/R,KAAKgS,EAAIhS,KAAK27D,cAAc3pD,EAC5BhS,KAAKy7D,GAAKz7D,KAAK27D,cAAcF,GAC7Bz7D,KAAK07D,GAAK17D,KAAK27D,cAAcD,IAO/Bn4D,EAAK4P,UAAUyoD,aAAe,WAE5B57D,KAAKo8D,eAAiB51D,OACtBxG,KAAKq8D,YAAc,EACnBr8D,KAAKs8D,kBACLt8D,KAAKu8D,kBACLv8D,KAAKw8D,oBAOPj5D,EAAK4P,UAAU4hD,WAAa,SAASrH,GACH,IAA5B1tD,KAAK89C,MAAMn3C,QAAQ+mD,IACrB1tD,KAAK89C,MAAM91C,KAAK0lD,GAEqB,IAAnC1tD,KAAKmvD,aAAaxoD,QAAQ+mD,IAC5B1tD,KAAKmvD,aAAannD,KAAK0lD,GAEzB1tD,KAAK67D,mBAAqB77D,KAAKmvD,aAAaxpD,QAO9CpC,EAAK4P,UAAU6hD,WAAa,SAAStH,GACnC,GAAIvlD,GAAQnI,KAAK89C,MAAMn3C,QAAQ+mD,EAClB,KAATvlD,GACFnI,KAAK89C,MAAM11C,OAAOD,EAAO,GAE3BA,EAAQnI,KAAKmvD,aAAaxoD,QAAQ+mD,GACrB,IAATvlD,GACFnI,KAAKmvD,aAAa/mD,OAAOD,EAAO,GAElCnI,KAAK67D,mBAAqB77D,KAAKmvD,aAAaxpD,QAS9CpC,EAAK4P,UAAUu7C,cAAgB,SAASC,EAAYjN,GAClD,GAAKiN,EAAL,CAIA,GAAIzgD,IAAU,cAAc,sBAAsB,QAAQ,QAAQ,cAAc,SAAS,YACvF,WAAW,WAAW,WAAW,kBAAkB,kBAAkB,QAAQ,OAkB/E,IAhBAvN,EAAKwF,oBAAoB+H,EAAQlO,KAAKyO,QAASkgD,GAGzBnoD,SAAlBmoD,EAAWtuD,KAA0BL,KAAKK,GAAKsuD,EAAWtuD,IACrCmG,SAArBmoD,EAAWlmC,QAA0BzoB,KAAKyoB,MAAQkmC,EAAWlmC,MAAOzoB,KAAKy8D,cAAgB9N,EAAWlmC,OAC/EjiB,SAArBmoD,EAAW3pB,QAA0BhlC,KAAKglC,MAAQ2pB,EAAW3pB,OAC5Cx+B,SAAjBmoD,EAAW58C,IAA0B/R,KAAK+R,EAAI48C,EAAW58C,GACxCvL,SAAjBmoD,EAAW38C,IAA0BhS,KAAKgS,EAAI28C,EAAW38C,GACpCxL,SAArBmoD,EAAWtnD,QAA0BrH,KAAKqH,MAAQsnD,EAAWtnD,OACxCb,SAArBmoD,EAAW/Q,QAA0B59C,KAAK49C,MAAQ+Q,EAAW/Q,MAAO59C,KAAKq7D,kBAAmB,GAGzD70D,SAAnCmoD,EAAWsM,sBAAoCj7D,KAAKi7D,oBAAsBtM,EAAWsM,qBAClDz0D,SAAnCmoD,EAAWuM,mBAAoCl7D,KAAKk7D,iBAAsBvM,EAAWuM,kBAClD10D,SAAnCmoD,EAAW+N,kBAAoC18D,KAAK08D,gBAAsB/N,EAAW+N,iBAEzEl2D,SAAZxG,KAAKK,GACP,KAAM,sBAIR,IAAkC,gBAAvBL,MAAKyO,QAAQwD,OAAqD,gBAAvBjS,MAAKyO,QAAQwD,OAA4C,IAAtBjS,KAAKyO,QAAQwD,MAAc,CAClH,GAAI0qD,GAAW38D,KAAK86D,UAAU5lD,IAAIlV,KAAKyO,QAAQwD,MAC/CtR,GAAK8F,WAAWzG,KAAKyO,QAASkuD,GAE9B38D,KAAKyO,QAAQtD,MAAQxK,EAAKuK,WAAWlL,KAAKyO,QAAQtD,OAMpD,GAF0B3E,SAAtBmoD,EAAWljC,SAA+BzrB,KAAKm7D,gBAAkBn7D,KAAKyO,QAAQgd,QACzDjlB,SAArBmoD,EAAWxjD,QAA+BnL,KAAKyO,QAAQtD,MAAQxK,EAAKuK,WAAWyjD,EAAWxjD,QACnE3E,SAAvBxG,KAAKyO,QAAQ4uC,OAA4C,IAArBr9C,KAAKyO,QAAQ4uC,MAAY,CAC/D,IAAIr9C,KAAK66D,UAIP,KAAM,uBAHN76D,MAAK48D,SAAW58D,KAAK66D,UAAUR,KAAKr6D,KAAKyO,QAAQ4uC,MAAOr9C,KAAKyO,QAAQouD,aAgCzE,OAzBkCr2D,SAA9BmoD,EAAW6D,gBACbxyD,KAAKmrD,QAAUwD,EAAW6D,eAC1BxyD,KAAKwyD,eAAiB7D,EAAW6D,gBAEThsD,SAAjBmoD,EAAW58C,GAA0C,GAAvB/R,KAAKwyD,iBAC1CxyD,KAAKmrD,QAAS,GAIkB3kD,SAA9BmoD,EAAW8D,gBACbzyD,KAAKorD,QAAUuD,EAAW8D,eAC1BzyD,KAAKyyD,eAAiB9D,EAAW8D,gBAETjsD,SAAjBmoD,EAAW38C,GAA0C,GAAvBhS,KAAKyyD,iBAC1CzyD,KAAKorD,QAAS,GAGhBprD,KAAKo7D,YAAcp7D,KAAKo7D,aAAsC50D,SAAtBmoD,EAAWljC,QAExB,UAAvBzrB,KAAKyO,QAAQ2uC,OAA4C,kBAAvBp9C,KAAKyO,QAAQ2uC,SACjDp9C,KAAKyO,QAAQyuC,UAAYwE,EAAU1E,MAAM91B,SACzClnB,KAAKyO,QAAQ0uC,UAAYuE,EAAU1E,MAAM71B,UAInCnnB,KAAKyO,QAAQ2uC,OACnB,IAAK,WAAiBp9C,KAAK+rC,KAAO/rC,KAAK88D,cAAe98D,KAAKg2D,OAASh2D,KAAK+8D,eAAiB,MAC1F,KAAK,MAAiB/8D,KAAK+rC,KAAO/rC,KAAKg9D,SAAUh9D,KAAKg2D,OAASh2D,KAAKi9D,UAAY,MAChF,KAAK,SAAiBj9D,KAAK+rC,KAAO/rC,KAAKk9D,YAAal9D,KAAKg2D,OAASh2D,KAAKm9D,aAAe,MACtF,KAAK,UAAiBn9D,KAAK+rC,KAAO/rC,KAAKo9D,aAAcp9D,KAAKg2D,OAASh2D,KAAKq9D,cAAgB,MAExF,KAAK,QAAiBr9D,KAAK+rC,KAAO/rC,KAAKs9D,WAAYt9D,KAAKg2D,OAASh2D,KAAKu9D,YAAc,MACpF,KAAK,gBAAiBv9D,KAAK+rC,KAAO/rC,KAAKw9D,mBAAoBx9D,KAAKg2D,OAASh2D,KAAKy9D,oBAAsB,MACpG,KAAK,OAAiBz9D,KAAK+rC,KAAO/rC,KAAK09D,UAAW19D,KAAKg2D,OAASh2D,KAAK29D,WAAa,MAClF,KAAK,MAAiB39D,KAAK+rC,KAAO/rC,KAAK49D,SAAU59D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MAClF,KAAK,SAAiB79D,KAAK+rC,KAAO/rC,KAAK89D,YAAa99D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACrF,KAAK,WAAiB79D,KAAK+rC,KAAO/rC,KAAK+9D,cAAe/9D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACvF,KAAK,eAAiB79D,KAAK+rC,KAAO/rC,KAAKg+D,kBAAmBh+D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MAC3F,KAAK,OAAiB79D,KAAK+rC,KAAO/rC,KAAKi+D,UAAWj+D,KAAKg2D,OAASh2D,KAAK69D,YAAc,MACnF,SAAsB79D,KAAK+rC,KAAO/rC,KAAKo9D,aAAcp9D,KAAKg2D,OAASh2D,KAAKq9D,eAG1Er9D,KAAKk+D,WAOP36D,EAAK4P,UAAUg+B,OAAS,WACtBnxC,KAAKqzC,UAAW,EAChBrzC,KAAKk+D,UAMP36D,EAAK4P,UAAU+9B,SAAW,WACxBlxC,KAAKqzC,UAAW,EAChBrzC,KAAKk+D,UAOP36D,EAAK4P,UAAUgrD,eAAiB,WAC9Bn+D,KAAKk+D,UAOP36D,EAAK4P,UAAU+qD,OAAS,WACtBl+D,KAAKuS,MAAQ/L,OACbxG,KAAKwS,OAAShM,QAQhBjD,EAAK4P,UAAUq6C,SAAW,WACxB,MAA6B,kBAAfxtD,MAAKglC,MAAuBhlC,KAAKglC,QAAUhlC,KAAKglC,OAShEzhC,EAAK4P,UAAU0kD,iBAAmB,SAAU9wC,EAAKqnC,GAC/C,GAAIpuC,GAAc,CAMlB,QAJKhgB,KAAKuS,OACRvS,KAAKg2D,OAAOjvC,GAGN/mB,KAAKyO,QAAQ2uC,OACnB,IAAK,SACL,IAAK,MACH,MAAOp9C,MAAKyO,QAAQgd,OAAQzL,CAE9B,KAAK,UACH,GAAIza,GAAIvF,KAAKuS,MAAQ,EACjBnM,EAAIpG,KAAKwS,OAAS,EAClB68C,EAAKnqD,KAAKkZ,IAAIgwC,GAAS7oD,EACvBoG,EAAKzG,KAAKqZ,IAAI6vC,GAAShoD,CAC3B,OAAOb,GAAIa,EAAIlB,KAAKyqB,KAAK0/B,EAAIA,EAAI1jD,EAAIA,EAMvC,KAAK,MACL,IAAK,QACL,IAAK,OACL,QACE,MAAI3L,MAAKuS,MACArN,KAAK4G,IACR5G,KAAK2lB,IAAI7qB,KAAKuS,MAAQ,EAAIrN,KAAKqZ,IAAI6vC,IACnClpD,KAAK2lB,IAAI7qB,KAAKwS,OAAS,EAAItN,KAAKkZ,IAAIgwC,KAAWpuC,EAI5C,IAYfzc,EAAK4P,UAAUirD,UAAY,SAAS7C,EAAIC,GACtCx7D,KAAKu7D,GAAKA,EACVv7D,KAAKw7D,GAAKA,GASZj4D,EAAK4P,UAAUkrD,UAAY,SAAS9C,EAAIC,GACtCx7D,KAAKu7D,IAAMA,EACXv7D,KAAKw7D,IAAMA,GAMbj4D,EAAK4P,UAAUmrD,WAAa,WAC1Bt+D,KAAK27D,cAAc5pD,EAAI/R,KAAK+R,EAC5B/R,KAAK27D,cAAc3pD,EAAIhS,KAAKgS,EAC5BhS,KAAK27D,cAAcF,GAAKz7D,KAAKy7D,GAC7Bz7D,KAAK27D,cAAcD,GAAK17D,KAAK07D,IAO/Bn4D,EAAK4P,UAAU09C,aAAe,SAASr+B,GAErC,GADAxyB,KAAKs+D,aACAt+D,KAAKmrD,OAORnrD,KAAKu7D,GAAK,EACVv7D,KAAKy7D,GAAK,MARM,CAChB,GAAI78C,GAAO5e,KAAK++C,QAAU/+C,KAAKy7D,GAC3B79C,GAAQ5d,KAAKu7D,GAAK38C,GAAM5e,KAAKyO,QAAQwuC,IACzCj9C,MAAKy7D,IAAM79C,EAAK4U,EAChBxyB,KAAK+R,GAAM/R,KAAKy7D,GAAKjpC,EAOvB,GAAKxyB,KAAKorD,OAORprD,KAAKw7D,GAAK,EACVx7D,KAAK07D,GAAK,MARM,CAChB,GAAI78C,GAAO7e,KAAK++C,QAAU/+C,KAAK07D,GAC3B79C,GAAQ7d,KAAKw7D,GAAK38C,GAAM7e,KAAKyO,QAAQwuC,IACzCj9C,MAAK07D,IAAM79C,EAAK2U,EAChBxyB,KAAKgS,GAAMhS,KAAK07D,GAAKlpC,IAezBjvB,EAAK4P,UAAUy9C,oBAAsB,SAASp+B,EAAUyuB,GAEtD,GADAjhD,KAAKs+D,aACAt+D,KAAKmrD,OAQRnrD,KAAKu7D,GAAK,EACVv7D,KAAKy7D,GAAK,MATM,CAChB,GAAI78C,GAAO5e,KAAK++C,QAAU/+C,KAAKy7D,GAC3B79C,GAAQ5d,KAAKu7D,GAAK38C,GAAM5e,KAAKyO,QAAQwuC,IACzCj9C,MAAKy7D,IAAM79C,EAAK4U,EAChBxyB,KAAKy7D,GAAMv2D,KAAK2lB,IAAI7qB,KAAKy7D,IAAMxa,EAAiBjhD,KAAKy7D,GAAK,EAAKxa,GAAeA,EAAejhD,KAAKy7D,GAClGz7D,KAAK+R,GAAM/R,KAAKy7D,GAAKjpC,EAOvB,GAAKxyB,KAAKorD,OAQRprD,KAAKw7D,GAAK,EACVx7D,KAAK07D,GAAK,MATM,CAChB,GAAI78C,GAAO7e,KAAK++C,QAAU/+C,KAAK07D,GAC3B79C,GAAQ7d,KAAKw7D,GAAK38C,GAAM7e,KAAKyO,QAAQwuC,IACzCj9C,MAAK07D,IAAM79C,EAAK2U,EAChBxyB,KAAK07D,GAAMx2D,KAAK2lB,IAAI7qB,KAAK07D,IAAMza,EAAiBjhD,KAAK07D,GAAK,EAAKza,GAAeA,EAAejhD,KAAK07D,GAClG17D,KAAKgS,GAAMhS,KAAK07D,GAAKlpC,IAYzBjvB,EAAK4P,UAAUorD,QAAU,WACvB,MAAQv+D,MAAKmrD,QAAUnrD,KAAKorD,QAQ9B7nD,EAAK4P,UAAUs9C,SAAW,SAASD,GACjC,GAAIgO,GAAWt5D,KAAKyqB,KAAKzqB,KAAK4uB,IAAI9zB,KAAKy7D,GAAG,GAAKv2D,KAAK4uB,IAAI9zB,KAAK07D,GAAG,GAEhE,OAAQ8C,GAAWhO,GAOrBjtD,EAAK4P,UAAU23C,WAAa,WAC1B,MAAO9qD,MAAKqzC,UAOd9vC,EAAK4P,UAAUyB,SAAW,WACxB,MAAO5U,MAAKqH,OASd9D,EAAK4P,UAAUsrD,YAAc,SAAS1sD,EAAGC,GACvC,GAAI4M,GAAK5e,KAAK+R,EAAIA,EACd8M,EAAK7e,KAAKgS,EAAIA,CAClB,OAAO9M,MAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,IAUlCtb,EAAK4P,UAAUi8C,cAAgB,SAAStjD,EAAKY,GAC3C,IAAK1M,KAAKo7D,aAA8B50D,SAAfxG,KAAKqH,MAC5B,GAAIqF,GAAOZ,EACT9L,KAAKyO,QAAQgd,QAASzrB,KAAKyO,QAAQyuC,UAAYl9C,KAAKyO,QAAQ0uC,WAAa,MAEtE,CACH,GAAIlgC,IAASjd,KAAKyO,QAAQ0uC,UAAYn9C,KAAKyO,QAAQyuC,YAAcxwC,EAAMZ,EACvE9L,MAAKyO,QAAQgd,QAASzrB,KAAKqH,MAAQyE,GAAOmR,EAAQjd,KAAKyO,QAAQyuC,UAGnEl9C,KAAKm7D,gBAAkBn7D,KAAKyO,QAAQgd,QAQtCloB,EAAK4P,UAAU44B,KAAO,WACpB,KAAM,wCAQRxoC,EAAK4P,UAAU6iD,OAAS,WACtB,KAAM,0CAQRzyD,EAAK4P,UAAUo6C,kBAAoB,SAASxqC,GAC1C,MAAQ/iB,MAAKyH,KAAoBsb,EAAIsE,OAC7BrnB,KAAKyH,KAAOzH,KAAKuS,MAAQwQ,EAAItb,MAC7BzH,KAAK2H,IAAoBob,EAAIO,QAC7BtjB,KAAK2H,IAAM3H,KAAKwS,OAASuQ,EAAIpb,KAGvCpE,EAAK4P,UAAUoqD,aAAe,WAG5B,IAAKv9D,KAAKuS,QAAUvS,KAAKwS,OAAQ,CAC/B,GAAID,GAAOC,CACX,IAAIxS,KAAKqH,MAAO,CACdrH,KAAKyO,QAAQgd,OAAQzrB,KAAKm7D,eAC1B,IAAIl+C,GAAQjd,KAAK48D,SAASpqD,OAASxS,KAAK48D,SAASrqD,KACnC/L,UAAVyW,GACF1K,EAAQvS,KAAKyO,QAAQgd,QAASzrB,KAAK48D,SAASrqD,MAC5CC,EAASxS,KAAKyO,QAAQgd,OAAQxO,GAASjd,KAAK48D,SAASpqD,SAGrDD,EAAQ,EACRC,EAAS,OAIXD,GAAQvS,KAAK48D,SAASrqD,MACtBC,EAASxS,KAAK48D,SAASpqD,MAEzBxS,MAAKuS,MAASA,EACdvS,KAAKwS,OAASA,EAEdxS,KAAKk8D,gBAAkB,EACnBl8D,KAAKuS,MAAQ,GAAKvS,KAAKwS,OAAS,IAClCxS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA0BjgD,KAAK+7D,uBAClF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQA,KAK1ChP,EAAK4P,UAAUurD,qBAAuB,SAAU33C,GAC9C,GAA2B,GAAvB/mB,KAAK48D,SAASrqD,MAAa,CAE7B,GAAIvS,KAAKq8D,YAAc,EAAG,CACxB,GAAI/0C,GAActnB,KAAKq8D,YAAc,EAAK,GAAK,CAC/C/0C,IAAatnB,KAAKm2D,gBAClB7uC,EAAYpiB,KAAK4G,IAAI,GAAM9L,KAAKuS,MAAM+U,GAEtCP,EAAI43C,YAAc,GAClB53C,EAAI63C,UAAU5+D,KAAK48D,SAAU58D,KAAKyH,KAAO6f,EAAWtnB,KAAK2H,IAAM2f,EAAWtnB,KAAKuS,MAAQ,EAAE+U,EAAWtnB,KAAKwS,OAAS,EAAE8U,GAItHP,EAAI43C,YAAc,EAClB53C,EAAI63C,UAAU5+D,KAAK48D,SAAU58D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,UAIvEjP,EAAK4P,UAAU0rD,gBAAkB,SAAU93C,GACzC,GAAIjN,GACA6P,EAAS,CAEb,IAAI3pB,KAAKwS,OAAO,CACdmX,EAAS3pB,KAAKwS,OAAS,CACvB,IAAIshD,GAAkB9zD,KAAK8+D,YAAY/3C,EAEnC+sC,GAAgB0C,WAAa,IAC/B7sC,GAAUmqC,EAAgBthD,OAAS,EACnCmX,GAAU,GAId7P,EAAS9Z,KAAKgS,EAAI2X,EAElB3pB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG+H,EAAQtT,SAG/CjD,EAAK4P,UAAUmqD,WAAa,SAAUv2C,GACpC/mB,KAAKu9D,aAAax2C,GAClB/mB,KAAKyH,KAASzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EACpCvS,KAAK2H,IAAS3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAErCxS,KAAK0+D,qBAAqB33C,GAE1B/mB,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK6+D,gBAAgB93C,GACrB/mB,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,SAG7GjP,EAAK4P,UAAUsqD,qBAAuB,SAAU12C,GAC9C,GAAI/mB,KAAK48D,SAAShX,KAAQ5lD,KAAK48D,SAASrqD,OAAUvS,KAAK48D,SAASpqD,OAe1DxS,KAAK++D,oCACP/+D,KAAKuS,MAAQ,EACbvS,KAAKwS,OAAS,QACPxS,MAAK++D,mCAEd/+D,KAAKu9D,aAAax2C,OAnBlB,KAAK/mB,KAAKuS,MAAO,CACf,GAAIysD,GAAiC,EAAtBh/D,KAAKyO,QAAQgd,MAC5BzrB,MAAKuS,MAAQysD,EACbh/D,KAAKwS,OAASwsD,EAKdh/D,KAAKyO,QAAQgd,QAAuE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC/Fj8D,KAAKk8D,gBAAkBl8D,KAAKyO,QAAQgd,OAAQ,GAAIuzC,EAChDh/D,KAAK++D,mCAAoC,IAc/Cx7D,EAAK4P,UAAUqqD,mBAAqB,SAAUz2C,GAC5C/mB,KAAKy9D,qBAAqB12C,GAE1B/mB,KAAKyH,KAASzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EACpCvS,KAAK2H,IAAS3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAErC,IAAIysD,GAAUj/D,KAAKyH,KAAQzH,KAAKuS,MAAQ,EACpC2sD,EAAUl/D,KAAK2H,IAAO3H,KAAKwS,OAAS,EACpCiZ,EAASvmB,KAAK2lB,IAAI7qB,KAAKwS,OAAS,EAEpCxS,MAAKm/D,eAAep4C,EAAKk4C,EAASC,EAASzzC,GAE3C1E,EAAIuoC,OACJvoC,EAAIq4C,OAAOp/D,KAAK+R,EAAG/R,KAAKgS,EAAGyZ,GAC3B1E,EAAIlH,SACJkH,EAAIs4C,OAEJr/D,KAAK0+D,qBAAqB33C,GAE1BA,EAAI0oC,UAEJzvD,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAEhDzrB,KAAK6+D,gBAAgB93C,GAErB/mB,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,SAG7GjP,EAAK4P,UAAU8pD,WAAa,SAAUl2C,GACpC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,EAChC/mB,MAAKuS,MAAQ+sD,EAAS/sD,MAAQ,EAAImH,EAClC1Z,KAAKwS,OAAS8sD,EAAS9sD,OAAS,EAAIkH,EAEpC1Z,KAAKuS,OAAuE,GAA7DrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAK+7D,uBACvF/7D,KAAKwS,QAAuE,GAA7DtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKg8D,wBACvFh8D,KAAKk8D,gBAAkBl8D,KAAKuS,OAAS+sD,EAAS/sD,MAAQ,EAAImH,KAM9DnW,EAAK4P,UAAU6pD,SAAW,SAAUj2C,GAClC/mB,KAAKi9D,WAAWl2C,GAEhB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI04C,UAAUz/D,KAAKyH,KAAK,EAAEsf,EAAIO,UAAWtnB,KAAK2H,IAAI,EAAEof,EAAIO,UAAWtnB,KAAKuS,MAAM,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAO,EAAEuU,EAAIO,UAAWtnB,KAAKyO,QAAQgd,QACzI1E,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAEhJ4a,EAAI04C,UAAUz/D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,OAAQxS,KAAKyO,QAAQgd,QACzE1E,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAI5CzO,EAAK4P,UAAU4pD,gBAAkB,SAAUh2C,GACzC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,GAC5B1U,EAAOitD,EAAS/sD,MAAQ,EAAImH,CAChC1Z,MAAKuS,MAAQF,EACbrS,KAAKwS,OAASH,EAGdrS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQF,IAIxC9O,EAAK4P,UAAU2pD,cAAgB,SAAU/1C,GACvC/mB,KAAK+8D,gBAAgBh2C,GACrB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI24C,SAAS1/D,KAAK+R,EAAI/R,KAAKuS,MAAM,EAAI,EAAEwU,EAAIO,UAAWtnB,KAAKgS,EAAgB,GAAZhS,KAAKwS,OAAa,EAAEuU,EAAIO,UAAWtnB,KAAKuS,MAAQ,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAS,EAAEuU,EAAIO,WACpJP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAI24C,SAAS1/D,KAAK+R,EAAI/R,KAAKuS,MAAM,EAAGvS,KAAKgS,EAAgB,GAAZhS,KAAKwS,OAAYxS,KAAKuS,MAAOvS,KAAKwS,QAC/EuU,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS;EAI5CzO,EAAK4P,UAAUgqD,cAAgB,SAAUp2C,GACvC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,GAC5Bi4C,EAAW95D,KAAKwH,IAAI4yD,EAAS/sD,MAAO+sD,EAAS9sD,QAAU,EAAIkH,CAC/D1Z,MAAKyO,QAAQgd,OAASuzC,EAAW,EAEjCh/D,KAAKuS,MAAQysD,EACbh/D,KAAKwS,OAASwsD,EAKdh/D,KAAKyO,QAAQgd,QAAuE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC/Fj8D,KAAKk8D,gBAAkBl8D,KAAKyO,QAAQgd,OAAQ,GAAIuzC,IAIpDz7D,EAAK4P,UAAUgsD,eAAiB,SAAUp4C,EAAKhV,EAAGC,EAAGyZ,GACnD,GAAI8zC,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIq4C,OAAOrtD,EAAGC,EAAGyZ,EAAO,EAAE1E,EAAIO,WAC9BP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAIq4C,OAAOp/D,KAAK+R,EAAG/R,KAAKgS,EAAGyZ,GAC3B1E,EAAInH,OACJmH,EAAIlH,UAGNtc,EAAK4P,UAAU+pD,YAAc,SAAUn2C,GACrC/mB,KAAKm9D,cAAcp2C,GACnB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAElCxS,KAAKm/D,eAAep4C,EAAK/mB,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,QAEtDzrB,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAEhDzrB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAG5CzO,EAAK4P,UAAUkqD,eAAiB,SAAUt2C,GACxC,IAAK/mB,KAAKuS,MAAO,CACf,GAAI+sD,GAAWt/D,KAAK8+D,YAAY/3C,EAEhC/mB,MAAKuS,MAAyB,IAAjB+sD,EAAS/sD,MACtBvS,KAAKwS,OAA2B,EAAlB8sD,EAAS9sD,OACnBxS,KAAKuS,MAAQvS,KAAKwS,SACpBxS,KAAKuS,MAAQvS,KAAKwS,OAEpB,IAAImtD,GAAc3/D,KAAKuS,KAGvBvS,MAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAAUvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACzFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQotD,IAIxCp8D,EAAK4P,UAAUiqD,aAAe,SAAUr2C,GACtC/mB,KAAKq9D,eAAet2C,GACpB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,WAE9E+G,GAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAGtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAI64C,QAAQ5/D,KAAKyH,KAAK,EAAEsf,EAAIO,UAAWtnB,KAAK2H,IAAI,EAAEof,EAAIO,UAAWtnB,KAAKuS,MAAM,EAAEwU,EAAIO,UAAWtnB,KAAKwS,OAAO,EAAEuU,EAAIO,WAC/GP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAEhJ4a,EAAI64C,QAAQ5/D,KAAKyH,KAAMzH,KAAK2H,IAAK3H,KAAKuS,MAAOvS,KAAKwS,QAClDuU,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,OAE1CxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,IAG5CzO,EAAK4P,UAAUyqD,SAAW,SAAU72C,GAClC/mB,KAAK6/D,WAAW94C,EAAK,WAGvBxjB,EAAK4P,UAAU4qD,cAAgB,SAAUh3C,GACvC/mB,KAAK6/D,WAAW94C,EAAK,aAGvBxjB,EAAK4P,UAAU6qD,kBAAoB,SAAUj3C,GAC3C/mB,KAAK6/D,WAAW94C,EAAK,iBAGvBxjB,EAAK4P,UAAU2qD,YAAc,SAAU/2C,GACrC/mB,KAAK6/D,WAAW94C,EAAK,WAGvBxjB,EAAK4P,UAAU8qD,UAAY,SAAUl3C,GACnC/mB,KAAK6/D,WAAW94C,EAAK,SAGvBxjB,EAAK4P,UAAU0qD,aAAe,WAC5B,IAAK79D,KAAKuS,MAAO,CACfvS,KAAKyO,QAAQgd,OAAQzrB,KAAKm7D,eAC1B,IAAI9oD,GAAO,EAAIrS,KAAKyO,QAAQgd,MAC5BzrB,MAAKuS,MAAQF,EACbrS,KAAKwS,OAASH,EAGdrS,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAAsE,GAA7DvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAA+BjgD,KAAKi8D,wBAC9Fj8D,KAAKk8D,gBAAkBl8D,KAAKuS,MAAQF,IAIxC9O,EAAK4P,UAAU0sD,WAAa,SAAU94C,EAAKq2B,GACzCp9C,KAAK69D,aAAa92C,GAElB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,CAElC,IAAI+sD,GAAmB,IACnBv/C,EAAchgB,KAAKyO,QAAQuR,YAC3Bw/C,EAAqBx/D,KAAKyO,QAAQovC,qBAAuB,EAAI79C,KAAKyO,QAAQuR,YAC1E8/C,EAAmB,CAGvB,QAAQ1iB,GACN,IAAK,MAAiB0iB,EAAmB,CAAG,MAC5C,KAAK,SAAiBA,EAAmB,CAAG,MAC5C,KAAK,WAAiBA,EAAmB,CAAG,MAC5C,KAAK,eAAiBA,EAAmB,CAAG,MAC5C,KAAK,OAAiBA,EAAmB,EAG3C/4C,EAAIY,YAAc3nB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUD,OAASpM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMF,OAASpM,KAAKyO,QAAQtD,MAAMiB,OAEtIpM,KAAKq8D,YAAc,IACrBt1C,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIq2B,GAAOp9C,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,OAAQq0C,EAAmB/4C,EAAIO,WACvEP,EAAIlH,UAENkH,EAAIO,WAAatnB,KAAKqzC,SAAWmsB,EAAqBx/C,IAAiBhgB,KAAKq8D,YAAc,EAAKkD,EAAmB,GAClHx4C,EAAIO,WAAatnB,KAAKm2D,gBACtBpvC,EAAIO,UAAYpiB,KAAK4G,IAAI9L,KAAKuS,MAAMwU,EAAIO,WAExCP,EAAIiB,UAAYhoB,KAAKqzC,SAAWrzC,KAAKyO,QAAQtD,MAAMkB,UAAUF,WAAanM,KAAKsM,MAAQtM,KAAKyO,QAAQtD,MAAMmB,MAAMH,WAAanM,KAAKyO,QAAQtD,MAAMgB,WAChJ4a,EAAIq2B,GAAOp9C,KAAK+R,EAAG/R,KAAKgS,EAAGhS,KAAKyO,QAAQgd,QACxC1E,EAAInH,OACJmH,EAAIlH,SAEJ7f,KAAKomD,YAAYz+C,IAAM3H,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAC7CzrB,KAAKomD,YAAY3+C,KAAOzH,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC9CzrB,KAAKomD,YAAY/+B,MAAQrnB,KAAK+R,EAAI/R,KAAKyO,QAAQgd,OAC/CzrB,KAAKomD,YAAY9iC,OAAStjB,KAAKgS,EAAIhS,KAAKyO,QAAQgd,OAE5CzrB,KAAKyoB,QACPzoB,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,EAAIhS,KAAKwS,OAAS,EAAGhM,OAAW,WAAU,GACpFxG,KAAKomD,YAAY3+C,KAAOvC,KAAK4G,IAAI9L,KAAKomD,YAAY3+C,KAAMzH,KAAK8zD,gBAAgBrsD,MAC7EzH,KAAKomD,YAAY/+B,MAAQniB,KAAKwH,IAAI1M,KAAKomD,YAAY/+B,MAAOrnB,KAAK8zD,gBAAgBrsD,KAAOzH,KAAK8zD,gBAAgBvhD,OAC3GvS,KAAKomD,YAAY9iC,OAASpe,KAAKwH,IAAI1M,KAAKomD,YAAY9iC,OAAQtjB,KAAKomD,YAAY9iC,OAAStjB,KAAK8zD,gBAAgBthD,UAI/GjP,EAAK4P,UAAUwqD,YAAc,SAAU52C,GACrC,IAAK/mB,KAAKuS,MAAO,CACf,GAAImH,GAAS,EACT4lD,EAAWt/D,KAAK8+D,YAAY/3C,EAChC/mB,MAAKuS,MAAQ+sD,EAAS/sD,MAAQ,EAAImH,EAClC1Z,KAAKwS,OAAS8sD,EAAS9sD,OAAS,EAAIkH,EAGpC1Z,KAAKuS,OAAUrN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAK+7D,uBACjF/7D,KAAKwS,QAAUtN,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKg8D,wBACjFh8D,KAAKyO,QAAQgd,QAASvmB,KAAK4G,IAAI9L,KAAKq8D,YAAc,EAAGr8D,KAAKigD,uBAAyBjgD,KAAKi8D,wBACxFj8D,KAAKk8D,gBAAkBl8D,KAAKuS,OAAS+sD,EAAS/sD,MAAQ,EAAImH,KAI9DnW,EAAK4P,UAAUuqD,UAAY,SAAU32C,GACnC/mB,KAAK29D,YAAY52C,GACjB/mB,KAAKyH,KAAOzH,KAAK+R,EAAI/R,KAAKuS,MAAQ,EAClCvS,KAAK2H,IAAM3H,KAAKgS,EAAIhS,KAAKwS,OAAS,EAElCxS,KAAK+1D,OAAOhvC,EAAK/mB,KAAKyoB,MAAOzoB,KAAK+R,EAAG/R,KAAKgS,GAE1ChS,KAAKomD,YAAYz+C,IAAM3H,KAAK2H,IAC5B3H,KAAKomD,YAAY3+C,KAAOzH,KAAKyH,KAC7BzH,KAAKomD,YAAY/+B,MAAQrnB,KAAKyH,KAAOzH,KAAKuS,MAC1CvS,KAAKomD,YAAY9iC,OAAStjB,KAAK2H,IAAM3H,KAAKwS,QAI5CjP,EAAK4P,UAAU4iD,OAAS,SAAUhvC,EAAKwC,EAAMxX,EAAGC,EAAG+8B,EAAOgxB,EAAUC,GAClE,GAAIz2C,GAAQrlB,OAAOlE,KAAKyO,QAAQ8uC,UAAYv9C,KAAKm8D,aAAen8D,KAAKg7D,kBAAmB,CACtFj0C,EAAIQ,MAAQvnB,KAAKqzC,SAAW,QAAU,IAAMrzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAEzF,IAAIhX,GAAQjd,EAAKxhB,MAAM,MACnByuD,EAAYhwB,EAAM7gC,OAClB43C,EAAWr5C,OAAOlE,KAAKyO,QAAQ8uC,UAC/BwW,EAAQ/hD,GAAK,EAAIwkD,GAAa,EAAIjZ,CAChB,IAAlByiB,IACFjM,EAAQ/hD,GAAK,EAAIwkD,IAAc,EAAIjZ,GAKrC,KAAK,GADDhrC,GAAQwU,EAAI0vC,YAAYjwB,EAAM,IAAIj0B,MAC7B/M,EAAI,EAAOgxD,EAAJhxD,EAAeA,IAAK,CAClC,GAAI8hB,GAAYP,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,KAC1CA,GAAQ+U,EAAY/U,EAAQ+U,EAAY/U,EAE1C,GAAIC,GAASxS,KAAKyO,QAAQ8uC,SAAWiZ,EACjC/uD,EAAOsK,EAAIQ,EAAQ,EACnB5K,EAAMqK,EAAIQ,EAAS,CACP,YAAZutD,IACFp4D,GAAO,GAAM41C,EACb51C,GAAO,EACPosD,GAAS,GAEX/zD,KAAK8zD,iBAAmBnsD,IAAIA,EAAIF,KAAKA,EAAK8K,MAAMA,EAAMC,OAAOA,EAAOuhD,MAAMA,GAG5CvtD,SAA1BxG,KAAKyO,QAAQgvC,UAAoD,OAA1Bz9C,KAAKyO,QAAQgvC,UAA+C,SAA1Bz9C,KAAKyO,QAAQgvC,WACxF12B,EAAIiB,UAAYhoB,KAAKyO,QAAQgvC,SAC7B12B,EAAIkwC,SAASxvD,EAAME,EAAK4K,EAAOC,IAIjCuU,EAAIiB,UAAYhoB,KAAKyO,QAAQ6uC,WAAa,QAC1Cv2B,EAAIuB,UAAYymB,GAAS,SACzBhoB,EAAIwB,aAAew3C,GAAY,SAC3B//D,KAAKyO,QAAQivC,gBAAkB,IACjC32B,EAAIO,UAActnB,KAAKyO,QAAQivC,gBAC/B32B,EAAIY,YAAc3nB,KAAKyO,QAAQkvC,gBAC/B52B,EAAImwC,SAAc,QAEpB,KAAK,GAAI1xD,GAAI,EAAOgxD,EAAJhxD,EAAeA,IAC1BxF,KAAKyO,QAAQivC,iBACd32B,EAAIowC,WAAW3wB,EAAMhhC,GAAIuM,EAAGgiD,GAE9BhtC,EAAIyB,SAASge,EAAMhhC,GAAIuM,EAAGgiD,GAC1BA,GAASxW,IAMfh6C,EAAK4P,UAAU2rD,YAAc,SAAS/3C,GACpC,GAAmBvgB,SAAfxG,KAAKyoB,MAAqB,CAC5B1B,EAAIQ,MAAQvnB,KAAKqzC,SAAW,QAAU,IAAMrzC,KAAKyO,QAAQ8uC,SAAW,MAAQv9C,KAAKyO,QAAQ+uC,QAMzF,KAAK,GAJDhX,GAAQxmC,KAAKyoB,MAAM1gB,MAAM,MACzByK,GAAUtO,OAAOlE,KAAKyO,QAAQ8uC,UAAY,GAAK/W,EAAM7gC,OACrD4M,EAAQ,EAEH/M,EAAI,EAAGq7B,EAAO2F,EAAM7gC,OAAYk7B,EAAJr7B,EAAUA,IAC7C+M,EAAQrN,KAAKwH,IAAI6F,EAAOwU,EAAI0vC,YAAYjwB,EAAMhhC,IAAI+M,MAGpD,QAAQA,MAASA,EAAOC,OAAUA,EAAQgkD,UAAWhwB,EAAM7gC,QAG3D,OAAQ4M,MAAS,EAAGC,OAAU,EAAGgkD,UAAW,IAUhDjzD,EAAK4P,UAAU48C,OAAS,WACtB,MAAmBvpD,UAAfxG,KAAKuS,MACDvS,KAAK+R,EAAI/R,KAAKuS,MAAOvS,KAAKm2D,iBAAoBn2D,KAAK+jD,cAAchyC,GACjE/R,KAAK+R,EAAI/R,KAAKuS,MAAOvS,KAAKm2D,gBAAoBn2D,KAAKgkD,kBAAkBjyC,GACrE/R,KAAKgS,EAAIhS,KAAKwS,OAAOxS,KAAKm2D,iBAAoBn2D,KAAK+jD,cAAc/xC,GACjEhS,KAAKgS,EAAIhS,KAAKwS,OAAOxS,KAAKm2D,gBAAoBn2D,KAAKgkD,kBAAkBhyC,GAGpE,GAQXzO,EAAK4P,UAAU8sD,OAAS,WACtB,MAAQjgE,MAAK+R,GAAK/R,KAAK+jD,cAAchyC,GAC7B/R,KAAK+R,EAAI/R,KAAKgkD,kBAAkBjyC,GAChC/R,KAAKgS,GAAKhS,KAAK+jD,cAAc/xC,GAC7BhS,KAAKgS,EAAIhS,KAAKgkD,kBAAkBhyC,GAW1CzO,EAAK4P,UAAU28C,eAAiB,SAAS7yC,EAAM8mC,EAAcC,GAC3DhkD,KAAKm2D,gBAAkB,EAAIl5C,EAC3Bjd,KAAKm8D,aAAel/C,EACpBjd,KAAK+jD,cAAgBA,EACrB/jD,KAAKgkD,kBAAoBA,GAS3BzgD,EAAK4P,UAAU6vB,SAAW,SAAS/lB,GACjCjd,KAAKm2D,gBAAkB,EAAIl5C,EAC3Bjd,KAAKm8D,aAAel/C,GAQtB1Z,EAAK4P,UAAU+sD,cAAgB,WAC7BlgE,KAAKy7D,GAAK,EACVz7D,KAAK07D,GAAK,GASZn4D,EAAK4P,UAAUgtD,eAAiB,SAASC,GACvC,GAAIC,GAAergE,KAAKy7D,GAAKz7D,KAAKy7D,GAAK2E,CAEvCpgE,MAAKy7D,GAAKv2D,KAAKyqB,KAAK0wC,EAAargE,KAAKyO,QAAQwuC,MAC9CojB,EAAergE,KAAK07D,GAAK17D,KAAK07D,GAAK0E,EAEnCpgE,KAAK07D,GAAKx2D,KAAKyqB,KAAK0wC,EAAargE,KAAKyO,QAAQwuC,OAGhDp9C,EAAOD,QAAU2D,GAKb,SAAS1D,GAWb,QAAS2D,GAAM+V,EAAWxH,EAAGC,EAAGuX,EAAMtc,GAElCjN,KAAKuZ,UADHA,EACeA,EAGAhI,SAASojB,KAIdnuB,SAAVyG,IACe,gBAAN8E,IACT9E,EAAQ8E,EACRA,EAAIvL,QACqB,gBAAT+iB,IAChBtc,EAAQsc,EACRA,EAAO/iB,QAGPyG,GACEqwC,UAAW,QACXC,SAAU,GACVC,SAAU,UACVryC,OACEiB,OAAQ,OACRD,WAAY,aAMpBnM,KAAK+R,EAAI,EACT/R,KAAKgS,EAAI,EACThS,KAAKgkB,QAAU,EAELxd,SAANuL,GAAyBvL,SAANwL,GACrBhS,KAAK6tD,YAAY97C,EAAGC,GAETxL,SAAT+iB,GACFvpB,KAAK8tD,QAAQvkC,GAIfvpB,KAAKsf,MAAQ/N,SAASM,cAAc,MACpC,IAAIyuD,GAAYtgE,KAAKsf,MAAMrS,KAC3BqzD,GAAU18C,SAAW,WACrB08C,EAAU9oC,WAAa,SACvB8oC,EAAUl0D,OAAS,aAAea,EAAM9B,MAAMiB,OAC9Ck0D,EAAUn1D,MAAQ8B,EAAMqwC,UACxBgjB,EAAU/iB,SAAWtwC,EAAMswC,SAAW,KACtC+iB,EAAUC,WAAatzD,EAAMuwC,SAC7B8iB,EAAUt8C,QAAUhkB,KAAKgkB,QAAU,KACnCs8C,EAAU3gD,gBAAkB1S,EAAM9B,MAAMgB,WACxCm0D,EAAUtwC,aAAe,MACzBswC,EAAUxuC,gBAAkB,MAC5BwuC,EAAUE,mBAAqB,MAC/BF,EAAUrwC,UAAY,wCACtBqwC,EAAUG,WAAa,SACvBzgE,KAAKuZ,UAAU9H,YAAYzR,KAAKsf,OAOlC9b,EAAM2P,UAAU06C,YAAc,SAAS97C,EAAGC,GACxChS,KAAK+R,EAAInH,SAASmH,GAClB/R,KAAKgS,EAAIpH,SAASoH,IAOpBxO,EAAM2P,UAAU26C,QAAU,SAASl+B,GAC7BA,YAAmB+c,UACrB3sC,KAAKsf,MAAM2E,UAAY,GACvBjkB,KAAKsf,MAAM7N,YAAYme,IAGvB5vB,KAAKsf,MAAM2E,UAAY2L,GAQ3BpsB,EAAM2P,UAAUw0B,KAAO,SAAUA,GAK/B,GAJanhC,SAATmhC,IACFA,GAAO,GAGLA,EAAM,CACR,GAAIn1B,GAASxS,KAAKsf,MAAMuF,aACpBtS,EAASvS,KAAKsf,MAAME,YACpBgV,EAAYx0B,KAAKsf,MAAMzV,WAAWgb,aAClC62B,EAAW17C,KAAKsf,MAAMzV,WAAW2V,YAEjC7X,EAAO3H,KAAKgS,EAAIQ,CAChB7K,GAAM6K,EAASxS,KAAKgkB,QAAUwQ,IAChC7sB,EAAM6sB,EAAYhiB,EAASxS,KAAKgkB,SAE9Brc,EAAM3H,KAAKgkB,UACbrc,EAAM3H,KAAKgkB,QAGb,IAAIvc,GAAOzH,KAAK+R,CACZtK,GAAO8K,EAAQvS,KAAKgkB,QAAU03B,IAChCj0C,EAAOi0C,EAAWnpC,EAAQvS,KAAKgkB,SAE7Bvc,EAAOzH,KAAKgkB,UACdvc,EAAOzH,KAAKgkB,SAGdhkB,KAAKsf,MAAMrS,MAAMxF,KAAOA,EAAO,KAC/BzH,KAAKsf,MAAMrS,MAAMtF,IAAMA,EAAM,KAC7B3H,KAAKsf,MAAMrS,MAAMuqB,WAAa,cAG9Bx3B,MAAK0nC,QAOTlkC,EAAM2P,UAAUu0B,KAAO,WACrB1nC,KAAKsf,MAAMrS,MAAMuqB,WAAa,UAGhC33B,EAAOD,QAAU4D,GAKb,SAAS3D,EAAQD,GAarB,QAAS8gE,GAAUhuD,GAEjB,MADAod,GAAMpd,EACCiuD,IAoCT,QAAS1+B,KACP95B,EAAQ,EACR1H,EAAIqvB,EAAI1K,OAAO,GAQjB,QAASiD,KACPlgB,IACA1H,EAAIqvB,EAAI1K,OAAOjd,GAOjB,QAASy4D,KACP,MAAO9wC,GAAI1K,OAAOjd,EAAQ,GAS5B,QAAS04D,GAAepgE,GACtB,MAAOqgE,GAAkB9yD,KAAKvN,GAShC,QAASsgE,GAAOx7D,EAAGa,GAKjB,GAJKb,IACHA,MAGEa,EACF,IAAK,GAAI6P,KAAQ7P,GACXA,EAAEN,eAAemQ,KACnB1Q,EAAE0Q,GAAQ7P,EAAE6P,GAIlB,OAAO1Q,GAeT,QAASqS,GAASmL,EAAKooB,EAAM9jC,GAG3B,IAFA,GAAI+F,GAAO+9B,EAAKpjC,MAAM,KAClBi5D,EAAIj+C,EACD3V,EAAKzH,QAAQ,CAClB,GAAI+C,GAAM0E,EAAKkE,OACXlE,GAAKzH,QAEFq7D,EAAEt4D,KACLs4D,EAAEt4D,OAEJs4D,EAAIA,EAAEt4D,IAINs4D,EAAEt4D,GAAOrB,GAWf,QAAS45D,GAAQhwC,EAAO60B,GAOtB,IANA,GAAItgD,GAAGC,EACHq0B,EAAU,KAGVonC,GAAUjwC,GACVvxB,EAAOuxB,EACJvxB,EAAKolC,QACVo8B,EAAOl5D,KAAKtI,EAAKolC,QACjBplC,EAAOA,EAAKolC,MAId,IAAIplC,EAAKs9C,MACP,IAAKx3C,EAAI,EAAGC,EAAM/F,EAAKs9C,MAAMr3C,OAAYF,EAAJD,EAASA,IAC5C,GAAIsgD,EAAKzlD,KAAOX,EAAKs9C,MAAMx3C,GAAGnF,GAAI,CAChCy5B,EAAUp6B,EAAKs9C,MAAMx3C,EACrB,OAiBN,IAZKs0B,IAEHA,GACEz5B,GAAIylD,EAAKzlD,IAEP4wB,EAAM60B,OAERhsB,EAAQqnC,KAAOJ,EAAMjnC,EAAQqnC,KAAMlwC,EAAM60B,QAKxCtgD,EAAI07D,EAAOv7D,OAAS,EAAGH,GAAK,EAAGA,IAAK,CACvC,GAAIkF,GAAIw2D,EAAO17D,EAEVkF,GAAEsyC,QACLtyC,EAAEsyC,UAE4B,IAA5BtyC,EAAEsyC,MAAMr2C,QAAQmzB,IAClBpvB,EAAEsyC,MAAMh1C,KAAK8xB,GAKbgsB,EAAKqb,OACPrnC,EAAQqnC,KAAOJ,EAAMjnC,EAAQqnC,KAAMrb,EAAKqb,OAS5C,QAASC,GAAQnwC,EAAOy8B,GAKtB,GAJKz8B,EAAM6sB,QACT7sB,EAAM6sB,UAER7sB,EAAM6sB,MAAM91C,KAAK0lD,GACbz8B,EAAMy8B,KAAM,CACd,GAAIyT,GAAOJ,KAAU9vC,EAAMy8B,KAC3BA,GAAKyT,KAAOJ,EAAMI,EAAMzT,EAAKyT,OAajC,QAASE,GAAWpwC,EAAO7H,EAAMC,EAAIviB,EAAMq6D,GACzC,GAAIzT,IACFtkC,KAAMA,EACNC,GAAIA,EACJviB,KAAMA,EAQR,OALImqB,GAAMy8B,OACRA,EAAKyT,KAAOJ,KAAU9vC,EAAMy8B,OAE9BA,EAAKyT,KAAOJ,EAAMrT,EAAKyT,SAAYA,GAE5BzT,EAOT,QAAS4T,KAKP,IAJAC,EAAYC,EAAUC,KACtBC,EAAQ,GAGI,KAALjhE,GAAiB,KAALA,GAAkB,MAALA,GAAkB,MAALA,GAC3C4nB,GAGF,GAAG,CACD,GAAIs5C,IAAY,CAGhB,IAAS,KAALlhE,EAAU,CAGZ,IADA,GAAI+E,GAAI2C,EAAQ,EACQ,KAAjB2nB,EAAI1K,OAAO5f,IAA8B,KAAjBsqB,EAAI1K,OAAO5f,IACxCA,GAEF,IAAqB,MAAjBsqB,EAAI1K,OAAO5f,IAA+B,IAAjBsqB,EAAI1K,OAAO5f,GAAU,CAEhD,KAAY,IAAL/E,GAAgB,MAALA,GAChB4nB,GAEFs5C,IAAY,GAGhB,GAAS,KAALlhE,GAA6B,KAAjBmgE,IAAsB,CAEpC,KAAY,IAALngE,GAAgB,MAALA,GAChB4nB,GAEFs5C,IAAY,EAEd,GAAS,KAALlhE,GAA6B,KAAjBmgE,IAAsB,CAEpC,KAAY,IAALngE,GAAS,CACd,GAAS,KAALA,GAA6B,KAAjBmgE,IAAsB,CAEpCv4C,IACAA,GACA,OAGAA,IAGJs5C,GAAY,EAId,KAAY,KAALlhE,GAAiB,KAALA,GAAkB,MAALA,GAAkB,MAALA,GAC3C4nB,UAGGs5C,EAGP,IAAS,IAALlhE,EAGF,YADA8gE,EAAYC,EAAUI,UAKxB,IAAIC,GAAKphE,EAAImgE,GACb,IAAIkB,EAAWD,GAKb,MAJAN,GAAYC,EAAUI,UACtBF,EAAQG,EACRx5C,QACAA,IAKF,IAAIy5C,EAAWrhE,GAIb,MAHA8gE,GAAYC,EAAUI,UACtBF,EAAQjhE,MACR4nB,IAMF,IAAIw4C,EAAepgE,IAAW,KAALA,EAAU,CAIjC,IAHAihE,GAASjhE,EACT4nB,IAEOw4C,EAAepgE,IACpBihE,GAASjhE,EACT4nB,GAYF,OAVa,SAATq5C,EACFA,GAAQ,EAEQ,QAATA,EACPA,GAAQ,EAEAh9D,MAAMR,OAAOw9D,MACrBA,EAAQx9D,OAAOw9D,SAEjBH,EAAYC,EAAUO,YAKxB,GAAS,KAALthE,EAAU,CAEZ,IADA4nB,IACY,IAAL5nB,IAAiB,KAALA,GAAkB,KAALA,GAA6B,KAAjBmgE,MAC1Cc,GAASjhE,EACA,KAALA,GACF4nB,IAEFA,GAEF,IAAS,KAAL5nB,EACF,KAAMuhE,GAAe,2BAIvB,OAFA35C,UACAk5C,EAAYC,EAAUO,YAMxB,IADAR,EAAYC,EAAUS,QACV,IAALxhE,GACLihE,GAASjhE,EACT4nB,GAEF,MAAM,IAAI7O,aAAY,yBAA2B0oD,EAAKR,EAAO,IAAM,KAOrE,QAASf,KACP,GAAI1vC,KAwBJ,IAtBAgR,IACAq/B,IAGa,UAATI,IACFzwC,EAAMkxC,QAAS,EACfb,MAIW,SAATI,GAA6B,WAATA,KACtBzwC,EAAMnqB,KAAO46D,EACbJ,KAIEC,GAAaC,EAAUO,aACzB9wC,EAAM5wB,GAAKqhE,EACXJ,KAIW,KAATI,EACF,KAAMM,GAAe,2BAQvB,IANAV,IAGAc,EAAgBnxC,GAGH,KAATywC,EACF,KAAMM,GAAe,2BAKvB,IAHAV,IAGc,KAAVI,EACF,KAAMM,GAAe,uBASvB,OAPAV,WAGOrwC,GAAM60B,WACN70B,GAAMy8B,WACNz8B,GAAMA,MAENA,EAOT,QAASmxC,GAAiBnxC,GACxB,KAAiB,KAAVywC,GAAyB,KAATA,GACrBW,EAAepxC,GACF,KAATywC,GACFJ,IAWN,QAASe,GAAepxC,GAEtB,GAAIqxC,GAAWC,EAActxC,EAC7B,IAAIqxC,EAIF,WAFAE,GAAUvxC,EAAOqxC,EAMnB,IAAInB,GAAOsB,EAAwBxxC,EACnC,KAAIkwC,EAAJ,CAKA,GAAII,GAAaC,EAAUO,WACzB,KAAMC,GAAe,sBAEvB,IAAI3hE,GAAKqhE,CAGT,IAFAJ,IAEa,KAATI,EAAc,CAGhB,GADAJ,IACIC,GAAaC,EAAUO,WACzB,KAAMC,GAAe,sBAEvB/wC,GAAM5wB,GAAMqhE,EACZJ,QAIAoB,GAAmBzxC,EAAO5wB,IAS9B,QAASkiE,GAAetxC,GACtB,GAAIqxC,GAAW,IAgBf,IAba,YAATZ,IACFY,KACAA,EAASx7D,KAAO,WAChBw6D,IAGIC,GAAaC,EAAUO,aACzBO,EAASjiE,GAAKqhE,EACdJ,MAKS,KAATI,EAAc,CAehB,GAdAJ,IAEKgB,IACHA,MAEFA,EAASx9B,OAAS7T,EAClBqxC,EAASxc,KAAO70B,EAAM60B,KACtBwc,EAAS5U,KAAOz8B,EAAMy8B,KACtB4U,EAASrxC,MAAQA,EAAMA,MAGvBmxC,EAAgBE,GAGH,KAATZ,EACF,KAAMM,GAAe,2BAEvBV,WAGOgB,GAASxc,WACTwc,GAAS5U,WACT4U,GAASrxC,YACTqxC,GAASx9B,OAGX7T,EAAM0xC,YACT1xC,EAAM0xC,cAER1xC,EAAM0xC,UAAU36D,KAAKs6D,GAGvB,MAAOA,GAYT,QAASG,GAAyBxxC,GAEhC,MAAa,QAATywC,GACFJ,IAGArwC,EAAM60B,KAAO8c,IACN,QAES,QAATlB,GACPJ,IAGArwC,EAAMy8B,KAAOkV,IACN,QAES,SAATlB,GACPJ,IAGArwC,EAAMA,MAAQ2xC,IACP,SAGF,KAQT,QAASF,GAAmBzxC,EAAO5wB,GAEjC,GAAIylD,IACFzlD,GAAIA,GAEF8gE,EAAOyB,GACPzB,KACFrb,EAAKqb,KAAOA,GAEdF,EAAQhwC,EAAO60B,GAGf0c,EAAUvxC,EAAO5wB,GAQnB,QAASmiE,GAAUvxC,EAAO7H,GACxB,KAAgB,MAATs4C,GAA0B,MAATA,GAAe,CACrC,GAAIr4C,GACAviB,EAAO46D,CACXJ,IAEA,IAAIgB,GAAWC,EAActxC,EAC7B,IAAIqxC,EACFj5C,EAAKi5C,MAEF,CACH,GAAIf,GAAaC,EAAUO,WACzB,KAAMC,GAAe,kCAEvB34C,GAAKq4C,EACLT,EAAQhwC,GACN5wB,GAAIgpB,IAENi4C,IAIF,GAAIH,GAAOyB,IAGPlV,EAAO2T,EAAWpwC,EAAO7H,EAAMC,EAAIviB,EAAMq6D,EAC7CC,GAAQnwC,EAAOy8B,GAEftkC,EAAOC,GASX,QAASu5C,KAGP,IAFA,GAAIzB,GAAO,KAEK,KAATO,GAAc,CAGnB,IAFAJ,IACAH,KACiB,KAAVO,GAAyB,KAATA,GAAc,CACnC,GAAIH,GAAaC,EAAUO,WACzB,KAAMC,GAAe,0BAEvB,IAAI/rD,GAAOyrD,CAGX,IADAJ,IACa,KAATI,EACF,KAAMM,GAAe,wBAIvB,IAFAV,IAEIC,GAAaC,EAAUO,WACzB,KAAMC,GAAe,2BAEvB,IAAI36D,GAAQq6D,CACZ9pD,GAASupD,EAAMlrD,EAAM5O,GAErBi6D,IACY,KAARI,GACFJ,IAIJ,GAAa,KAATI,EACF,KAAMM,GAAe,qBAEvBV,KAGF,MAAOH,GAQT,QAASa,GAAea,GACtB,MAAO,IAAIrpD,aAAYqpD,EAAU,UAAYX,EAAKR,EAAO,IAAM,WAAav5D,EAAQ,KAStF,QAAS+5D,GAAM34C,EAAMu5C,GACnB,MAAQv5C,GAAK5jB,QAAUm9D,EAAav5C,EAAQA,EAAKje,OAAO,EAAG,IAAM,MASnE,QAASy3D,GAASC,EAAQC,EAAQ/pD,GAC5BjT,MAAMC,QAAQ88D,GAChBA,EAAO36D,QAAQ,SAAU66D,GACnBj9D,MAAMC,QAAQ+8D,GAChBA,EAAO56D,QAAQ,SAAU86D,GACvBjqD,EAAGgqD,EAAOC,KAIZjqD,EAAGgqD,EAAOD,KAKVh9D,MAAMC,QAAQ+8D,GAChBA,EAAO56D,QAAQ,SAAU86D,GACvBjqD,EAAG8pD,EAAQG,KAIbjqD,EAAG8pD,EAAQC,GAWjB,QAAS5b,GAAY30C,GAEnB,GAAI00C,GAAUsZ,EAAShuD,GACnB0wD,GACFpmB,SACAc,SACArvC,WAmBF,IAfI24C,EAAQpK,OACVoK,EAAQpK,MAAM30C,QAAQ,SAAUg7D,GAC9B,GAAIC,IACFjjE,GAAIgjE,EAAQhjE,GACZooB,MAAOrkB,OAAOi/D,EAAQ56C,OAAS46C,EAAQhjE,IAEzC0gE,GAAMuC,EAAWD,EAAQlC,MACrBmC,EAAUjmB,QACZimB,EAAUlmB,MAAQ,SAEpBgmB,EAAUpmB,MAAMh1C,KAAKs7D,KAKrBlc,EAAQtJ,MAAO,CAMjB,GAAIylB,GAAc,SAAUC,GAC1B,GAAIC,IACFr6C,KAAMo6C,EAAQp6C,KACdC,GAAIm6C,EAAQn6C,GAId,OAFA03C,GAAM0C,EAAWD,EAAQrC,MACzBsC,EAAUx2D,MAAyB,MAAhBu2D,EAAQ18D,KAAgB,QAAU,OAC9C28D,EAGTrc,GAAQtJ,MAAMz1C,QAAQ,SAAUm7D,GAC9B,GAAIp6C,GAAMC,CAERD,GADEo6C,EAAQp6C,eAAgB7iB,QACnBi9D,EAAQp6C,KAAK4zB,OAIlB38C,GAAImjE,EAAQp6C,MAKdC,EADEm6C,EAAQn6C,aAAc9iB,QACnBi9D,EAAQn6C,GAAG2zB,OAId38C,GAAImjE,EAAQn6C,IAIZm6C,EAAQp6C,eAAgB7iB,SAAUi9D,EAAQp6C,KAAK00B,OACjD0lB,EAAQp6C,KAAK00B,MAAMz1C,QAAQ,SAAUq7D,GACnC,GAAID,GAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,KAIzBV,EAAS35C,EAAMC,EAAI,SAAUD,EAAMC,GACjC,GAAIq6C,GAAUrC,EAAW+B,EAAWh6C,EAAK/oB,GAAIgpB,EAAGhpB,GAAImjE,EAAQ18D,KAAM08D,EAAQrC,MACtEsC,EAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,KAGnBD,EAAQn6C,aAAc9iB,SAAUi9D,EAAQn6C,GAAGy0B,OAC7C0lB,EAAQn6C,GAAGy0B,MAAMz1C,QAAQ,SAAUq7D,GACjC,GAAID,GAAYF,EAAYG,EAC5BN,GAAUtlB,MAAM91C,KAAKy7D,OAW7B,MAJIrc,GAAQ+Z,OACViC,EAAU30D,QAAU24C,EAAQ+Z,MAGvBiC,EAnyBT,GAAI5B,IACFC,KAAO,EACPG,UAAY,EACZG,WAAY,EACZE,QAAU,GAIRH,GACF6B,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EACLC,KAAK,EAELC,MAAM,EACNC,MAAM,GAGJr0C,EAAM,GACN3nB,EAAQ,EACR1H,EAAI,GACJihE,EAAQ,GACRH,EAAYC,EAAUC,KAmCtBX,EAAoB,iBA2uBxBlhE,GAAQ8gE,SAAWA,EACnB9gE,EAAQynD,WAAaA,GAKjB,SAASxnD,EAAQD,GAGrB,QAAS4nD,GAAW4c,EAAW31D,GAC7B,GAAIqvC,MACAd,IACJh9C,MAAKyO,SACHqvC,OACEQ,cAAc,GAEhBtB,OACEqnB,eAAe,EACfn5D,YAAY,IAIA1E,SAAZiI,IACFzO,KAAKyO,QAAQuuC,MAAqB,cAAIvuC,EAAQ41D,eAAgB,EAC9DrkE,KAAKyO,QAAQuuC,MAAkB,WAAOvuC,EAAQvD,YAAgB,EAC9DlL,KAAKyO,QAAQqvC,MAAoB,aAAKrvC,EAAQ6vC,cAAgB,EAKhE,KAAK,GAFDgmB,GAASF,EAAUtmB,MACnBymB,EAASH,EAAUpnB,MACdx3C,EAAI,EAAGA,EAAI8+D,EAAO3+D,OAAQH,IAAK,CACtC,GAAIkoD,MACA8W,EAAQF,EAAO9+D,EACnBkoD,GAAS,GAAI8W,EAAMnkE,GACnBqtD,EAAW,KAAI8W,EAAMC,OACrB/W,EAAS,GAAI8W,EAAM96D,OACnBgkD,EAAiB,WAAI8W,EAAMvpB,WAG3ByS,EAAY,MAAI8W,EAAMr5D,MACtBuiD,EAAmB,aAAsBlnD,SAAlBknD,EAAY,OAAkB,EAAQ1tD,KAAKyO,QAAQ6vC,aAC1ER,EAAM91C,KAAK0lD,GAGb,IAAK,GAAIloD,GAAI,EAAGA,EAAI++D,EAAO5+D,OAAQH,IAAK,CACtC,GAAIsgD,MACA4e,EAAQH,EAAO/+D,EACnBsgD,GAAS,GAAI4e,EAAMrkE,GACnBylD,EAAiB,WAAI4e,EAAMzpB,WAC3B6K,EAAQ,EAAI4e,EAAM3yD,EAClB+zC,EAAQ,EAAI4e,EAAM1yD,EAClB8zC,EAAY,MAAI4e,EAAMj8C,MAEpBq9B,EAAY,MADuB,GAAjC9lD,KAAKyO,QAAQuuC,MAAM9xC,WACLw5D,EAAMv5D,MAGU3E,SAAhBk+D,EAAMv5D,OAAuBgB,WAAWu4D,EAAMv5D,MAAOiB,OAAOs4D,EAAMv5D,OAAS3E,OAE7Fs/C,EAAa,OAAI4e,EAAMryD,KACvByzC,EAAqB,eAAI9lD,KAAKyO,QAAQuuC,MAAMqnB,cAC5Cve,EAAqB,eAAI9lD,KAAKyO,QAAQuuC,MAAMqnB,cAC5CrnB,EAAMh1C,KAAK89C,GAGb,OAAQ9I,MAAMA,EAAOc,MAAMA,GAG7Bl+C,EAAQ4nD,WAAaA,GAIjB,SAAS3nD,EAAQD,EAASM,GAI9BL,EAAOD,QAA6B,mBAAX2J,SAA2BA,OAAe,QAAKrJ,EAAoB,KAKxF,SAASL,EAAQD,EAASM,GAI9B,GAAsB,mBAAXqJ,QAAwB,CACjC,GAAIo7D,GAAczkE,EAAoB,IAClC6D,EAASwF,OAAe,QAAKrJ,EAAoB,GACrDL,GAAOD,QAAU+kE,EAAY5gE,OAG7BlE,GAAOD,QAAU,WACf,KAAMgE,OAAM,+DAOZ,SAAS/D,EAAQD,EAASM,GAmB9B,QAASg2B,MAjBT,GAAInZ,GAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,GAK3BolD,GAJUplD,EAAoB,GACnBA,EAAoB,GACvBA,EAAoB,IAClBA,EAAoB,IAClBA,EAAoB,KAChCyB,EAAWzB,EAAoB,GAYnC6c,GAAQmZ,EAAK/iB,WASb+iB,EAAK/iB,UAAUuhB,QAAU,SAAUnb,GAmHjC,QAASqrD,GAAat7D,GAChB6K,EAAGs1C,YACLt1C,EAAGyZ,KAAK,aAActkB,GApH1BtJ,KAAK+vB,OAEL/vB,KAAK+vB,IAAIrwB,KAAuB6R,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI5jB,WAAuBoF,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIgV,mBAAuBxzB,SAASM,cAAc,OACvD7R,KAAK+vB,IAAImY,qBAAuB32B,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI8H,gBAAuBtmB,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI80C,cAAuBtzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI+0C,eAAuBvzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI7D,OAAuB3a,SAASM,cAAc,OACvD7R,KAAK+vB,IAAItoB,KAAuB8J,SAASM,cAAc,OACvD7R,KAAK+vB,IAAI1I,MAAuB9V,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIpoB,IAAuB4J,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIzM,OAAuB/R,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIg1C,UAAuBxzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIi1C,aAAuBzzD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIk1C,cAAuB1zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIm1C,iBAAuB3zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIo1C,eAAuB5zD,SAASM,cAAc,OACvD7R,KAAK+vB,IAAIq1C,kBAAuB7zD,SAASM,cAAc,OAEvD7R,KAAK+vB,IAAIrwB,KAAKmI,UAA4B,oBAC1C7H,KAAK+vB,IAAI5jB,WAAWtE,UAAsB,sBAC1C7H,KAAK+vB,IAAIgV,mBAAmBl9B,UAAc,+BAC1C7H,KAAK+vB,IAAImY,qBAAqBrgC,UAAY,iCAC1C7H,KAAK+vB,IAAI8H,gBAAgBhwB,UAAiB,kBAC1C7H,KAAK+vB,IAAI80C,cAAch9D,UAAmB,gBAC1C7H,KAAK+vB,IAAI+0C,eAAej9D,UAAkB,iBAC1C7H,KAAK+vB,IAAIpoB,IAAIE,UAA6B,eAC1C7H,KAAK+vB,IAAIzM,OAAOzb,UAA0B,kBAC1C7H,KAAK+vB,IAAItoB,KAAKI,UAA4B,UAC1C7H,KAAK+vB,IAAI7D,OAAOrkB,UAA0B,UAC1C7H,KAAK+vB,IAAI1I,MAAMxf,UAA2B,UAC1C7H,KAAK+vB,IAAIg1C,UAAUl9D,UAAuB,aAC1C7H,KAAK+vB,IAAIi1C,aAAan9D,UAAoB,gBAC1C7H,KAAK+vB,IAAIk1C,cAAcp9D,UAAmB,aAC1C7H,KAAK+vB,IAAIm1C,iBAAiBr9D,UAAgB,gBAC1C7H,KAAK+vB,IAAIo1C,eAAet9D,UAAkB,aAC1C7H,KAAK+vB,IAAIq1C,kBAAkBv9D,UAAe,gBAE1C7H,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI5jB,YACnCnM,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIgV,oBACnC/kC,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAImY,sBACnCloC,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI8H,iBACnC73B,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI80C,eACnC7kE,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAI+0C,gBACnC9kE,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIpoB,KACnC3H,KAAK+vB,IAAIrwB,KAAK+R,YAAYzR,KAAK+vB,IAAIzM,QAEnCtjB,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAI7D,QAC9ClsB,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAItoB,MAC5CzH,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAI1I,OAE7CrnB,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAIg1C,WAC9C/kE,KAAK+vB,IAAI8H,gBAAgBpmB,YAAYzR,KAAK+vB,IAAIi1C,cAC9ChlE,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAIk1C,eAC5CjlE,KAAK+vB,IAAI80C,cAAcpzD,YAAYzR,KAAK+vB,IAAIm1C,kBAC5CllE,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAIo1C,gBAC7CnlE,KAAK+vB,IAAI+0C,eAAerzD,YAAYzR,KAAK+vB,IAAIq1C,mBAE7CplE,KAAKuT,GAAG,cAAevT,KAAKyhB,OAAOqT,KAAK90B,MAExC,IAAImU,GAAKnU,IACTA,MAAKuT,GAAG,SAAU,SAAUo7C,GACtBA,GAAkC,GAApBA,EAAWv7C,MAEtBe,EAAGkxD,eACNlxD,EAAGkxD,aAAe/rD,WAAW,WAC3BnF,EAAGkxD,aAAe,KAClBlxD,EAAGsN,UACF,IAKLtN,EAAGsN,WAMPzhB,KAAK8D,OAAS,GAAIC,GAAO/D,KAAK+vB,IAAIrwB,MAAO4lE,YAAa,UACtDtlE,KAAK8D,OAAOoR,IAAI,SAASi0C,KAAK/lB,QAAQ,IACtCpjC,KAAKulE,YAEL,IAAIC,IACF,MAAO,YAAa,QACpB,QACA,MAAO,WAAY,UAAW,SAsDhC,IA/CAA,EAAOn9D,QAAQ,SAAUvB,GACvB,GAAIgC,GAAW,SAAUQ,GACnB6K,EAAGs1C,YACLt1C,EAAGyZ,KAAK9mB,EAAMwC,GAGlB6K,GAAGrQ,OAAOyP,GAAGzM,EAAMgC,GACnBqL,EAAGoxD,UAAUz+D,GAAQgC,IAIvB9I,KAAK8D,OAAOyP,GAAG,eAAgB,SAAUjK,GACnCA,EAAMqnC,SACJx8B,EAAGs1C,YACLt1C,EAAGyZ,KAAK,QAAStkB,IAGrBwrB,KAAK90B,OAOPA,KAAK+vB,IAAIrwB,KAAKiJ,iBAAiB,aAAci8D,GAC7C5kE,KAAK+vB,IAAIrwB,KAAKiJ,iBAAiB,iBAAkBi8D,GAGjD5kE,KAAKgG,OACHtG,QACAyM,cACA0rB,mBACAgtC,iBACAC,kBACA54C,UACAzkB,QACA4f,SACA1f,OACA2b,UACAlX,UACAi7B,UAAW,EACXo+B,aAAc,GAGhBzlE,KAAK0lE,YAAc,GAGdnsD,EAAW,KAAM,IAAI3V,OAAM,wBAChC2V,GAAU9H,YAAYzR,KAAK+vB,IAAIrwB,OA4BjCw2B,EAAK/iB,UAAUD,WAAa,SAAUzE,GACpC,GAAIA,EAAS,CAEX,GAAIP,IAAU,QAAS,SAAU,YAAa,YAAa,aAAc,QAAS,MAAO,cAAe,aAAc,iBAAkB,cACxIvN,GAAKoF,gBAAgBmI,EAAQlO,KAAKyO,QAASA,GAEvC,eAAiBzO,MAAKyO,SACxB9M,EAAS81B,qBAAqBz3B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAGpD,cAAgBtmB,KACdA,EAAQ45C,WACLroD,KAAKsoD,YACRtoD,KAAKsoD,UAAY,GAAIhD,GAAUtlD,KAAK+vB,IAAIrwB,OAItCM,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,YAMlBtoD,KAAK2lE,kBASP,GALA3lE,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAU1yD,WAAWzE,KAInBA,GAAWA,EAAQgH,MACrB,KAAM,IAAI7R,OAAM,wEAIlB5D,MAAKyhB,UAOPyU,EAAK/iB,UAAUs2C,SAAW,WACxB,OAAQzpD,KAAKsoD,WAAatoD,KAAKsoD,UAAUgL,QAM3Cp9B,EAAK/iB,UAAUG,QAAU,WAEvBtT,KAAKyW,QAGLzW,KAAK0T,MAGL1T,KAAK6lE,kBAGD7lE,KAAK+vB,IAAIrwB,KAAKmK,YAChB7J,KAAK+vB,IAAIrwB,KAAKmK,WAAWsH,YAAYnR,KAAK+vB,IAAIrwB,MAEhDM,KAAK+vB,IAAM,KAGP/vB,KAAKsoD,YACPtoD,KAAKsoD,UAAUh1C,gBACRtT,MAAKsoD,UAId,KAAK,GAAIh/C,KAAStJ,MAAKulE,UACjBvlE,KAAKulE,UAAUz/D,eAAewD,UACzBtJ,MAAKulE,UAAUj8D,EAG1BtJ,MAAKulE,UAAY,KACjBvlE,KAAK8D,OAAS,KAGd9D,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAUtyD,YAGZtT,KAAK20B,KAAO,MAQduB,EAAK/iB,UAAUsyB,cAAgB,SAAUtL,GACvC,IAAKn6B,KAAK41B,WACR,KAAM,IAAIhyB,OAAM,yDAGlB5D,MAAK41B,WAAW6P,cAActL,IAOhCjE,EAAK/iB,UAAUuyB,cAAgB,WAC7B,IAAK1lC,KAAK41B,WACR,KAAM,IAAIhyB,OAAM,yDAGlB,OAAO5D,MAAK41B,WAAW8P,iBAQzBxP,EAAK/iB,UAAUi+B,gBAAkB,WAC/B,MAAOpxC,MAAK61B,SAAW71B,KAAK61B,QAAQub,uBAetClb,EAAK/iB,UAAUsD,MAAQ,SAASqvD,KAEzBA,GAAQA,EAAK7jE,QAChBjC,KAAKi2B,SAAS,QAIX6vC,GAAQA,EAAK3xC,SAChBn0B,KAAKg2B,UAAU,QAIZ8vC,GAAQA,EAAKr3D,WAChBzO,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCA,EAAU1yD,WAAW0yD,EAAUvxC,kBAGjCr0B,KAAKkT,WAAWlT,KAAKq0B,kBAazB6B,EAAK/iB,UAAUsjB,IAAM,SAAShoB,GAC5B,GAAIgnB,GAAQz1B,KAAKs2B,eAGjB,IAAoB,OAAhBb,EAAM7lB,OAAgC,OAAd6lB,EAAM5lB,IAAlC,CAIA,GAAI2mB,GAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7Ex2B,MAAKy1B,MAAMlC,SAASkC,EAAM7lB,MAAO6lB,EAAM5lB,IAAK2mB,KAQ9CN,EAAK/iB,UAAUmjB,cAAgB,WAE7B,GAAID,GAAYr2B,KAAK+2B,eAGjBnnB,EAAQymB,EAAUvqB,IAClB+D,EAAMwmB,EAAU3pB,GACpB,IAAa,MAATkD,GAAwB,MAAPC,EAAa,CAChC,GAAI2iB,GAAY3iB,EAAI7I,UAAY4I,EAAM5I,SACtB,IAAZwrB,IAEFA,EAAW,OAEb5iB,EAAQ,GAAItL,MAAKsL,EAAM5I,UAAuB,IAAXwrB,GACnC3iB,EAAM,GAAIvL,MAAKuL,EAAI7I,UAAuB,IAAXwrB,GAGjC,OACE5iB,MAAOA,EACPC,IAAKA,IAuBTqmB,EAAK/iB,UAAUojB,UAAY,SAAS3mB,EAAOC,EAAKpB,GAC9C,GAAI+nB,GAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAC7E,IAAwB,GAApB9wB,UAAUC,OAAa,CACzB,GAAI8vB,GAAQ/vB,UAAU,EACtB1F,MAAKy1B,MAAMlC,SAASkC,EAAM7lB,MAAO6lB,EAAM5lB,IAAK2mB,OAG5Cx2B,MAAKy1B,MAAMlC,SAAS3jB,EAAOC,EAAK2mB,IAcpCN,EAAK/iB,UAAU0U,OAAS,SAASsS,EAAM1rB,GACrC,GAAI+jB,GAAWxyB,KAAKy1B,MAAM5lB,IAAM7P,KAAKy1B,MAAM7lB,MACvC9B,EAAInN,EAAKkG,QAAQszB,EAAM,QAAQnzB,UAE/B4I,EAAQ9B,EAAI0kB,EAAW,EACvB3iB,EAAM/B,EAAI0kB,EAAW,EACrBgE,EAAW/nB,GAA+BjI,SAApBiI,EAAQ+nB,QAAyB/nB,EAAQ+nB,SAAU,CAE7Ex2B,MAAKy1B,MAAMlC,SAAS3jB,EAAOC,EAAK2mB,IAOlCN,EAAK/iB,UAAU4yD,UAAY,WACzB,GAAItwC,GAAQz1B,KAAKy1B,MAAM2J,UACvB,QACExvB,MAAO,GAAItL,MAAKmxB,EAAM7lB,OACtBC,IAAK,GAAIvL,MAAKmxB,EAAM5lB,OAQxBqmB,EAAK/iB,UAAUsO,OAAS,WACtB,GAAIgjB,IAAU,EACVh2B,EAAUzO,KAAKyO,QACfzI,EAAQhG,KAAKgG,MACb+pB,EAAM/vB,KAAK+vB,GAEf,IAAKA,EAAL,CAEApuB,EAASi2B,kBAAkB53B,KAAK20B,KAAM30B,KAAKyO,QAAQsmB,aAGxB,OAAvBtmB,EAAQ8lB,aACV5zB,EAAKiH,aAAamoB,EAAIrwB,KAAM,OAC5BiB,EAAKuH,gBAAgB6nB,EAAIrwB,KAAM,YAG/BiB,EAAKuH,gBAAgB6nB,EAAIrwB,KAAM,OAC/BiB,EAAKiH,aAAamoB,EAAIrwB,KAAM,WAI9BqwB,EAAIrwB,KAAKuN,MAAMunB,UAAY7zB,EAAKmJ,OAAOK,OAAOsE,EAAQ+lB,UAAW,IACjEzE,EAAIrwB,KAAKuN,MAAMwnB,UAAY9zB,EAAKmJ,OAAOK,OAAOsE,EAAQgmB,UAAW,IACjE1E,EAAIrwB,KAAKuN,MAAMsF,MAAQ5R,EAAKmJ,OAAOK,OAAOsE,EAAQ8D,MAAO,IAGzDvM,EAAMoG,OAAO3E,MAAUsoB,EAAI8H,gBAAgBzH,YAAcL,EAAI8H,gBAAgBrY,aAAe,EAC5FxZ,EAAMoG,OAAOib,MAASrhB,EAAMoG,OAAO3E,KACnCzB,EAAMoG,OAAOzE,KAAUooB,EAAI8H,gBAAgBvH,aAAeP,EAAI8H,gBAAgBhT,cAAgB,EAC9F7e,EAAMoG,OAAOkX,OAAStd,EAAMoG,OAAOzE,GACnC,IAAIq+D,GAAkBj2C,EAAIrwB,KAAK4wB,aAAeP,EAAIrwB,KAAKmlB,aACnDohD,EAAkBl2C,EAAIrwB,KAAK0wB,YAAcL,EAAIrwB,KAAK8f,WAIb,KAArCuQ,EAAI8H,gBAAgBhT,eACtB7e,EAAMoG,OAAO3E,KAAOzB,EAAMoG,OAAOzE,IACjC3B,EAAMoG,OAAOib,MAASrhB,EAAMoG,OAAO3E,MAEP,IAA1BsoB,EAAIrwB,KAAKmlB,eACXohD,EAAkBD,GAKpBhgE,EAAMkmB,OAAO1Z,OAASud,EAAI7D,OAAOoE,aACjCtqB,EAAMyB,KAAK+K,OAAWud,EAAItoB,KAAK6oB,aAC/BtqB,EAAMqhB,MAAM7U,OAAUud,EAAI1I,MAAMiJ,aAChCtqB,EAAM2B,IAAI6K,OAAYud,EAAIpoB,IAAIkd,eAAoB7e,EAAMoG,OAAOzE,IAC/D3B,EAAMsd,OAAO9Q,OAASud,EAAIzM,OAAOuB,eAAiB7e,EAAMoG,OAAOkX,MAM/D,IAAI+M,GAAgBnrB,KAAKwH,IAAI1G,EAAMyB,KAAK+K,OAAQxM,EAAMkmB,OAAO1Z,OAAQxM,EAAMqhB,MAAM7U,QAC7E0zD,EAAalgE,EAAM2B,IAAI6K,OAAS6d,EAAgBrqB,EAAMsd,OAAO9Q,OAC/DwzD,EAAmBhgE,EAAMoG,OAAOzE,IAAM3B,EAAMoG,OAAOkX,MACrDyM,GAAIrwB,KAAKuN,MAAMuF,OAAS7R,EAAKmJ,OAAOK,OAAOsE,EAAQ+D,OAAQ0zD,EAAa,MAGxElgE,EAAMtG,KAAK8S,OAASud,EAAIrwB,KAAK4wB,aAC7BtqB,EAAMmG,WAAWqG,OAASxM,EAAMtG,KAAK8S,OAASwzD,CAC9C,IAAI3qC,GAAkBr1B,EAAMtG,KAAK8S,OAASxM,EAAM2B,IAAI6K,OAASxM,EAAMsd,OAAO9Q,OACxEwzD,CACFhgE,GAAM6xB,gBAAgBrlB,OAAU6oB,EAChCr1B,EAAM6+D,cAAcryD,OAAY6oB,EAChCr1B,EAAM8+D,eAAetyD,OAAWxM,EAAM6+D,cAAcryD,OAGpDxM,EAAMtG,KAAK6S,MAAQwd,EAAIrwB,KAAK0wB,YAC5BpqB,EAAMmG,WAAWoG,MAAQvM,EAAMtG,KAAK6S,MAAQ0zD,EAC5CjgE,EAAMyB,KAAK8K,MAAQwd,EAAI80C,cAAcrlD,cAAkBxZ,EAAMoG,OAAO3E,KACpEzB,EAAM6+D,cAActyD,MAAQvM,EAAMyB,KAAK8K,MACvCvM,EAAMqhB,MAAM9U,MAAQwd,EAAI+0C,eAAetlD,cAAgBxZ,EAAMoG,OAAOib,MACpErhB,EAAM8+D,eAAevyD,MAAQvM,EAAMqhB,MAAM9U,KACzC,IAAI4zD,GAAcngE,EAAMtG,KAAK6S,MAAQvM,EAAMyB,KAAK8K,MAAQvM,EAAMqhB,MAAM9U,MAAQ0zD,CAC5EjgE,GAAMkmB,OAAO3Z,MAAiB4zD,EAC9BngE,EAAM6xB,gBAAgBtlB,MAAQ4zD,EAC9BngE,EAAM2B,IAAI4K,MAAoB4zD,EAC9BngE,EAAMsd,OAAO/Q,MAAiB4zD,EAG9Bp2C,EAAI5jB,WAAWc,MAAMuF,OAAmBxM,EAAMmG,WAAWqG,OAAS,KAClEud,EAAIgV,mBAAmB93B,MAAMuF,OAAWxM,EAAMmG,WAAWqG,OAAS,KAClEud,EAAImY,qBAAqBj7B,MAAMuF,OAASxM,EAAM6xB,gBAAgBrlB,OAAS,KACvEud,EAAI8H,gBAAgB5qB,MAAMuF,OAAcxM,EAAM6xB,gBAAgBrlB,OAAS,KACvEud,EAAI80C,cAAc53D,MAAMuF,OAAgBxM,EAAM6+D,cAAcryD,OAAS,KACrEud,EAAI+0C,eAAe73D,MAAMuF,OAAexM,EAAM8+D,eAAetyD,OAAS,KAEtEud,EAAI5jB,WAAWc,MAAMsF,MAAmBvM,EAAMmG,WAAWoG,MAAQ,KACjEwd,EAAIgV,mBAAmB93B,MAAMsF,MAAWvM,EAAM6xB,gBAAgBtlB,MAAQ,KACtEwd,EAAImY,qBAAqBj7B,MAAMsF,MAASvM,EAAMmG,WAAWoG,MAAQ,KACjEwd,EAAI8H,gBAAgB5qB,MAAMsF,MAAcvM,EAAMkmB,OAAO3Z,MAAQ,KAC7Dwd,EAAIpoB,IAAIsF,MAAMsF,MAA0BvM,EAAM2B,IAAI4K,MAAQ,KAC1Dwd,EAAIzM,OAAOrW,MAAMsF,MAAuBvM,EAAMsd,OAAO/Q,MAAQ,KAG7Dwd,EAAI5jB,WAAWc,MAAMxF,KAAiB,IACtCsoB,EAAI5jB,WAAWc,MAAMtF,IAAiB,IACtCooB,EAAIgV,mBAAmB93B,MAAMxF,KAAUzB,EAAMyB,KAAK8K,MAAQvM,EAAMoG,OAAO3E,KAAQ,KAC/EsoB,EAAIgV,mBAAmB93B,MAAMtF,IAAS,IACtCooB,EAAImY,qBAAqBj7B,MAAMxF,KAAO,IACtCsoB,EAAImY,qBAAqBj7B,MAAMtF,IAAO3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI8H,gBAAgB5qB,MAAMxF,KAAYzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAI8H,gBAAgB5qB,MAAMtF,IAAY3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI80C,cAAc53D,MAAMxF,KAAc,IACtCsoB,EAAI80C,cAAc53D,MAAMtF,IAAc3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAI+0C,eAAe73D,MAAMxF,KAAczB,EAAMyB,KAAK8K,MAAQvM,EAAMkmB,OAAO3Z,MAAS,KAChFwd,EAAI+0C,eAAe73D,MAAMtF,IAAa3B,EAAM2B,IAAI6K,OAAS,KACzDud,EAAIpoB,IAAIsF,MAAMxF,KAAwBzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAIpoB,IAAIsF,MAAMtF,IAAwB,IACtCooB,EAAIzM,OAAOrW,MAAMxF,KAAqBzB,EAAMyB,KAAK8K,MAAQ,KACzDwd,EAAIzM,OAAOrW,MAAMtF,IAAsB3B,EAAM2B,IAAI6K,OAASxM,EAAM6xB,gBAAgBrlB,OAAU,KAI1FxS,KAAKomE,kBAGL,IAAIz8C,GAAS3pB,KAAKgG,MAAMqhC,SACG,WAAvB54B,EAAQ8lB,cACV5K,GAAUzkB,KAAKwH,IAAI1M,KAAKgG,MAAM6xB,gBAAgBrlB,OAASxS,KAAKgG,MAAMkmB,OAAO1Z,OACvExS,KAAKgG,MAAMoG,OAAOzE,IAAM3H,KAAKgG,MAAMoG,OAAOkX,OAAQ,IAEtDyM,EAAI7D,OAAOjf,MAAMxF,KAAO,IACxBsoB,EAAI7D,OAAOjf,MAAMtF,IAAOgiB,EAAS,KACjCoG,EAAItoB,KAAKwF,MAAMxF,KAAS,IACxBsoB,EAAItoB,KAAKwF,MAAMtF,IAASgiB,EAAS,KACjCoG,EAAI1I,MAAMpa,MAAMxF,KAAQ,IACxBsoB,EAAI1I,MAAMpa,MAAMtF,IAAQgiB,EAAS,IAGjC,IAAI08C,GAAwC,GAAxBrmE,KAAKgG,MAAMqhC,UAAiB,SAAW,GACvDi/B,EAAmBtmE,KAAKgG,MAAMqhC,WAAarnC,KAAKgG,MAAMy/D,aAAe,SAAW,EAYpF,IAXA11C,EAAIg1C,UAAU93D,MAAMuqB,WAAsB6uC,EAC1Ct2C,EAAIi1C,aAAa/3D,MAAMuqB,WAAmB8uC,EAC1Cv2C,EAAIk1C,cAAch4D,MAAMuqB,WAAkB6uC,EAC1Ct2C,EAAIm1C,iBAAiBj4D,MAAMuqB,WAAe8uC,EAC1Cv2C,EAAIo1C,eAAel4D,MAAMuqB,WAAiB6uC,EAC1Ct2C,EAAIq1C,kBAAkBn4D,MAAMuqB,WAAc8uC,EAG1CtmE,KAAKgC,WAAWqG,QAAQ,SAAUu9D,GAChCnhC,EAAUmhC,EAAUnkD,UAAYgjB,IAE9BA,EAAS,CAEX,GAAI8hC,GAAc,CACdvmE,MAAK0lE,YAAca,GACrBvmE,KAAK0lE,cACL1lE,KAAKyhB,UAGLkX,QAAQhF,IAAI,qCAEd3zB,KAAK0lE,YAAc,EAGrB1lE,KAAK4tB,KAAK,oBAIZsI,EAAK/iB,UAAUqzD,QAAU,WACvB,KAAM,IAAI5iE,OAAM,wDAUlBsyB,EAAK/iB,UAAUiyB,eAAiB,SAASjL,GACvC,IAAKn6B,KAAK21B,YACR,KAAM,IAAI/xB,OAAM,sCAGlB5D,MAAK21B,YAAYyP,eAAejL,IAQlCjE,EAAK/iB,UAAUkyB,eAAiB,WAC9B,IAAKrlC,KAAK21B,YACR,KAAM,IAAI/xB,OAAM,sCAGlB,OAAO5D,MAAK21B,YAAY0P,kBAU1BnP,EAAK/iB,UAAUmiB,QAAU,SAASvjB,GAChC,MAAOpQ,GAAS0zB,OAAOr1B,KAAM+R,EAAG/R,KAAKgG,MAAMkmB,OAAO3Z,QAUpD2jB,EAAK/iB,UAAUqiB,cAAgB,SAASzjB,GACtC,MAAOpQ,GAAS0zB,OAAOr1B,KAAM+R,EAAG/R,KAAKgG,MAAMtG,KAAK6S,QAalD2jB,EAAK/iB,UAAU+hB,UAAY,SAASiF,GAClC,MAAOx4B,GAASszB,SAASj1B,KAAMm6B,EAAMn6B,KAAKgG,MAAMkmB,OAAO3Z,QAczD2jB,EAAK/iB,UAAUiiB,gBAAkB,SAAS+E,GACxC,MAAOx4B,GAASszB,SAASj1B,KAAMm6B,EAAMn6B,KAAKgG,MAAMtG,KAAK6S,QAUvD2jB,EAAK/iB,UAAUwyD,gBAAkB,WACA,GAA3B3lE,KAAKyO,QAAQ6lB,WACft0B,KAAKymE,mBAGLzmE,KAAK6lE,mBAST3vC,EAAK/iB,UAAUszD,iBAAmB,WAChC,GAAItyD,GAAKnU,IAETA,MAAK6lE,kBAEL7lE,KAAK0mE,UAAY,WACf,MAA6B,IAAzBvyD,EAAG1F,QAAQ6lB,eAEbngB,GAAG0xD,uBAID1xD,EAAG4b,IAAIrwB,OAKJyU,EAAG4b,IAAIrwB,KAAK0wB,aAAejc,EAAGnO,MAAM2rC,WACtCx9B,EAAG4b,IAAIrwB,KAAK4wB,cAAgBnc,EAAGnO,MAAM2gE,cACtCxyD,EAAGnO,MAAM2rC,UAAYx9B,EAAG4b,IAAIrwB,KAAK0wB,YACjCjc,EAAGnO,MAAM2gE,WAAaxyD,EAAG4b,IAAIrwB,KAAK4wB,aAElCnc,EAAGyZ,KAAK,aAMdjtB,EAAKgI,iBAAiBY,OAAQ,SAAUvJ,KAAK0mE,WAE7C1mE,KAAK4mE,WAAaC,YAAY7mE,KAAK0mE,UAAW,MAOhDxwC,EAAK/iB,UAAU0yD,gBAAkB,WAC3B7lE,KAAK4mE,aACPn0C,cAAczyB,KAAK4mE,YACnB5mE,KAAK4mE,WAAapgE,QAIpB7F,EAAKwI,oBAAoBI,OAAQ,SAAUvJ,KAAK0mE,WAChD1mE,KAAK0mE,UAAY,MASnBxwC,EAAK/iB,UAAU2zD,cAAgB,SAAUz/B,GAGvC,MAFArnC,MAAKgG,MAAMqhC,UAAYA,EACvBrnC,KAAKomE,mBACEpmE,KAAKgG,MAAMqhC,WAQpBnR,EAAK/iB,UAAUizD,iBAAmB,WAEhC,GAAIX,GAAevgE,KAAK4G,IAAI9L,KAAKgG,MAAM6xB,gBAAgBrlB,OAASxS,KAAKgG,MAAMkmB,OAAO1Z,OAAQ,EAc1F,OAbIizD,IAAgBzlE,KAAKgG,MAAMy/D,eAGG,UAA5BzlE,KAAKyO,QAAQ8lB,cACfv0B,KAAKgG,MAAMqhC,WAAco+B,EAAezlE,KAAKgG,MAAMy/D,cAErDzlE,KAAKgG,MAAMy/D,aAAeA,GAIxBzlE,KAAKgG,MAAMqhC,UAAY,IAAGrnC,KAAKgG,MAAMqhC,UAAY,GACjDrnC,KAAKgG,MAAMqhC,UAAYo+B,IAAczlE,KAAKgG,MAAMqhC,UAAYo+B,GAEzDzlE,KAAKgG,MAAMqhC,WAQpBnR,EAAK/iB,UAAU4zD,cAAgB,WAC7B,MAAO/mE,MAAKgG,MAAMqhC,WAGpBxnC,EAAOD,QAAUs2B,GAKb,SAASr2B,EAAQD,EAASM,GAEjBA,EAAoB,GAOjCN,GAAQ0pD,QAAU,SAAUxlD,EAAQwE,GAClCA,EAAS0+D,aAAe,SAAU19D,GAC5BA,EAAMqnC,SACRroC,EAASgB,IAIbxF,EAAOyP,GAAG,eAAgBjL,EAAS0+D,eAQrCpnE,EAAQqnE,UAAY,SAAUnjE,EAAQwE,GAOpC,MANAA,GAAS0+D,aAAe,SAAU19D,GAC5BA,EAAM49D,SACR5+D,EAASgB,IAINxF,EAAOyP,GAAG,eAAgBjL,EAAS0+D,eAQ5CpnE,EAAQunE,SAAW,SAAUrjE,EAAQwE,GACnCxE,EAAO4P,IAAI,eAAgBpL,EAAS0+D,eAQtCpnE,EAAQwnE,WAAaxnE,EAAQunE,UAKzB,SAAStnE,EAAQD,GAGrBA,EAAY,IACVk6B,QAAS,UACTK,KAAM,QAERv6B,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,GAG/BA,EAAY,IACVynE,OAAQ,aACRltC,KAAM,QAERv6B,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,IAK3B,SAASC,EAAQD,GAGrBA,EAAY,IACVg9C,KAAM,OACNG,IAAK,kBACLuqB,KAAM,OACNrG,QAAS,WACTG,QAAS,WACTmG,SAAU,YACV1qB,SAAU,YACV2qB,eAAgB,+CAChBC,gBAAiB,qEACjBC,oBAAqB,wEACrBC,gBAAiB,kCACjBC,mBAAoB,+BAEtBhoE,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,GAG/BA,EAAY,IACVg9C,KAAM,WACNG,IAAK,uBACLuqB,KAAM,QACNrG,QAAS,iBACTG,QAAS,iBACTmG,SAAU,gBACV1qB,SAAU,gBACV2qB,eAAgB,uDAChBC,gBAAiB,6EACjBC,oBAAqB,kFACrBC,gBAAiB,wCACjBC,mBAAoB,2CAEtBhoE,EAAe,MAAIA,EAAY,GAC/BA,EAAe,MAAIA,EAAY,IAK3B,WAKoC,mBAA7BioE,4BAKTA,yBAAyB10D,UAAUisD,OAAS,SAASrtD,EAAGC,EAAGvH,GACzDzK,KAAK4nB,YACL5nB,KAAK0rB,IAAI3Z,EAAGC,EAAGvH,EAAG,EAAG,EAAEvF,KAAKymB,IAAI,IASlCk8C,yBAAyB10D,UAAU20D,OAAS,SAAS/1D,EAAGC,EAAGvH,GACzDzK,KAAK4nB,YACL5nB,KAAKyS,KAAKV,EAAItH,EAAGuH,EAAIvH,EAAO,EAAJA,EAAW,EAAJA,IASjCo9D,yBAAyB10D,UAAU4b,SAAW,SAAShd,EAAGC,EAAGvH,GAE3DzK,KAAK4nB,WAEL,IAAIhc,GAAQ,EAAJnB,EACJs9D,EAAKn8D,EAAI,EACTo8D,EAAK9iE,KAAKyqB,KAAK,GAAK,EAAI/jB,EACxBD,EAAIzG,KAAKyqB,KAAK/jB,EAAIA,EAAIm8D,EAAKA,EAE/B/nE,MAAK6nB,OAAO9V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAKioB,aASP4/C,yBAAyB10D,UAAU80D,aAAe,SAASl2D,EAAGC,EAAGvH,GAE/DzK,KAAK4nB,WAEL,IAAIhc,GAAQ,EAAJnB,EACJs9D,EAAKn8D,EAAI,EACTo8D,EAAK9iE,KAAKyqB,KAAK,GAAK,EAAI/jB,EACxBD,EAAIzG,KAAKyqB,KAAK/jB,EAAIA,EAAIm8D,EAAKA,EAE/B/nE,MAAK6nB,OAAO9V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAIg2D,EAAI/1D,EAAIg2D,GACxBhoE,KAAK8nB,OAAO/V,EAAGC,GAAKrG,EAAIq8D,IACxBhoE,KAAKioB,aASP4/C,yBAAyB10D,UAAU+0D,KAAO,SAASn2D,EAAGC,EAAGvH,GAEvDzK,KAAK4nB,WAEL,KAAK,GAAIugD,GAAI,EAAO,GAAJA,EAAQA,IAAK,CAC3B,GAAI18C,GAAU08C,EAAI,IAAM,EAAS,IAAJ19D,EAAc,GAAJA,CACvCzK,MAAK8nB,OACD/V,EAAI0Z,EAASvmB,KAAKkZ,IAAQ,EAAJ+pD,EAAQjjE,KAAKymB,GAAK,IACxC3Z,EAAIyZ,EAASvmB,KAAKqZ,IAAQ,EAAJ4pD,EAAQjjE,KAAKymB,GAAK,KAI9C3rB,KAAKioB,aAMP4/C,yBAAyB10D,UAAUssD,UAAY,SAAS1tD,EAAGC,EAAGq9C,EAAG1jD,EAAGlB,GAClE,GAAI29D,GAAMljE,KAAKymB,GAAG,GACE,GAAhB0jC,EAAM,EAAI5kD,IAAYA,EAAM4kD,EAAI,GAChB,EAAhB1jD,EAAM,EAAIlB,IAAYA,EAAMkB,EAAI,GACpC3L,KAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAEtH,EAAEuH,GAChBhS,KAAK8nB,OAAO/V,EAAEs9C,EAAE5kD,EAAEuH,GAClBhS,KAAK0rB,IAAI3Z,EAAEs9C,EAAE5kD,EAAEuH,EAAEvH,EAAEA,EAAM,IAAJ29D,EAAY,IAAJA,GAAQ,GACrCpoE,KAAK8nB,OAAO/V,EAAEs9C,EAAEr9C,EAAErG,EAAElB,GACpBzK,KAAK0rB,IAAI3Z,EAAEs9C,EAAE5kD,EAAEuH,EAAErG,EAAElB,EAAEA,EAAE,EAAM,GAAJ29D,GAAO,GAChCpoE,KAAK8nB,OAAO/V,EAAEtH,EAAEuH,EAAErG,GAClB3L,KAAK0rB,IAAI3Z,EAAEtH,EAAEuH,EAAErG,EAAElB,EAAEA,EAAM,GAAJ29D,EAAW,IAAJA,GAAQ,GACpCpoE,KAAK8nB,OAAO/V,EAAEC,EAAEvH,GAChBzK,KAAK0rB,IAAI3Z,EAAEtH,EAAEuH,EAAEvH,EAAEA,EAAM,IAAJ29D,EAAY,IAAJA,GAAQ,IAMrCP,yBAAyB10D,UAAUysD,QAAU,SAAS7tD,EAAGC,EAAGq9C,EAAG1jD,GAC7D,GAAI08D,GAAQ,SACRC,EAAMjZ,EAAI,EAAKgZ,EACfE,EAAM58D,EAAI,EAAK08D,EACfG,EAAKz2D,EAAIs9C,EACToZ,EAAKz2D,EAAIrG,EACT+8D,EAAK32D,EAAIs9C,EAAI,EACbsZ,EAAK32D,EAAIrG,EAAI,CAEjB3L;KAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAG42D,GACf3oE,KAAK4oE,cAAc72D,EAAG42D,EAAKJ,EAAIG,EAAKJ,EAAIt2D,EAAG02D,EAAI12D,GAC/ChS,KAAK4oE,cAAcF,EAAKJ,EAAIt2D,EAAGw2D,EAAIG,EAAKJ,EAAIC,EAAIG,GAChD3oE,KAAK4oE,cAAcJ,EAAIG,EAAKJ,EAAIG,EAAKJ,EAAIG,EAAIC,EAAID,GACjDzoE,KAAK4oE,cAAcF,EAAKJ,EAAIG,EAAI12D,EAAG42D,EAAKJ,EAAIx2D,EAAG42D,IAQjDd,yBAAyB10D,UAAUusD,SAAW,SAAS3tD,EAAGC,EAAGq9C,EAAG1jD,GAC9D,GAAIiC,GAAI,EAAE,EACNi7D,EAAWxZ,EACXyZ,EAAWn9D,EAAIiC,EAEfy6D,EAAQ,SACRC,EAAMO,EAAW,EAAKR,EACtBE,EAAMO,EAAW,EAAKT,EACtBG,EAAKz2D,EAAI82D,EACTJ,EAAKz2D,EAAI82D,EACTJ,EAAK32D,EAAI82D,EAAW,EACpBF,EAAK32D,EAAI82D,EAAW,EACpBC,EAAM/2D,GAAKrG,EAAIm9D,EAAS,GACxBE,EAAMh3D,EAAIrG,CAEd3L,MAAK4nB,YACL5nB,KAAK6nB,OAAO2gD,EAAIG,GAEhB3oE,KAAK4oE,cAAcJ,EAAIG,EAAKJ,EAAIG,EAAKJ,EAAIG,EAAIC,EAAID,GACjDzoE,KAAK4oE,cAAcF,EAAKJ,EAAIG,EAAI12D,EAAG42D,EAAKJ,EAAIx2D,EAAG42D,GAE/C3oE,KAAK4oE,cAAc72D,EAAG42D,EAAKJ,EAAIG,EAAKJ,EAAIt2D,EAAG02D,EAAI12D,GAC/ChS,KAAK4oE,cAAcF,EAAKJ,EAAIt2D,EAAGw2D,EAAIG,EAAKJ,EAAIC,EAAIG,GAEhD3oE,KAAK8nB,OAAO0gD,EAAIO,GAEhB/oE,KAAK4oE,cAAcJ,EAAIO,EAAMR,EAAIG,EAAKJ,EAAIU,EAAKN,EAAIM,GACnDhpE,KAAK4oE,cAAcF,EAAKJ,EAAIU,EAAKj3D,EAAGg3D,EAAMR,EAAIx2D,EAAGg3D,GAEjD/oE,KAAK8nB,OAAO/V,EAAG42D,IAOjBd,yBAAyB10D,UAAUukD,MAAQ,SAAS3lD,EAAGC,EAAGo8C,EAAOzoD,GAE/D,GAAIsjE,GAAKl3D,EAAIpM,EAAST,KAAKqZ,IAAI6vC,GAC3B8a,EAAKl3D,EAAIrM,EAAST,KAAKkZ,IAAIgwC,GAI3B+a,EAAKp3D,EAAa,GAATpM,EAAeT,KAAKqZ,IAAI6vC,GACjCgb,EAAKp3D,EAAa,GAATrM,EAAeT,KAAKkZ,IAAIgwC,GAGjCib,EAAKJ,EAAKtjE,EAAS,EAAIT,KAAKqZ,IAAI6vC,EAAQ,GAAMlpD,KAAKymB,IACnD29C,EAAKJ,EAAKvjE,EAAS,EAAIT,KAAKkZ,IAAIgwC,EAAQ,GAAMlpD,KAAKymB,IAGnD49C,EAAKN,EAAKtjE,EAAS,EAAIT,KAAKqZ,IAAI6vC,EAAQ,GAAMlpD,KAAKymB,IACnD69C,EAAKN,EAAKvjE,EAAS,EAAIT,KAAKkZ,IAAIgwC,EAAQ,GAAMlpD,KAAKymB,GAEvD3rB,MAAK4nB,YACL5nB,KAAK6nB,OAAO9V,EAAGC,GACfhS,KAAK8nB,OAAOuhD,EAAIC,GAChBtpE,KAAK8nB,OAAOqhD,EAAIC,GAChBppE,KAAK8nB,OAAOyhD,EAAIC,GAChBxpE,KAAKioB,aASP4/C,yBAAyB10D,UAAUqkD,WAAa,SAASzlD,EAAEC,EAAEwmD,EAAGC,EAAGgR,GAC5DA,IAAWA,GAAW,GAAG,IACd,GAAZC,IAAeA,EAAa,KAChC,IAAIC,GAAYF,EAAU9jE,MAC1B3F,MAAK6nB,OAAO9V,EAAGC,EAKf,KAJA,GAAI4M,GAAM45C,EAAGzmD,EAAI8M,EAAM45C,EAAGzmD,EACtB43D,EAAQ/qD,EAAGD,EACXirD,EAAgB3kE,KAAKyqB,KAAM/Q,EAAGA,EAAKC,EAAGA,GACtCirD,EAAU,EAAG/9B,GAAK,EACf89B,GAAe,IAAI,CACxB,GAAIH,GAAaD,EAAUK,IAAYH,EACnCD,GAAaG,IAAeH,EAAaG,EAC7C,IAAInuD,GAAQxW,KAAKyqB,KAAM+5C,EAAWA,GAAc,EAAIE,EAAMA,GACnD,GAAHhrD,IAAMlD,GAASA,GACnB3J,GAAK2J,EACL1J,GAAK43D,EAAMluD,EACX1b,KAAK+rC,EAAO,SAAW,UAAUh6B,EAAEC,GACnC63D,GAAiBH,EACjB39B,GAAQA,MAUV,SAASlsC,EAAQD,EAASM,GAQ9B,QAASyqC,GAAKrT,EAAS7oB,GACrBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EALjB,GAAI7N,GAAUV,EAAoB,GAC9B2qC,EAAS3qC,EAAoB,GAOjCyqC,GAAKx3B,UAAU04B,UAAY,SAASC,GAGlC,IAAK,GAFDlwB,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,mBAU/DjB,EAAKx3B,UAAU44B,KAAO,SAAU/U,EAAS/kB,EAAO+5B,GAC9C,GAAe,MAAXhV,GACEA,EAAQrxB,OAAS,EAAG,CACtB,GAAIwlC,GAAMx+B,EACNosC,EAAY70C,OAAO8nC,EAAUpG,IAAI34B,MAAMuF,OAAOhI,QAAQ,KAAK,IAgB/D,IAfA2gC,EAAOvqC,EAAQwQ,cAAc,OAAQ46B,EAAU/E,YAAa+E,EAAUpG,KACtEuF,EAAK/4B,eAAe,KAAM,QAASH,EAAMpK,WACtBrB,SAAhByL,EAAMhF,OACPk+B,EAAK/4B,eAAe,KAAM,QAASH,EAAMhF,OAKzCN,EADsC,GAApCsF,EAAMxD,QAAQs8B,WAAWr8B,QACvBi8B,EAAKo/B,YAAY/yC,EAAS/kB,GAG1B04B,EAAKq/B,QAAQhzC,GAIiB,GAAhC/kB,EAAMxD,QAAQ88B,OAAO78B,QAAiB,CACxC,GACIu7D,GADA7+B,EAAWxqC,EAAQwQ,cAAc,OAAQ46B,EAAU/E,YAAa+E,EAAUpG,IAG5EqkC,GADsC,OAApCh4D,EAAMxD,QAAQ88B,OAAOhX,YACf,IAAMyC,EAAQ,GAAGjlB,EAAI,MAAgBpF,EAAI,IAAMqqB,EAAQA,EAAQrxB,OAAS,GAAGoM,EAAI,KAG/E,IAAMilB,EAAQ,GAAGjlB,EAAI,IAAMgnC,EAAY,IAAMpsC,EAAI,IAAMqqB,EAAQA,EAAQrxB,OAAS,GAAGoM,EAAI,IAAMgnC,EAEvG3N,EAASh5B,eAAe,KAAM,QAASH,EAAMpK,UAAY,SACvBrB,SAA/ByL,EAAMxD,QAAQ88B,OAAOt+B,OACtBm+B,EAASh5B,eAAe,KAAM,QAASH,EAAMxD,QAAQ88B,OAAOt+B,OAE9Dm+B,EAASh5B,eAAe,KAAM,IAAK63D,GAGrC9+B,EAAK/4B,eAAe,KAAM,IAAK,IAAMzF,GAGG,GAApCsF,EAAMxD,QAAQ0D,WAAWzD,SAC3Bm8B,EAAOkB,KAAK/U,EAAS/kB,EAAO+5B,KAepCrB,EAAKu/B,mBAAqB,SAASx3D,GAMjC,IAAK,GAJDy3D,GAAIC,EAAIC,EAAIC,EAAIC,EAAKC,EACrB79D,EAAIzH,KAAKwoB,MAAMhb,EAAK,GAAGX,GAAK,IAAM7M,KAAKwoB,MAAMhb,EAAK,GAAGV,GAAK,IAC1Dy4D,EAAgB,EAAE,EAClB9kE,EAAS+M,EAAK/M,OACTH,EAAI,EAAOG,EAAS,EAAbH,EAAgBA,IAE9B2kE,EAAW,GAAL3kE,EAAUkN,EAAK,GAAKA,EAAKlN,EAAE,GACjC4kE,EAAK13D,EAAKlN,GACV6kE,EAAK33D,EAAKlN,EAAE,GACZ8kE,EAAc3kE,EAARH,EAAI,EAAckN,EAAKlN,EAAE,GAAK6kE,EAUpCE,GAAQx4D,IAAMo4D,EAAGp4D,EAAI,EAAEq4D,EAAGr4D,EAAIs4D,EAAGt4D,GAAI04D,EAAgBz4D,IAAMm4D,EAAGn4D,EAAI,EAAEo4D,EAAGp4D,EAAIq4D,EAAGr4D,GAAIy4D,GAClFD,GAAQz4D,GAAMq4D,EAAGr4D,EAAI,EAAEs4D,EAAGt4D,EAAIu4D,EAAGv4D,GAAI04D,EAAgBz4D,GAAMo4D,EAAGp4D,EAAI,EAAEq4D,EAAGr4D,EAAIs4D,EAAGt4D,GAAIy4D,GAGlF99D,GAAK,IACL49D,EAAIx4D,EAAI,IACRw4D,EAAIv4D,EAAI,IACRw4D,EAAIz4D,EAAI,IACRy4D,EAAIx4D,EAAI,IACRq4D,EAAGt4D,EAAI,IACPs4D,EAAGr4D,EAAI,GAGT,OAAOrF,IAcTg+B,EAAKo/B,YAAc,SAASr3D,EAAMT,GAChC,GAAIg5B,GAAQh5B,EAAMxD,QAAQs8B,WAAWE,KACrC,IAAa,GAATA,GAAwBzkC,SAAVykC,EAChB,MAAOjrC,MAAKkqE,mBAAmBx3D,EAO/B,KAAK,GAJDy3D,GAAIC,EAAIC,EAAIC,EAAIC,EAAKC,EAAKE,EAAGC,EAAGC,EAAIC,EAAGpgD,EAAGqgD,EAAGC,EAC7CC,EAAQC,EAAQC,EAASC,EAASC,EAASC,EAC3C1+D,EAAIzH,KAAKwoB,MAAMhb,EAAK,GAAGX,GAAK,IAAM7M,KAAKwoB,MAAMhb,EAAK,GAAGV,GAAK,IAC1DrM,EAAS+M,EAAK/M,OACTH,EAAI,EAAOG,EAAS,EAAbH,EAAgBA,IAE9B2kE,EAAW,GAAL3kE,EAAUkN,EAAK,GAAKA,EAAKlN,EAAE,GACjC4kE,EAAK13D,EAAKlN,GACV6kE,EAAK33D,EAAKlN,EAAE,GACZ8kE,EAAc3kE,EAARH,EAAI,EAAckN,EAAKlN,EAAE,GAAK6kE,EAEpCK,EAAKxlE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIq2C,EAAGp4D,EAAIq4D,EAAGr4D,EAAE,GAAK7M,KAAK4uB,IAAIq2C,EAAGn4D,EAAIo4D,EAAGp4D,EAAE,IAC9D24D,EAAKzlE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIs2C,EAAGr4D,EAAIs4D,EAAGt4D,EAAE,GAAK7M,KAAK4uB,IAAIs2C,EAAGp4D,EAAIq4D,EAAGr4D,EAAE,IAC9D44D,EAAK1lE,KAAKyqB,KAAKzqB,KAAK4uB,IAAIu2C,EAAGt4D,EAAIu4D,EAAGv4D,EAAE,GAAK7M,KAAK4uB,IAAIu2C,EAAGr4D,EAAIs4D,EAAGt4D,EAAE,IAY9Dg5D,EAAU9lE,KAAK4uB,IAAI82C,EAAK3/B,GACxBigC,EAAUhmE,KAAK4uB,IAAI82C,EAAG,EAAE3/B,GACxBggC,EAAU/lE,KAAK4uB,IAAI62C,EAAK1/B,GACxBkgC,EAAUjmE,KAAK4uB,IAAI62C,EAAG,EAAE1/B,GACxBogC,EAAUnmE,KAAK4uB,IAAI42C,EAAKz/B,GACxBmgC,EAAUlmE,KAAK4uB,IAAI42C,EAAG,EAAEz/B,GAExB4/B,EAAI,EAAEO,EAAU,EAAEC,EAASJ,EAASE,EACpC1gD,EAAI,EAAEygD,EAAU,EAAEF,EAASC,EAASE,EACpCL,EAAI,EAAEO,GAAUA,EAASJ,GACrBH,EAAI,IAAIA,EAAI,EAAIA,GACpBC,EAAI,EAAEC,GAAUA,EAASC,GACrBF,EAAI,IAAIA,EAAI,EAAIA,GAEpBR,GAAQx4D,IAAMo5D,EAAUhB,EAAGp4D,EAAI84D,EAAET,EAAGr4D,EAAIq5D,EAAUf,EAAGt4D,GAAK+4D,EACxD94D,IAAMm5D,EAAUhB,EAAGn4D,EAAI64D,EAAET,EAAGp4D,EAAIo5D,EAAUf,EAAGr4D,GAAK84D,GAEpDN,GAAQz4D,GAAMm5D,EAAUd,EAAGr4D,EAAI0Y,EAAE4/C,EAAGt4D,EAAIo5D,EAAUb,EAAGv4D,GAAKg5D,EACxD/4D,GAAMk5D,EAAUd,EAAGp4D,EAAIyY,EAAE4/C,EAAGr4D,EAAIm5D,EAAUb,EAAGt4D,GAAK+4D,GAEvC,GAATR,EAAIx4D,GAAmB,GAATw4D,EAAIv4D,IAASu4D,EAAMH,GACxB,GAATI,EAAIz4D,GAAmB,GAATy4D,EAAIx4D,IAASw4D,EAAMH,GACrC19D,GAAK,IACL49D,EAAIx4D,EAAI,IACRw4D,EAAIv4D,EAAI,IACRw4D,EAAIz4D,EAAI,IACRy4D,EAAIx4D,EAAI,IACRq4D,EAAGt4D,EAAI,IACPs4D,EAAGr4D,EAAI,GAGT,OAAOrF,IAUXg+B,EAAKq/B,QAAU,SAASt3D,GAGtB,IAAK,GADD/F,GAAI,GACCnH,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAE7BmH,GADO,GAALnH,EACGkN,EAAKlN,GAAGuM,EAAI,IAAMW,EAAKlN,GAAGwM,EAG1B,IAAMU,EAAKlN,GAAGuM,EAAI,IAAMW,EAAKlN,GAAGwM,CAGzC,OAAOrF,IAGT9M,EAAOD,QAAU+qC,GAKb,SAAS9qC,EAAQD,EAASM,GAQ9B,QAASorE,GAASh0C,EAAS7oB,GACzBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EALjB,CAAA,GAAI7N,GAAUV,EAAoB,EACrBA,GAAoB,IAOjCorE,EAASn4D,UAAU04B,UAAY,SAASC,GACtC,GAA2C,SAAvC9rC,KAAKyO,QAAQymC,SAASC,cAA0B,CAGlD,IAAK,GAFDv5B,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,kBAI7D,IAAK,GADD2/B,MACK3/C,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpC2/C,EAAgBvjE,MACd+J,EAAG+5B,EAAUlgB,GAAG7Z,EAChBC,EAAG85B,EAAUlgB,GAAG5Z,EAChBslB,QAASt3B,KAAKs3B,SAGlB,OAAOi0C,IAYXD,EAASv/B,KAAO,SAAUmE,EAAUsG,EAAoBxK,GACtD,GAEIw/B,GACA9iE,EAAK+iE,EACLx5D,EACAzM,EAAEomB,EALF8/C,KACAC,KAKAC,EAAY,CAGhB,KAAKpmE,EAAI,EAAGA,EAAI0qC,EAASvqC,OAAQH,IAE/B,GADAyM,EAAQ+5B,EAAU7X,OAAO+b,EAAS1qC,IACP,OAAvByM,EAAMxD,QAAQxB,OACK,GAAjBgF,EAAMyW,UAAyEliB,SAArDwlC,EAAUv9B,QAAQ0lB,OAAOqD,WAAW0Y,EAAS1qC,KAAyE,GAApDwmC,EAAUv9B,QAAQ0lB,OAAOqD,WAAW0Y,EAAS1qC,KAC3I,IAAKomB,EAAI,EAAGA,EAAI4qB,EAAmBtG,EAAS1qC,IAAIG,OAAQimB,IACtD8/C,EAAa1jE,MACX+J,EAAGykC,EAAmBtG,EAAS1qC,IAAIomB,GAAG7Z,EACtCC,EAAGwkC,EAAmBtG,EAAS1qC,IAAIomB,GAAG5Z,EACtCslB,QAAS4Y,EAAS1qC,KAEpBomE,GAAa,CAMrB,IAAiB,GAAbA,EAeJ,IAZAF,EAAax1D,KAAK,SAAU3Q,EAAGa,GAC7B,MAAIb,GAAEwM,GAAK3L,EAAE2L,EACJxM,EAAE+xB,QAAUlxB,EAAEkxB,QAEd/xB,EAAEwM,EAAI3L,EAAE2L,IAKnBu5D,EAASO,sBAAsBF,EAAeD,GAGzClmE,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IAAK,CACxCyM,EAAQ+5B,EAAU7X,OAAOu3C,EAAalmE,GAAG8xB,QACzC,IAAIqP,GAAW,GAAM10B,EAAMxD,QAAQymC,SAAS3iC,KAE5C7J,GAAMgjE,EAAalmE,GAAGuM,CACtB,IAAI+5D,GAAe,CACnB,IAA2BtlE,SAAvBmlE,EAAcjjE,GACZlD,EAAE,EAAIkmE,EAAa/lE,SAAS6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAalmE,EAAE,GAAGuM,EAAIrJ,IAC1ElD,EAAI,IAAwBgmE,EAAetmE,KAAK4G,IAAI0/D,EAAatmE,KAAK2lB,IAAI6gD,EAAalmE,EAAE,GAAGuM,EAAIrJ,KACpG+iE,EAAWH,EAASS,iBAAiBP,EAAcv5D,EAAO00B,OAEvD,CACH,GAAIqlC,GAAUxmE,GAAKmmE,EAAcjjE,GAAKujE,OAASN,EAAcjjE,GAAKwjE,UAC9DC,EAAU3mE,GAAKmmE,EAAcjjE,GAAKwjE,SAAW,EAC7CF,GAAUN,EAAa/lE,SAAS6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAaM,GAASj6D,EAAIrJ,IAClFyjE,EAAU,IAAsBX,EAAetmE,KAAK4G,IAAI0/D,EAAatmE,KAAK2lB,IAAI6gD,EAAaS,GAASp6D,EAAIrJ,KAC5G+iE,EAAWH,EAASS,iBAAiBP,EAAcv5D,EAAO00B,GAC1DglC,EAAcjjE,GAAKwjE,UAAY,EAEa,SAAxCj6D,EAAMxD,QAAQymC,SAASC,eACzB22B,EAAeH,EAAcjjE,GAAK0jE,YAClCT,EAAcjjE,GAAK0jE,aAAen6D,EAAMy4B,aAAeghC,EAAalmE,GAAGwM,GAExB,cAAxCC,EAAMxD,QAAQymC,SAASC,gBAC9Bs2B,EAASl5D,MAAQk5D,EAASl5D,MAAQo5D,EAAcjjE,GAAKujE,OACrDR,EAAS9hD,QAAWgiD,EAAcjjE,GAAa,SAAI+iE,EAASl5D,MAAS,GAAIk5D,EAASl5D,OAASo5D,EAAcjjE,GAAKujE,OAAO,GACjF,QAAhCh6D,EAAMxD,QAAQymC,SAASnG,MAAwB08B,EAAS9hD,QAAU,GAAI8hD,EAASl5D,MAC1C,SAAhCN,EAAMxD,QAAQymC,SAASnG,QAAmB08B,EAAS9hD,QAAU,GAAI8hD,EAASl5D,QAGvF3R,EAAQ0R,QAAQo5D,EAAalmE,GAAGuM,EAAI05D,EAAS9hD,OAAQ+hD,EAAalmE,GAAGwM,EAAI85D,EAAcL,EAASl5D,MAAON,EAAMy4B,aAAeghC,EAAalmE,GAAGwM,EAAGC,EAAMpK,UAAY,OAAQmkC,EAAU/E,YAAa+E,EAAUpG,KAElK,GAApC3zB,EAAMxD,QAAQ0D,WAAWzD,SAC3B9N,EAAQkR,UAAU45D,EAAalmE,GAAGuM,EAAI05D,EAAS9hD,OAAQ+hD,EAAalmE,GAAGwM,EAAGC,EAAO+5B,EAAU/E,YAAa+E,EAAUpG,OAYxH0lC,EAASO,sBAAwB,SAAUF,EAAeD,GAGxD,IAAK,GADDF,GACKhmE,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IACnCA,EAAI,EAAIkmE,EAAa/lE,SACvB6lE,EAAetmE,KAAK2lB,IAAI6gD,EAAalmE,EAAI,GAAGuM,EAAI25D,EAAalmE,GAAGuM,IAE9DvM,EAAI,IACNgmE,EAAetmE,KAAK4G,IAAI0/D,EAActmE,KAAK2lB,IAAI6gD,EAAalmE,EAAI,GAAGuM,EAAI25D,EAAalmE,GAAGuM,KAErE,GAAhBy5D,IACuChlE,SAArCmlE,EAAcD,EAAalmE,GAAGuM,KAChC45D,EAAcD,EAAalmE,GAAGuM,IAAMk6D,OAAQ,EAAGC,SAAU,EAAGE,YAAa,IAE3ET,EAAcD,EAAalmE,GAAGuM,GAAGk6D,QAAU,IAejDX,EAASS,iBAAmB,SAAUP,EAAcv5D,EAAO00B,GACzD,GAAIp0B,GAAOoX,CAwBX,OAvBI6hD,GAAev5D,EAAMxD,QAAQymC,SAAS3iC,OAASi5D,EAAe,GAChEj5D,EAAuBo0B,EAAf6kC,EAA0B7kC,EAAW6kC,EAE7C7hD,EAAS,EAC2B,QAAhC1X,EAAMxD,QAAQymC,SAASnG,MACzBplB,GAAU,GAAM6hD,EAEuB,SAAhCv5D,EAAMxD,QAAQymC,SAASnG,QAC9BplB,GAAU,GAAM6hD,KAKlBj5D,EAAQN,EAAMxD,QAAQymC,SAAS3iC,MAC/BoX,EAAS,EAC2B,QAAhC1X,EAAMxD,QAAQymC,SAASnG,MACzBplB,GAAU,GAAM1X,EAAMxD,QAAQymC,SAAS3iC,MAEA,SAAhCN,EAAMxD,QAAQymC,SAASnG,QAC9BplB,GAAU,GAAM1X,EAAMxD,QAAQymC,SAAS3iC,SAInCA,MAAOA,EAAOoX,OAAQA,IAGhC2hD,EAASxzB,oBAAsB,SAASyzB,EAAiB90B,EAAavG,EAAUm8B,EAAY93C,GAC1F,GAAIg3C,EAAgB5lE,OAAS,EAAG,CAE9B4lE,EAAgBr1D,KAAK,SAAU3Q,EAAGa,GAChC,MAAIb,GAAEwM,GAAK3L,EAAE2L,EACJxM,EAAE+xB,QAAUlxB,EAAEkxB,QAEd/xB,EAAEwM,EAAI3L,EAAE2L,GAGnB,IAAI45D,KAEJL,GAASO,sBAAsBF,EAAeJ,GAC9C90B,EAAY41B,GAAcf,EAASgB,qBAAqBX,EAAeJ,GACvE90B,EAAY41B,GAAYzgC,iBAAmBrX,EAC3C2b,EAASloC,KAAKqkE,KAIlBf,EAASgB,qBAAuB,SAAUX,EAAeD,GAIvD,IAAK,GAHDhjE,GACAkT,EAAO8vD,EAAa,GAAG15D,EACvB8J,EAAO4vD,EAAa,GAAG15D,EAClBxM,EAAI,EAAGA,EAAIkmE,EAAa/lE,OAAQH,IACvCkD,EAAMgjE,EAAalmE,GAAGuM,EACKvL,SAAvBmlE,EAAcjjE,IAChBkT,EAAOA,EAAO8vD,EAAalmE,GAAGwM,EAAI05D,EAAalmE,GAAGwM,EAAI4J,EACtDE,EAAOA,EAAO4vD,EAAalmE,GAAGwM,EAAI05D,EAAalmE,GAAGwM,EAAI8J,GAGtD6vD,EAAcjjE,GAAK0jE,aAAeV,EAAalmE,GAAGwM,CAGtD,KAAK,GAAIu6D,KAAQZ,GACXA,EAAc7lE,eAAeymE,KAC/B3wD,EAAOA,EAAO+vD,EAAcY,GAAMH,YAAcT,EAAcY,GAAMH,YAAcxwD,EAClFE,EAAOA,EAAO6vD,EAAcY,GAAMH,YAAcT,EAAcY,GAAMH,YAActwD,EAItF,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,IAG1Bjc,EAAOD,QAAU0rE,GAIb,SAASzrE,EAAQD,EAASM,GAO9B,QAAS2qC,GAAOvT,EAAS7oB,GACvBzO,KAAKs3B,QAAUA,EACft3B,KAAKyO,QAAUA,EAJjB,GAAI7N,GAAUV,EAAoB,EAQlC2qC,GAAO13B,UAAU04B,UAAY,SAASC,GAGpC,IAAK,GAFDlwB,GAAOkwB,EAAU,GAAG95B,EACpB8J,EAAOgwB,EAAU,GAAG95B,EACf4Z,EAAI,EAAGA,EAAIkgB,EAAUnmC,OAAQimB,IACpChQ,EAAOA,EAAOkwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI4J,EAChDE,EAAOA,EAAOgwB,EAAUlgB,GAAG5Z,EAAI85B,EAAUlgB,GAAG5Z,EAAI8J,CAElD,QAAQhQ,IAAK8P,EAAMlP,IAAKoP,EAAM8vB,iBAAkB5rC,KAAKyO,QAAQm9B,mBAG/Df,EAAO13B,UAAU44B,KAAO,SAAS/U,EAAS/kB,EAAO+5B,EAAWriB,GAC1DkhB,EAAOkB,KAAK/U,EAAS/kB,EAAO+5B,EAAWriB,IAYzCkhB,EAAOkB,KAAO,SAAU/U,EAAS/kB,EAAO+5B,EAAWriB,GAClCnjB,SAAXmjB,IAAuBA,EAAS,EACpC,KAAK,GAAInkB,GAAI,EAAGA,EAAIwxB,EAAQrxB,OAAQH,IAClC5E,EAAQkR,UAAUklB,EAAQxxB,GAAGuM,EAAI4X,EAAQqN,EAAQxxB,GAAGwM,EAAGC,EAAO+5B,EAAU/E,YAAa+E,EAAUpG,MAKnG/lC,EAAOD,QAAUirC,GAIb,SAAShrC,EAAQD,EAASM,GAE9B,GAAIssE,GAAetsE,EAAoB,IACnCusE,EAAevsE,EAAoB,IACnCwsE,EAAexsE,EAAoB,IACnCysE,EAAiBzsE,EAAoB,IACrC0sE,EAAoB1sE,EAAoB,IACxC2sE,EAAkB3sE,EAAoB,IACtC4sE,EAA0B5sE,EAAoB,GAQlDN,GAAQmtE,WAAa,SAAUC,GAC7B,IAAK,GAAIC,KAAiBD,GACpBA,EAAelnE,eAAemnE,KAChCjtE,KAAKitE,GAAiBD,EAAeC,KAY3CrtE,EAAQstE,YAAc,SAAUF,GAC9B,IAAK,GAAIC,KAAiBD,GACpBA,EAAelnE,eAAemnE,KAChCjtE,KAAKitE,GAAiBzmE,SAW5B5G,EAAQqjD,mBAAqB,WAC3BjjD,KAAK+sE,WAAWP,GAChBxsE,KAAKmtE,2BACkC,GAAnCntE,KAAK0hD,UAAUnD,iBACjBv+C,KAAKotE,4BAGLptE,KAAKoqD,gCAUTxqD,EAAQujD,mBAAqB,WAC3BnjD,KAAK87D,eAAiB,EACtB97D,KAAKqtE,aAAe,EACpBrtE,KAAK+sE,WAAWN,IASlB7sE,EAAQsjD,kBAAoB,WAC1BljD,KAAKkvD,WACLlvD,KAAKstE,cAAgB,WACrBttE,KAAKkvD,QAAgB,UACrBlvD,KAAKkvD,QAAgB,OAAE,YAAclS,SACnCc,SACAgG,eACAsY,eAAkB,EAClBmR,YAAe/mE,QACjBxG,KAAKkvD,QAAgB,UACrBlvD,KAAKkvD,QAAiB,SAAKlS,SACzBc,SACAgG,eACAsY,eAAkB,EAClBmR,YAAe/mE,QAEjBxG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAE,WAAwB,YAElElvD,KAAK+sE,WAAWL,IASlB9sE,EAAQwjD,qBAAuB,WAC7BpjD,KAAKkrD,cAAgBlO,SAAWc,UAEhC99C,KAAK+sE,WAAWJ,IASlB/sE,EAAQ6oD,wBAA0B,WAEhCzoD,KAAKwtE,8BAA+B,EACpCxtE,KAAKytE,sBAAuB,EAEmB,GAA3CztE,KAAK0hD,UAAUnB,iBAAiB7xC,SAELlI,SAAzBxG,KAAK0tE,kBACP1tE,KAAK0tE,gBAAkBn8D,SAASM,cAAc,OAC9C7R,KAAK0tE,gBAAgB7lE,UAAY,0BAE/B7H,KAAK0tE,gBAAgBzgE,MAAM26B,QADR,GAAjB5nC,KAAKkoD,SAC8B,QAGA,OAEvCloD,KAAKsf,MAAM7N,YAAYzR,KAAK0tE,kBAGLlnE,SAArBxG,KAAK2tE,cACP3tE,KAAK2tE,YAAcp8D,SAASM,cAAc,OAC1C7R,KAAK2tE,YAAY9lE,UAAY,gCAE3B7H,KAAK2tE,YAAY1gE,MAAM26B,QADJ,GAAjB5nC,KAAKkoD,SAC0B,OAGA,QAEnCloD,KAAKsf,MAAM7N,YAAYzR,KAAK2tE,cAGRnnE,SAAlBxG,KAAK4tE,WACP5tE,KAAK4tE,SAAWr8D,SAASM,cAAc,OACvC7R,KAAK4tE,SAAS/lE,UAAY,gCAC1B7H,KAAK4tE,SAAS3gE,MAAM26B,QAAU5nC,KAAK0tE,gBAAgBzgE,MAAM26B,QACzD5nC,KAAKsf,MAAM7N,YAAYzR,KAAK4tE,WAI9B5tE,KAAK+sE,WAAWH,GAGhB5sE,KAAKmnD,yBAGwB3gD,SAAzBxG,KAAK0tE,kBAEP1tE,KAAKmnD,wBAGLnnD,KAAKsf,MAAMnO,YAAYnR,KAAK0tE,iBAC5B1tE,KAAKsf,MAAMnO,YAAYnR,KAAK2tE,aAC5B3tE,KAAKsf,MAAMnO,YAAYnR,KAAK4tE,UAE5B5tE,KAAK0tE,gBAAkBlnE,OACvBxG,KAAK2tE,YAAcnnE,OACnBxG,KAAK4tE,SAAWpnE,OAEhBxG,KAAKktE,YAAYN,KAWvBhtE,EAAQ4oD,wBAA0B,WAChCxoD,KAAK+sE,WAAWF,GAEhB7sE,KAAK6tE,mBACoC,GAArC7tE,KAAK0hD,UAAUtB,WAAW1xC,SAC5B1O,KAAK8tE,2BAUTluE,EAAQyjD,qBAAuB,WAC7BrjD,KAAK+sE,WAAWD,KAMd,SAASjtE,EAAQD,EAASM,GAiB9B,QAASolD,GAAU/rC,GACjBvZ,KAAKszD,QAAS,EAEdtzD,KAAK+vB,KACHxW,UAAWA,GAGbvZ,KAAK+vB,IAAIg+C,QAAUx8D,SAASM,cAAc,OAC1C7R,KAAK+vB,IAAIg+C,QAAQlmE,UAAY,UAE7B7H,KAAK+vB,IAAIxW,UAAU9H,YAAYzR,KAAK+vB,IAAIg+C,SAExC/tE,KAAK8D,OAASC,EAAO/D,KAAK+vB,IAAIg+C,SAAUC,iBAAiB,IACzDhuE,KAAK8D,OAAOyP,GAAG,MAAOvT,KAAKiuE,cAAcn5C,KAAK90B,MAG9C,IAAImU,GAAKnU,KACLwlE,GACF,QAAS,QACT,YAAa,OACb,YAAa,OAAQ,UACrB,aAAc,iBAEhBA,GAAOn9D,QAAQ,SAAUiB,GACvB6K,EAAGrQ,OAAOyP,GAAGjK,EAAO,SAAUA,GAC5BA,EAAMq8B,sBAKV3lC,KAAKkuE,aAAenqE,EAAOwF,QAASykE,iBAAiB,IACrDhuE,KAAKkuE,aAAa36D,GAAG,MAAO,SAAUjK,GAE/B6kE,EAAW7kE,EAAMI,OAAQ6P,IAC5BpF,EAAGi6D,eAIe5nE,SAAlBxG,KAAKmlD,UACPnlD,KAAKmlD,SAAS7xC,UAEhBtT,KAAKmlD,SAAWA,IAGhBnlD,KAAKquE,YAAcruE,KAAKouE,WAAWt5C,KAAK90B,MAiF1C,QAASmuE,GAAWvlE,EAASk8B,GAC3B,KAAOl8B,GAAS,CACd,GAAIA,IAAYk8B,EACd,OAAO,CAETl8B,GAAUA,EAAQiB,WAEpB,OAAO,EAnJT,GAAIs7C,GAAWjlD,EAAoB,IAC/B6c,EAAU7c,EAAoB,IAC9B6D,EAAS7D,EAAoB,IAC7BS,EAAOT,EAAoB,EA4D/B6c,GAAQuoC,EAAUnyC,WAGlBmyC,EAAUxrB,QAAU,KAKpBwrB,EAAUnyC,UAAUG,QAAU,WAC5BtT,KAAKouE,aAGLpuE,KAAK+vB,IAAIg+C,QAAQlkE,WAAWsH,YAAYnR,KAAK+vB,IAAIg+C,SAGjD/tE,KAAK8D,OAAS,KACd9D,KAAKkuE,aAAe,MAQtB5oB,EAAUnyC,UAAUm7D,SAAW,WAEzBhpB,EAAUxrB,SACZwrB,EAAUxrB,QAAQs0C,aAEpB9oB,EAAUxrB,QAAU95B,KAEpBA,KAAKszD,QAAS,EACdtzD,KAAK+vB,IAAIg+C,QAAQ9gE,MAAM26B,QAAU,OACjCjnC,EAAKiH,aAAa5H,KAAK+vB,IAAIxW,UAAW,cAEtCvZ,KAAK4tB,KAAK,UACV5tB,KAAK4tB,KAAK,YAIV5tB,KAAKmlD,SAASrwB,KAAK,MAAO90B,KAAKquE,cAOjC/oB,EAAUnyC,UAAUi7D,WAAa,WAC/BpuE,KAAKszD,QAAS,EACdtzD,KAAK+vB,IAAIg+C,QAAQ9gE,MAAM26B,QAAU,GACjCjnC,EAAKuH,gBAAgBlI,KAAK+vB,IAAIxW,UAAW,cACzCvZ,KAAKmlD,SAASopB,OAAO,MAAOvuE,KAAKquE,aAEjCruE,KAAK4tB,KAAK,UACV5tB,KAAK4tB,KAAK,eAQZ03B,EAAUnyC,UAAU86D,cAAgB,SAAU3kE,GAE5CtJ,KAAKsuE,WACLhlE,EAAMq8B,mBAsBR9lC,EAAOD,QAAU0lD,GAKb,SAASzlD,GAeb,QAASkd,GAAQgG,GACf,MAAIA,GAAYqvC,EAAMrvC,GAAtB,OAWF,QAASqvC,GAAMrvC,GACb,IAAK,GAAIra,KAAOqU,GAAQ5J,UACtB4P,EAAIra,GAAOqU,EAAQ5J,UAAUzK,EAE/B,OAAOqa,GAxBTljB,EAAOD,QAAUmd,EAoCjBA,EAAQ5J,UAAUI,GAClBwJ,EAAQ5J,UAAUxK,iBAAmB,SAASW,EAAO4P,GAInD,MAHAlZ,MAAKwuE,WAAaxuE,KAAKwuE,gBACtBxuE,KAAKwuE,WAAWllE,GAAStJ,KAAKwuE,WAAWllE,QACvCtB,KAAKkR,GACDlZ,MAaT+c,EAAQ5J,UAAUs7D,KAAO,SAASnlE,EAAO4P,GAIvC,QAAS3F,KACPm7D,EAAKh7D,IAAIpK,EAAOiK,GAChB2F,EAAGnB,MAAM/X,KAAM0F,WALjB,GAAIgpE,GAAO1uE,IAUX,OATAA,MAAKwuE,WAAaxuE,KAAKwuE,eAOvBj7D,EAAG2F,GAAKA,EACRlZ,KAAKuT,GAAGjK,EAAOiK,GACRvT,MAaT+c,EAAQ5J,UAAUO,IAClBqJ,EAAQ5J,UAAUw7D,eAClB5xD,EAAQ5J,UAAUy7D,mBAClB7xD,EAAQ5J,UAAUhK,oBAAsB,SAASG,EAAO4P,GAItD,GAHAlZ,KAAKwuE,WAAaxuE,KAAKwuE,eAGnB,GAAK9oE,UAAUC,OAEjB,MADA3F,MAAKwuE,cACExuE,IAIT,IAAI6uE,GAAY7uE,KAAKwuE,WAAWllE,EAChC,KAAKulE,EAAW,MAAO7uE,KAGvB,IAAI,GAAK0F,UAAUC,OAEjB,aADO3F,MAAKwuE,WAAWllE,GAChBtJ,IAKT,KAAK,GADD8uE,GACKtpE,EAAI,EAAGA,EAAIqpE,EAAUlpE,OAAQH,IAEpC,GADAspE,EAAKD,EAAUrpE,GACXspE,IAAO51D,GAAM41D,EAAG51D,KAAOA,EAAI,CAC7B21D,EAAUzmE,OAAO5C,EAAG,EACpB,OAGJ,MAAOxF,OAWT+c,EAAQ5J,UAAUya,KAAO,SAAStkB,GAChCtJ,KAAKwuE,WAAaxuE,KAAKwuE,cACvB,IAAIv1D,MAAUhO,MAAM1K,KAAKmF,UAAW,GAChCmpE,EAAY7uE,KAAKwuE,WAAWllE,EAEhC,IAAIulE,EAAW,CACbA,EAAYA,EAAU5jE,MAAM,EAC5B,KAAK,GAAIzF,GAAI,EAAGC,EAAMopE,EAAUlpE,OAAYF,EAAJD,IAAWA,EACjDqpE,EAAUrpE,GAAGuS,MAAM/X,KAAMiZ,GAI7B,MAAOjZ,OAWT+c,EAAQ5J,UAAUoyD,UAAY,SAASj8D,GAErC,MADAtJ,MAAKwuE,WAAaxuE,KAAKwuE,eAChBxuE,KAAKwuE,WAAWllE,QAWzByT,EAAQ5J,UAAU47D,aAAe,SAASzlE,GACxC,QAAUtJ,KAAKulE,UAAUj8D,GAAO3D,SAM9B,SAAS9F,EAAQD,EAASM,GAE9B,GAAI8uE,IAA0D,SAASC,EAAQpvE,IAM/E,SAAW2G,GA+RP,QAAS0oE,GAAI3pE,EAAGa,EAAG3F,GACf,OAAQiF,UAAUC,QACd,IAAK,GAAG,MAAY,OAALJ,EAAYA,EAAIa,CAC/B,KAAK,GAAG,MAAY,OAALb,EAAYA,EAAS,MAALa,EAAYA,EAAI3F,CAC/C,SAAS,KAAM,IAAImD,OAAM,iBAIjC,QAASurE,GAAW5pE,EAAGa,GACnB,MAAON,IAAevF,KAAKgF,EAAGa,GAGlC,QAASgpE,KAGL,OACIC,OAAQ,EACRC,gBACAC,eACA1rD,SAAW,GACX2rD,cAAgB,EAChBC,WAAY,EACZC,aAAe,KACfC,eAAgB,EAChBC,iBAAkB,EAClBC,KAAK,GAIb,QAASC,GAASC,GACVlsE,GAAOmsE,+BAAgC,GAChB,mBAAZr3C,UAA2BA,QAAQs3C,MAC9Ct3C,QAAQs3C,KAAK,wBAA0BF,GAI/C,QAASG,GAAUH,EAAK72D,GACpB,GAAIi3D,IAAY,CAChB,OAAO7qE,GAAO,WAKV,MAJI6qE,KACAL,EAASC,GACTI,GAAY,GAETj3D,EAAGnB,MAAM/X,KAAM0F,YACvBwT,GAGP,QAASk3D,GAAgBn6D,EAAM85D,GACtBM,GAAap6D,KACd65D,EAASC,GACTM,GAAap6D,IAAQ,GAI7B,QAASq6D,GAASC,EAAMv5D,GACpB,MAAO,UAAUzR,GACb,MAAOirE,GAAaD,EAAKhwE,KAAKP,KAAMuF,GAAIyR,IAGhD,QAASy5D,GAAgBF,EAAMG,GAC3B,MAAO,UAAUnrE,GACb,MAAOvF,MAAK2wE,aAAaC,QAAQL,EAAKhwE,KAAKP,KAAMuF,GAAImrE,IAI7D,QAASG,GAAUtrE,EAAGa,GAElB,GAGI0qE,GAASC,EAHTC,EAA0C,IAAvB5qE,EAAEmyB,OAAShzB,EAAEgzB,SAAiBnyB,EAAEsyB,QAAUnzB,EAAEmzB,SAE/D+hB,EAASl1C,EAAE6yB,QAAQnlB,IAAI+9D,EAAgB,SAa3C,OAViB,GAAb5qE,EAAIq0C,GACJq2B,EAAUvrE,EAAE6yB,QAAQnlB,IAAI+9D,EAAiB,EAAG,UAE5CD,GAAU3qE,EAAIq0C,IAAWA,EAASq2B,KAElCA,EAAUvrE,EAAE6yB,QAAQnlB,IAAI+9D,EAAiB,EAAG,UAE5CD,GAAU3qE,EAAIq0C,IAAWq2B,EAAUr2B,MAG9Bu2B,EAAiBD,GAc9B,QAASE,GAAgB5sC,EAAQzC,EAAMsvC,GACnC,GAAIC,EAEJ,OAAgB,OAAZD,EAEOtvC,EAEgB,MAAvByC,EAAO+sC,aACA/sC,EAAO+sC,aAAaxvC,EAAMsvC,GACX,MAAf7sC,EAAOgtC,MAEdF,EAAO9sC,EAAOgtC,KAAKH,GACfC,GAAe,GAAPvvC,IACRA,GAAQ,IAEPuvC,GAAiB,KAATvvC,IACTA,EAAO,GAEJA,GAGAA,EAQf,QAAS0vC,MAIT,QAASC,GAAOC,EAAQC,GAChBA,KAAiB,GACjBC,EAAcF,GAElBG,EAAW3xE,KAAMwxE,GACjBxxE,KAAKk4B,GAAK,GAAI5zB,OAAMktE,EAAOt5C,IAGvB05C,MAAqB,IACrBA,IAAmB,EACnB/tE,GAAOguE,aAAa7xE,MACpB4xE,IAAmB,GAK3B,QAASE,GAAShiE,GACd,GAAIiiE,GAAkBC,EAAqBliE,GACvCmiE,EAAQF,EAAgBx5C,MAAQ,EAChC25C,EAAWH,EAAgBI,SAAW,EACtCC,EAASL,EAAgBr5C,OAAS,EAClC25C,EAAQN,EAAgBO,MAAQ,EAChCC,EAAOR,EAAgB15C,KAAO,EAC9B+E,EAAQ20C,EAAgBnwC,MAAQ,EAChCvE,EAAU00C,EAAgBpwC,QAAU,EACpCrE,EAAUy0C,EAAgBrwC,QAAU,EACpCnE,EAAew0C,EAAgBtwC,aAAe,CAGlDzhC,MAAKwyE,eAAiBj1C,EACR,IAAVD,EACU,IAAVD,EACQ,KAARD,EAGJp9B,KAAKyyE,OAASF,EACF,EAARF,EAIJryE,KAAK0yE,SAAWN,EACD,EAAXF,EACQ,GAARD,EAEJjyE,KAAK4S,SAEL5S,KAAK2yE,QAAU9uE,GAAO8sE,aAEtB3wE,KAAK4yE,UAQT,QAASttE,GAAOC,EAAGa,GACf,IAAK,GAAIZ,KAAKY,GACN+oE,EAAW/oE,EAAGZ,KACdD,EAAEC,GAAKY,EAAEZ,GAYjB,OARI2pE,GAAW/oE,EAAG,cACdb,EAAEF,SAAWe,EAAEf,UAGf8pE,EAAW/oE,EAAG,aACdb,EAAEyB,QAAUZ,EAAEY,SAGXzB,EAGX,QAASosE,GAAWtoD,EAAID,GACpB,GAAI5jB,GAAGK,EAAMgtE,CAiCb,IA/BqC,mBAA1BzpD,GAAK0pD,mBACZzpD,EAAGypD,iBAAmB1pD,EAAK0pD,kBAER,mBAAZ1pD,GAAK2pD,KACZ1pD,EAAG0pD,GAAK3pD,EAAK2pD,IAEM,mBAAZ3pD,GAAK4pD,KACZ3pD,EAAG2pD,GAAK5pD,EAAK4pD,IAEM,mBAAZ5pD,GAAK6pD,KACZ5pD,EAAG4pD,GAAK7pD,EAAK6pD,IAEW,mBAAjB7pD,GAAK8pD,UACZ7pD,EAAG6pD,QAAU9pD,EAAK8pD,SAEG,mBAAd9pD,GAAK+pD,OACZ9pD,EAAG8pD,KAAO/pD,EAAK+pD,MAEQ,mBAAhB/pD,GAAKgqD,SACZ/pD,EAAG+pD,OAAShqD,EAAKgqD,QAEO,mBAAjBhqD,GAAKiqD,UACZhqD,EAAGgqD,QAAUjqD,EAAKiqD,SAEE,mBAAbjqD,GAAKkqD,MACZjqD,EAAGiqD,IAAMlqD,EAAKkqD,KAEU,mBAAjBlqD,GAAKupD,UACZtpD,EAAGspD,QAAUvpD,EAAKupD,SAGlBY,GAAiB5tE,OAAS,EAC1B,IAAKH,IAAK+tE,IACN1tE,EAAO0tE,GAAiB/tE,GACxBqtE,EAAMzpD,EAAKvjB,GACQ,mBAARgtE,KACPxpD,EAAGxjB,GAAQgtE,EAKvB,OAAOxpD,GAGX,QAASmqD,GAASC,GACd,MAAa,GAATA,EACOvuE,KAAKwyC,KAAK+7B,GAEVvuE,KAAKC,MAAMsuE,GAM1B,QAASjD,GAAaiD,EAAQC,EAAcC,GAIxC,IAHA,GAAIC,GAAS,GAAK1uE,KAAK2lB,IAAI4oD,GACvBzkD,EAAOykD,GAAU,EAEdG,EAAOjuE,OAAS+tE,GACnBE,EAAS,IAAMA,CAEnB,QAAQ5kD,EAAQ2kD,EAAY,IAAM,GAAM,KAAOC,EAGnD,QAASC,GAA0BC,EAAMluE,GACrC,GAAImuE,IAAOx2C,aAAc,EAAG60C,OAAQ,EAUpC,OARA2B,GAAI3B,OAASxsE,EAAM8yB,QAAUo7C,EAAKp7C,QACC,IAA9B9yB,EAAM2yB,OAASu7C,EAAKv7C,QACrBu7C,EAAK17C,QAAQnlB,IAAI8gE,EAAI3B,OAAQ,KAAK4B,QAAQpuE,MACxCmuE,EAAI3B,OAGV2B,EAAIx2C,cAAgB33B,GAAUkuE,EAAK17C,QAAQnlB,IAAI8gE,EAAI3B,OAAQ,KAEpD2B,EAGX,QAASE,GAAkBH,EAAMluE,GAC7B,GAAImuE,EAUJ,OATAnuE,GAAQsuE,EAAOtuE,EAAOkuE,GAClBA,EAAKK,SAASvuE,GACdmuE,EAAMF,EAA0BC,EAAMluE,IAEtCmuE,EAAMF,EAA0BjuE,EAAOkuE,GACvCC,EAAIx2C,cAAgBw2C,EAAIx2C,aACxBw2C,EAAI3B,QAAU2B,EAAI3B,QAGf2B,EAIX,QAASK,GAAYl5C,EAAWjlB,GAC5B,MAAO,UAAU48D,EAAKnC,GAClB,GAAI2D,GAAKC,CAUT,OARe,QAAX5D,GAAoBhsE,OAAOgsE,KAC3BN,EAAgBn6D,EAAM,YAAcA,EAAQ,uDAAyDA,EAAO,qBAC5Gq+D,EAAMzB,EAAKA,EAAMnC,EAAQA,EAAS4D,GAGtCzB,EAAqB,gBAARA,IAAoBA,EAAMA,EACvCwB,EAAMxwE,GAAOiM,SAAS+iE,EAAKnC,GAC3B6D,EAAgCv0E,KAAMq0E,EAAKn5C,GACpCl7B,MAIf,QAASu0E,GAAgCC,EAAK1kE,EAAU2kE,EAAU5C,GAC9D,GAAIt0C,GAAeztB,EAAS0iE,cACxBD,EAAOziE,EAAS2iE,MAChBL,EAAStiE,EAAS4iE,OACtBb,GAA+B,MAAhBA,GAAuB,EAAOA,EAEzCt0C,GACAi3C,EAAIt8C,GAAGw8C,SAASF,EAAIt8C,GAAKqF,EAAek3C,GAExClC,GACAoC,GAAUH,EAAK,OAAQI,GAAUJ,EAAK,QAAUjC,EAAOkC,GAEvDrC,GACAyC,GAAeL,EAAKI,GAAUJ,EAAK,SAAWpC,EAASqC,GAEvD5C,GACAhuE,GAAOguE,aAAa2C,EAAKjC,GAAQH,GAKzC,QAASlsE,GAAQ4uE,GACb,MAAiD,mBAA1CvuE,OAAO4M,UAAU9N,SAAS9E,KAAKu0E,GAG1C,QAASzwE,GAAOywE,GACZ,MAAiD,kBAA1CvuE,OAAO4M,UAAU9N,SAAS9E,KAAKu0E,IAClCA,YAAiBxwE,MAIzB,QAASywE,GAAc/R,EAAQC,EAAQ+R,GACnC,GAGIxvE,GAHAC,EAAMP,KAAK4G,IAAIk3D,EAAOr9D,OAAQs9D,EAAOt9D,QACrCsvE,EAAa/vE,KAAK2lB,IAAIm4C,EAAOr9D,OAASs9D,EAAOt9D,QAC7CuvE,EAAQ,CAEZ,KAAK1vE,EAAI,EAAOC,EAAJD,EAASA,KACZwvE,GAAehS,EAAOx9D,KAAOy9D,EAAOz9D,KACnCwvE,GAAeG,EAAMnS,EAAOx9D,MAAQ2vE,EAAMlS,EAAOz9D,MACnD0vE,GAGR,OAAOA,GAAQD,EAGnB,QAASG,GAAeC,GACpB,GAAIA,EAAO,CACP,GAAIC,GAAUD,EAAM9wC,cAAc/5B,QAAQ,QAAS,KACnD6qE,GAAQE,GAAYF,IAAUG,GAAeF,IAAYA,EAE7D,MAAOD,GAGX,QAASrD,GAAqByD,GAC1B,GACIC,GACA7vE,EAFAksE,IAIJ,KAAKlsE,IAAQ4vE,GACLtG,EAAWsG,EAAa5vE,KACxB6vE,EAAiBN,EAAevvE,GAC5B6vE,IACA3D,EAAgB2D,GAAkBD,EAAY5vE,IAK1D,OAAOksE,GAGX,QAAS4D,GAAS7mE,GACd,GAAIkI,GAAO4+D,CAEX,IAA8B,IAA1B9mE,EAAMnI,QAAQ,QACdqQ,EAAQ,EACR4+D,EAAS,UAER,CAAA,GAA+B,IAA3B9mE,EAAMnI,QAAQ,SAKnB,MAJAqQ,GAAQ,GACR4+D,EAAS,QAMb/xE,GAAOiL,GAAS,SAAUwyB,EAAQn5B,GAC9B,GAAI3C,GAAGqwE,EACH78D,EAASnV,GAAO8uE,QAAQ7jE,GACxBgnE,IAYJ,IAVsB,gBAAXx0C,KACPn5B,EAAQm5B,EACRA,EAAS96B,GAGbqvE,EAAS,SAAUrwE,GACf,GAAIhF,GAAIqD,KAASkyE,MAAM5sB,IAAIysB,EAAQpwE,EACnC,OAAOwT,GAAOzY,KAAKsD,GAAO8uE,QAASnyE,EAAG8gC,GAAU,KAGvC,MAATn5B,EACA,MAAO0tE,GAAO1tE,EAGd,KAAK3C,EAAI,EAAOwR,EAAJxR,EAAWA,IACnBswE,EAAQ9tE,KAAK6tE,EAAOrwE,GAExB,OAAOswE,IAKnB,QAASX,GAAMa,GACX,GAAIC,IAAiBD,EACjB3uE,EAAQ,CAUZ,OARsB,KAAlB4uE,GAAuBC,SAASD,KAE5B5uE,EADA4uE,GAAiB,EACT/wE,KAAKC,MAAM8wE,GAEX/wE,KAAKwyC,KAAKu+B,IAInB5uE,EAGX,QAAS8uE,GAAY59C,EAAMG,GACvB,MAAO,IAAIp0B,MAAKA,KAAK8xE,IAAI79C,EAAMG,EAAQ,EAAG,IAAI29C,aAGlD,QAASC,GAAY/9C,EAAMg+C,EAAKC,GAC5B,MAAOC,IAAW5yE,IAAQ00B,EAAM,GAAI,GAAKg+C,EAAMC,IAAOD,EAAKC,GAAKlE,KAGpE,QAASoE,GAAWn+C,GAChB,MAAOo+C,GAAWp+C,GAAQ,IAAM,IAGpC,QAASo+C,GAAWp+C,GAChB,MAAQA,GAAO,IAAM,GAAKA,EAAO,MAAQ,GAAMA,EAAO,MAAQ,EAGlE,QAASm5C,GAAclxE,GACnB,GAAIqjB,EACArjB,GAAEo2E,IAAyB,KAAnBp2E,EAAE8yE,IAAIzvD,WACdA,EACIrjB,EAAEo2E,GAAGC,IAAS,GAAKr2E,EAAEo2E,GAAGC,IAAS,GAAKA,GACtCr2E,EAAEo2E,GAAGE,IAAQ,GAAKt2E,EAAEo2E,GAAGE,IAAQX,EAAY31E,EAAEo2E,GAAGG,IAAOv2E,EAAEo2E,GAAGC,KAAUC,GACtEt2E,EAAEo2E,GAAGI,IAAQ,GAAKx2E,EAAEo2E,GAAGI,IAAQ,IACX,KAAfx2E,EAAEo2E,GAAGI,MAAkC,IAAjBx2E,EAAEo2E,GAAGK,KACY,IAAjBz2E,EAAEo2E,GAAGM,KACiB,IAAtB12E,EAAEo2E,GAAGO,KAAuBH,GACvDx2E,EAAEo2E,GAAGK,IAAU,GAAKz2E,EAAEo2E,GAAGK,IAAU,GAAKA,GACxCz2E,EAAEo2E,GAAGM,IAAU,GAAK12E,EAAEo2E,GAAGM,IAAU,GAAKA,GACxC12E,EAAEo2E,GAAGO,IAAe,GAAK32E,EAAEo2E,GAAGO,IAAe,IAAMA,GACnD,GAEA32E,EAAE8yE,IAAI8D,qBAAkCL,GAAXlzD,GAAmBA,EAAWizD,MAC3DjzD,EAAWizD,IAGft2E,EAAE8yE,IAAIzvD,SAAWA,GAIzB,QAASwzD,GAAQ72E,GAiBb,MAhBkB,OAAdA,EAAE82E,WACF92E,EAAE82E,UAAY5yE,MAAMlE,EAAE03B,GAAGq/C,YACrB/2E,EAAE8yE,IAAIzvD,SAAW,IAChBrjB,EAAE8yE,IAAIjE,QACN7uE,EAAE8yE,IAAI5D,eACNlvE,EAAE8yE,IAAI7D,YACNjvE,EAAE8yE,IAAI3D,gBACNnvE,EAAE8yE,IAAI1D,gBAEPpvE,EAAE0yE,UACF1yE,EAAE82E,SAAW92E,EAAE82E,UACa,IAAxB92E,EAAE8yE,IAAI9D,eACwB,IAA9BhvE,EAAE8yE,IAAIhE,aAAa3pE,QACnBnF,EAAE8yE,IAAIkE,UAAYhxE,IAGvBhG,EAAE82E,SAGb,QAASG,GAAgB/uE,GACrB,MAAOA,GAAMA,EAAI67B,cAAc/5B,QAAQ,IAAK,KAAO9B,EAMvD,QAASgvE,GAAaC,GAGlB,IAFA,GAAW/rD,GAAGvD,EAAMgc,EAAQt8B,EAAxBvC,EAAI,EAEDA,EAAImyE,EAAMhyE,QAAQ,CAKrB,IAJAoC,EAAQ0vE,EAAgBE,EAAMnyE,IAAIuC,MAAM,KACxC6jB,EAAI7jB,EAAMpC,OACV0iB,EAAOovD,EAAgBE,EAAMnyE,EAAI,IACjC6iB,EAAOA,EAAOA,EAAKtgB,MAAM,KAAO,KACzB6jB,EAAI,GAAG,CAEV,GADAyY,EAASuzC,EAAW7vE,EAAMkD,MAAM,EAAG2gB,GAAG3jB,KAAK,MAEvC,MAAOo8B,EAEX,IAAIhc,GAAQA,EAAK1iB,QAAUimB,GAAKmpD,EAAchtE,EAAOsgB,GAAM,IAASuD,EAAI,EAEpE,KAEJA,KAEJpmB,IAEJ,MAAO,MAGX,QAASoyE,GAAW3hE,GAChB,GAAI4hE,GAAY,IAChB,KAAKhzC,GAAQ5uB,IAAS6hE,GAClB,IACID,EAAYh0E,GAAOwgC,UACjB,WAAkC,GAAIvN,GAAI,GAAIlzB,OAAM,gCAAiE,MAA7BkzB,GAAEihD,KAAO,mBAA0BjhD,KAE7HjzB,GAAOwgC,OAAOwzC,GAChB,MAAO/gD,IAEb,MAAO+N,IAAQ5uB,GAKnB,QAASi+D,GAAOY,EAAOkD,GACnB,GAAIjE,GAAK1nD,CACT,OAAI2rD,GAAM5E,QACNW,EAAMiE,EAAM5/C,QACZ/L,GAAQxoB,GAAOoD,SAAS6tE,IAAUzwE,EAAOywE,IAChCA,GAASjxE,GAAOixE,KAAYf,EAErCA,EAAI77C,GAAGw8C,SAASX,EAAI77C,GAAK7L,GACzBxoB,GAAOguE,aAAakC,GAAK,GAClBA,GAEAlwE,GAAOixE,GAAOmD,QA6N7B,QAASC,GAAuBpD,GAC5B,MAAIA,GAAMvwE,MAAM,YACLuwE,EAAMtqE,QAAQ,WAAY,IAE9BsqE,EAAMtqE,QAAQ,MAAO,IAGhC,QAAS2tE,GAAmB72C,GACxB,GAA4C97B,GAAGG,EAA3C6C,EAAQ84B,EAAO/8B,MAAM6zE,GAEzB,KAAK5yE,EAAI,EAAGG,EAAS6C,EAAM7C,OAAYA,EAAJH,EAAYA,IAEvCgD,EAAMhD,GADN6yE,GAAqB7vE,EAAMhD,IAChB6yE,GAAqB7vE,EAAMhD,IAE3B0yE,EAAuB1vE,EAAMhD,GAIhD,OAAO,UAAUgvE,GACb,GAAIZ,GAAS,EACb,KAAKpuE,EAAI,EAAOG,EAAJH,EAAYA,IACpBouE,GAAUprE,EAAMhD,YAAcwrC,UAAWxoC,EAAMhD,GAAGjF,KAAKi0E,EAAKlzC,GAAU94B,EAAMhD,EAEhF,OAAOouE,IAKf,QAAS0E,GAAa93E,EAAG8gC,GACrB,MAAK9gC,GAAE62E,WAIP/1C,EAASi3C,EAAaj3C,EAAQ9gC,EAAEmwE,cAE3B6H,GAAgBl3C,KACjBk3C,GAAgBl3C,GAAU62C,EAAmB72C,IAG1Ck3C,GAAgBl3C,GAAQ9gC,IATpBA,EAAEmwE,aAAa8H,cAY9B,QAASF,GAAaj3C,EAAQ+C,GAG1B,QAASq0C,GAA4B5D,GACjC,MAAOzwC,GAAOs0C,eAAe7D,IAAUA,EAH3C,GAAItvE,GAAI,CAOR,KADAozE,GAAsBC,UAAY,EAC3BrzE,GAAK,GAAKozE,GAAsB5qE,KAAKszB,IACxCA,EAASA,EAAO92B,QAAQouE,GAAuBF,GAC/CE,GAAsBC,UAAY,EAClCrzE,GAAK,CAGT,OAAO87B,GAUX,QAASw3C,GAAsBpX,EAAO8P,GAClC,GAAIjsE,GAAG48D,EAASqP,EAAO0B,OACvB,QAAQxR,GACR,IAAK,IACD,MAAOqX,GACX,KAAK,OACD,MAAOC,GACX,KAAK,OACL,IAAK,OACL,IAAK,OACD,MAAO7W,GAAS8W,GAAuBC,EAC3C,KAAK,IACL,IAAK,IACL,IAAK,IACD,MAAOC,GACX,KAAK,SACL,IAAK,QACL,IAAK,QACL,IAAK,QACD,MAAOhX,GAASiX,GAAsBC,EAC1C,KAAK,IACD,GAAIlX,EACA,MAAO4W,GAGf,KAAK,KACD,GAAI5W,EACA,MAAOmX,GAGf,KAAK,MACD,GAAInX,EACA,MAAO6W,GAGf,KAAK,MACD,MAAOO,GACX,KAAK,MACL,IAAK,OACL,IAAK,KACL,IAAK,MACL,IAAK,OACD,MAAOC,GACX,KAAK,IACL,IAAK,IACD,MAAOhI,GAAOmB,QAAQ8G,cAC1B,KAAK,IACD,MAAOC,GACX,KAAK,IACD,MAAOC,GACX,KAAK,IACL,IAAK,KACD,MAAOC,GACX,KAAK,IACD,MAAOC,GACX,KAAK,OACD,MAAOC,GACX,KAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACL,IAAK,KACD,MAAO3X,GAASmX,GAAsBS,EAC1C,KAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACL,IAAK,IACD,MAAOA,GACX,KAAK,KACD,MAAO5X,GAASqP,EAAOmB,QAAQqH,cAAgBxI,EAAOmB,QAAQsH,oBAClE,SAEI,MADA10E,GAAI,GAAI20E,QAAOC,GAAaC,GAAe1Y,EAAMl3D,QAAQ,KAAM,KAAM,OAK7E,QAAS6vE,GAAoBC,GACzBA,EAASA,GAAU,EACnB,IAAIC,GAAqBD,EAAO/1E,MAAMq1E,QAClCY,EAAUD,EAAkBA,EAAkB50E,OAAS,OACvDwH,GAASqtE,EAAU,IAAIj2E,MAAMk2E,MAA0B,IAAK,EAAG,GAC/Dp9C,IAAuB,GAAXlwB,EAAM,IAAWgoE,EAAMhoE,EAAM,GAE7C,OAAoB,MAAbA,EAAM,GAAakwB,GAAWA,EAIzC,QAASq9C,GAAwBhZ,EAAOoT,EAAOtD,GAC3C,GAAIjsE,GAAGo1E,EAAgBnJ,EAAOoF,EAE9B,QAAQlV,GAER,IAAK,IACY,MAAToT,IACA6F,EAAc9D,IAA8B,GAApB1B,EAAML,GAAS,GAE3C,MAEJ,KAAK,IACL,IAAK,KACY,MAATA,IACA6F,EAAc9D,IAAS1B,EAAML,GAAS,EAE1C,MACJ,KAAK,MACL,IAAK,OACDvvE,EAAIisE,EAAOmB,QAAQiI,YAAY9F,EAAOpT,EAAO8P,EAAO0B,SAE3C,MAAL3tE,EACAo1E,EAAc9D,IAAStxE,EAEvBisE,EAAO8B,IAAI5D,aAAeoF,CAE9B,MAEJ,KAAK,IACL,IAAK,KACY,MAATA,IACA6F,EAAc7D,IAAQ3B,EAAML,GAEhC,MACJ,KAAK,KACY,MAATA,IACA6F,EAAc7D,IAAQ3B,EAAMvqE,SAChBkqE,EAAMvwE,MAAM,WAAW,GAAI,KAE3C,MAEJ,KAAK,MACL,IAAK,OACY,MAATuwE,IACAtD,EAAOqJ,WAAa1F,EAAML,GAG9B,MAEJ,KAAK,KACD6F,EAAc5D,IAAQlzE,GAAOi3E,kBAAkBhG,EAC/C,MACJ,KAAK,OACL,IAAK,QACL,IAAK,SACD6F,EAAc5D,IAAQ5B,EAAML,EAC5B,MAEJ,KAAK,IACL,IAAK,IACDtD,EAAOuJ,UAAYjG,CAEnB,MAEJ,KAAK,IACL,IAAK,KACDtD,EAAO8B,IAAIkE,SAAU,CAEzB,KAAK,IACL,IAAK,KACDmD,EAAc3D,IAAQ7B,EAAML,EAC5B,MAEJ,KAAK,IACL,IAAK,KACD6F,EAAc1D,IAAU9B,EAAML,EAC9B,MAEJ,KAAK,IACL,IAAK,KACD6F,EAAczD,IAAU/B,EAAML,EAC9B,MAEJ,KAAK,IACL,IAAK,KACL,IAAK,MACL,IAAK,OACD6F,EAAcxD,IAAehC,EAAuB,KAAhB,KAAOL,GAC3C,MAEJ,KAAK,IACDtD,EAAOt5C,GAAK,GAAI5zB,MAAK6wE,EAAML,GAC3B,MAEJ,KAAK,IACDtD,EAAOt5C,GAAK,GAAI5zB,MAAyB,IAApB+gB,WAAWyvD,GAChC,MAEJ,KAAK,IACL,IAAK,KACDtD,EAAOwJ,SAAU,EACjBxJ,EAAO2B,KAAOkH,EAAoBvF,EAClC,MAEJ,KAAK,KACL,IAAK,MACL,IAAK,OACDvvE,EAAIisE,EAAOmB,QAAQsI,cAAcnG,GAExB,MAALvvE,GACAisE,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAM,EAAI31E,GAEjBisE,EAAO8B,IAAI6H,eAAiBrG,CAEhC,MAEJ,KAAK,IACL,IAAK,KACL,IAAK,IACL,IAAK,KACL,IAAK,IACL,IAAK,IACL,IAAK,IACDpT,EAAQA,EAAMp2D,OAAO,EAAG,EAE5B,KAAK,OACL,IAAK,OACL,IAAK,QACDo2D,EAAQA,EAAMp2D,OAAO,EAAG,GACpBwpE,IACAtD,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAGxZ,GAASyT,EAAML,GAE7B,MACJ,KAAK,KACL,IAAK,KACDtD,EAAO0J,GAAK1J,EAAO0J,OACnB1J,EAAO0J,GAAGxZ,GAAS79D,GAAOi3E,kBAAkBhG,IAIpD,QAASsG,GAAsB5J,GAC3B,GAAIniB,GAAGgsB,EAAU/I,EAAMzwC,EAAS00C,EAAKC,EAAK8E,CAE1CjsB,GAAImiB,EAAO0J,GACC,MAAR7rB,EAAEksB,IAAqB,MAAPlsB,EAAEmsB,GAAoB,MAAPnsB,EAAEosB,GACjClF,EAAM,EACNC,EAAM,EAMN6E,EAAWnM,EAAI7f,EAAEksB,GAAI/J,EAAOoF,GAAGG,IAAON,GAAW5yE,KAAU,EAAG,GAAG00B,MACjE+5C,EAAOpD,EAAI7f,EAAEmsB,EAAG,GAChB35C,EAAUqtC,EAAI7f,EAAEosB,EAAG,KAEnBlF,EAAM/E,EAAOmB,QAAQ+I,MAAMnF,IAC3BC,EAAMhF,EAAOmB,QAAQ+I,MAAMlF,IAE3B6E,EAAWnM,EAAI7f,EAAEssB,GAAInK,EAAOoF,GAAGG,IAAON,GAAW5yE,KAAU0yE,EAAKC,GAAKj+C,MACrE+5C,EAAOpD,EAAI7f,EAAEA,EAAG,GAEL,MAAPA,EAAE1iD,GAEFk1B,EAAUwtB,EAAE1iD,EACE4pE,EAAV10C,KACEywC,GAINzwC,EAFc,MAAPwtB,EAAEv4B,EAECu4B,EAAEv4B,EAAIy/C,EAGNA,GAGlB+E,EAAOM,GAAmBP,EAAU/I,EAAMzwC,EAAS20C,EAAKD,GAExD/E,EAAOoF,GAAGG,IAAQuE,EAAK/iD,KACvBi5C,EAAOqJ,WAAaS,EAAKhjD,UAO7B,QAASujD,GAAerK,GACpB,GAAIhsE,GAAGizB,EAAkBqjD,EAAaC,EAAzBjH,IAEb,KAAItD,EAAOt5C,GAAX,CA6BA,IAzBA4jD,EAAcE,GAAiBxK,GAG3BA,EAAO0J,IAAyB,MAAnB1J,EAAOoF,GAAGE,KAAqC,MAApBtF,EAAOoF,GAAGC,KAClDuE,EAAsB5J,GAItBA,EAAOqJ,aACPkB,EAAY7M,EAAIsC,EAAOoF,GAAGG,IAAO+E,EAAY/E,KAEzCvF,EAAOqJ,WAAanE,EAAWqF,KAC/BvK,EAAO8B,IAAI8D,oBAAqB,GAGpC3+C,EAAOwjD,GAAYF,EAAW,EAAGvK,EAAOqJ,YACxCrJ,EAAOoF,GAAGC,IAASp+C,EAAKyjD,cACxB1K,EAAOoF,GAAGE,IAAQr+C,EAAK49C,cAQtB7wE,EAAI,EAAO,EAAJA,GAAyB,MAAhBgsE,EAAOoF,GAAGpxE,KAAcA,EACzCgsE,EAAOoF,GAAGpxE,GAAKsvE,EAAMtvE,GAAKs2E,EAAYt2E,EAI1C,MAAW,EAAJA,EAAOA,IACVgsE,EAAOoF,GAAGpxE,GAAKsvE,EAAMtvE,GAAsB,MAAhBgsE,EAAOoF,GAAGpxE,GAAqB,IAANA,EAAU,EAAI,EAAKgsE,EAAOoF,GAAGpxE,EAI7D,MAApBgsE,EAAOoF,GAAGI,KACgB,IAAtBxF,EAAOoF,GAAGK,KACY,IAAtBzF,EAAOoF,GAAGM,KACiB,IAA3B1F,EAAOoF,GAAGO,MACd3F,EAAO2K,UAAW,EAClB3K,EAAOoF,GAAGI,IAAQ,GAGtBxF,EAAOt5C,IAAMs5C,EAAOwJ,QAAUiB,GAAcG,IAAUrkE,MAAM,KAAM+8D,GAG/C,MAAftD,EAAO2B,MACP3B,EAAOt5C,GAAGmkD,cAAc7K,EAAOt5C,GAAGokD,gBAAkB9K,EAAO2B,MAG3D3B,EAAO2K,WACP3K,EAAOoF,GAAGI,IAAQ,KAI1B,QAASuF,GAAe/K,GACpB,GAAIO,EAEAP,GAAOt5C,KAIX65C,EAAkBC,EAAqBR,EAAOuB,IAC9CvB,EAAOoF,IACH7E,EAAgBx5C,KAChBw5C,EAAgBr5C,MAChBq5C,EAAgB15C,KAAO05C,EAAgBt5C,KACvCs5C,EAAgBnwC,KAChBmwC,EAAgBpwC,OAChBowC,EAAgBrwC,OAChBqwC,EAAgBtwC,aAGpBo6C,EAAerK,IAGnB,QAASwK,IAAiBxK,GACtB,GAAIr0C,GAAM,GAAI74B,KACd,OAAIktE,GAAOwJ,SAEH79C,EAAIq/C,iBACJr/C,EAAI++C,cACJ/+C,EAAIk5C,eAGAl5C,EAAIgF,cAAehF,EAAI4F,WAAY5F,EAAI2F,WAKvD,QAAS25C,IAA4BjL,GACjC,GAAIA,EAAOwB,KAAOnvE,GAAO64E,SAErB,WADAC,IAASnL,EAIbA,GAAOoF,MACPpF,EAAO8B,IAAIjE,OAAQ,CAGnB,IACI7pE,GAAGo3E,EAAaC,EAAQnb,EAAOob,EAD/BxC,EAAS,GAAK9I,EAAOuB,GAErBgK,EAAezC,EAAO30E,OACtBq3E,EAAyB,CAI7B,KAFAH,EAAStE,EAAa/G,EAAOwB,GAAIxB,EAAOmB,SAASpuE,MAAM6zE,QAElD5yE,EAAI,EAAGA,EAAIq3E,EAAOl3E,OAAQH,IAC3Bk8D,EAAQmb,EAAOr3E,GACfo3E,GAAetC,EAAO/1E,MAAMu0E,EAAsBpX,EAAO8P,SAAgB,GACrEoL,IACAE,EAAUxC,EAAOhvE,OAAO,EAAGgvE,EAAO3zE,QAAQi2E,IACtCE,EAAQn3E,OAAS,GACjB6rE,EAAO8B,IAAI/D,YAAYvnE,KAAK80E,GAEhCxC,EAASA,EAAOrvE,MAAMqvE,EAAO3zE,QAAQi2E,GAAeA,EAAYj3E,QAChEq3E,GAA0BJ,EAAYj3E,QAGtC0yE,GAAqB3W,IACjBkb,EACApL,EAAO8B,IAAIjE,OAAQ,EAGnBmC,EAAO8B,IAAIhE,aAAatnE,KAAK05D,GAEjCgZ,EAAwBhZ,EAAOkb,EAAapL,IAEvCA,EAAO0B,UAAY0J,GACxBpL,EAAO8B,IAAIhE,aAAatnE,KAAK05D,EAKrC8P,GAAO8B,IAAI9D,cAAgBuN,EAAeC,EACtC1C,EAAO30E,OAAS,GAChB6rE,EAAO8B,IAAI/D,YAAYvnE,KAAKsyE,GAI5B9I,EAAO8B,IAAIkE,WAAY,GAAQhG,EAAOoF,GAAGI,KAAS,KAClDxF,EAAO8B,IAAIkE,QAAUhxE,GAGzBgrE,EAAOoF,GAAGI,IAAQ/F,EAAgBO,EAAOmB,QAASnB,EAAOoF,GAAGI,IACpDxF,EAAOuJ,WACfc,EAAerK,GACfE,EAAcF,GAGlB,QAAS4I,IAAexuE,GACpB,MAAOA,GAAEpB,QAAQ,sCAAuC,SAAUyyE,EAAS7S,EAAIC,EAAIC,EAAI4S,GACnF,MAAO9S,IAAMC,GAAMC,GAAM4S,IAKjC,QAAS/C,IAAavuE,GAClB,MAAOA,GAAEpB,QAAQ,yBAA0B,QAI/C,QAAS2yE,IAA2B3L,GAChC,GAAI4L,GACAC,EAEAC,EACA93E,EACA+3E,CAEJ,IAAyB,IAArB/L,EAAOwB,GAAGrtE,OAGV,MAFA6rE,GAAO8B,IAAI3D,eAAgB,OAC3B6B,EAAOt5C,GAAK,GAAI5zB,MAAKk5E,KAIzB,KAAKh4E,EAAI,EAAGA,EAAIgsE,EAAOwB,GAAGrtE,OAAQH,IAC9B+3E,EAAe,EACfH,EAAazL,KAAeH,GACN,MAAlBA,EAAOwJ,UACPoC,EAAWpC,QAAUxJ,EAAOwJ,SAEhCoC,EAAW9J,IAAMlE,IACjBgO,EAAWpK,GAAKxB,EAAOwB,GAAGxtE,GAC1Bi3E,GAA4BW,GAEvB/F,EAAQ+F,KAKbG,GAAgBH,EAAW9J,IAAI9D,cAG/B+N,GAAqD,GAArCH,EAAW9J,IAAIhE,aAAa3pE,OAE5Cy3E,EAAW9J,IAAImK,MAAQF,GAEJ,MAAfD,GAAsCA,EAAfC,KACvBD,EAAcC,EACdF,EAAaD,GAIrB93E,GAAOksE,EAAQ6L,GAAcD,GAIjC,QAAST,IAASnL,GACd,GAAIhsE,GAAGk4E,EACHpD,EAAS9I,EAAOuB,GAChBxuE,EAAQo5E,GAASl5E,KAAK61E,EAE1B,IAAI/1E,EAAO,CAEP,IADAitE,EAAO8B,IAAIzD,KAAM,EACZrqE,EAAI,EAAGk4E,EAAIE,GAASj4E,OAAY+3E,EAAJl4E,EAAOA,IACpC,GAAIo4E,GAASp4E,GAAG,GAAGf,KAAK61E,GAAS,CAE7B9I,EAAOwB,GAAK4K,GAASp4E,GAAG,IAAMjB,EAAM,IAAM,IAC1C,OAGR,IAAKiB,EAAI,EAAGk4E,EAAIG,GAASl4E,OAAY+3E,EAAJl4E,EAAOA,IACpC,GAAIq4E,GAASr4E,GAAG,GAAGf,KAAK61E,GAAS,CAC7B9I,EAAOwB,IAAM6K,GAASr4E,GAAG,EACzB,OAGJ80E,EAAO/1E,MAAMq1E,MACbpI,EAAOwB,IAAM,KAEjByJ,GAA4BjL,OAE5BA,GAAO8F,UAAW,EAK1B,QAASwG,IAAmBtM,GACxBmL,GAASnL,GACLA,EAAO8F,YAAa,UACb9F,GAAO8F,SACdzzE,GAAOk6E,wBAAwBvM,IAIvC,QAASnkE,IAAI+sC,EAAKlhC,GACd,GAAc1T,GAAVuuE,IACJ,KAAKvuE,EAAI,EAAGA,EAAI40C,EAAIz0C,SAAUH,EAC1BuuE,EAAI/rE,KAAKkR,EAAGkhC,EAAI50C,GAAIA,GAExB,OAAOuuE,GAGX,QAASiK,IAAkBxM,GACvB,GAAuByL,GAAnBnI,EAAQtD,EAAOuB,EACf+B,KAAUtuE,EACVgrE,EAAOt5C,GAAK,GAAI5zB,MACTD,EAAOywE,GACdtD,EAAOt5C,GAAK,GAAI5zB,OAAMwwE,GAC6B,QAA3CmI,EAAUgB,GAAgBx5E,KAAKqwE,IACvCtD,EAAOt5C,GAAK,GAAI5zB,OAAM24E,EAAQ,IACN,gBAAVnI,GACdgJ,GAAmBtM,GACZtrE,EAAQ4uE,IACftD,EAAOoF,GAAKvpE,GAAIynE,EAAM7pE,MAAM,GAAI,SAAU8X,GACtC,MAAOnY,UAASmY,EAAK,MAEzB84D,EAAerK,IACU,gBAAZ,GACb+K,EAAe/K,GACU,gBAAZ,GAEbA,EAAOt5C,GAAK,GAAI5zB,MAAKwwE,GAErBjxE,GAAOk6E,wBAAwBvM,GAIvC,QAAS4K,IAASpqE,EAAGxR,EAAGmM,EAAGhB,EAAGo/D,EAAGn/D,EAAGsyE,GAGhC,GAAIzlD,GAAO,GAAIn0B,MAAK0N,EAAGxR,EAAGmM,EAAGhB,EAAGo/D,EAAGn/D,EAAGsyE,EAMtC,OAHQ,MAAJlsE,GACAymB,EAAKyJ,YAAYlwB,GAEdymB,EAGX,QAASwjD,IAAYjqE,GACjB,GAAIymB,GAAO,GAAIn0B,MAAKA,KAAK8xE,IAAIr+D,MAAM,KAAMrS,WAIzC,OAHQ,MAAJsM,GACAymB,EAAK0lD,eAAensE,GAEjBymB,EAGX,QAAS2lD,IAAatJ,EAAOzwC,GACzB,GAAqB,gBAAVywC,GACP,GAAKpwE,MAAMowE,IAKP,GADAA,EAAQzwC,EAAO42C,cAAcnG,GACR,gBAAVA,GACP,MAAO,UALXA,GAAQlqE,SAASkqE,EAAO,GAShC,OAAOA,GASX,QAASuJ,IAAkB/D,EAAQ7G,EAAQ6K,EAAeC,EAAUl6C,GAChE,MAAOA,GAAOm6C,aAAa/K,GAAU,IAAK6K,EAAehE,EAAQiE,GAGrE,QAASC,IAAaC,EAAgBH,EAAej6C,GACjD,GAAIv0B,GAAWjM,GAAOiM,SAAS2uE,GAAgB5zD,MAC3CyS,EAAU5P,GAAM5d,EAASmf,GAAG,MAC5BoO,EAAU3P,GAAM5d,EAASmf,GAAG,MAC5BmO,EAAQ1P,GAAM5d,EAASmf,GAAG,MAC1BsjD,EAAO7kD,GAAM5d,EAASmf,GAAG,MACzBmjD,EAAS1kD,GAAM5d,EAASmf,GAAG,MAC3BgjD,EAAQvkD,GAAM5d,EAASmf,GAAG,MAE1BhW,EAAOqkB,EAAUohD,GAAuB9yE,IAAM,IAAK0xB,IACnC,IAAZD,IAAkB,MAClBA,EAAUqhD,GAAuBl+E,IAAM,KAAM68B,IACnC,IAAVD,IAAgB,MAChBA,EAAQshD,GAAuB/yE,IAAM,KAAMyxB,IAClC,IAATm1C,IAAe,MACfA,EAAOmM,GAAuB/xE,IAAM,KAAM4lE,IAC/B,IAAXH,IAAiB,MACjBA,EAASsM,GAAuB3T,IAAM,KAAMqH,IAClC,IAAVH,IAAgB,OAAS,KAAMA,EAKvC,OAHAh5D,GAAK,GAAKqlE,EACVrlE,EAAK,IAAMwlE,EAAiB,EAC5BxlE,EAAK,GAAKorB,EACHg6C,GAAkBtmE,SAAUkB,GAgBvC,QAASw9D,IAAWjC,EAAKmK,EAAgBC,GACrC,GAEIC,GAFAhvE,EAAM+uE,EAAuBD,EAC7BG,EAAkBF,EAAuBpK,EAAIn8C,KAajD,OATIymD,GAAkBjvE,IAClBivE,GAAmB,GAGDjvE,EAAM,EAAxBivE,IACAA,GAAmB,GAGvBD,EAAiBh7E,GAAO2wE,GAAKvhE,IAAI6rE,EAAiB,MAE9CxM,KAAMptE,KAAKwyC,KAAKmnC,EAAevmD,YAAc,GAC7CC,KAAMsmD,EAAetmD,QAK7B,QAASqjD,IAAmBrjD,EAAM+5C,EAAMzwC,EAAS+8C,EAAsBD,GACnE,GAA6CI,GAAWzmD,EAApD3rB,EAAIsvE,GAAY1jD,EAAM,EAAG,GAAGymD,WAOhC,OALAryE,GAAU,IAANA,EAAU,EAAIA,EAClBk1B,EAAqB,MAAXA,EAAkBA,EAAU88C,EACtCI,EAAYJ,EAAiBhyE,GAAKA,EAAIiyE,EAAuB,EAAI,IAAUD,EAAJhyE,EAAqB,EAAI,GAChG2rB,EAAY,GAAKg6C,EAAO,IAAMzwC,EAAU88C,GAAkBI,EAAY,GAGlExmD,KAAMD,EAAY,EAAIC,EAAOA,EAAO,EACpCD,UAAWA,EAAY,EAAKA,EAAYo+C,EAAWn+C,EAAO,GAAKD,GAQvE,QAAS2mD,IAAWzN,GAChB,GAEIuC,GAFAe,EAAQtD,EAAOuB,GACfzxC,EAASkwC,EAAOwB,EAKpB,OAFAxB,GAAOmB,QAAUnB,EAAOmB,SAAW9uE,GAAO8sE,WAAWa,EAAOyB,IAE9C,OAAV6B,GAAmBxzC,IAAW96B,GAAuB,KAAVsuE,EACpCjxE,GAAOq7E,SAASzP,WAAW,KAGjB,gBAAVqF,KACPtD,EAAOuB,GAAK+B,EAAQtD,EAAOmB,QAAQwM,SAASrK,IAG5CjxE,GAAOoD,SAAS6tE,GACT,GAAIvD,GAAOuD,GAAO,IAClBxzC,EACHp7B,EAAQo7B,GACR67C,GAA2B3L,GAE3BiL,GAA4BjL,GAGhCwM,GAAkBxM,GAGtBuC,EAAM,GAAIxC,GAAOC,GACbuC,EAAIoI,WAEJpI,EAAI9gE,IAAI,EAAG,KACX8gE,EAAIoI,SAAW31E,GAGZutE,IAyCX,QAASqL,IAAOlmE,EAAImmE,GAChB,GAAItL,GAAKvuE,CAIT,IAHuB,IAAnB65E,EAAQ15E,QAAgBO,EAAQm5E,EAAQ,MACxCA,EAAUA,EAAQ,KAEjBA,EAAQ15E,OACT,MAAO9B,KAGX,KADAkwE,EAAMsL,EAAQ,GACT75E,EAAI,EAAGA,EAAI65E,EAAQ15E,SAAUH,EAC1B65E,EAAQ75E,GAAG0T,GAAI66D,KACfA,EAAMsL,EAAQ75E,GAGtB,OAAOuuE,GAsvBX,QAASc,IAAeL,EAAKntE,GACzB,GAAIi4E,EAGJ,OAAqB,gBAAVj4E,KACPA,EAAQmtE,EAAI7D,aAAaiK,YAAYvzE,GAEhB,gBAAVA,IACAmtE,GAIf8K,EAAap6E,KAAK4G,IAAI0oE,EAAI/7C,OAClB09C,EAAY3B,EAAIj8C,OAAQlxB,IAChCmtE,EAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAM,SAAS/rE,EAAOi4E,GACpD9K,GAGX,QAASI,IAAUJ,EAAK+K,GACpB,MAAO/K,GAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAMmM,KAGtD,QAAS5K,IAAUH,EAAK+K,EAAMl4E,GAC1B,MAAa,UAATk4E,EACO1K,GAAeL,EAAKntE,GAEpBmtE,EAAIt8C,GAAG,OAASs8C,EAAIpB,OAAS,MAAQ,IAAMmM,GAAMl4E,GAIhE,QAASm4E,IAAaD,EAAME,GACxB,MAAO,UAAUp4E,GACb,MAAa,OAATA,GACAstE,GAAU30E,KAAMu/E,EAAMl4E,GACtBxD,GAAOguE,aAAa7xE,KAAMy/E,GACnBz/E,MAEA40E,GAAU50E,KAAMu/E,IAqCnC,QAASG,IAAanN,GAElB,MAAc,KAAPA,EAAa,OAGxB,QAASoN,IAAa1N,GAGlB,MAAe,QAARA,EAAiB,IAuL5B,QAAS2N,IAAmB3pE,GACxBpS,GAAOiM,SAASoJ,GAAGjD,GAAQ,WACvB,MAAOjW,MAAK4S,MAAMqD,IA2D1B,QAAS4pE,IAAWC,GAEK,mBAAVC,SAGXC,GAAkBC,GAAYp8E,OAE1Bo8E,GAAYp8E,OADZi8E,EACqB5P,EACb,uGAGArsE,IAEaA,IAplF7B,IA/WA,GAAIA,IAIAm8E,GAGAx6E,GANA06E,GAAU,QAEVD,GAAiC,mBAAXhR,IAA6C,mBAAX1lE,SAA0BA,SAAW0lE,EAAO1lE,OAAoBvJ,KAATivE,EAE/GvhD,GAAQxoB,KAAKwoB,MACb5nB,GAAiBS,OAAO4M,UAAUrN,eAGlCixE,GAAO,EACPF,GAAQ,EACRC,GAAO,EACPE,GAAO,EACPC,GAAS,EACTC,GAAS,EACTC,GAAc,EAGdtyC,MAGA0uC,MAGAuE,GAA+B,mBAAXj4E,IAA0BA,GAAUA,EAAOD,QAG/Dq+E,GAAkB,sBAClBkC,GAA0B,uDAI1BC,GAAmB,gIAGnBhI,GAAmB,qKACnBQ,GAAwB,6CAGxBmB,GAA2B,QAC3BR,GAA6B,UAC7BL,GAA4B,UAC5BG,GAA2B,gBAC3BS,GAAmB,MACnBN,GAAiB,mHACjBI,GAAqB,uBACrBC,GAAc,KACdH,GAAqB,aACrBC,GAAwB,yBAGxBZ,GAAqB,KACrBO,GAAsB,OACtBN,GAAwB,QACxBC,GAAuB,QACvBG,GAAsB,aACtBD,GAAyB,WAIzBwE,GAAW,4IAEX0C,GAAY,uBAEZzC,KACK,eAAgB,0BAChB,aAAc,sBACd,eAAgB,oBAChB,aAAc,iBACd,WAAY,gBAIjBC,KACK,gBAAiB,6BACjB,WAAY,wBACZ,QAAS,mBACT,KAAM,cAIXpD,GAAuB,kBAIvB6F,IADyB,0CAA0Cv4E,MAAM,MAErEw4E,aAAiB,EACjBC,QAAY,IACZC,QAAY,IACZC,MAAU,KACVC,KAAS,MACTC,OAAW,OACXC,MAAU,UAGdtL,IACI2I,GAAK,cACLtyE,EAAI,SACJpL,EAAI,SACJmL,EAAI,OACJgB,EAAI,MACJm0E,EAAI,OACJzxB,EAAI,OACJmsB,EAAI,UACJzQ,EAAI,QACJgW,EAAI,UACJ/uE,EAAI,OACJgvE,IAAM,YACNlqD,EAAI,UACJ2kD,EAAI,aACJE,GAAI,WACJJ,GAAI,eAGR/F,IACIyL,UAAY,YACZC,WAAa,aACbC,QAAU,UACVC,SAAW,WACXC,YAAc,eAIlB7I,MAGAkG,IACI9yE,EAAG,GACHpL,EAAG,GACHmL,EAAG,GACHgB,EAAG,GACHo+D,EAAG,IAIPuW,GAAmB,gBAAgBv5E,MAAM,KACzCw5E,GAAe,kBAAkBx5E,MAAM,KAEvCswE,IACItN,EAAO,WACH,MAAO/qE,MAAK04B,QAAU,GAE1B8oD,IAAO,SAAUlgD,GACb,MAAOthC,MAAK2wE,aAAa8Q,YAAYzhF,KAAMshC,IAE/CogD,KAAO,SAAUpgD,GACb,MAAOthC,MAAK2wE,aAAayB,OAAOpyE,KAAMshC,IAE1Cw/C,EAAO,WACH,MAAO9gF,MAAKy4B,QAEhBuoD,IAAO,WACH,MAAOhhF,MAAKs4B,aAEhB3rB,EAAO,WACH,MAAO3M,MAAKq4B,OAEhBspD,GAAO,SAAUrgD,GACb,MAAOthC,MAAK2wE,aAAaiR,YAAY5hF,KAAMshC,IAE/CugD,IAAO,SAAUvgD,GACb,MAAOthC,MAAK2wE,aAAamR,cAAc9hF,KAAMshC,IAEjDygD,KAAO,SAAUzgD,GACb,MAAOthC,MAAK2wE,aAAaqR,SAAShiF,KAAMshC,IAE5C+tB,EAAO,WACH,MAAOrvD,MAAKsyE,QAEhBkJ,EAAO,WACH,MAAOx7E,MAAKiiF,WAEhBC,GAAO,WACH,MAAO1R,GAAaxwE,KAAKu4B,OAAS,IAAK,IAE3C4pD,KAAO,WACH,MAAO3R,GAAaxwE,KAAKu4B,OAAQ,IAErC6pD,MAAQ,WACJ,MAAO5R,GAAaxwE,KAAKu4B,OAAQ,IAErC8pD,OAAS,WACL,GAAIrwE,GAAIhS,KAAKu4B,OAAQvJ,EAAOhd,GAAK,EAAI,IAAM,GAC3C,OAAOgd,GAAOwhD,EAAatrE,KAAK2lB,IAAI7Y,GAAI,IAE5C2pE,GAAO,WACH,MAAOnL,GAAaxwE,KAAKq7E,WAAa,IAAK,IAE/CiH,KAAO,WACH,MAAO9R,GAAaxwE,KAAKq7E,WAAY,IAEzCkH,MAAQ,WACJ,MAAO/R,GAAaxwE,KAAKq7E,WAAY,IAEzCE,GAAO,WACH,MAAO/K,GAAaxwE,KAAKwiF,cAAgB,IAAK,IAElDC,KAAO,WACH,MAAOjS,GAAaxwE,KAAKwiF,cAAe,IAE5CE,MAAQ,WACJ,MAAOlS,GAAaxwE,KAAKwiF,cAAe,IAE5C1rD,EAAI,WACA,MAAO92B,MAAK6hC,WAEhB45C,EAAI,WACA,MAAOz7E,MAAK2iF,cAEhBp9E,EAAO,WACH,MAAOvF,MAAK2wE,aAAaO,SAASlxE,KAAKo9B,QAASp9B,KAAKq9B,WAAW,IAEpEwtC,EAAO,WACH,MAAO7qE,MAAK2wE,aAAaO,SAASlxE,KAAKo9B,QAASp9B,KAAKq9B,WAAW,IAEpEjT,EAAO,WACH,MAAOpqB,MAAKo9B,SAEhBzxB,EAAO,WACH,MAAO3L,MAAKo9B,QAAU,IAAM,IAEhC58B,EAAO,WACH,MAAOR,MAAKq9B,WAEhBzxB,EAAO,WACH,MAAO5L,MAAKs9B,WAEhBjT,EAAO,WACH,MAAO8qD,GAAMn1E,KAAKu9B,eAAiB,MAEvCqlD,GAAO,WACH,MAAOpS,GAAa2E,EAAMn1E,KAAKu9B,eAAiB,IAAK,IAEzDslD,IAAO,WACH,MAAOrS,GAAaxwE,KAAKu9B,eAAgB,IAE7CulD,KAAO,WACH,MAAOtS,GAAaxwE,KAAKu9B,eAAgB,IAE7CwlD,EAAO,WACH,GAAIx9E,GAAIvF,KAAKgjF,YACT58E,EAAI,GAKR,OAJQ,GAAJb,IACAA,GAAKA,EACLa,EAAI,KAEDA,EAAIoqE,EAAa2E,EAAM5vE,EAAI,IAAK,GAAK,IAAMirE,EAAa2E,EAAM5vE,GAAK,GAAI,IAElF09E,GAAO,WACH,GAAI19E,GAAIvF,KAAKgjF,YACT58E,EAAI,GAKR,OAJQ,GAAJb,IACAA,GAAKA,EACLa,EAAI,KAEDA,EAAIoqE,EAAa2E,EAAM5vE,EAAI,IAAK,GAAKirE,EAAa2E,EAAM5vE,GAAK,GAAI,IAE5E2X,EAAI,WACA,MAAOld,MAAKkjF,YAEhBC,GAAK,WACD,MAAOnjF,MAAKojF,YAEhBrxE,EAAO,WACH,MAAO/R,MAAKgH,WAEhB4jB,EAAO,WACH,MAAO5qB,MAAKqjF,QAEhBtC,EAAI,WACA,MAAO/gF,MAAKmyE,YAIpB9B,MAEAiT,IAAS,SAAU,cAAe,WAAY,gBAAiB,eAE/D1R,IAAmB,EAyFhB0P,GAAiB37E,QACpBH,GAAI87E,GAAiBjnC,MACrBg+B,GAAqB7yE,GAAI,KAAOirE,EAAgB4H,GAAqB7yE,IAAIA,GAE7E,MAAO+7E,GAAa57E,QAChBH,GAAI+7E,GAAalnC,MACjBg+B,GAAqB7yE,GAAIA,IAAK8qE,EAAS+H,GAAqB7yE,IAAI,EAEpE6yE,IAAqBkL,KAAOjT,EAAS+H,GAAqB2I,IAAK,GA0d/D17E,EAAOgsE,EAAOn+D,WAEVg2C,IAAM,SAAUqoB,GACZ,GAAI3rE,GAAML,CACV,KAAKA,IAAKgsE,GACN3rE,EAAO2rE,EAAOhsE,GACM,kBAATK,GACP7F,KAAKwF,GAAKK,EAEV7F,KAAK,IAAMwF,GAAKK,CAKxB7F,MAAKi6E,qBAAuB,GAAIC,QAAOl6E,KAAKg6E,cAAcvV,OAAS,IAAM,UAAUA,SAGvFiO,QAAU,wFAAwF3qE,MAAM,KACxGqqE,OAAS,SAAU5xE,GACf,MAAOR,MAAK0yE,QAAQlyE,EAAEk4B,UAG1B8qD,aAAe,kDAAkDz7E,MAAM,KACvE05E,YAAc,SAAUjhF,GACpB,MAAOR,MAAKwjF,aAAahjF,EAAEk4B,UAG/BkiD,YAAc,SAAU6I,EAAWniD,EAAQ6gC,GACvC,GAAI38D,GAAGgvE,EAAKkP,CAQZ,KANK1jF,KAAK2jF,eACN3jF,KAAK2jF,gBACL3jF,KAAK4jF,oBACL5jF,KAAK6jF,sBAGJr+E,EAAI,EAAO,GAAJA,EAAQA,IAAK,CAYrB,GAVAgvE,EAAM3wE,GAAOkyE,KAAK,IAAMvwE,IACpB28D,IAAWniE,KAAK4jF,iBAAiBp+E,KACjCxF,KAAK4jF,iBAAiBp+E,GAAK,GAAI00E,QAAO,IAAMl6E,KAAKoyE,OAAOoC,EAAK,IAAIhqE,QAAQ,IAAK,IAAM,IAAK,KACzFxK,KAAK6jF,kBAAkBr+E,GAAK,GAAI00E,QAAO,IAAMl6E,KAAKyhF,YAAYjN,EAAK,IAAIhqE,QAAQ,IAAK,IAAM,IAAK,MAE9F23D,GAAWniE,KAAK2jF,aAAan+E,KAC9Bk+E,EAAQ,IAAM1jF,KAAKoyE,OAAOoC,EAAK,IAAM,KAAOx0E,KAAKyhF,YAAYjN,EAAK,IAClEx0E,KAAK2jF,aAAan+E,GAAK,GAAI00E,QAAOwJ,EAAMl5E,QAAQ,IAAK,IAAK,MAG1D23D,GAAqB,SAAX7gC,GAAqBthC,KAAK4jF,iBAAiBp+E,GAAGwI,KAAKy1E,GAC7D,MAAOj+E,EACJ,IAAI28D,GAAqB,QAAX7gC,GAAoBthC,KAAK6jF,kBAAkBr+E,GAAGwI,KAAKy1E,GACpE,MAAOj+E,EACJ,KAAK28D,GAAUniE,KAAK2jF,aAAan+E,GAAGwI,KAAKy1E,GAC5C,MAAOj+E,KAKnBs+E,UAAY,2DAA2D/7E,MAAM,KAC7Ei6E,SAAW,SAAUxhF,GACjB,MAAOR,MAAK8jF,UAAUtjF,EAAE63B,QAG5B0rD,eAAiB,8BAA8Bh8E,MAAM,KACrD+5E,cAAgB,SAAUthF,GACtB,MAAOR,MAAK+jF,eAAevjF,EAAE63B,QAGjC2rD,aAAe,uBAAuBj8E,MAAM,KAC5C65E,YAAc,SAAUphF,GACpB,MAAOR,MAAKgkF,aAAaxjF,EAAE63B,QAG/B4iD,cAAgB,SAAUgJ,GACtB,GAAIz+E,GAAGgvE,EAAKkP,CAMZ,KAJK1jF,KAAKkkF,iBACNlkF,KAAKkkF,mBAGJ1+E,EAAI,EAAO,EAAJA,EAAOA,IAQf,GANKxF,KAAKkkF,eAAe1+E,KACrBgvE,EAAM3wE,IAAQ,IAAM,IAAIw0B,IAAI7yB,GAC5Bk+E,EAAQ,IAAM1jF,KAAKgiF,SAASxN,EAAK,IAAM,KAAOx0E,KAAK8hF,cAActN,EAAK,IAAM,KAAOx0E,KAAK4hF,YAAYpN,EAAK,IACzGx0E,KAAKkkF,eAAe1+E,GAAK,GAAI00E,QAAOwJ,EAAMl5E,QAAQ,IAAK,IAAK,MAG5DxK,KAAKkkF,eAAe1+E,GAAGwI,KAAKi2E,GAC5B,MAAOz+E,IAKnB2+E,iBACIC,IAAM,YACNC,GAAK,SACLC,EAAI,aACJC,GAAK,eACLC,IAAM,kBACNC,KAAO,yBAEX9L,eAAiB,SAAUjwE,GACvB,GAAIkrE,GAAS5zE,KAAKmkF,gBAAgBz7E,EAOlC;OANKkrE,GAAU5zE,KAAKmkF,gBAAgBz7E,EAAIu8B,iBACpC2uC,EAAS5zE,KAAKmkF,gBAAgBz7E,EAAIu8B,eAAez6B,QAAQ,mBAAoB,SAAUqoE,GACnF,MAAOA,GAAI5nE,MAAM,KAErBjL,KAAKmkF,gBAAgBz7E,GAAOkrE,GAEzBA,GAGXvC,KAAO,SAAUyD,GAGb,MAAiD,OAAxCA,EAAQ,IAAIvwC,cAAcnf,OAAO,IAG9Cq0D,eAAiB,gBACjBvI,SAAW,SAAU9zC,EAAOC,EAASqnD,GACjC,MAAItnD,GAAQ,GACDsnD,EAAU,KAAO,KAEjBA,EAAU,KAAO,MAKhCC,WACIC,QAAU,gBACVC,QAAU,mBACVC,SAAW,eACXC,QAAU,oBACVC,SAAW,sBACXC,SAAW,KAEfC,SAAW,SAAUx8E,EAAK8rE,EAAKr3C,GAC3B,GAAIy2C,GAAS5zE,KAAK2kF,UAAUj8E,EAC5B,OAAyB,kBAAXkrE,GAAwBA,EAAO77D,MAAMy8D,GAAMr3C,IAAQy2C,GAGrEuR,eACIC,OAAS,QACTC,KAAO,SACPz5E,EAAI,gBACJpL,EAAI,WACJ8kF,GAAK,aACL35E,EAAI,UACJ45E,GAAK,WACL54E,EAAI,QACJg1E,GAAK,UACL5W,EAAI,UACJya,GAAK,YACLxzE,EAAI,SACJyzE,GAAK,YAGTjH,aAAe,SAAU/K,EAAQ6K,EAAehE,EAAQiE,GACpD,GAAI3K,GAAS5zE,KAAKmlF,cAAc7K,EAChC,OAA0B,kBAAX1G,GACXA,EAAOH,EAAQ6K,EAAehE,EAAQiE,GACtC3K,EAAOppE,QAAQ,MAAOipE,IAG9BiS,WAAa,SAAUr5D,EAAMunD,GACzB,GAAItyC,GAASthC,KAAKmlF,cAAc94D,EAAO,EAAI,SAAW,OACtD,OAAyB,kBAAXiV,GAAwBA,EAAOsyC,GAAUtyC,EAAO92B,QAAQ,MAAOopE,IAGjFhD,QAAU,SAAU6C,GAChB,MAAOzzE,MAAK2lF,SAASn7E,QAAQ,KAAMipE,IAEvCkS,SAAW,KACX3L,cAAgB,UAEhBmF,SAAW,SAAU7E,GACjB,MAAOA,IAGXsL,WAAa,SAAUtL,GACnB,MAAOA,IAGXhI,KAAO,SAAUkC,GACb,MAAOiC,IAAWjC,EAAKx0E,KAAK07E,MAAMnF,IAAKv2E,KAAK07E,MAAMlF,KAAKlE,MAG3DoJ,OACInF,IAAM,EACNC,IAAM,GAGVmI,eAAiB,WACb,MAAO3+E,MAAK07E,MAAMnF,KAGtBsP,eAAiB,WACb,MAAO7lF,MAAK07E,MAAMlF,KAGtBsP,aAAc,eACdrN,YAAa,WACT,MAAOz4E,MAAK8lF,gBA0yBpBjiF,GAAS,SAAUixE,EAAOxzC,EAAQ+C,EAAQ89B,GACtC,GAAI1hE,EAiBJ,OAfuB,iBAAb,KACN0hE,EAAS99B,EACTA,EAAS79B,GAIb/F,KACAA,EAAEqyE,kBAAmB,EACrBryE,EAAEsyE,GAAK+B,EACPr0E,EAAEuyE,GAAK1xC,EACP7gC,EAAEwyE,GAAK5uC,EACP5jC,EAAEyyE,QAAU/Q,EACZ1hE,EAAE2yE,QAAS,EACX3yE,EAAE6yE,IAAMlE,IAED6P,GAAWx+E,IAGtBoD,GAAOmsE,6BAA8B,EAErCnsE,GAAOk6E,wBAA0B7N,EAC7B,4LAIA,SAAUsB,GACNA,EAAOt5C,GAAK,GAAI5zB,MAAKktE,EAAOuB,IAAMvB,EAAOwJ,QAAU,OAAS,OA0BpEn3E,GAAOiI,IAAM,WACT,GAAImN,MAAUhO,MAAM1K,KAAKmF,UAAW,EAEpC,OAAO05E,IAAO,WAAYnmE,IAG9BpV,GAAO6I,IAAM,WACT,GAAIuM,MAAUhO,MAAM1K,KAAKmF,UAAW,EAEpC,OAAO05E,IAAO,UAAWnmE,IAI7BpV,GAAOkyE,IAAM,SAAUjB,EAAOxzC,EAAQ+C,EAAQ89B,GAC1C,GAAI1hE,EAkBJ,OAhBuB,iBAAb,KACN0hE,EAAS99B,EACTA,EAAS79B,GAIb/F,KACAA,EAAEqyE,kBAAmB,EACrBryE,EAAEu6E,SAAU,EACZv6E,EAAE2yE,QAAS,EACX3yE,EAAEwyE,GAAK5uC,EACP5jC,EAAEsyE,GAAK+B,EACPr0E,EAAEuyE,GAAK1xC,EACP7gC,EAAEyyE,QAAU/Q,EACZ1hE,EAAE6yE,IAAMlE,IAED6P,GAAWx+E,GAAGs1E,OAIzBlyE,GAAOw/E,KAAO,SAAUvO,GACpB,MAAOjxE,IAAe,IAARixE,IAIlBjxE,GAAOiM,SAAW,SAAUglE,EAAOpsE,GAC/B,GAGIsmB,GACA+2D,EACAC,EACAC,EANAn2E,EAAWglE,EAEXvwE,EAAQ,IAiEZ,OA3DIV,IAAOqiF,WAAWpR,GAClBhlE,GACIouE,GAAIpJ,EAAMtC,cACV7lE,EAAGmoE,EAAMrC,MACT1H,EAAG+J,EAAMpC,SAEW,gBAAVoC,IACdhlE,KACIpH,EACAoH,EAASpH,GAAOosE,EAEhBhlE,EAASytB,aAAeu3C,IAElBvwE,EAAQ47E,GAAwB17E,KAAKqwE,KAC/C9lD,EAAqB,MAAbzqB,EAAM,GAAc,GAAK,EACjCuL,GACIkC,EAAG,EACHrF,EAAGwoE,EAAM5wE,EAAMuyE,KAAS9nD,EACxBrjB,EAAGwpE,EAAM5wE,EAAMyyE,KAAShoD,EACxBxuB,EAAG20E,EAAM5wE,EAAM0yE,KAAWjoD,EAC1BpjB,EAAGupE,EAAM5wE,EAAM2yE,KAAWloD,EAC1BkvD,GAAI/I,EAAM5wE,EAAM4yE,KAAgBnoD,KAE1BzqB,EAAQ67E,GAAiB37E,KAAKqwE,KACxC9lD,EAAqB,MAAbzqB,EAAM,GAAc,GAAK,EACjCyhF,EAAW,SAAUG,GAIjB,GAAIpS,GAAMoS,GAAO9gE,WAAW8gE,EAAI37E,QAAQ,IAAK,KAE7C,QAAQ9F,MAAMqvE,GAAO,EAAIA,GAAO/kD,GAEpClf,GACIkC,EAAGg0E,EAASzhF,EAAM,IAClBwmE,EAAGib,EAASzhF,EAAM,IAClBoI,EAAGq5E,EAASzhF,EAAM,IAClBoH,EAAGq6E,EAASzhF,EAAM,IAClB/D,EAAGwlF,EAASzhF,EAAM,IAClBqH,EAAGo6E,EAASzhF,EAAM,IAClB8qD,EAAG22B,EAASzhF,EAAM,MAEH,MAAZuL,EACPA,KAC2B,gBAAbA,KACT,QAAUA,IAAY,MAAQA,MACnCm2E,EAAUhS,EAAkBpwE,GAAOiM,EAASsZ,MAAOvlB,GAAOiM,EAASuZ,KAEnEvZ,KACAA,EAASouE,GAAK+H,EAAQ1oD,aACtBztB,EAASi7D,EAAIkb,EAAQ7T,QAGzB2T,EAAM,GAAIjU,GAAShiE,GAEfjM,GAAOqiF,WAAWpR,IAAU3F,EAAW2F,EAAO,aAC9CiR,EAAIpT,QAAUmC,EAAMnC,SAGjBoT,GAIXliF,GAAOuiF,QAAUlG,GAGjBr8E,GAAOm+B,cAAgBq+C,GAGvBx8E,GAAO64E,SAAW,aAIlB74E,GAAO0vE,iBAAmBA,GAI1B1vE,GAAOguE,aAAe,aAGtBhuE,GAAOwiF,sBAAwB,SAAUruB,EAAWsuB,GAChD,MAAI5H,IAAuB1mB,KAAexxD,GAC/B,EAEP8/E,IAAU9/E,EACHk4E,GAAuB1mB,IAElC0mB,GAAuB1mB,GAAasuB,GAC7B,IAGXziF,GAAOygC,KAAO4rC,EACV,wDACA,SAAUxnE,EAAKrB,GACX,MAAOxD,IAAOwgC,OAAO37B,EAAKrB,KAOlCxD,GAAOwgC,OAAS,SAAU37B,EAAKoO,GAC3B,GAAIpE,EAcJ,OAbIhK,KAEIgK,EADmB,mBAAb,GACC7O,GAAO0iF,aAAa79E,EAAKoO,GAGzBjT,GAAO8sE,WAAWjoE,GAGzBgK,IACA7O,GAAOiM,SAAS6iE,QAAU9uE,GAAO8uE,QAAUjgE,IAI5C7O,GAAO8uE,QAAQ6T,OAG1B3iF,GAAO0iF,aAAe,SAAUtwE,EAAMa,GAClC,MAAe,QAAXA,GACAA,EAAO2vE,KAAOxwE,EACT4uB,GAAQ5uB,KACT4uB,GAAQ5uB,GAAQ,GAAIq7D,IAExBzsC,GAAQ5uB,GAAMkzC,IAAIryC,GAGlBjT,GAAOwgC,OAAOpuB,GAEP4uB,GAAQ5uB,WAGR4uB,IAAQ5uB,GACR,OAIfpS,GAAO6iF,SAAWxW,EACd,gEACA,SAAUxnE,GACN,MAAO7E,IAAO8sE,WAAWjoE,KAKjC7E,GAAO8sE,WAAa,SAAUjoE,GAC1B,GAAI27B,EAMJ,IAJI37B,GAAOA,EAAIiqE,SAAWjqE,EAAIiqE,QAAQ6T,QAClC99E,EAAMA,EAAIiqE,QAAQ6T,QAGjB99E,EACD,MAAO7E,IAAO8uE,OAGlB,KAAKzsE,EAAQwC,GAAM,CAGf,GADA27B,EAASuzC,EAAWlvE,GAEhB,MAAO27B,EAEX37B,IAAOA,GAGX,MAAOgvE,GAAahvE,IAIxB7E,GAAOoD,SAAW,SAAU8b,GACxB,MAAOA,aAAewuD,IACV,MAAPxuD,GAAeosD,EAAWpsD,EAAK,qBAIxClf,GAAOqiF,WAAa,SAAUnjE,GAC1B,MAAOA,aAAe+uD,GAG1B,KAAKtsE,GAAI89E,GAAM39E,OAAS,EAAGH,IAAK,IAAKA,GACjCmwE,EAAS2N,GAAM99E,IAGnB3B,IAAOuxE,eAAiB,SAAUC,GAC9B,MAAOD,GAAeC,IAG1BxxE,GAAOq7E,QAAU,SAAUyH,GACvB,GAAInmF,GAAIqD,GAAOkyE,IAAIyH,IAQnB,OAPa,OAATmJ,EACArhF,EAAO9E,EAAE8yE,IAAKqT,GAGdnmF,EAAE8yE,IAAI1D,iBAAkB,EAGrBpvE,GAGXqD,GAAO+iF,UAAY,WACf,MAAO/iF,IAAOkU,MAAM,KAAMrS,WAAWkhF,aAGzC/iF,GAAOi3E,kBAAoB,SAAUhG,GACjC,MAAOK,GAAML,IAAUK,EAAML,GAAS,GAAK,KAAO,MAGtDjxE,GAAOQ,OAASA,EAOhBiB,EAAOzB,GAAOqV,GAAKq4D,EAAOp+D,WAEtBilB,MAAQ,WACJ,MAAOv0B,IAAO7D,OAGlBgH,QAAU,WACN,OAAQhH,KAAKk4B,GAA4B,KAArBl4B,KAAKqzE,SAAW,IAGxCgQ,KAAO,WACH,MAAOn+E,MAAKC,OAAOnF,KAAO,MAG9BqF,SAAW,WACP,MAAOrF,MAAKo4B,QAAQiM,OAAO,MAAM/C,OAAO,qCAG5Cp6B,OAAS,WACL,MAAOlH,MAAKqzE,QAAU,GAAI/uE,OAAMtE,MAAQA,KAAKk4B,IAGjD9wB,YAAc,WACV,GAAI5G,GAAIqD,GAAO7D,MAAM+1E,KACrB,OAAI,GAAIv1E,EAAE+3B,QAAU/3B,EAAE+3B,QAAU,KACxB,kBAAsBj0B,MAAK6O,UAAU/L,YAE9BpH,KAAKkH,SAASE,cAEdkxE,EAAa93E,EAAG,gCAGpB83E,EAAa93E,EAAG,mCAI/B+H,QAAU,WACN,GAAI/H,GAAIR,IACR,QACIQ,EAAE+3B,OACF/3B,EAAEk4B,QACFl4B,EAAEi4B,OACFj4B,EAAE48B,QACF58B,EAAE68B,UACF78B,EAAE88B,UACF98B,EAAE+8B,iBAIV85C,QAAU,WACN,MAAOA,GAAQr3E,OAGnB6mF,aAAe,WACX,MAAI7mF,MAAK42E,GACE52E,KAAKq3E,WAAatC,EAAc/0E,KAAK42E,IAAK52E,KAAKozE,OAASvvE,GAAOkyE,IAAI/1E,KAAK42E,IAAM/yE,GAAO7D,KAAK42E,KAAKruE,WAAa,GAGhH,GAGXu+E,aAAe,WACX,MAAOxhF,MAAWtF,KAAKszE,MAG3ByT,UAAW,WACP,MAAO/mF,MAAKszE,IAAIzvD,UAGpBkyD,IAAM,SAAUiR,GACZ,MAAOhnF,MAAKgjF,UAAU,EAAGgE,IAG7B/O,MAAQ,SAAU+O,GASd,MARIhnF,MAAKozE,SACLpzE,KAAKgjF,UAAU,EAAGgE,GAClBhnF,KAAKozE,QAAS,EAEV4T,GACAhnF,KAAKqrB,SAASrrB,KAAKinF,iBAAkB,MAGtCjnF,MAGXshC,OAAS,SAAU4lD,GACf,GAAItT,GAAS0E,EAAat4E,KAAMknF,GAAerjF,GAAOm+B,cACtD,OAAOhiC,MAAK2wE,aAAaiV,WAAWhS,IAGxC3gE,IAAMmhE,EAAY,EAAG,OAErB/oD,SAAW+oD,EAAY,GAAI,YAE3B/nD,KAAO,SAAUyoD,EAAOO,EAAO8R,GAC3B,GAEY96D,GAAMunD,EAFdwT,EAAOlT,EAAOY,EAAO90E,MACrBqnF,EAAmD,KAAvCD,EAAKpE,YAAchjF,KAAKgjF,YAqBxC,OAlBA3N,GAAQD,EAAeC,GAET,SAAVA,GAA8B,UAAVA,GAA+B,YAAVA,GACzCzB,EAAS/C,EAAU7wE,KAAMonF,GACX,YAAV/R,EACAzB,GAAkB,EACD,SAAVyB,IACPzB,GAAkB,MAGtBvnD,EAAOrsB,KAAOonF,EACdxT,EAAmB,WAAVyB,EAAqBhpD,EAAO,IACvB,WAAVgpD,EAAqBhpD,EAAO,IAClB,SAAVgpD,EAAmBhpD,EAAO,KAChB,QAAVgpD,GAAmBhpD,EAAOg7D,GAAY,MAC5B,SAAVhS,GAAoBhpD,EAAOg7D,GAAY,OACvCh7D,GAED86D,EAAUvT,EAASJ,EAASI,IAGvCxqD,KAAO,SAAU+Q,EAAMmkD,GACnB,MAAOz6E,IAAOiM,UAAUuZ,GAAIrpB,KAAMopB,KAAM+Q,IAAOkK,OAAOrkC,KAAKqkC,UAAUijD,UAAUhJ,IAGnFiJ,QAAU,SAAUjJ,GAChB,MAAOt+E,MAAKopB,KAAKvlB,KAAUy6E,IAG/B4G,SAAW,SAAU/qD,GAIjB,GAAIgD,GAAMhD,GAAQt2B,KACd2jF,EAAMtT,EAAO/2C,EAAKn9B,MAAMynF,QAAQ,OAChCp7D,EAAOrsB,KAAKqsB,KAAKm7D,EAAK,QAAQ,GAC9BlmD,EAAgB,GAAPjV,EAAY,WACV,GAAPA,EAAY,WACL,EAAPA,EAAW,UACJ,EAAPA,EAAW,UACJ,EAAPA,EAAW,UACJ,EAAPA,EAAW,WAAa,UAChC,OAAOrsB,MAAKshC,OAAOthC,KAAK2wE,aAAauU,SAAS5jD,EAAQthC,KAAM6D,GAAOs5B,MAGvEw5C,WAAa,WACT,MAAOA,GAAW32E,KAAKu4B,SAG3BmvD,MAAQ,WACJ,MAAQ1nF,MAAKgjF,YAAchjF,KAAKo4B,QAAQM,MAAM,GAAGsqD,aAC7ChjF,KAAKgjF,YAAchjF,KAAKo4B,QAAQM,MAAM,GAAGsqD,aAGjD3qD,IAAM,SAAUy8C,GACZ,GAAIz8C,GAAMr4B,KAAKozE,OAASpzE,KAAKk4B,GAAG8mD,YAAch/E,KAAKk4B,GAAGyvD,QACtD,OAAa,OAAT7S,GACAA,EAAQsJ,GAAatJ,EAAO90E,KAAK2wE,cAC1B3wE,KAAKiT,IAAI6hE,EAAQz8C,EAAK,MAEtBA,GAIfK,MAAQ8mD,GAAa,SAAS,GAE9BiI,QAAU,SAAUpS,GAIhB,OAHAA,EAAQD,EAAeC,IAIvB,IAAK,OACDr1E,KAAK04B,MAAM,EAEf,KAAK,UACL,IAAK,QACD14B,KAAKy4B,KAAK,EAEd,KAAK,OACL,IAAK,UACL,IAAK,MACDz4B,KAAKo9B,MAAM,EAEf,KAAK,OACDp9B,KAAKq9B,QAAQ,EAEjB,KAAK,SACDr9B,KAAKs9B,QAAQ,EAEjB,KAAK,SACDt9B,KAAKu9B,aAAa,GAgBtB,MAXc,SAAV83C,EACAr1E,KAAK6hC,QAAQ,GACI,YAAVwzC,GACPr1E,KAAK2iF,WAAW,GAIN,YAAVtN,GACAr1E,KAAK04B,MAAqC,EAA/BxzB,KAAKC,MAAMnF,KAAK04B,QAAU,IAGlC14B,MAGX4nF,MAAO,SAAUvS,GAEb,MADAA,GAAQD,EAAeC,GACnBA,IAAU7uE,GAAuB,gBAAV6uE,EAChBr1E,KAEJA,KAAKynF,QAAQpS,GAAOpiE,IAAI,EAAc,YAAVoiE,EAAsB,OAASA,GAAQhqD,SAAS,EAAG,OAG1F2oD,QAAS,SAAUc,EAAOO,GACtB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAgC,mBAAVC,GAAwBA,EAAQ,eAChD,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IACxC90E,MAAQ80E,IAEhB+S,EAAUhkF,GAAOoD,SAAS6tE,IAAUA,GAASjxE,GAAOixE,GAC7C+S,GAAW7nF,KAAKo4B,QAAQqvD,QAAQpS,KAI/ClB,SAAU,SAAUW,EAAOO,GACvB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAgC,mBAAVC,GAAwBA,EAAQ,eAChD,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IAChCA,GAAR90E,OAER6nF,EAAUhkF,GAAOoD,SAAS6tE,IAAUA,GAASjxE,GAAOixE,IAC5C90E,KAAKo4B,QAAQwvD,MAAMvS,GAASwS,IAI5CC,UAAW,SAAU1+D,EAAMC,EAAIgsD,GAC3B,MAAOr1E,MAAKg0E,QAAQ5qD,EAAMisD,IAAUr1E,KAAKm0E,SAAS9qD,EAAIgsD,IAG1DpxC,OAAQ,SAAU6wC,EAAOO,GACrB,GAAIwS,EAEJ,OADAxS,GAAQD,EAAeC,GAAS,eAClB,gBAAVA,GACAP,EAAQjxE,GAAOoD,SAAS6tE,GAASA,EAAQjxE,GAAOixE,IACxC90E,QAAU80E,IAElB+S,GAAWhkF,GAAOixE,IACT90E,KAAKo4B,QAAQqvD,QAAQpS,IAAWwS,GAAWA,IAAa7nF,KAAKo4B,QAAQwvD,MAAMvS,KAI5FvpE,IAAKokE,EACI,mGACA,SAAUtqE,GAEN,MADAA,GAAQ/B,GAAOkU,MAAM,KAAMrS,WACZ1F,KAAR4F,EAAe5F,KAAO4F,IAI1C8G,IAAKwjE,EACG,mGACA,SAAUtqE,GAEN,MADAA,GAAQ/B,GAAOkU,MAAM,KAAMrS,WACpBE,EAAQ5F,KAAOA,KAAO4F,IAIzCmiF,KAAO7X,EACC,4GAEA,SAAU4E,EAAOkS,GACb,MAAa,OAATlS,GACqB,gBAAVA,KACPA,GAASA,GAGb90E,KAAKgjF,UAAUlO,EAAOkS,GAEfhnF,OAECA,KAAKgjF,cAe7BA,UAAY,SAAUlO,EAAOkS,GACzB,GACIgB,GADAr+D,EAAS3pB,KAAKqzE,SAAW,CAE7B,OAAa,OAATyB,GACqB,gBAAVA,KACPA,EAAQuF,EAAoBvF,IAE5B5vE,KAAK2lB,IAAIiqD,GAAS,KAClBA,EAAgB,GAARA,IAEP90E,KAAKozE,QAAU4T,IAChBgB,EAAchoF,KAAKinF,kBAEvBjnF,KAAKqzE,QAAUyB,EACf90E,KAAKozE,QAAS,EACK,MAAf4U,GACAhoF,KAAKiT,IAAI+0E,EAAa,KAEtBr+D,IAAWmrD,KACNkS,GAAiBhnF,KAAKioF,kBACvB1T,EAAgCv0E,KACxB6D,GAAOiM,SAASglE,EAAQnrD,EAAQ,KAAM,GAAG,GACzC3pB,KAAKioF,oBACbjoF,KAAKioF,mBAAoB,EACzBpkF,GAAOguE,aAAa7xE,MAAM,GAC1BA,KAAKioF,kBAAoB,OAI1BjoF,MAEAA,KAAKozE,OAASzpD,EAAS3pB,KAAKinF,kBAI3CiB,QAAU,WACN,OAAQloF,KAAKozE,QAGjB+U,YAAc,WACV,MAAOnoF,MAAKozE,QAGhBgV,MAAQ,WACJ,MAAOpoF,MAAKozE,QAA2B,IAAjBpzE,KAAKqzE,SAG/B6P,SAAW,WACP,MAAOljF,MAAKozE,OAAS,MAAQ,IAGjCgQ,SAAW,WACP,MAAOpjF,MAAKozE,OAAS,6BAA+B,IAGxDwT,UAAY,WAMR,MALI5mF,MAAKmzE,KACLnzE,KAAKgjF,UAAUhjF,KAAKmzE,MACM,gBAAZnzE,MAAK+yE,IACnB/yE,KAAKgjF,UAAU3I,EAAoBr6E,KAAK+yE,KAErC/yE,MAGXqoF,qBAAuB,SAAUvT,GAQ7B,MAHIA,GAJCA,EAIOjxE,GAAOixE,GAAOkO,YAHd,GAMJhjF,KAAKgjF,YAAclO,GAAS,KAAO,GAG/CqB,YAAc,WACV,MAAOA,GAAYn2E,KAAKu4B,OAAQv4B,KAAK04B,UAGzCJ,UAAY,SAAUw8C,GAClB,GAAIx8C,GAAY5K,IAAO7pB,GAAO7D,MAAMynF,QAAQ,OAAS5jF,GAAO7D,MAAMynF,QAAQ,SAAW,OAAS,CAC9F,OAAgB,OAAT3S,EAAgBx8C,EAAYt4B,KAAKiT,IAAK6hE,EAAQx8C,EAAY,MAGrE65C,QAAU,SAAU2C,GAChB,MAAgB,OAATA,EAAgB5vE,KAAKwyC,MAAM13C,KAAK04B,QAAU,GAAK,GAAK14B,KAAK04B,MAAoB,GAAbo8C,EAAQ,GAAS90E,KAAK04B,QAAU,IAG3G2iD,SAAW,SAAUvG,GACjB,GAAIv8C,GAAOk+C,GAAWz2E,KAAMA,KAAK2wE,aAAa+K,MAAMnF,IAAKv2E,KAAK2wE,aAAa+K,MAAMlF,KAAKj+C,IACtF,OAAgB,OAATu8C,EAAgBv8C,EAAOv4B,KAAKiT,IAAK6hE,EAAQv8C,EAAO,MAG3DiqD,YAAc,SAAU1N,GACpB,GAAIv8C,GAAOk+C,GAAWz2E,KAAM,EAAG,GAAGu4B,IAClC,OAAgB,OAATu8C,EAAgBv8C,EAAOv4B,KAAKiT,IAAK6hE,EAAQv8C,EAAO,MAG3D+5C,KAAO,SAAUwC,GACb,GAAIxC,GAAOtyE,KAAK2wE,aAAa2B,KAAKtyE,KAClC,OAAgB,OAAT80E,EAAgBxC,EAAOtyE,KAAKiT,IAAqB,GAAhB6hE,EAAQxC,GAAW,MAG/D2P,QAAU,SAAUnN,GAChB,GAAIxC,GAAOmE,GAAWz2E,KAAM,EAAG,GAAGsyE,IAClC,OAAgB,OAATwC,EAAgBxC,EAAOtyE,KAAKiT,IAAqB,GAAhB6hE,EAAQxC,GAAW,MAG/DzwC,QAAU,SAAUizC,GAChB,GAAIjzC,IAAW7hC,KAAKq4B,MAAQ,EAAIr4B,KAAK2wE,aAAa+K,MAAMnF,KAAO,CAC/D,OAAgB,OAATzB,EAAgBjzC,EAAU7hC,KAAKiT,IAAI6hE,EAAQjzC,EAAS,MAG/D8gD,WAAa,SAAU7N,GAInB,MAAgB,OAATA,EAAgB90E,KAAKq4B,OAAS,EAAIr4B,KAAKq4B,IAAIr4B,KAAKq4B,MAAQ,EAAIy8C,EAAQA,EAAQ,IAGvFwT,eAAiB,WACb,MAAOhS,GAAYt2E,KAAKu4B,OAAQ,EAAG,IAGvC+9C,YAAc,WACV,GAAIiS,GAAWvoF,KAAK2wE,aAAa+K,KACjC,OAAOpF,GAAYt2E,KAAKu4B,OAAQgwD,EAAShS,IAAKgS,EAAS/R,MAG3DthE,IAAM,SAAUmgE,GAEZ,MADAA,GAAQD,EAAeC,GAChBr1E,KAAKq1E,MAGhBlsB,IAAM,SAAUksB,EAAOhuE,GACnB,GAAIk4E,EACJ,IAAqB,gBAAVlK,GACP,IAAKkK,IAAQlK,GACTr1E,KAAKmpD,IAAIo2B,EAAMlK,EAAMkK,QAIzBlK,GAAQD,EAAeC,GACI,kBAAhBr1E,MAAKq1E,IACZr1E,KAAKq1E,GAAOhuE,EAGpB,OAAOrH,OAMXqkC,OAAS,SAAU37B,GACf,GAAI8/E,EAEJ,OAAI9/E,KAAQlC,EACDxG,KAAK2yE,QAAQ6T,OAEpBgC,EAAgB3kF,GAAO8sE,WAAWjoE,GACb,MAAjB8/E,IACAxoF,KAAK2yE,QAAU6V,GAEZxoF,OAIfskC,KAAO4rC,EACH,kJACA,SAAUxnE,GACN,MAAIA,KAAQlC,EACDxG,KAAK2wE,aAEL3wE,KAAKqkC,OAAO37B,KAK/BioE,WAAa,WACT,MAAO3wE,MAAK2yE,SAGhBsU,eAAiB,WAGb,MAAuD,KAA/C/hF,KAAKwoB,MAAM1tB,KAAKk4B,GAAGuwD,oBAAsB,OA+CzD5kF,GAAOqV,GAAGuoB,YAAc59B,GAAOqV,GAAGqkB,aAAeiiD,GAAa,gBAAgB,GAC9E37E,GAAOqV,GAAGwoB,OAAS79B,GAAOqV,GAAGokB,QAAUkiD,GAAa,WAAW,GAC/D37E,GAAOqV,GAAGyoB,OAAS99B,GAAOqV,GAAGmkB,QAAUmiD,GAAa,WAAW,GAK/D37E,GAAOqV,GAAG0oB,KAAO/9B,GAAOqV,GAAGkkB,MAAQoiD,GAAa,SAAS,GAEzD37E,GAAOqV,GAAGuf,KAAO+mD,GAAa,QAAQ,GACtC37E,GAAOqV,GAAGsgB,MAAQ02C,EAAU,kDAAmDsP,GAAa,QAAQ,IACpG37E,GAAOqV,GAAGqf,KAAOinD,GAAa,YAAY,GAC1C37E,GAAOqV,GAAG+4D,MAAQ/B,EAAU,kDAAmDsP,GAAa,YAAY,IAGxG37E,GAAOqV,GAAGq5D,KAAO1uE,GAAOqV,GAAGmf,IAC3Bx0B,GAAOqV,GAAGk5D,OAASvuE,GAAOqV,GAAGwf,MAC7B70B,GAAOqV,GAAGm5D,MAAQxuE,GAAOqV,GAAGo5D,KAC5BzuE,GAAOqV,GAAGwvE,SAAW7kF,GAAOqV,GAAG+oE,QAC/Bp+E,GAAOqV,GAAGg5D,SAAWruE,GAAOqV,GAAGi5D,QAG/BtuE,GAAOqV,GAAGyvE,OAAS9kF,GAAOqV,GAAG9R,YAG7BvD,GAAOqV,GAAG0vE,MAAQ/kF,GAAOqV,GAAGkvE,MAkB5B9iF,EAAOzB,GAAOiM,SAASoJ,GAAK44D,EAAS3+D,WAEjCy/D,QAAU,WACN,GAIIt1C,GAASD,EAASD,EAJlBG,EAAev9B,KAAKwyE,cACpBD,EAAOvyE,KAAKyyE,MACZL,EAASpyE,KAAK0yE,QACdhgE,EAAO1S,KAAK4S,MACaq/D,EAAQ,CAIrCv/D,GAAK6qB,aAAeA,EAAe,IAEnCD,EAAUk2C,EAASj2C,EAAe,KAClC7qB,EAAK4qB,QAAUA,EAAU,GAEzBD,EAAUm2C,EAASl2C,EAAU,IAC7B5qB,EAAK2qB,QAAUA,EAAU,GAEzBD,EAAQo2C,EAASn2C,EAAU,IAC3B3qB,EAAK0qB,MAAQA,EAAQ,GAErBm1C,GAAQiB,EAASp2C,EAAQ,IAGzB60C,EAAQuB,EAASkM,GAAYnN,IAC7BA,GAAQiB,EAASmM,GAAY1N,IAI7BG,GAAUoB,EAASjB,EAAO,IAC1BA,GAAQ,GAGRN,GAASuB,EAASpB,EAAS,IAC3BA,GAAU,GAEV1/D,EAAK6/D,KAAOA,EACZ7/D,EAAK0/D,OAASA,EACd1/D,EAAKu/D,MAAQA,GAGjBpnD,IAAM,WAYF,MAXA7qB,MAAKwyE,cAAgBttE,KAAK2lB,IAAI7qB,KAAKwyE,eACnCxyE,KAAKyyE,MAAQvtE,KAAK2lB,IAAI7qB,KAAKyyE,OAC3BzyE,KAAK0yE,QAAUxtE,KAAK2lB,IAAI7qB,KAAK0yE,SAE7B1yE,KAAK4S,MAAM2qB,aAAer4B,KAAK2lB,IAAI7qB,KAAK4S,MAAM2qB,cAC9Cv9B,KAAK4S,MAAM0qB,QAAUp4B,KAAK2lB,IAAI7qB,KAAK4S,MAAM0qB,SACzCt9B,KAAK4S,MAAMyqB,QAAUn4B,KAAK2lB,IAAI7qB,KAAK4S,MAAMyqB,SACzCr9B,KAAK4S,MAAMwqB,MAAQl4B,KAAK2lB,IAAI7qB,KAAK4S,MAAMwqB,OACvCp9B,KAAK4S,MAAMw/D,OAASltE,KAAK2lB,IAAI7qB,KAAK4S,MAAMw/D,QACxCpyE,KAAK4S,MAAMq/D,MAAQ/sE,KAAK2lB,IAAI7qB,KAAK4S,MAAMq/D,OAEhCjyE,MAGXqyE,MAAQ,WACJ,MAAOmB,GAASxzE,KAAKuyE,OAAS,IAGlCvrE,QAAU,WACN,MAAOhH,MAAKwyE,cACG,MAAbxyE,KAAKyyE,MACJzyE,KAAK0yE,QAAU,GAAM,OACK,QAA3ByC,EAAMn1E,KAAK0yE,QAAU,KAG3B4U,SAAW,SAAUuB,GACjB,GAAIjV,GAAS4K,GAAax+E,MAAO6oF,EAAY7oF,KAAK2wE,aAMlD,OAJIkY,KACAjV,EAAS5zE,KAAK2wE,aAAa+U,YAAY1lF,KAAM4zE,IAG1C5zE,KAAK2wE,aAAaiV,WAAWhS,IAGxC3gE,IAAM,SAAU6hE,EAAOjC,GAEnB,GAAIwB,GAAMxwE,GAAOiM,SAASglE,EAAOjC,EAQjC,OANA7yE,MAAKwyE,eAAiB6B,EAAI7B,cAC1BxyE,KAAKyyE,OAAS4B,EAAI5B,MAClBzyE,KAAK0yE,SAAW2B,EAAI3B,QAEpB1yE,KAAK4yE,UAEE5yE,MAGXqrB,SAAW,SAAUypD,EAAOjC,GACxB,GAAIwB,GAAMxwE,GAAOiM,SAASglE,EAAOjC,EAQjC,OANA7yE,MAAKwyE,eAAiB6B,EAAI7B,cAC1BxyE,KAAKyyE,OAAS4B,EAAI5B,MAClBzyE,KAAK0yE,SAAW2B,EAAI3B,QAEpB1yE,KAAK4yE,UAEE5yE,MAGXkV,IAAM,SAAUmgE,GAEZ,MADAA,GAAQD,EAAeC,GAChBr1E,KAAKq1E,EAAM9wC,cAAgB,QAGtCtV,GAAK,SAAUomD,GACX,GAAI9C,GAAMH,CAGV,IAFAiD,EAAQD,EAAeC,GAET,UAAVA,GAA+B,SAAVA,EAGrB,MAFA9C,GAAOvyE,KAAKyyE,MAAQzyE,KAAKwyE,cAAgB,MACzCJ,EAASpyE,KAAK0yE,QAA8B,GAApBgN,GAAYnN,GACnB,UAAV8C,EAAoBjD,EAASA,EAAS,EAI7C,QADAG,EAAOvyE,KAAKyyE,MAAQvtE,KAAKwoB,MAAMiyD,GAAY3/E,KAAK0yE,QAAU,KAClD2C,GACJ,IAAK,OAAQ,MAAO9C,GAAO,EAAIvyE,KAAKwyE,cAAgB,MACpD,KAAK,MAAO,MAAOD,GAAOvyE,KAAKwyE,cAAgB,KAC/C,KAAK,OAAQ,MAAc,IAAPD,EAAYvyE,KAAKwyE,cAAgB,IACrD,KAAK,SAAU,MAAc,IAAPD,EAAY,GAAKvyE,KAAKwyE,cAAgB,GAC5D,KAAK,SAAU,MAAc,IAAPD,EAAY,GAAK,GAAKvyE,KAAKwyE,cAAgB,GAEjE,KAAK,cAAe,MAAOttE,MAAKC,MAAa,GAAPotE,EAAY,GAAK,GAAK,KAAQvyE,KAAKwyE,aACzE,SAAS,KAAM,IAAI5uE,OAAM,gBAAkByxE,KAKvD/wC,KAAOzgC,GAAOqV,GAAGorB,KACjBD,OAASxgC,GAAOqV,GAAGmrB,OAEnBykD,YAAc5Y,EACV,sFAEA,WACI,MAAOlwE,MAAKoH,gBAIpBA,YAAc,WAEV,GAAI6qE,GAAQ/sE,KAAK2lB,IAAI7qB,KAAKiyE,SACtBG,EAASltE,KAAK2lB,IAAI7qB,KAAKoyE,UACvBG,EAAOrtE,KAAK2lB,IAAI7qB,KAAKuyE,QACrBn1C,EAAQl4B,KAAK2lB,IAAI7qB,KAAKo9B,SACtBC,EAAUn4B,KAAK2lB,IAAI7qB,KAAKq9B,WACxBC,EAAUp4B,KAAK2lB,IAAI7qB,KAAKs9B,UAAYt9B,KAAKu9B,eAAiB,IAE9D,OAAKv9B,MAAK+oF,aAMF/oF,KAAK+oF,YAAc,EAAI,IAAM,IACjC,KACC9W,EAAQA,EAAQ,IAAM,KACtBG,EAASA,EAAS,IAAM,KACxBG,EAAOA,EAAO,IAAM,KACnBn1C,GAASC,GAAWC,EAAW,IAAM,KACtCF,EAAQA,EAAQ,IAAM,KACtBC,EAAUA,EAAU,IAAM,KAC1BC,EAAUA,EAAU,IAAM,IAXpB,OAcfqzC,WAAa,WACT,MAAO3wE,MAAK2yE,SAGhBgW,OAAS,WACL,MAAO3oF,MAAKoH,iBAIpBvD,GAAOiM,SAASoJ,GAAG7T,SAAWxB,GAAOiM,SAASoJ,GAAG9R,WAQjD,KAAK5B,KAAK86E,IACFnR,EAAWmR,GAAwB96E,KACnCo6E,GAAmBp6E,GAAE++B,cAI7B1gC,IAAOiM,SAASoJ,GAAG8vE,eAAiB,WAChC,MAAOhpF,MAAKivB,GAAG,OAEnBprB,GAAOiM,SAASoJ,GAAG6vE,UAAY,WAC3B,MAAO/oF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAG+vE,UAAY,WAC3B,MAAOjpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGgwE,QAAU,WACzB,MAAOlpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGiwE,OAAS,WACxB,MAAOnpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGkwE,QAAU,WACzB,MAAOppF,MAAKivB,GAAG,UAEnBprB,GAAOiM,SAASoJ,GAAGmwE,SAAW,WAC1B,MAAOrpF,MAAKivB,GAAG,MAEnBprB,GAAOiM,SAASoJ,GAAGowE,QAAU,WACzB,MAAOtpF,MAAKivB,GAAG,MASnBprB,GAAOwgC,OAAO,MACVklD,aAAc,uBACd3Y,QAAU,SAAU6C,GAChB,GAAIrtE,GAAIqtE,EAAS,GACbG,EAAuC,IAA7BuB,EAAM1B,EAAS,IAAM,IAAa,KACrC,IAANrtE,EAAW,KACL,IAANA,EAAW,KACL,IAANA,EAAW,KAAO,IACvB,OAAOqtE,GAASG,KA4BpBkE,GACAj4E,EAAOD,QAAUiE,IAEfmrE,EAAgC,SAAUwa,EAAS5pF,EAASC,GAM1D,MALIA,GAAO2xE,QAAU3xE,EAAO2xE,UAAY3xE,EAAO2xE,SAASiY,YAAa,IAEjExJ,GAAYp8E,OAASm8E,IAGlBn8E,IACTtD,KAAKX,EAASM,EAAqBN,EAASC,KAASmvE,IAAkCxoE,IAAc3G,EAAOD,QAAUovE,IACxH6Q,IAAW,MAIhBt/E,KAAKP,QAEqBO,KAAKX,EAAU,WAAa,MAAOI,SAAYE,EAAoB,IAAIL,KAIhG,SAASA,EAAQD,GAErB,GAAI8pF,GAAgCC,EAA8B3a,GAOjE,SAAUtvE,EAAMC,GAGXgqF,KAAmCD,EAAiC,EAAW1a,EAA2E,kBAAnC0a,GAAiDA,EAA+B3xE,MAAMnY,EAAS+pF,GAAiCD,IAAmEljF,SAAlCwoE,IAAgDnvE,EAAOD,QAAUovE,KAU7VhvE,KAAM,WAEN,QAASmlD,GAAS12C,GAChB,GAOIjJ,GAPA6D,EAAiBoF,GAAWA,EAAQpF,iBAAkB,EAEtDkQ,EAAY9K,GAAWA,EAAQ8K,WAAahQ,OAE5CqgF,KACAC,GAAUC,WAAYC,UACtBC,IAIJ,KAAKxkF,EAAI,GAAS,KAALA,EAAUA,IAAMwkF,EAAM5lF,OAAO6lF,aAAazkF,KAAOuyE,KAAK,IAAMvyE,EAAI,IAAK8L,OAAO,EAEzF,KAAK9L,EAAI,GAAS,IAALA,EAASA,IAAMwkF,EAAM5lF,OAAO6lF,aAAazkF,KAAOuyE,KAAKvyE,EAAG8L,OAAO,EAE5E,KAAK9L,EAAI,EAAS,GAALA,EAAUA,IAAMwkF,EAAM,GAAKxkF,IAAMuyE,KAAK,GAAKvyE,EAAG8L,OAAO,EAElE,KAAK9L,EAAI,EAAS,IAALA,EAAWA,IAAMwkF,EAAM,IAAMxkF,IAAMuyE,KAAK,IAAMvyE,EAAG8L,OAAO,EAErE,KAAK9L,EAAI,EAAS,GAALA,EAAUA,IAAMwkF,EAAM,MAAQxkF,IAAMuyE,KAAK,GAAKvyE,EAAG8L,OAAO,EAGrE04E,GAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAClC04E,EAAM,SAAWjS,KAAK,IAAKzmE,OAAO,GAElC04E,EAAY,MAAMjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAU,IAAQjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAY,MAAMjS,KAAK,GAAIzmE,OAAO,GAElC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAa,OAAKjS,KAAK,GAAIzmE,MAAO9K,QAClCwjF,EAAW,KAAOjS,KAAK,GAAIzmE,OAAO,GAClC04E,EAAiB,WAAKjS,KAAK,EAAGzmE,OAAO,GACrC04E,EAAW,KAAWjS,KAAK,EAAGzmE,OAAO,GACrC04E,EAAY,MAAUjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAW,KAAWjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAM,WAAgBjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAc,QAAQjS,KAAK,GAAIzmE,OAAO,GACtC04E,EAAgB,UAAMjS,KAAK,GAAIzmE,OAAO,GAEtC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,GACnC04E,EAAM,MAAYjS,KAAK,IAAKzmE,OAAO,EAInC,IAAI44E,GAAO,SAAS5gF,GAAQ6gF,EAAY7gF,EAAM,YAC1C8gF,EAAK,SAAS9gF,GAAQ6gF,EAAY7gF,EAAM,UAGxC6gF,EAAc,SAAS7gF,EAAMxC,GAC/B,GAAoCN,SAAhCqjF,EAAO/iF,GAAMwC,EAAM+gF,SAAwB,CAE7C,IAAK,GADDC,GAAQT,EAAO/iF,GAAMwC,EAAM+gF,SACtB7kF,EAAI,EAAGA,EAAI8kF,EAAM3kF,OAAQH,IACTgB,SAAnB8jF,EAAM9kF,GAAG8L,MACXg5E,EAAM9kF,GAAG0T,GAAG5P,GAEa,GAAlBghF,EAAM9kF,GAAG8L,OAAmC,GAAlBhI,EAAMyqC,SACvCu2C,EAAM9kF,GAAG0T,GAAG5P,GAEa,GAAlBghF,EAAM9kF,GAAG8L,OAAoC,GAAlBhI,EAAMyqC,UACxCu2C,EAAM9kF,GAAG0T,GAAG5P,EAIM,IAAlBD,GACFC,EAAMD,kBA4FZ,OAtFAugF,GAAiB90D,KAAO,SAASpsB,EAAKJ,EAAUxB,GAI9C,GAHaN,SAATM,IACFA,EAAO,WAEUN,SAAfwjF,EAAMthF,GACR,KAAM,IAAI9E,OAAM,oBAAsB8E,EAEFlC,UAAlCqjF,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,QAC1B8R,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,UAE1B8R,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAM/vE,MAAMkR,GAAG5Q,EAAUgJ,MAAM04E,EAAMthF,GAAK4I,SAKpEs4E,EAAiBW,QAAU,SAASjiF,EAAUxB,GAC/BN,SAATM,IACFA,EAAO,UAET,KAAK,GAAI4B,KAAOshF,GACVA,EAAMlkF,eAAe4C,IACvBkhF,EAAiB90D,KAAKpsB,EAAIJ,EAASxB,IAMzC8iF,EAAiBY,OAAS,SAASlhF,GACjC,IAAK,GAAIZ,KAAOshF,GACd,GAAIA,EAAMlkF,eAAe4C,GAAM,CAC7B,GAAsB,GAAlBY,EAAMyqC,UAAwC,GAApBi2C,EAAMthF,GAAK4I,OAAiBhI,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,KACpF,MAAOrvE,EAEJ,IAAsB,GAAlBY,EAAMyqC,UAAyC,GAApBi2C,EAAMthF,GAAK4I,OAAkBhI,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,KAC3F,MAAOrvE,EAEJ,IAAIY,EAAM+gF,SAAWL,EAAMthF,GAAKqvE,MAAe,SAAPrvE,EAC3C,MAAOA,GAIb,MAAO,wCAITkhF,EAAiBrb,OAAS,SAAS7lE,EAAKJ,EAAUxB,GAIhD,GAHaN,SAATM,IACFA,EAAO,WAEUN,SAAfwjF,EAAMthF,GACR,KAAM,IAAI9E,OAAM,oBAAsB8E,EAExC,IAAiBlC,SAAb8B,EAAwB,CAC1B,GAAImiF,MACAH,EAAQT,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,KACpC,IAAcvxE,SAAV8jF,EACF,IAAK,GAAI9kF,GAAI,EAAGA,EAAI8kF,EAAM3kF,OAAQH,KAC1B8kF,EAAM9kF,GAAG0T,IAAM5Q,GAAYgiF,EAAM9kF,GAAG8L,OAAS04E,EAAMthF,GAAK4I,QAC5Dm5E,EAAYziF,KAAK6hF,EAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAMvyE,GAIrDqkF,GAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,MAAQ0S,MAGhCZ,GAAO/iF,GAAMkjF,EAAMthF,GAAKqvE,UAK5B6R,EAAiBpgC,MAAQ,WACvBqgC,GAAUC,WAAYC,WAIxBH,EAAiBt2E,QAAU,WACzBu2E,GAAUC,WAAYC,UACtBxwE,EAAUpQ,oBAAoB,UAAW+gF,GAAM,GAC/C3wE,EAAUpQ,oBAAoB,QAASihF,GAAI,IAI7C7wE,EAAU5Q,iBAAiB,UAAUuhF,GAAK,GAC1C3wE,EAAU5Q,iBAAiB,QAAQyhF,GAAG,GAG/BR,EAGT,MAAOzkC,MAQL,SAAStlD,EAAQD,GAErB,GAAI8pF,GAAgCC,EAA8B3a,GAEjE,SAAUtvE,EAAMC,GAGXgqF,KAAmCD,EAAiC,EAAW1a,EAA2E,kBAAnC0a,GAAiDA,EAA+B3xE,MAAMnY,EAAS+pF,GAAiCD,IAAmEljF,SAAlCwoE,IAAgDnvE,EAAOD,QAAUovE,KAU7VhvE,KAAM,WAEN,GAAI0qF,GAAe,IAkBnB,OAAO,SAAS/lB,GAAY7gE,GAuG1B,QAASiE,GAAMy9D,GACb,MAAOA,GAAOjhE,MAAM,UAOtB,QAASomF,GAAkBrhF,GAEzB,GAAmB,iBAAfA,EAAMxC,KAAyB,CACjC,GAAIwC,EAAMwqC,SAAS82C,UAAYthF,EAAMwqC,SAAS82C,SAASthF,EAAMxC,MAC3D,MAIAwC,GAAMwqC,SAAS82C,YACfthF,EAAMwqC,SAAS82C,SAASthF,EAAMxC,OAAQ,EAK1C,GAAI+jF,IAAU,CACdvhF,GAAMq8B,gBAAkB,WACtBklD,GAAU,GAIZvhF,EAAMwhF,YAAcJ,CAIpB,KADA,GAAInjF,GAAOmjF,EACJnjF,IAASsjF,GAAS,CACvB,GAAIE,GAAYxjF,EAAKzD,QAAUyD,EAAKzD,OAAOinF,UAAUzhF,EAAMxC,KAC3D,IAAIikF,EACF,IAAK,GAAIvlF,GAAI,EAAGA,EAAIulF,EAAUplF,SAAWklF,EAASrlF,IAChDulF,EAAUvlF,GAAG8D,EAIjB/B,GAAOA,EAAKsC,YA9IhB,GAAI/F,EAAOknF,QAAS,CAGlB,GAAIjnF,GAASD,EAETmnF,EAAoB,SAASriF,EAAS6F,GACxC,MAAOk2D,GAAY,GAAI5gE,GAAO6E,EAAS6F,IAOzC,OALA1K,GAAOuB,OAAO2lF,EAAmBlnF,GACjCknF,EAAkBD,QAAU,SAAUpiF,EAAS6F,GAC7C,MAAOk2D,GAAY,GAAI5gE,GAAOinF,QAAQpiF,EAAS6F,KAG1Cw8E,EAIT,GAAIriF,GAAU9E,EAAO8E,OAiIrB,OAhIAA,GAAQ9E,OAASA,EAGjBA,EAAOonF,IAAMpnF,EAAOyP,GACpBzP,EAAOqnF,KAAOrnF,EAAO4P,IACrB5P,EAAOsnF,SAAWtnF,EAAOwP,QAGzBxP,EAAOinF,aAIPjnF,EAAOonF,IAAI,eAAgB,SAAU5hF,GAC/BA,EAAMqnC,UACR+5C,EAAephF,EAAMI,UAUzB5F,EAAOyP,GAAK,SAAUiyD,EAAQ6lB,GAa5B,MAXAtjF,GAAMy9D,GAAQn9D,QAAQ,SAAUiB,GAC9B,GAAIyhF,GAAYjnF,EAAOinF,UAAUzhF,EAC5ByhF,KACHjnF,EAAOinF,UAAUzhF,GAASyhF,KAG1BjnF,EAAOonF,IAAI5hF,EAAOqhF,IAEpBI,EAAU/iF,KAAKqjF,KAGVvnF,GAWTA,EAAO4P,IAAM,SAAU8xD,EAAQ6lB,GAoB7B,MAlBAtjF,GAAMy9D,GAAQn9D,QAAQ,SAAUiB,GAC9B,GAAIyhF,GAAYjnF,EAAOinF,UAAUzhF,EAC7ByhF,KACFA,EAAYM,EAAUN,EAAUp3E,OAAO,SAAUhI,GAC/C,MAAOA,KAAM0/E,OAGXN,EAAUplF,OAAS,EACrB7B,EAAOinF,UAAUzhF,GAASyhF,GAI1BjnF,EAAOqnF,KAAK7hF,EAAOqhF,SACZ7mF,GAAOinF,UAAUzhF,OAKvBxF,GAGTA,EAAOwP,QAAU,WAEf,GAAI1K,GAAU9E,EAAO8E,cACdA,GAAQ9E,OAGfA,EAAOinF,aAGPjnF,EAAOsnF,YAgDFtnF,MAOP,SAASjE,EAAQD,EAASM,GAE9B,GAAI8uE,IAKJ,SAAUzlE,EAAQgI,EAAU+5E,EAAY9kF,GAmBxC,QAAS+kF,GAAkBryE,EAAIsyE,EAASryE,GACpC,MAAOG,YAAWmyE,EAAOvyE,EAAIC,GAAUqyE,GAY3C,QAASE,GAAeC,EAAKzyE,EAAIC,GAC7B,MAAIlT,OAAMC,QAAQylF,IACdC,EAAKD,EAAKxyE,EAAQD,GAAKC,IAChB,IAEJ,EASX,QAASyyE,GAAK7oE,EAAK8oE,EAAU1yE,GACzB,GAAI3T,EAEJ,IAAKud,EAIL,GAAIA,EAAI1a,QACJ0a,EAAI1a,QAAQwjF,EAAU1yE,OACnB,IAAI4J,EAAIpd,SAAWa,EAEtB,IADAhB,EAAI,EACGA,EAAIud,EAAIpd,QACXkmF,EAAStrF,KAAK4Y,EAAS4J,EAAIvd,GAAIA,EAAGud,GAClCvd,QAGJ,KAAKA,IAAKud,GACNA,EAAIjd,eAAeN,IAAMqmF,EAAStrF,KAAK4Y,EAAS4J,EAAIvd,GAAIA,EAAGud,GAavE,QAASzd,GAAOwmF,EAAMlmC,EAAKmb,GAGvB,IAFA,GAAI3zD,GAAO7G,OAAO6G,KAAKw4C,GACnBpgD,EAAI,EACDA,EAAI4H,EAAKzH,UACPo7D,GAAUA,GAAS+qB,EAAK1+E,EAAK5H,MAAQgB,KACtCslF,EAAK1+E,EAAK5H,IAAMogD,EAAIx4C,EAAK5H,KAE7BA,GAEJ,OAAOsmF,GAUX,QAAS/qB,GAAM+qB,EAAMlmC,GACjB,MAAOtgD,GAAOwmF,EAAMlmC,GAAK,GAS7B,QAASmmC,GAAQC,EAAOlY,EAAMnlB,GAC1B,GACIs9B,GADAC,EAAQpY,EAAK3gE,SAGjB84E,GAASD,EAAM74E,UAAY5M,OAAO8H,OAAO69E,GACzCD,EAAO3lF,YAAc0lF,EACrBC,EAAOE,OAASD,EAEZv9B,GACArpD,EAAO2mF,EAAQt9B,GAUvB,QAAS88B,GAAOvyE,EAAIC,GAChB,MAAO,YACH,MAAOD,GAAGnB,MAAMoB,EAASzT,YAWjC,QAAS0mF,GAASvZ,EAAK55D,GACnB,aAAW45D,IAAOwZ,GACPxZ,EAAI96D,MAAMkB,EAAOA,EAAK,IAAMzS,EAAYA,EAAWyS,GAEvD45D,EASX,QAASyZ,GAAYC,EAAMC,GACvB,MAAQD,KAAS/lF,EAAagmF,EAAOD,EASzC,QAASE,GAAkB/iF,EAAQwN,EAAOm0E,GACtCO,EAAKc,EAASx1E,GAAQ,SAASpQ,GAC3B4C,EAAOf,iBAAiB7B,EAAMukF,GAAS,KAU/C,QAASsB,GAAqBjjF,EAAQwN,EAAOm0E,GACzCO,EAAKc,EAASx1E,GAAQ,SAASpQ,GAC3B4C,EAAOP,oBAAoBrC,EAAMukF,GAAS,KAWlD,QAASuB,GAAU9mC,EAAMhhB,GACrB,KAAOghB,GAAM,CACT,GAAIA,GAAQhhB,EACR,OAAO,CAEXghB,GAAOA,EAAKj8C,WAEhB,OAAO,EASX,QAASgjF,GAAMC,EAAKC,GAChB,MAAOD,GAAInmF,QAAQomF,GAAQ,GAQ/B,QAASL,GAASI,GACd,MAAOA,GAAI5/E,OAAOnF,MAAM,QAU5B,QAASilF,GAAQpnC,EAAKmnC,EAAME,GACxB,GAAIrnC,EAAIj/C,UAAYsmF,EAChB,MAAOrnC,GAAIj/C,QAAQomF,EAGnB,KADA,GAAIvnF,GAAI,EACDA,EAAIogD,EAAIjgD,QAAQ,CACnB,GAAKsnF,GAAarnC,EAAIpgD,GAAGynF,IAAcF,IAAWE,GAAarnC,EAAIpgD,KAAOunF,EACtE,MAAOvnF,EAEXA,KAEJ,MAAO,GASf,QAAS+C,GAAQwa,GACb,MAAO9c,OAAMkN,UAAUlI,MAAM1K,KAAKwiB,EAAK,GAU3C,QAASmqE,GAAYtnC,EAAKl9C,EAAKwN,GAK3B,IAJA,GAAI4/D,MACAh/D,KACAtR,EAAI,EAEDA,EAAIogD,EAAIjgD,QAAQ,CACnB,GAAIktE,GAAMnqE,EAAMk9C,EAAIpgD,GAAGkD,GAAOk9C,EAAIpgD,EAC9BwnF,GAAQl2E,EAAQ+7D,GAAO,GACvBiD,EAAQ9tE,KAAK49C,EAAIpgD,IAErBsR,EAAOtR,GAAKqtE,EACZrtE,IAaJ,MAVI0Q,KAII4/D,EAHCptE,EAGSotE,EAAQ5/D,KAAK,SAAyB3Q,EAAGa,GAC/C,MAAOb,GAAEmD,GAAOtC,EAAEsC,KAHZotE,EAAQ5/D,QAQnB4/D,EASX,QAASqX,GAASpqE,EAAKqqE,GAKnB,IAJA,GAAIC,GAAQxnF,EACRynF,EAAYF,EAAS,GAAGnoD,cAAgBmoD,EAASniF,MAAM,GAEvDzF,EAAI,EACDA,EAAI+nF,GAAgB5nF,QAAQ,CAI/B,GAHA0nF,EAASE,GAAgB/nF,GACzBK,EAAO,EAAWwnF,EAASC,EAAYF,EAEnCvnF,IAAQkd,GACR,MAAOld,EAEXL,KAEJ,MAAOgB,GAQX,QAASgnF,KACL,MAAOC,MAQX,QAASC,GAAoB9kF,GACzB,GAAI+kF,GAAM/kF,EAAQglF,aAClB,OAAQD,GAAIE,aAAeF,EAAIG,aAyCnC,QAASC,GAAMC,EAAS1lF,GACpB,GAAIomE,GAAO1uE,IACXA,MAAKguF,QAAUA,EACfhuF,KAAKsI,SAAWA,EAChBtI,KAAK4I,QAAUolF,EAAQplF,QACvB5I,KAAK0J,OAASskF,EAAQv/E,QAAQw/E,YAI9BjuF,KAAKkuF,WAAa,SAASC,GACnB/B,EAAS4B,EAAQv/E,QAAQ20B,QAAS4qD,KAClCtf,EAAK2c,QAAQ8C,IAIrBnuF,KAAKouF,OAoCT,QAASC,GAAoBL,GACzB,GAAIM,GACAC,EAAaP,EAAQv/E,QAAQ8/E,UAajC,OAAO,KAVHD,EADAC,EACOA,EACAC,GACAC,EACAC,GACAC,EACCC,GAGDC,EAFAC,GAIOd,EAAShnB,GAS/B,QAASA,GAAagnB,EAASe,EAAWja,GACtC,GAAIka,GAAcla,EAAMma,SAAStpF,OAC7BupF,EAAqBpa,EAAMqa,gBAAgBxpF,OAC3CgrC,EAAWo+C,EAAYK,IAAgBJ,EAAcE,IAAuB,EAC5EhoB,EAAW6nB,GAAaM,GAAYC,KAAkBN,EAAcE,IAAuB,CAE/Fpa,GAAMnkC,UAAYA,EAClBmkC,EAAM5N,UAAYA,EAEdv2B,IACAq9C,EAAQuB,YAKZza,EAAMia,UAAYA,EAGlBS,EAAiBxB,EAASlZ,GAG1BkZ,EAAQpgE,KAAK,eAAgBknD,GAE7BkZ,EAAQyB,UAAU3a,GAClBkZ,EAAQuB,QAAQG,UAAY5a,EAQhC,QAAS0a,GAAiBxB,EAASlZ,GAC/B,GAAIya,GAAUvB,EAAQuB,QAClBN,EAAWna,EAAMma,SACjBU,EAAiBV,EAAStpF,MAGzB4pF,GAAQK,aACTL,EAAQK,WAAaC,EAAqB/a,IAI1C6a,EAAiB,IAAMJ,EAAQO,cAC/BP,EAAQO,cAAgBD,EAAqB/a,GACnB,IAAnB6a,IACPJ,EAAQO,eAAgB,EAG5B,IAAIF,GAAaL,EAAQK,WACrBE,EAAgBP,EAAQO,cACxBC,EAAeD,EAAgBA,EAAc5jE,OAAS0jE,EAAW1jE,OAEjEA,EAAS4oD,EAAM5oD,OAAS8jE,EAAUf,EACtCna,GAAMmb,UAAY9yD,KAClB23C,EAAMob,UAAYpb,EAAMmb,UAAYL,EAAWK,UAE/Cnb,EAAM1mB,MAAQ+hC,EAASJ,EAAc7jE,GACrC4oD,EAAMnvD,SAAW84C,EAAYsxB,EAAc7jE,GAE3CkkE,EAAeb,EAASza,GACxBA,EAAMub,gBAAkBC,EAAaxb,EAAMt1C,OAAQs1C,EAAMr1C,QAEzDq1C,EAAM73D,MAAQ6yE,EAAgBv8B,EAASu8B,EAAcb,SAAUA,GAAY,EAC3Ena,EAAMyb,SAAWT,EAAgBU,EAAYV,EAAcb,SAAUA,GAAY,EAEjFwB,EAAyBlB,EAASza,EAGlC,IAAIprE,GAASskF,EAAQplF,OACjBgkF,GAAU9X,EAAMhhC,SAASpqC,OAAQA,KACjCA,EAASorE,EAAMhhC,SAASpqC,QAE5BorE,EAAMprE,OAASA,EAGnB,QAAS0mF,GAAeb,EAASza,GAC7B,GAAI5oD,GAAS4oD,EAAM5oD,OACfvC,EAAS4lE,EAAQmB,gBACjBC,EAAYpB,EAAQoB,cACpBjB,EAAYH,EAAQG,eAEpB5a,EAAMia,YAAcK,IAAeM,EAAUX,YAAcM,MAC3DsB,EAAYpB,EAAQoB,WAChB5+E,EAAG29E,EAAUlwD,QAAU,EACvBxtB,EAAG09E,EAAUjwD,QAAU,GAG3B9V,EAAS4lE,EAAQmB,aACb3+E,EAAGma,EAAOna,EACVC,EAAGka,EAAOla,IAIlB8iE,EAAMt1C,OAASmxD,EAAU5+E,GAAKma,EAAOna,EAAI4X,EAAO5X,GAChD+iE,EAAMr1C,OAASkxD,EAAU3+E,GAAKka,EAAOla,EAAI2X,EAAO3X,GAQpD,QAASy+E,GAAyBlB,EAASza,GACvC,GAEItW,GAAUoyB,EAAWC,EAAW31D,EAFhC41D,EAAOvB,EAAQwB,cAAgBjc,EAC/Bob,EAAYpb,EAAMmb,UAAYa,EAAKb,SAGvC,IAAInb,EAAMia,WAAaO,KAAiBY,EAAYc,IAAoBF,EAAKtyB,WAAah4D,GAAY,CAClG,GAAIg5B,GAASsxD,EAAKtxD,OAASs1C,EAAMt1C,OAC7BC,EAASqxD,EAAKrxD,OAASq1C,EAAMr1C,OAE7B5zB,EAAIolF,EAAYf,EAAW1wD,EAAQC,EACvCmxD,GAAY/kF,EAAEkG,EACd8+E,EAAYhlF,EAAEmG,EACdwsD,EAAY3zC,GAAIhf,EAAEkG,GAAK8Y,GAAIhf,EAAEmG,GAAMnG,EAAEkG,EAAIlG,EAAEmG,EAC3CkpB,EAAYo1D,EAAa9wD,EAAQC,GAEjC8vD,EAAQwB,aAAejc,MAGvBtW,GAAWsyB,EAAKtyB,SAChBoyB,EAAYE,EAAKF,UACjBC,EAAYC,EAAKD,UACjB31D,EAAY41D,EAAK51D,SAGrB45C,GAAMtW,SAAWA,EACjBsW,EAAM8b,UAAYA,EAClB9b,EAAM+b,UAAYA,EAClB/b,EAAM55C,UAAYA,EAQtB,QAAS20D,GAAqB/a,GAK1B,IAFA,GAAIma,MACAzpF,EAAI,EACDA,EAAIsvE,EAAMma,SAAStpF,QACtBspF,EAASzpF,IACLmX,QAAS+Q,GAAMonD,EAAMma,SAASzpF,GAAGmX,SACjCG,QAAS4Q,GAAMonD,EAAMma,SAASzpF,GAAGsX,UAErCtX,GAGJ,QACIyqF,UAAW9yD,KACX8xD,SAAUA,EACV/iE,OAAQ8jE,EAAUf,GAClBzvD,OAAQs1C,EAAMt1C,OACdC,OAAQq1C,EAAMr1C,QAStB,QAASuwD,GAAUf,GACf,GAAIU,GAAiBV,EAAStpF,MAG9B,IAAuB,IAAnBgqF,EACA,OACI59E,EAAG2b,GAAMuhE,EAAS,GAAGtyE,SACrB3K,EAAG0b,GAAMuhE,EAAS,GAAGnyE,SAK7B,KADA,GAAI/K,GAAI,EAAGC,EAAI,EAAGxM,EAAI,EACXmqF,EAAJnqF,GACHuM,GAAKk9E,EAASzpF,GAAGmX,QACjB3K,GAAKi9E,EAASzpF,GAAGsX,QACjBtX,GAGJ,QACIuM,EAAG2b,GAAM3b,EAAI49E,GACb39E,EAAG0b,GAAM1b,EAAI29E,IAWrB,QAASsB,GAAYf,EAAWn+E,EAAGC,GAC/B,OACID,EAAGA,EAAIm+E,GAAa,EACpBl+E,EAAGA,EAAIk+E,GAAa,GAU5B,QAASI,GAAav+E,EAAGC,GACrB,MAAID,KAAMC,EACCk/E,GAGPrmE,GAAI9Y,IAAM8Y,GAAI7Y,GACPD,EAAI,EAAIo/E,GAAiBC,GAE7Bp/E,EAAI,EAAIq/E,GAAeC,GAUlC,QAAS7yB,GAAY2L,EAAIC,EAAIrkE,GACpBA,IACDA,EAAQurF,GAEZ,IAAIx/E,GAAIs4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,IAC5BgM,EAAIq4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,GAEhC,OAAOd,MAAKyqB,KAAM5d,EAAIA,EAAMC,EAAIA,GAUpC,QAASm+E,GAAS/lB,EAAIC,EAAIrkE,GACjBA,IACDA,EAAQurF,GAEZ,IAAIx/E,GAAIs4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,IAC5BgM,EAAIq4D,EAAGrkE,EAAM,IAAMokE,EAAGpkE,EAAM,GAChC,OAA0B,KAAnBd,KAAK4xD,MAAM9kD,EAAGD,GAAW7M,KAAKymB,GASzC,QAAS6kE,GAAY5gF,EAAOC,GACxB,MAAOsgF,GAAStgF,EAAI,GAAIA,EAAI,GAAI2hF,IAAmBrB,EAASvgF,EAAM,GAAIA,EAAM,GAAI4hF,IAUpF,QAASj+B,GAAS3jD,EAAOC,GACrB,MAAO4uD,GAAY5uD,EAAI,GAAIA,EAAI,GAAI2hF,IAAmB/yB,EAAY7uD,EAAM,GAAIA,EAAM,GAAI4hF,IAiB1F,QAAS1C,KACL9uF,KAAKyxF,KAAOC,GACZ1xF,KAAK2xF,MAAQC,GAEb5xF,KAAK6xF,OAAQ,EACb7xF,KAAK8xF,SAAU,EAEf/D,EAAMh2E,MAAM/X,KAAM0F,WAoEtB,QAAS+oF,KACLzuF,KAAKyxF,KAAOM,GACZ/xF,KAAK2xF,MAAQK,GAEbjE,EAAMh2E,MAAM/X,KAAM0F,WAElB1F,KAAKiyF,MAASjyF,KAAKguF,QAAQuB,QAAQ2C,iBAoEvC,QAASC,KACLnyF,KAAKoyF,SAAWC,GAChBryF,KAAK2xF,MAAQW,GACbtyF,KAAKuyF,SAAU,EAEfxE,EAAMh2E,MAAM/X,KAAM0F,WAsCtB,QAAS8sF,GAAuBrE,EAAIrnF,GAChC,GAAI2rF,GAAMlqF,EAAQ4lF,EAAGuE,SACjBzzD,EAAU12B,EAAQ4lF,EAAGwE,eAMzB,OAJI7rF,IAAQuoF,GAAYC,MACpBmD,EAAMvF,EAAYuF,EAAIz+E,OAAOirB,GAAU,cAAc,KAGjDwzD,EAAKxzD,GAiBjB,QAAS0vD,KACL3uF,KAAKoyF,SAAWQ,GAChB5yF,KAAK6yF,aAEL9E,EAAMh2E,MAAM/X,KAAM0F,WA0BtB,QAASotF,GAAW3E,EAAIrnF,GACpB,GAAIisF,GAAaxqF,EAAQ4lF,EAAGuE,SACxBG,EAAY7yF,KAAK6yF,SAGrB,IAAI/rF,GAAQsoF,GAAc4D,KAAqC,IAAtBD,EAAWptF,OAEhD,MADAktF,GAAUE,EAAW,GAAGE,aAAc,GAC9BF,EAAYA,EAGxB,IAAIvtF,GACAoX,EACA+1E,EAAiBpqF,EAAQ4lF,EAAGwE,gBAC5BO,KACAxpF,EAAS1J,KAAK0J,MAQlB,IALAkT,EAAgBm2E,EAAWp/E,OAAO,SAASmqB,GACvC,MAAO8uD,GAAU9uD,EAAMp0B,OAAQA,KAI/B5C,IAASsoF,GAET,IADA5pF,EAAI,EACGA,EAAIoX,EAAcjX,QACrBktF,EAAUj2E,EAAcpX,GAAGytF,aAAc,EACzCztF,GAMR,KADAA,EAAI,EACGA,EAAImtF,EAAehtF,QAClBktF,EAAUF,EAAentF,GAAGytF,aAC5BC,EAAqBlrF,KAAK2qF,EAAentF,IAIzCsB,GAAQuoF,GAAYC,WACbuD,GAAUF,EAAentF,GAAGytF,YAEvCztF,GAGJ,OAAK0tF,GAAqBvtF,QAMtBunF,EAAYtwE,EAAc5I,OAAOk/E,GAAuB,cAAc,GACtEA,GAPJ,OAoBJ,QAASrE,KACLd,EAAMh2E,MAAM/X,KAAM0F,UAElB,IAAI2lF,GAAUI,EAAOzrF,KAAKqrF,QAASrrF,KACnCA,MAAK89B,MAAQ,GAAI6wD,GAAW3uF,KAAKguF,QAAS3C,GAC1CrrF,KAAKmzF,MAAQ,GAAIrE,GAAW9uF,KAAKguF,QAAS3C,GAyD9C,QAAS+H,GAAYpF,EAAS3mF,GAC1BrH,KAAKguF,QAAUA,EACfhuF,KAAKmpD,IAAI9hD,GAuFb,QAASgsF,GAAkBC,GAEvB,GAAIzG,EAAMyG,EAASC,IACf,MAAOA,GAGX,IAAIC,GAAU3G,EAAMyG,EAASG,IACzBC,EAAU7G,EAAMyG,EAASK,GAG7B,OAAIH,IAAWE,EACJD,GAAqB,IAAME,GAIlCH,GAAWE,EACJF,EAAUC,GAAqBE,GAItC9G,EAAMyG,EAASM,IACRA,GAGJC,GA4CX,QAASC,GAAWrlF,GAChBzO,KAAKK,GAAKmtF,IAEVxtF,KAAKguF,QAAU,KACfhuF,KAAKyO,QAAUsyD,EAAMtyD,MAAezO,KAAK+zF,UAGzC/zF,KAAKyO,QAAQ20B,OAASkpD,EAAYtsF,KAAKyO,QAAQ20B,QAAQ,GAEvDpjC,KAAKg0F,MAAQC,GAEbj0F,KAAKk0F,gBACLl0F,KAAKm0F,eAiOT,QAASC,GAASJ,GACd,MAAIA,GAAQK,GACD,SACAL,EAAQM,GACR,MACAN,EAAQO,GACR,OACAP,EAAQQ,GACR,QAEJ,GAQX,QAASC,GAAav5D,GAClB,MAAIA,IAAao2D,GACN,OACAp2D,GAAam2D,GACb,KACAn2D,GAAai2D,GACb,OACAj2D,GAAak2D,GACb,QAEJ,GASX,QAASsD,IAA6BC,EAAiBC,GACnD,GAAI5G,GAAU4G,EAAW5G,OACzB,OAAIA,GACOA,EAAQ94E,IAAIy/E,GAEhBA,EAQX,QAASE,MACLf,EAAW/7E,MAAM/X,KAAM0F,WA6D3B,QAASovF,MACLD,GAAe98E,MAAM/X,KAAM0F,WAE3B1F,KAAK+0F,GAAK,KACV/0F,KAAKg1F,GAAK,KA2Ed,QAASC,MACLJ,GAAe98E,MAAM/X,KAAM0F,WAsC/B,QAASwvF,MACLpB,EAAW/7E,MAAM/X,KAAM0F,WAEvB1F,KAAKm1F,OAAS,KACdn1F,KAAKo1F,OAAS,KAmElB,QAASC,MACLR,GAAe98E,MAAM/X,KAAM0F,WA8B/B,QAAS4vF,MACLT,GAAe98E,MAAM/X,KAAM0F,WA0D/B,QAAS6vF,MACLzB,EAAW/7E,MAAM/X,KAAM0F,WAIvB1F,KAAKw1F,OAAQ,EACbx1F,KAAKy1F,SAAU,EAEfz1F,KAAKm1F,OAAS,KACdn1F,KAAKo1F,OAAS,KACdp1F,KAAKgX,MAAQ,EAqGjB,QAASjT,IAAO6E,EAAS6F,GAGrB,MAFAA,GAAUA,MACVA,EAAQinF,YAAcpJ,EAAY79E,EAAQinF,YAAa3xF,GAAOgwF,SAAS4B,QAChE,GAAI3K,IAAQpiF,EAAS6F,GAiIhC,QAASu8E,IAAQpiF,EAAS6F,GACtBA,EAAUA,MAEVzO,KAAKyO,QAAUsyD,EAAMtyD,EAAS1K,GAAOgwF,UACrC/zF,KAAKyO,QAAQw/E,YAAcjuF,KAAKyO,QAAQw/E,aAAerlF,EAEvD5I,KAAK41F,YACL51F,KAAKuvF,WACLvvF,KAAK01F,eAEL11F,KAAK4I,QAAUA,EACf5I,KAAK80E,MAAQuZ,EAAoBruF,MACjCA,KAAKslE,YAAc,GAAI8tB,GAAYpzF,KAAMA,KAAKyO,QAAQ62D,aAEtDuwB,GAAe71F,MAAM,GAErB4rF,EAAKn9E,EAAQinF,YAAa,SAASrmF,GAC/B,GAAIulF,GAAa50F,KAAKiT,IAAI,GAAK5D,GAAK,GAAIA,EAAK,IAC7CA,GAAK,IAAMulF,EAAWkB,cAAczmF,EAAK,IACzCA,EAAK,IAAMulF,EAAWmB,eAAe1mF,EAAK,KAC3CrP,MAyOP,QAAS61F,IAAe7H,EAAS/6E,GAC7B,GAAIrK,GAAUolF,EAAQplF,OACtBgjF,GAAKoC,EAAQv/E,QAAQunF,SAAU,SAAS3uF,EAAO4O,GAC3CrN,EAAQqE,MAAMkgF,EAASvkF,EAAQqE,MAAOgJ,IAAShD,EAAM5L,EAAQ,KASrE,QAAS4uF,IAAgB3sF,EAAOoJ,GAC5B,GAAIwjF,GAAe3kF,EAAS4kF,YAAY,QACxCD,GAAaE,UAAU9sF,GAAO,GAAM,GACpC4sF,EAAaG,QAAU3jF,EACvBA,EAAKhJ,OAAO4sF,cAAcJ,GAr1E9B,GAAI3I,KAAmB,GAAI,SAAU,MAAO,KAAM,KAAM,KACpDgJ,GAAehlF,EAASM,cAAc,OAEtCw6E,GAAgB,WAEhB3+D,GAAQxoB,KAAKwoB,MACb7C,GAAM3lB,KAAK2lB,IACXsS,GAAM74B,KAAK64B,IAwSXswD,GAAY,EAeZ+I,GAAe,wCAEf5H,GAAiB,gBAAkBrlF,GACnCilF,GAAyBrB,EAAS5jF,EAAQ,kBAAoB/C,EAC9DkoF,GAAqBE,IAAiB4H,GAAaxoF,KAAKhF,UAAUC,WAElEwtF,GAAmB,QACnBC,GAAiB,MACjBC,GAAmB,QACnBC,GAAoB,SAEpB5F,GAAmB,GAEnB5B,GAAc,EACd4D,GAAa,EACb3D,GAAY,EACZC,GAAe,EAEf4B,GAAiB,EACjBC,GAAiB,EACjBC,GAAkB,EAClBC,GAAe,EACfC,GAAiB,GAEjBuF,GAAuB1F,GAAiBC,GACxC0F,GAAqBzF,GAAeC,GACpCyF,GAAgBF,GAAuBC,GAEvCvF,IAAY,IAAK,KACjBC,IAAmB,UAAW,UA4BlCzD,GAAM56E,WAKFk4E,QAAS,aAKT+C,KAAM,WACFpuF,KAAKyxF,MAAQhF,EAAkBzsF,KAAK4I,QAAS5I,KAAKyxF,KAAMzxF,KAAKkuF,YAC7DluF,KAAKoyF,UAAY3F,EAAkBzsF,KAAK0J,OAAQ1J,KAAKoyF,SAAUpyF,KAAKkuF,YACpEluF,KAAK2xF,OAASlF,EAAkBiB,EAAoB1tF,KAAK4I,SAAU5I,KAAK2xF,MAAO3xF,KAAKkuF,aAMxF56E,QAAS,WACLtT,KAAKyxF,MAAQ9E,EAAqB3sF,KAAK4I,QAAS5I,KAAKyxF,KAAMzxF,KAAKkuF,YAChEluF,KAAKoyF,UAAYzF,EAAqB3sF,KAAK0J,OAAQ1J,KAAKoyF,SAAUpyF,KAAKkuF,YACvEluF,KAAK2xF,OAAShF,EAAqBe,EAAoB1tF,KAAK4I,SAAU5I,KAAK2xF,MAAO3xF,KAAKkuF,aAoT/F,IAAI8I,KACAC,UAAW7H,GACX8H,UAAWlE,GACXmE,QAAS9H,IAGTqC,GAAuB,YACvBE,GAAsB,mBAiB1B7F,GAAQ+C,EAAYf,GAKhB1C,QAAS,SAAmB8C,GACxB,GAAIY,GAAYiI,GAAgB7I,EAAGrnF,KAG/BioF,GAAYK,IAA6B,IAAdjB,EAAG1hE,SAC9BzsB,KAAK8xF,SAAU,GAGf/C,EAAYiE,IAA2B,IAAb7E,EAAG3hE,QAC7BuiE,EAAYM,IAIXrvF,KAAK8xF,SAAY9xF,KAAK6xF,QAIvB9C,EAAYM,KACZrvF,KAAK8xF,SAAU,GAGnB9xF,KAAKsI,SAAStI,KAAKguF,QAASe,GACxBE,UAAWd,GACXgB,iBAAkBhB,GAClBiJ,YAAaT,GACb7iD,SAAUq6C,OAKtB,IAAIkJ,KACAC,YAAalI,GACbmI,YAAavE,GACbwE,UAAWnI,GACXoI,cAAenI,GACfoI,WAAYpI,IAIZqI,IACAC,EAAGnB,GACHoB,EAAGnB,GACHoB,EAAGnB,GACHoB,EAAGnB,IAGH7E,GAAyB,cACzBC,GAAwB,qCAGxBzoF,GAAOyuF,iBACPjG,GAAyB,gBACzBC,GAAwB,6CAiB5BjG,EAAQ0C,EAAmBV,GAKvB1C,QAAS,SAAmB8C,GACxB,GAAI8D,GAAQjyF,KAAKiyF,MACbgG,GAAgB,EAEhBC,EAAsB/J,EAAGrnF,KAAKy9B,cAAc/5B,QAAQ,KAAM,IAC1DukF,EAAYsI,GAAkBa,GAC9Bd,EAAcO,GAAuBxJ,EAAGiJ,cAAgBjJ,EAAGiJ,YAE3De,EAAWf,GAAeX,GAG1B2B,EAAapL,EAAQiF,EAAO9D,EAAGkK,UAAW,YAG1CtJ,GAAYK,KAA8B,IAAdjB,EAAG1hE,QAAgB0rE,GAC9B,EAAbC,IACAnG,EAAMjqF,KAAKmmF,GACXiK,EAAanG,EAAMtsF,OAAS,GAEzBopF,GAAaM,GAAYC,MAChC2I,GAAgB,GAIH,EAAbG,IAKJnG,EAAMmG,GAAcjK,EAEpBnuF,KAAKsI,SAAStI,KAAKguF,QAASe,GACxBE,SAAUgD,EACV9C,iBAAkBhB,GAClBiJ,YAAaA,EACbtjD,SAAUq6C,IAGV8J,GAEAhG,EAAM7pF,OAAOgwF,EAAY,MAKrC,IAAIE,KACAC,WAAYnJ,GACZoJ,UAAWxF,GACXyF,SAAUpJ,GACVqJ,YAAapJ,IAGb+C,GAA6B,aAC7BC,GAA6B,2CAejCvG,GAAQoG,EAAkBpE,GACtB1C,QAAS,SAAmB8C,GACxB,GAAIrnF,GAAOwxF,GAAuBnK,EAAGrnF,KAOrC,IAJIA,IAASsoF,KACTpvF,KAAKuyF,SAAU,GAGdvyF,KAAKuyF,QAAV,CAIA,GAAIG,GAAUF,EAAuBjyF,KAAKP,KAAMmuF,EAAIrnF,EAGhDA,IAAQuoF,GAAYC,KAAiBoD,EAAQ,GAAG/sF,OAAS+sF,EAAQ,GAAG/sF,SAAW,IAC/E3F,KAAKuyF,SAAU,GAGnBvyF,KAAKsI,SAAStI,KAAKguF,QAASlnF,GACxBmoF,SAAUyD,EAAQ,GAClBvD,gBAAiBuD,EAAQ,GACzB0E,YAAaX,GACb3iD,SAAUq6C,OAsBtB,IAAIwK,KACAJ,WAAYnJ,GACZoJ,UAAWxF,GACXyF,SAAUpJ,GACVqJ,YAAapJ,IAGbsD,GAAsB,2CAc1B7G,GAAQ4C,EAAYZ,GAChB1C,QAAS,SAAoB8C,GACzB,GAAIrnF,GAAO6xF,GAAgBxK,EAAGrnF,MAC1B4rF,EAAUI,EAAWvyF,KAAKP,KAAMmuF,EAAIrnF,EACnC4rF,IAIL1yF,KAAKsI,SAAStI,KAAKguF,QAASlnF,GACxBmoF,SAAUyD,EAAQ,GAClBvD,gBAAiBuD,EAAQ,GACzB0E,YAAaX,GACb3iD,SAAUq6C,OAmFtBpC,EAAQ8C,EAAiBd,GAOrB1C,QAAS,SAAoB2C,EAAS4K,EAAYC,GAC9C,GAAIV,GAAWU,EAAUzB,aAAeX,GACpCqC,EAAWD,EAAUzB,aAAeT,EAIxC,IAAIwB,EACAn4F,KAAKmzF,MAAMtB,OAAQ,MAChB,IAAIiH,IAAY94F,KAAKmzF,MAAMtB,MAC9B,MAIA+G,IAAcvJ,GAAYC,MAC1BtvF,KAAKmzF,MAAMtB,OAAQ,GAGvB7xF,KAAKsI,SAAS0lF,EAAS4K,EAAYC,IAMvCvlF,QAAS,WACLtT,KAAK89B,MAAMxqB,UACXtT,KAAKmzF,MAAM7/E,YAInB,IAAIylF,IAAwB5L,EAASoJ,GAAatpF,MAAO,eACrD+rF,GAAsBD,KAA0BvyF,EAGhDyyF,GAAuB,UACvBpF,GAAoB,OACpBD,GAA4B,eAC5BL,GAAoB,OACpBE,GAAqB,QACrBE,GAAqB,OAczBP,GAAYjgF,WAKRg2C,IAAK,SAAS9hD,GAENA,GAAS4xF,KACT5xF,EAAQrH,KAAKk5F,WAGbF,KACAh5F,KAAKguF,QAAQplF,QAAQqE,MAAM8rF,IAAyB1xF,GAExDrH,KAAKszF,QAAUjsF,EAAMk9B,cAAcr3B,QAMvC2H,OAAQ,WACJ7U,KAAKmpD,IAAInpD,KAAKguF,QAAQv/E,QAAQ62D,cAOlC4zB,QAAS,WACL,GAAI5F,KAMJ,OALA1H,GAAK5rF,KAAKguF,QAAQ0H,YAAa,SAASd,GAChCxI,EAASwI,EAAWnmF,QAAQ20B,QAASwxD,MACrCtB,EAAUA,EAAQt/E,OAAO4gF,EAAWuE,qBAGrC9F,EAAkBC,EAAQrrF,KAAK,OAO1CmxF,gBAAiB,SAAStkB,GAEtB,IAAIkkB,GAAJ,CAIA,GAAIllD,GAAWghC,EAAMhhC,SACjB5Y,EAAY45C,EAAMub,eAGtB,IAAIrwF,KAAKguF,QAAQuB,QAAQ8J,UAErB,WADAvlD,GAASzqC,gBAIb,IAAIiqF,GAAUtzF,KAAKszF,QACfgG,EAAUzM,EAAMyG,EAASC,IACzBG,EAAU7G,EAAMyG,EAASK,IACzBH,EAAU3G,EAAMyG,EAASG,GAE7B,OAAI6F,IACC5F,GAAWx4D,EAAY27D,IACvBrD,GAAWt4D,EAAY47D,GACjB92F,KAAKu5F,WAAWzlD,GAH3B,SAWJylD,WAAY,SAASzlD,GACjB9zC,KAAKguF,QAAQuB,QAAQ8J,WAAY,EACjCvlD,EAASzqC,kBA+DjB,IAAI4qF,IAAiB,EACjBO,GAAc,EACdD,GAAgB,EAChBD,GAAc,EACdkF,GAAmBlF,GACnBD,GAAkB,GAClBoF,GAAe,EAuBnB3F,GAAW3gF,WAKP4gF,YAOA5qC,IAAK,SAAS16C,GAKV,MAJAnJ,GAAOtF,KAAKyO,QAASA,GAGrBzO,KAAKguF,SAAWhuF,KAAKguF,QAAQ1oB,YAAYzwD,SAClC7U,MAQX81F,cAAe,SAASnB,GACpB,GAAIjJ,EAAeiJ,EAAiB,gBAAiB30F,MACjD,MAAOA,KAGX,IAAIk0F,GAAel0F,KAAKk0F,YAMxB,OALAS,GAAkBD,GAA6BC,EAAiB30F,MAC3Dk0F,EAAaS,EAAgBt0F,MAC9B6zF,EAAaS,EAAgBt0F,IAAMs0F,EACnCA,EAAgBmB,cAAc91F,OAE3BA,MAQX05F,kBAAmB,SAAS/E,GACxB,MAAIjJ,GAAeiJ,EAAiB,oBAAqB30F,MAC9CA,MAGX20F,EAAkBD,GAA6BC,EAAiB30F,YACzDA,MAAKk0F,aAAaS,EAAgBt0F,IAClCL,OAQX+1F,eAAgB,SAASpB,GACrB,GAAIjJ,EAAeiJ,EAAiB,iBAAkB30F,MAClD,MAAOA,KAGX,IAAIm0F,GAAcn0F,KAAKm0F,WAMvB,OALAQ,GAAkBD,GAA6BC,EAAiB30F,MAClB,KAA1CgtF,EAAQmH,EAAaQ,KACrBR,EAAYnsF,KAAK2sF,GACjBA,EAAgBoB,eAAe/1F,OAE5BA,MAQX25F,mBAAoB,SAAShF,GACzB,GAAIjJ,EAAeiJ,EAAiB,qBAAsB30F,MACtD,MAAOA,KAGX20F,GAAkBD,GAA6BC,EAAiB30F,KAChE,IAAImI,GAAQ6kF,EAAQhtF,KAAKm0F,YAAaQ,EAItC,OAHIxsF,GAAQ,IACRnI,KAAKm0F,YAAY/rF,OAAOD,EAAO,GAE5BnI,MAOX45F,mBAAoB,WAChB,MAAO55F,MAAKm0F,YAAYxuF,OAAS,GAQrCk0F,iBAAkB,SAASlF,GACvB,QAAS30F,KAAKk0F,aAAaS,EAAgBt0F,KAQ/CutB,KAAM,SAASknD,GAIX,QAASlnD,GAAKksE,GACVprB,EAAKsf,QAAQpgE,KAAK8gD,EAAKjgE,QAAQnF,OAASwwF,EAAY1F,EAASJ,GAAS,IAAKlf,GAJ/E,GAAIpG,GAAO1uE,KACPg0F,EAAQh0F,KAAKg0F,KAOLM,IAARN,GACApmE,GAAK,GAGTA,IAGIomE,GAASM,IACT1mE,GAAK,IAUbmsE,QAAS,SAASjlB,GACd,MAAI90E,MAAKg6F,UACEh6F,KAAK4tB,KAAKknD,QAGrB90E,KAAKg0F,MAAQyF,KAOjBO,QAAS,WAEL,IADA,GAAIx0F,GAAI,EACDA,EAAIxF,KAAKm0F,YAAYxuF,QAAQ,CAChC,KAAM3F,KAAKm0F,YAAY3uF,GAAGwuF,OAASyF,GAAexF,KAC9C,OAAO,CAEXzuF,KAEJ,OAAO,GAOXiqF,UAAW,SAASoJ,GAGhB,GAAIoB,GAAiB30F,KAAWuzF,EAGhC,OAAKzM,GAASpsF,KAAKyO,QAAQ20B,QAASpjC,KAAMi6F,KAOtCj6F,KAAKg0F,OAASwF,GAAmBnF,GAAkBoF,MACnDz5F,KAAKg0F,MAAQC,IAGjBj0F,KAAKg0F,MAAQh0F,KAAKk6F,QAAQD,QAItBj6F,KAAKg0F,OAASQ,GAAcD,GAAgBD,GAAcD,KAC1Dr0F,KAAK+5F,QAAQE,MAfbj6F,KAAKwpD,aACLxpD,KAAKg0F,MAAQyF,MAyBrBS,QAAS,aAOTf,eAAgB,aAOhB3vC,MAAO,cA8DXuiC,EAAQ8I,GAAgBf,GAKpBC,UAKI9E,SAAU,GASdkL,SAAU,SAASrlB,GACf,GAAIslB,GAAiBp6F,KAAKyO,QAAQwgF,QAClC,OAA0B,KAAnBmL,GAAwBtlB,EAAMma,SAAStpF,SAAWy0F,GAS7DF,QAAS,SAASplB,GACd,GAAIkf,GAAQh0F,KAAKg0F,MACbjF,EAAYja,EAAMia,UAElBsL,EAAerG,GAASQ,GAAcD,IACtCld,EAAUr3E,KAAKm6F,SAASrlB,EAG5B,OAAIulB,KAAiBtL,EAAYO,KAAiBjY,GACvC2c,EAAQK,GACRgG,GAAgBhjB,EACnB0X,EAAYM,GACL2E,EAAQM,GACNN,EAAQQ,GAGdR,EAAQO,GAFJC,GAIRiF,MAiBf1N,EAAQ+I,GAAeD,IAKnBd,UACIzqF,MAAO,MACP0uD,UAAW,GACXi3B,SAAU,EACV/zD,UAAW67D,IAGfoC,eAAgB,WACZ,GAAIj+D,GAAYl7B,KAAKyO,QAAQysB,UACzBo4D,IAOJ,OANIp4D,GAAY27D,IACZvD,EAAQtrF,KAAK2rF,IAEbz4D,EAAY47D,IACZxD,EAAQtrF,KAAKyrF,IAEVH,GAGXgH,cAAe,SAASxlB,GACpB,GAAIrmE,GAAUzO,KAAKyO,QACf8rF,GAAW,EACX50E,EAAWmvD,EAAMnvD,SACjBuV,EAAY45C,EAAM55C,UAClBnpB,EAAI+iE,EAAMt1C,OACVxtB,EAAI8iE,EAAMr1C,MAed,OAZMvE,GAAYzsB,EAAQysB,YAClBzsB,EAAQysB,UAAY27D,IACpB37D,EAAmB,IAANnpB,EAAWm/E,GAAsB,EAAJn/E,EAASo/E,GAAiBC,GACpEmJ,EAAWxoF,GAAK/R,KAAK+0F,GACrBpvE,EAAWzgB,KAAK2lB,IAAIiqD,EAAMt1C,UAE1BtE,EAAmB,IAANlpB,EAAWk/E,GAAsB,EAAJl/E,EAASq/E,GAAeC,GAClEiJ,EAAWvoF,GAAKhS,KAAKg1F,GACrBrvE,EAAWzgB,KAAK2lB,IAAIiqD,EAAMr1C,UAGlCq1C,EAAM55C,UAAYA,EACXq/D,GAAY50E,EAAWlX,EAAQupD,WAAa98B,EAAYzsB,EAAQysB,WAG3Ei/D,SAAU,SAASrlB,GACf,MAAO+f,IAAe1hF,UAAUgnF,SAAS55F,KAAKP,KAAM80E,KAC/C90E,KAAKg0F,MAAQQ,MAAkBx0F,KAAKg0F,MAAQQ,KAAgBx0F,KAAKs6F,cAAcxlB,KAGxFlnD,KAAM,SAASknD,GACX90E,KAAK+0F,GAAKjgB,EAAMt1C,OAChBx/B,KAAKg1F,GAAKlgB,EAAMr1C,MAEhB,IAAIvE,GAAYu5D,EAAa3f,EAAM55C,UAC/BA,IACAl7B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ4xB,EAAW45C,GAGtD90E,KAAKmsF,OAAOv+D,KAAKrtB,KAAKP,KAAM80E,MAcpCiX,EAAQkJ,GAAiBJ,IAKrBd,UACIzqF,MAAO,QACP0uD,UAAW,EACXi3B,SAAU,GAGdkK,eAAgB,WACZ,OAAQ5F,KAGZ4G,SAAU,SAASrlB,GACf,MAAO90E,MAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,KAClC5vE,KAAK2lB,IAAIiqD,EAAM73D,MAAQ,GAAKjd,KAAKyO,QAAQupD,WAAah4D,KAAKg0F,MAAQQ,KAG5E5mE,KAAM,SAASknD,GAEX,GADA90E,KAAKmsF,OAAOv+D,KAAKrtB,KAAKP,KAAM80E,GACR,IAAhBA,EAAM73D,MAAa,CACnB,GAAIu9E,GAAQ1lB,EAAM73D,MAAQ,EAAI,KAAO,KACrCjd,MAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQkxF,EAAO1lB,OAkB1DiX,EAAQmJ,GAAiBpB,GAKrBC,UACIzqF,MAAO,QACP2lF,SAAU,EACV90D,KAAM,IACN69B,UAAW,GAGfmhC,eAAgB,WACZ,OAAQtF,KAGZqG,QAAS,SAASplB,GACd,GAAIrmE,GAAUzO,KAAKyO,QACfgsF,EAAgB3lB,EAAMma,SAAStpF,SAAW8I,EAAQwgF,SAClDyL,EAAgB5lB,EAAMnvD,SAAWlX,EAAQupD,UACzC2iC,EAAY7lB,EAAMob,UAAYzhF,EAAQ0rB,IAM1C,IAJAn6B,KAAKo1F,OAAStgB,GAIT4lB,IAAkBD,GAAkB3lB,EAAMia,WAAaM,GAAYC,MAAkBqL,EACtF36F,KAAKwpD,YACF,IAAIsrB,EAAMia,UAAYK,GACzBpvF,KAAKwpD,QACLxpD,KAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQwF,GACbx5F,KAAK+5F;EACNtrF,EAAQ0rB,KAAMn6B,UACd,IAAI80E,EAAMia,UAAYM,GACzB,MAAOmK,GAEX,OAAOC,KAGXjwC,MAAO,WACHnwC,aAAarZ,KAAKm1F,SAGtBvnE,KAAM,SAASknD,GACP90E,KAAKg0F,QAAUwF,KAIf1kB,GAAUA,EAAMia,UAAYM,GAC5BrvF,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ,KAAMwrE,IAE7C90E,KAAKo1F,OAAOnF,UAAY9yD,KACxBn9B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOtJ,KAAKo1F,aAevDrJ,EAAQsJ,GAAkBR,IAKtBd,UACIzqF,MAAO,SACP0uD,UAAW,EACXi3B,SAAU,GAGdkK,eAAgB,WACZ,OAAQ5F,KAGZ4G,SAAU,SAASrlB,GACf,MAAO90E,MAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,KAClC5vE,KAAK2lB,IAAIiqD,EAAMyb,UAAYvwF,KAAKyO,QAAQupD,WAAah4D,KAAKg0F,MAAQQ,OAc/EzI,EAAQuJ,GAAiBT,IAKrBd,UACIzqF,MAAO,QACP0uD,UAAW,GACXwG,SAAU,IACVtjC,UAAW27D,GAAuBC,GAClC7H,SAAU,GAGdkK,eAAgB,WACZ,MAAOrE,IAAc3hF,UAAUgmF,eAAe54F,KAAKP,OAGvDm6F,SAAU,SAASrlB,GACf,GACItW,GADAtjC,EAAYl7B,KAAKyO,QAAQysB,SAW7B,OARIA,IAAa27D,GAAuBC,IACpCt4B,EAAWsW,EAAMtW,SACVtjC,EAAY27D,GACnBr4B,EAAWsW,EAAM8b,UACV11D,EAAY47D,KACnBt4B,EAAWsW,EAAM+b,WAGd7wF,KAAKmsF,OAAOgO,SAAS55F,KAAKP,KAAM80E,IACnC55C,EAAY45C,EAAM55C,WAClB45C,EAAMnvD,SAAW3lB,KAAKyO,QAAQupD,WAC9BntC,GAAI2zC,GAAYx+D,KAAKyO,QAAQ+vD,UAAYsW,EAAMia,UAAYM,IAGnEzhE,KAAM,SAASknD,GACX,GAAI55C,GAAYu5D,EAAa3f,EAAM55C,UAC/BA,IACAl7B,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAQ4xB,EAAW45C,GAGtD90E,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOwrE,MA2B9CiX,EAAQwJ,GAAezB,GAKnBC,UACIzqF,MAAO,MACP2lF,SAAU,EACV2L,KAAM,EACNpoE,SAAU,IACV2H,KAAM,IACN69B,UAAW,EACX6iC,aAAc,IAGlB1B,eAAgB,WACZ,OAAQvF,KAGZsG,QAAS,SAASplB,GACd,GAAIrmE,GAAUzO,KAAKyO,QAEfgsF,EAAgB3lB,EAAMma,SAAStpF,SAAW8I,EAAQwgF,SAClDyL,EAAgB5lB,EAAMnvD,SAAWlX,EAAQupD,UACzC8iC,EAAiBhmB,EAAMob,UAAYzhF,EAAQ0rB,IAI/C,IAFAn6B,KAAKwpD,QAEAsrB,EAAMia,UAAYK,IAAgC,IAAfpvF,KAAKgX,MACzC,MAAOhX,MAAK+6F,aAKhB,IAAIL,GAAiBI,GAAkBL,EAAe,CAClD,GAAI3lB,EAAMia,WAAaM,GACnB,MAAOrvF,MAAK+6F,aAGhB,IAAIC,GAAgBh7F,KAAKw1F,MAAS1gB,EAAMmb,UAAYjwF,KAAKw1F,MAAQ/mF,EAAQ+jB,UAAY,EACjFyoE,GAAiBj7F,KAAKy1F,SAAWh3B,EAAYz+D,KAAKy1F,QAAS3gB,EAAM5oD,QAAUzd,EAAQosF,YAEvF76F,MAAKw1F,MAAQ1gB,EAAMmb,UACnBjwF,KAAKy1F,QAAU3gB,EAAM5oD,OAEhB+uE,GAAkBD,EAGnBh7F,KAAKgX,OAAS,EAFdhX,KAAKgX,MAAQ,EAKjBhX,KAAKo1F,OAAStgB,CAId,IAAIomB,GAAWl7F,KAAKgX,MAAQvI,EAAQmsF,IACpC,IAAiB,IAAbM,EAGA,MAAKl7F,MAAK45F,sBAGN55F,KAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQwF,GACbx5F,KAAK+5F,WACNtrF,EAAQ+jB,SAAUxyB,MACdw0F,IANAgF,GAUnB,MAAOC,KAGXsB,YAAa,WAIT,MAHA/6F,MAAKm1F,OAAS5J,EAAkB,WAC5BvrF,KAAKg0F,MAAQyF,IACdz5F,KAAKyO,QAAQ+jB,SAAUxyB,MACnBy5F,IAGXjwC,MAAO,WACHnwC,aAAarZ,KAAKm1F,SAGtBvnE,KAAM,WACE5tB,KAAKg0F,OAASwF,KACdx5F,KAAKo1F,OAAO8F,SAAWl7F,KAAKgX,MAC5BhX,KAAKguF,QAAQpgE,KAAK5tB,KAAKyO,QAAQnF,MAAOtJ,KAAKo1F,YAoBvDrxF,GAAOm8E,QAAU,QAMjBn8E,GAAOgwF,UAOHoH,WAAW,EAQX71B,YAAa2zB,GAMb71D,QAAQ,EASR6qD,YAAa,KAObM,WAAY,KAOZoH,SAEKN,IAAoBjyD,QAAQ,KAC5B6xD,IAAmB7xD,QAAQ,IAAU,YACrCkyD,IAAkBp6D,UAAW27D,MAC7B/B,IAAiB55D,UAAW27D,KAAyB,WACrDtB,KACAA,IAAiBjsF,MAAO,YAAasxF,KAAM,IAAM,SACjD1F,KAQLc,UAMIoF,WAAY,OAOZC,YAAa,OASbC,aAAc,OAOdC,eAAgB,OAOhBC,SAAU,OAQVC,kBAAmB,iBAI3B,IAAIC,IAAO,EACPC,GAAc,CA+BlB3Q,IAAQ73E,WAMJg2C,IAAK,SAAS16C,GAaV,MAZAnJ,GAAOtF,KAAKyO,QAASA,GAGjBA,EAAQ62D,aACRtlE,KAAKslE,YAAYzwD,SAEjBpG,EAAQw/E,cAERjuF,KAAK80E,MAAMxhE,UACXtT,KAAK80E,MAAMprE,OAAS+E,EAAQw/E,YAC5BjuF,KAAK80E,MAAMsZ,QAERpuF,MASXklB,KAAM,SAAS0b,GACX5gC,KAAKuvF,QAAQ1E,QAAUjqD,EAAQ+6D,GAAcD,IASjDjM,UAAW,SAASoJ,GAChB,GAAItJ,GAAUvvF,KAAKuvF,OACnB,KAAIA,EAAQ1E,QAAZ,CAKA7qF,KAAKslE,YAAY8zB,gBAAgBP,EAEjC,IAAIjE,GACAc,EAAc11F,KAAK01F,YAKnBkG,EAAgBrM,EAAQqM,gBAIvBA,GAAkBA,GAAiBA,EAAc5H,MAAQwF,MAC1DoC,EAAgBrM,EAAQqM,cAAgB,KAI5C,KADA,GAAIp2F,GAAI,EACDA,EAAIkwF,EAAY/vF,QACnBivF,EAAac,EAAYlwF,GAQrB+pF,EAAQ1E,UAAY8Q,IACfC,GAAiBhH,GAAcgH,IAChChH,EAAWiF,iBAAiB+B,GAGhChH,EAAWprC,QAFXorC,EAAWnF,UAAUoJ,IAOpB+C,GAAiBhH,EAAWZ,OAASQ,GAAcD,GAAgBD,MACpEsH,EAAgBrM,EAAQqM,cAAgBhH,GAE5CpvF,MASR0P,IAAK,SAAS0/E,GACV,GAAIA,YAAsBd,GACtB,MAAOc,EAIX,KAAK,GADDc,GAAc11F,KAAK01F,YACdlwF,EAAI,EAAGA,EAAIkwF,EAAY/vF,OAAQH,IACpC,GAAIkwF,EAAYlwF,GAAGiJ,QAAQnF,OAASsrF,EAChC,MAAOc,GAAYlwF,EAG3B,OAAO,OASXyN,IAAK,SAAS2hF,GACV,GAAIlJ,EAAekJ,EAAY,MAAO50F,MAClC,MAAOA,KAIX,IAAI+hD,GAAW/hD,KAAKkV,IAAI0/E,EAAWnmF,QAAQnF,MAS3C,OARIy4C,IACA/hD,KAAKqW,OAAO0rC,GAGhB/hD,KAAK01F,YAAY1tF,KAAK4sF,GACtBA,EAAW5G,QAAUhuF,KAErBA,KAAKslE,YAAYzwD,SACV+/E,GAQXv+E,OAAQ,SAASu+E,GACb,GAAIlJ,EAAekJ,EAAY,SAAU50F,MACrC,MAAOA,KAGX,IAAI01F,GAAc11F,KAAK01F,WAKvB,OAJAd,GAAa50F,KAAKkV,IAAI0/E,GACtBc,EAAYttF,OAAO4kF,EAAQ0I,EAAad,GAAa,GAErD50F,KAAKslE,YAAYzwD,SACV7U,MASXuT,GAAI,SAASiyD,EAAQ6lB,GACjB,GAAIuK,GAAW51F,KAAK41F,QAKpB,OAJAhK,GAAKc,EAASlnB,GAAS,SAASl8D,GAC5BssF,EAAStsF,GAASssF,EAAStsF,OAC3BssF,EAAStsF,GAAOtB,KAAKqjF,KAElBrrF,MASX0T,IAAK,SAAS8xD,EAAQ6lB,GAClB,GAAIuK,GAAW51F,KAAK41F,QAQpB,OAPAhK,GAAKc,EAASlnB,GAAS,SAASl8D,GACvB+hF,EAGDuK,EAAStsF,GAAOlB,OAAO4kF,EAAQ4I,EAAStsF,GAAQ+hF,GAAU,SAFnDuK,GAAStsF,KAKjBtJ,MAQX4tB,KAAM,SAAStkB,EAAOoJ,GAEd1S,KAAKyO,QAAQ0sF,WACblF,GAAgB3sF,EAAOoJ,EAI3B,IAAIkjF,GAAW51F,KAAK41F,SAAStsF,IAAUtJ,KAAK41F,SAAStsF,GAAO2B,OAC5D,IAAK2qF,GAAaA,EAASjwF,OAA3B,CAIA+M,EAAK5L,KAAOwC,EACZoJ,EAAKrJ,eAAiB,WAClBqJ,EAAKohC,SAASzqC,iBAIlB,KADA,GAAI7D,GAAI,EACDA,EAAIowF,EAASjwF,QAChBiwF,EAASpwF,GAAGkN,GACZlN,MAQR8N,QAAS,WACLtT,KAAK4I,SAAWitF,GAAe71F,MAAM,GAErCA,KAAK41F,YACL51F,KAAKuvF,WACLvvF,KAAK80E,MAAMxhE,UACXtT,KAAK4I,QAAU,OA4BvBtD,EAAOvB,IACHqrF,YAAaA,GACb4D,WAAYA,GACZ3D,UAAWA,GACXC,aAAcA,GAEd2E,eAAgBA,GAChBO,YAAaA,GACbD,cAAeA,GACfD,YAAaA,GACbkF,iBAAkBA,GAClBnF,gBAAiBA,GACjBoF,aAAcA,GAEdvI,eAAgBA,GAChBC,eAAgBA,GAChBC,gBAAiBA,GACjBC,aAAcA,GACdC,eAAgBA,GAChBuF,qBAAsBA,GACtBC,mBAAoBA,GACpBC,cAAeA,GAEf/L,QAASA,GACT+C,MAAOA,EACPqF,YAAaA,EAEbzE,WAAYA,EACZG,WAAYA,EACZL,kBAAmBA,EACnBI,gBAAiBA,EACjBsD,iBAAkBA,EAElB2B,WAAYA,EACZe,eAAgBA,GAChBgH,IAAKtG,GACLuG,IAAKhH,GACLiH,MAAOzG,GACP0G,MAAO/G,GACPgH,OAAQ5G,GACR6G,MAAOhH,GAEP3hF,GAAIk5E,EACJ/4E,IAAKi5E,EACLf,KAAMA,EACN7qB,MAAOA,EACPz7D,OAAQA,EACRymF,QAASA,EACTN,OAAQA,EACR0B,SAAUA,IAGV,YAAcd,IAAiBnsF,EAAoB,KACjD8uE,EAAgC,WAC9B,MAAOjrE,KACTxD,KAAKX,EAASM,EAAqBN,EAASC,KAASmvE,IAAkCxoE,IAAc3G,EAAOD,QAAUovE,KAChG,mBAAVnvE,IAAyBA,EAAOD,QAC9CC,EAAOD,QAAUmE,GAEjBwF,EAAO+hF,GAAcvnF,IAGtBwF,OAAQgI,SAAU,WAKjB,SAAS1R,EAAQD,GAYrBA,EAAQslD,oBAAsB,WAE7BllD,KAAKm8F,aAAan8F,KAAK0hD,UAAUvC,WAAWC,iBAAiB,GAG7Dp/C,KAAKwuD,eAIDxuD,KAAKmhD,WACPnhD,KAAK6nD,aAEP7nD,KAAK4P,SASNhQ,EAAQu8F,aAAe,SAASC,EAAkBC,GAOhD,IANA,GAAI31C,GAAgB1mD,KAAK8jD,YAAYn+C,OAEjC22F,EAAY,GACZ1+C,EAAQ,EAGL8I,EAAgB01C,GAA4BE,EAAR1+C,GACrCA,EAAQ,GAAK,GACf59C,KAAKu8F,oBAAmB,GACxBv8F,KAAKw8F,0BAGLx8F,KAAKy8F,uBAGP/1C,EAAgB1mD,KAAK8jD,YAAYn+C,OACjCi4C,GAAS,CAIPA,GAAQ,GAAmB,GAAdy+C,GACfr8F,KAAK08F,kBAEP18F,KAAKquD,2BASPzuD,EAAQ+8F,YAAc,SAAS72C,GAC7B,GAAI82C,GAA2B58F,KAAK8kD,MACpC,IAAIgB,EAAKuW,YAAcr8D,KAAK0hD,UAAUvC,WAAWM,iBAAmBz/C,KAAK68F,kBAAkB/2C,KACrE,WAAlB9lD,KAAK88F,WAAqD,GAA3B98F,KAAK8jD,YAAYn+C,QAAc,CAEhE3F,KAAK+8F,WAAWj3C,EAIhB,KAHA,GAAIlI,GAAQ,EAGJ59C,KAAK8jD,YAAYn+C,OAAS3F,KAAK0hD,UAAUvC,WAAWC,iBAA6B,GAARxB,GAC/E59C,KAAKg9F,uBACLp/C,GAAS,MAKX59C,MAAKi9F,mBAAmBn3C,GAAK,GAAM,GAGnC9lD,KAAKgnD,uBACLhnD,KAAKk9F,sBACLl9F,KAAKquD,0BACLruD,KAAKwuD,cAIHxuD,MAAK8kD,QAAU83C,GACjB58F,KAAK4P,SAQThQ,EAAQ4sD,sBAAwB,WACW,GAArCxsD,KAAK0hD,UAAUvC,WAAWzwC,SAC5B1O,KAAKm9F,eAAe,GAAE,GAAM,IAUhCv9F,EAAQ68F,qBAAuB,WAC7Bz8F,KAAKm9F,eAAe,IAAG,GAAM,IAS/Bv9F,EAAQo9F,qBAAuB,WAC7Bh9F,KAAKm9F,eAAe,GAAE,GAAM,IAgB9Bv9F,EAAQu9F,eAAiB,SAASC,EAAcC,EAAUz8D,EAAM08D,GAC9D,GAAIV,GAA2B58F,KAAK8kD,OAChCy4C,EAAgBv9F,KAAK8jD,YAAYn+C,MAGjC3F,MAAKmkD,cAAgBnkD,KAAKid,OAA0B,GAAjBmgF,GACrCp9F,KAAKw9F,kBAIHx9F,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,EAGrCp9F,KAAKy9F,cAAc78D,IAEZ5gC,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,GAAjBmgF,KAC7B,GAATx8D,EAGF5gC,KAAK09F,cAAcL,EAAUz8D,GAI7B5gC,KAAK29F,uBAGT39F,KAAKgnD,uBAGDhnD,KAAK8jD,YAAYn+C,QAAU43F,IAAkBv9F,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,KAClFp9F,KAAK49F,eAAeh9D,GACpB5gC,KAAKgnD,yBAIHhnD,KAAKmkD,cAAgBnkD,KAAKid,OAA0B,IAAjBmgF,KACrCp9F,KAAK69F,eACL79F,KAAKgnD,wBAGPhnD,KAAKmkD,cAAgBnkD,KAAKid,MAG1Bjd,KAAKk9F,sBACLl9F,KAAKwuD,eAGDxuD,KAAK8jD,YAAYn+C,OAAS43F,IAC5Bv9F,KAAK87D,gBAAkB,EAEvB97D,KAAKw8F,2BAGW,GAAdc,GAAsC92F,SAAf82F,IAErBt9F,KAAK8kD,QAAU83C,GACjB58F,KAAK4P,QAIT5P,KAAKquD,2BAMPzuD,EAAQi+F,aAAe,WAErB,GAAIC,GAAkB99F,KAAK+9F,mBACvBD,GAAkB99F,KAAK0hD,UAAUvC,WAAWI,gBAC9Cv/C,KAAKg+F,sBAAsB,EAAIh+F,KAAK0hD,UAAUvC,WAAWI,eAAiBu+C,IAW9El+F,EAAQg+F,eAAiB,SAASh9D,GAChC5gC,KAAKi+F,cACLj+F,KAAKk+F,mBAAmBt9D,GAAM,IAQhChhC,EAAQ28F,mBAAqB,SAASe,GACpC,GAAIV,GAA2B58F,KAAK8kD,OAChCy4C,EAAgBv9F,KAAK8jD,YAAYn+C,MAErC3F,MAAK49F,gBAAe,GAGpB59F,KAAKgnD,uBACLhnD,KAAKk9F,sBACLl9F,KAAKwuD,eAGDxuD,KAAK8jD,YAAYn+C,QAAU43F,IAC7Bv9F,KAAK87D,gBAAkB,IAGP,GAAdwhC,GAAsC92F,SAAf82F,IAErBt9F,KAAK8kD,QAAU83C,GACjB58F,KAAK4P,SAUXhQ,EAAQ+9F,oBAAsB,WAC5B,IAAK,GAAIx3C,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACD,IAAjBL,EAAKma,WACFna,EAAKvzC,MAAMvS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOC,aAC1FsmC,EAAKtzC,OAAOxS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOsF,eAC9F7kB,KAAK28F,YAAY72C,KAc3BlmD,EAAQ89F,cAAgB,SAASL,EAAUz8D,GACzC,IAAK,GAAIp7B,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvCxF,MAAKi9F,mBAAmBn3C,EAAKu3C,EAAUz8D,GACvC5gC,KAAKquD,4BAeTzuD,EAAQq9F,mBAAqB,SAASpzF,EAAYwzF,EAAWz8D,EAAOu9D,GAElE,GAAIt0F,EAAWwyD,YAAc,IAEvBxyD,EAAWwyD,YAAcr8D,KAAK0hD,UAAUvC,WAAWM,kBACrD0+C,GAAU,GAEZd,EAAYc,GAAU,EAAOd,EAGzBxzF,EAAWuyD,eAAiBp8D,KAAKid,OAAkB,GAAT2jB,GAE5C,IAAK,GAAIw9D,KAAmBv0F,GAAWyyD,eACrC,GAAIzyD,EAAWyyD,eAAex2D,eAAes4F,GAAkB,CAC7D,GAAIC,GAAYx0F,EAAWyyD,eAAe8hC,EAI7B,IAATx9D,GACEy9D,EAAUviC,gBAAkBjyD,EAAW2yD,gBAAgB3yD,EAAW2yD,gBAAgB72D,OAAO,IACtFw4F,IACLn+F,KAAKs+F,sBAAsBz0F,EAAWu0F,EAAgBf,EAAUz8D,EAAMu9D,GAIpEn+F,KAAK68F,kBAAkBhzF,IACzB7J,KAAKs+F,sBAAsBz0F,EAAWu0F,EAAgBf,EAAUz8D,EAAMu9D,KAwBpFv+F,EAAQ0+F,sBAAwB,SAASz0F,EAAYu0F,EAAiBf,EAAWz8D,EAAOu9D,GACtF,GAAIE,GAAYx0F,EAAWyyD,eAAe8hC,EAG1C,IAAIC,EAAUjiC,eAAiBp8D,KAAKid,OAAkB,GAAT2jB,EAAe,CAE1D5gC,KAAKu+F,eAGLv+F,KAAKg9C,MAAMohD,GAAmBC,EAG9Br+F,KAAKw+F,uBAAuB30F,EAAWw0F,GAGvCr+F,KAAKy+F,wBAAwB50F,EAAWw0F,GAGxCr+F,KAAK0+F,eAAe70F,GAGpBA,EAAW4E,QAAQwuC,MAAQohD,EAAU5vF,QAAQwuC,KAC7CpzC,EAAWwyD,aAAegiC,EAAUhiC,YACpCxyD,EAAW4E,QAAQ8uC,SAAWr4C,KAAK4G,IAAI9L,KAAK0hD,UAAUvC,WAAWS,YAAa5/C,KAAK0hD,UAAU1E,MAAMO,SAAWv9C,KAAK0hD,UAAUvC,WAAWQ,oBAAoB91C,EAAWwyD,YAAY,IACnLxyD,EAAWgyD,mBAAqBhyD,EAAWslD,aAAaxpD,OAGxD04F,EAAUtsF,EAAIlI,EAAWkI,EAAIlI,EAAWqyD,iBAAmB,GAAMh3D,KAAKE,UACtEi5F,EAAUrsF,EAAInI,EAAWmI,EAAInI,EAAWqyD,iBAAmB,GAAMh3D,KAAKE,gBAG/DyE,GAAWyyD,eAAe8hC,EAGjC,IAAIO,IAAgB,CACpB,KAAK,GAAIC,KAAe/0F,GAAWyyD,eACjC,GAAIzyD,EAAWyyD,eAAex2D,eAAe84F,IACvC/0F,EAAWyyD,eAAesiC,GAAa9iC,gBAAkBuiC,EAAUviC,eAAgB,CACrF6iC,GAAgB,CAChB,OAKe,GAAjBA,GACF90F,EAAW2yD,gBAAgBniB,MAG7Br6C,KAAK6+F,uBAAuBR,GAI5BA,EAAUviC,eAAiB,EAG3BjyD,EAAWs0D,iBAGXn+D,KAAK8kD,QAAS,EAIC,GAAbu4C,GACFr9F,KAAKi9F,mBAAmBoB,EAAUhB,EAAUz8D,EAAMu9D,IAWtDv+F,EAAQi/F,uBAAyB,SAAS/4C,GACxC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAC5CsgD,EAAKqJ,aAAa3pD,GAAG2sD,sBAczBvyD,EAAQ69F,cAAgB,SAAS78D,GAClB,GAATA,EACF5gC,KAAK8+F,sBAGL9+F,KAAK++F,wBAUTn/F,EAAQk/F,oBAAsB,WAC5B,GAAIlgF,GAAGC,EAAGlZ,EACNq5F,EAAYh/F,KAAK0hD,UAAUvC,WAAWK,qBAAqBx/C,KAAKid,KAIpE,KAAK,GAAI+vC,KAAUhtD,MAAK89C,MACtB,GAAI99C,KAAK89C,MAAMh4C,eAAeknD,GAAS,CACrC,GAAIU,GAAO1tD,KAAK89C,MAAMkP,EACtB,IAAIU,EAAKC,WACHD,EAAKkG,MAAQlG,EAAKiG,SACpB/0C,EAAM8uC,EAAKrkC,GAAGtX,EAAI27C,EAAKtkC,KAAKrX,EAC5B8M,EAAM6uC,EAAKrkC,GAAGrX,EAAI07C,EAAKtkC,KAAKpX,EAC5BrM,EAAST,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAGrBmgF,EAATr5F,GAAoB,CAEtB,GAAIkE,GAAa6jD,EAAKtkC,KAClBi1E,EAAY3wC,EAAKrkC,EACjBqkC,GAAKrkC,GAAG5a,QAAQwuC,KAAOyQ,EAAKtkC,KAAK3a,QAAQwuC,OAC3CpzC,EAAa6jD,EAAKrkC,GAClBg1E,EAAY3wC,EAAKtkC,MAGiB,GAAhCi1E,EAAUxiC,mBACZ77D,KAAKi/F,cAAcp1F,EAAWw0F,GAAU,GAEA,GAAjCx0F,EAAWgyD,oBAClB77D,KAAKi/F,cAAcZ,EAAUx0F,GAAW,MAetDjK,EAAQm/F,qBAAuB,WAC7B,IAAK,GAAI54C,KAAUnmD,MAAKg9C,MAEtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIk4C,GAAYr+F,KAAKg9C,MAAMmJ,EAG3B,IAAoC,GAAhCk4C,EAAUxiC,oBAA4D,GAAjCwiC,EAAUlvC,aAAaxpD,OAAa,CAC3E,GAAI+nD,GAAO2wC,EAAUlvC,aAAa,GAC9BtlD,EAAc6jD,EAAKkG,MAAQyqC,EAAUh+F,GAAML,KAAKg9C,MAAM0Q,EAAKiG,QAAU3zD,KAAKg9C,MAAM0Q,EAAKkG,KAGrFyqC,GAAUh+F,IAAMwJ,EAAWxJ,KACzBwJ,EAAW4E,QAAQwuC,KAAOohD,EAAU5vF,QAAQwuC,KAC9Cj9C,KAAKi/F,cAAcp1F,EAAWw0F,GAAU,GAGxCr+F,KAAKi/F,cAAcZ,EAAUx0F,GAAW,OAgBpDjK,EAAQs/F,4BAA8B,SAASp5C,GAG7C,IAAK,GAFDq5C,GAAoB,GACpBC,EAAwB,KACnB55F,EAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAC5C,GAA6BgB,SAAzBs/C,EAAKqJ,aAAa3pD,GAAkB,CACtC,GAAI65F,GAAY,IACZv5C,GAAKqJ,aAAa3pD,GAAGmuD,QAAU7N,EAAKzlD,GACtCg/F,EAAYv5C,EAAKqJ,aAAa3pD,GAAG4jB,KAE1B08B,EAAKqJ,aAAa3pD,GAAGouD,MAAQ9N,EAAKzlD,KACzCg/F,EAAYv5C,EAAKqJ,aAAa3pD,GAAG6jB,IAIlB,MAAbg2E,GAAqBF,EAAoBE,EAAU7iC,gBAAgB72D,SACrEw5F,EAAoBE,EAAU7iC,gBAAgB72D,OAC9Cy5F,EAAwBC,GAKb,MAAbA,GAAkD74F,SAA7BxG,KAAKg9C,MAAMqiD,EAAUh/F,KAC5CL,KAAKi/F,cAAcI,EAAWv5C,GAAM,IAYxClmD,EAAQs+F,mBAAqB,SAASt9D,EAAO0+D,GAE3C,IAAK,GAAIn5C,KAAUnmD,MAAKg9C,MAElBh9C,KAAKg9C,MAAMl3C,eAAeqgD,IAC5BnmD,KAAKu/F,oBAAoBv/F,KAAKg9C,MAAMmJ,GAAQvlB,EAAM0+D,IAcxD1/F,EAAQ2/F,oBAAsB,SAASC,EAAS5+D,EAAO0+D,EAAWG,GAKhE,GAJ6Bj5F,SAAzBi5F,IACFA,EAAuB,GAGpBD,EAAQ3jC,oBAAsB77D,KAAKqtE,cAA6B,GAAbiyB,GACrDE,EAAQ3jC,oBAAsB77D,KAAKqtE,cAA6B,GAAbiyB,EAAoB,CASxE,IAAK,GAPD1gF,GAAGC,EAAGlZ,EACNq5F,EAAYh/F,KAAK0hD,UAAUvC,WAAWK,qBAAqBx/C,KAAKid,MAChEyiF,GAAe,EAGfC,KACAC,EAAuBJ,EAAQrwC,aAAaxpD,OACvCimB,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IACxC+zE,EAAa33F,KAAKw3F,EAAQrwC,aAAavjC,GAAGvrB,GAK5C,IAAa,GAATugC,EAEF,IADA8+D,GAAe,EACV9zE,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IAAK,CACzC,GAAI8hC,GAAO1tD,KAAK89C,MAAM6hD,EAAa/zE,GACnC,IAAaplB,SAATknD,GACEA,EAAKC,WACHD,EAAKkG,MAAQlG,EAAKiG,SACpB/0C,EAAM8uC,EAAKrkC,GAAGtX,EAAI27C,EAAKtkC,KAAKrX,EAC5B8M,EAAM6uC,EAAKrkC,GAAGrX,EAAI07C,EAAKtkC,KAAKpX,EAC5BrM,EAAST,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAErBmgF,EAATr5F,GAAoB,CACtB+5F,GAAe,CACf,QASZ,IAAM9+D,GAAS8+D,GAAiB9+D,EAE9B,IAAKhV,EAAI,EAAOg0E,EAAJh0E,EAA0BA,IAGpC,GAFA8hC,EAAO1tD,KAAK89C,MAAM6hD,EAAa/zE,IAElBplB,SAATknD,EAAoB,CACtB,GAAI2wC,GAAYr+F,KAAKg9C,MAAO0Q,EAAKiG,QAAU6rC,EAAQn/F,GAAMqtD,EAAKkG,KAAOlG,EAAKiG,OAErE0qC,GAAUlvC,aAAaxpD,QAAW3F,KAAKqtE,aAAeoyB,GACtDpB,EAAUh+F,IAAMm/F,EAAQn/F,IAC3BL,KAAKi/F,cAAcO,EAAQnB,EAAUz9D,MAkBjDhhC,EAAQq/F,cAAgB,SAASp1F,EAAYw0F,EAAWz9D,GAEtD/2B,EAAWyyD,eAAe+hC,EAAUh+F,IAAMg+F,CAG1C,KAAK,GAAI74F,GAAI,EAAGA,EAAI64F,EAAUlvC,aAAaxpD,OAAQH,IAAK,CACtD,GAAIkoD,GAAO2wC,EAAUlvC,aAAa3pD,EAC9BkoD,GAAKkG,MAAQ/pD,EAAWxJ,IAAMqtD,EAAKiG,QAAU9pD,EAAWxJ,GAC1DL,KAAK6/F,qBAAqBh2F,EAAWw0F,EAAU3wC,GAG/C1tD,KAAK8/F,sBAAsBj2F,EAAWw0F,EAAU3wC,GAIpD2wC,EAAUlvC,gBAGVnvD,KAAK+/F,8BAA8Bl2F,EAAWw0F,SAIvCr+F,MAAKg9C,MAAMqhD,EAAUh+F,GAG5B,IAAI2/F,GAAan2F,EAAW4E,QAAQwuC,IACpCohD,GAAUviC,eAAiB97D,KAAK87D,eAChCjyD,EAAW4E,QAAQwuC,MAAQohD,EAAU5vF,QAAQwuC,KAC7CpzC,EAAWwyD,aAAegiC,EAAUhiC,YACpCxyD,EAAW4E,QAAQ8uC,SAAWr4C,KAAK4G,IAAI9L,KAAK0hD,UAAUvC,WAAWS,YAAa5/C,KAAK0hD,UAAU1E,MAAMO,SAAWv9C,KAAK0hD,UAAUvC,WAAWQ,mBAAmB91C,EAAWwyD,aAGlKxyD,EAAW2yD,gBAAgB3yD,EAAW2yD,gBAAgB72D,OAAS,IAAM3F,KAAK87D,gBAC5EjyD,EAAW2yD,gBAAgBx0D,KAAKhI,KAAK87D,gBAMrCjyD,EAAWuyD,eAFA,GAATx7B,EAE0B,EAGA5gC,KAAKid,MAInCpT,EAAWs0D,iBAGXt0D,EAAWyyD,eAAe+hC,EAAUh+F,IAAI+7D,eAAiBvyD,EAAWuyD,eAGpEiiC,EAAUn+B,gBAGVr2D,EAAWs2D,eAAe6/B,GAG1BhgG,KAAK8kD,QAAS,GAUhBllD,EAAQs9F,oBAAsB,WAC5B,IAAK,GAAI13F,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvCsgD,GAAK+V,mBAAqB/V,EAAKqJ,aAAaxpD,MAG5C,IAAIs6F,GAAa,CACjB,IAAIn6C,EAAK+V,mBAAqB,EAC5B,IAAK,GAAIjwC,GAAI,EAAGA,EAAIk6B,EAAK+V,mBAAqB,EAAGjwC,IAG/C,IAAK,GAFDs0E,GAAWp6C,EAAKqJ,aAAavjC,GAAGgoC,KAChCusC,EAAar6C,EAAKqJ,aAAavjC,GAAG+nC,OAC7BysC,EAAIx0E,EAAE,EAAGw0E,EAAIt6C,EAAK+V,mBAAoBukC,KACxCt6C,EAAKqJ,aAAaixC,GAAGxsC,MAAQssC,GAAYp6C,EAAKqJ,aAAaixC,GAAGzsC,QAAUwsC,GACxEr6C,EAAKqJ,aAAaixC,GAAGzsC,QAAUusC,GAAYp6C,EAAKqJ,aAAaixC,GAAGxsC,MAAQusC,KAC3EF,GAAc,EAKtBn6C,GAAK+V,oBAAsBokC,IAa/BrgG,EAAQigG,qBAAuB,SAASh2F,EAAYw0F,EAAW3wC,GAEvD7jD,EAAW0yD,eAAez2D,eAAeu4F,EAAUh+F,MACvDwJ,EAAW0yD,eAAe8hC,EAAUh+F,QAGtCwJ,EAAW0yD,eAAe8hC,EAAUh+F,IAAI2H,KAAK0lD,SAGtC1tD,MAAK89C,MAAM4P,EAAKrtD,GAGvB,KAAK,GAAImF,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAClD,GAAIqE,EAAWslD,aAAa3pD,GAAGnF,IAAMqtD,EAAKrtD,GAAI,CAC5CwJ,EAAWslD,aAAa/mD,OAAO5C,EAAE,EACjC,SAcN5F,EAAQkgG,sBAAwB,SAASj2F,EAAYw0F,EAAW3wC,GAE1DA,EAAKkG,MAAQlG,EAAKiG,OACpB3zD,KAAK6/F,qBAAqBh2F,EAAYw0F,EAAW3wC,IAG7CA,EAAKkG,MAAQyqC,EAAUh+F,IACzBqtD,EAAK0G,aAAapsD,KAAKq2F,EAAUh+F,IACjCqtD,EAAKrkC,GAAKxf,EACV6jD,EAAKkG,KAAO/pD,EAAWxJ,KAIvBqtD,EAAKyG,eAAensD,KAAKq2F,EAAUh+F,IACnCqtD,EAAKtkC,KAAOvf,EACZ6jD,EAAKiG,OAAS9pD,EAAWxJ,IAG3BL,KAAKqgG,oBAAoBx2F,EAAWw0F,EAAU3wC,KAalD9tD,EAAQmgG,8BAAgC,SAASl2F,EAAYw0F,GAE3D,IAAK,GAAI74F,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAAK,CACvD,GAAIkoD,GAAO7jD,EAAWslD,aAAa3pD,EAE/BkoD,GAAKkG,MAAQlG,EAAKiG,QACpB3zD,KAAK6/F,qBAAqBh2F,EAAYw0F,EAAW3wC,KAcvD9tD,EAAQygG,oBAAsB,SAASx2F,EAAYw0F,EAAW3wC,GAGtD7jD,EAAWkxD,cAAcj1D,eAAeu4F,EAAUh+F,MACtDwJ,EAAWkxD,cAAcsjC,EAAUh+F,QAErCwJ,EAAWkxD,cAAcsjC,EAAUh+F,IAAI2H,KAAK0lD,GAG5C7jD,EAAWslD,aAAannD,KAAK0lD,IAY/B9tD,EAAQ6+F,wBAA0B,SAAS50F,EAAYw0F,GACrD,GAAIx0F,EAAWkxD,cAAcj1D,eAAeu4F,EAAUh+F,IAAK,CACzD,IAAK,GAAImF,GAAI,EAAGA,EAAIqE,EAAWkxD,cAAcsjC,EAAUh+F,IAAIsF,OAAQH,IAAK,CACtE,GAAIkoD,GAAO7jD,EAAWkxD,cAAcsjC,EAAUh+F,IAAImF,EAC9CkoD,GAAKyG,eAAezG,EAAKyG,eAAexuD,OAAO,IAAM04F,EAAUh+F,IACjEqtD,EAAKyG,eAAe9Z,MACpBqT,EAAKiG,OAAS0qC,EAAUh+F,GACxBqtD,EAAKtkC,KAAOi1E,IAGZ3wC,EAAK0G,aAAa/Z,MAClBqT,EAAKkG,KAAOyqC,EAAUh+F,GACtBqtD,EAAKrkC,GAAKg1E,GAIZA,EAAUlvC,aAAannD,KAAK0lD,EAG5B,KAAK,GAAI9hC,GAAI,EAAGA,EAAI/hB,EAAWslD,aAAaxpD,OAAQimB,IAClD,GAAI/hB,EAAWslD,aAAavjC,GAAGvrB,IAAMqtD,EAAKrtD,GAAI,CAC5CwJ,EAAWslD,aAAa/mD,OAAOwjB,EAAE,EACjC,cAKC/hB,GAAWkxD,cAAcsjC,EAAUh+F,MAa9CT,EAAQ8+F,eAAiB,SAAS70F,GAChC,IAAK,GAAIrE,GAAI,EAAGA,EAAIqE,EAAWslD,aAAaxpD,OAAQH,IAAK,CACvD,GAAIkoD,GAAO7jD,EAAWslD,aAAa3pD,EAC/BqE,GAAWxJ,IAAMqtD,EAAKkG,MAAQ/pD,EAAWxJ,IAAMqtD,EAAKiG,QACtD9pD,EAAWslD,aAAa/mD,OAAO5C,EAAE,KAcvC5F,EAAQ4+F,uBAAyB,SAAS30F,EAAYw0F,GACpD,IAAK,GAAI74F,GAAI,EAAGA,EAAIqE,EAAW0yD,eAAe8hC,EAAUh+F,IAAIsF,OAAQH,IAAK,CACvE,GAAIkoD,GAAO7jD,EAAW0yD,eAAe8hC,EAAUh+F,IAAImF,EAGnDxF,MAAK89C,MAAM4P,EAAKrtD,IAAMqtD,EAGtB2wC,EAAUlvC,aAAannD,KAAK0lD,GAC5B7jD,EAAWslD,aAAannD,KAAK0lD,SAGxB7jD,GAAW0yD,eAAe8hC,EAAUh+F,KAa7CT,EAAQ4uD,aAAe,WACrB,GAAIrI,EAEJ,KAAKA,IAAUnmD,MAAKg9C,MAClB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EAClBL,GAAKuW,YAAc,IACrBvW,EAAKr9B,MAAQ,IAAIzU,OAAO5P,OAAO0hD,EAAKuW,aAAa,MAMvD,IAAKlW,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACM,GAApBL,EAAKuW,cAELvW,EAAKr9B,MADoBjiB,SAAvBs/C,EAAK2W,cACM3W,EAAK2W,cAGLr4D,OAAO0hD,EAAKzlD,OAuBnCT,EAAQ48F,uBAAyB,WAC/B,GAGIr2C,GAHAm6C,EAAW,EACXC,EAAW,IACXC,EAAe,CAInB,KAAKr6C,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5Bq6C,EAAexgG,KAAKg9C,MAAMmJ,GAAQqW,gBAAgB72D,OACnC66F,EAAXF,IAA0BA,EAAWE,GACrCD,EAAWC,IAAeD,EAAWC,GAI7C,IAAIF,EAAWC,EAAWvgG,KAAK0hD,UAAUvC,WAAWgB,uBAAwB,CAC1E,GAAIo9C,GAAgBv9F,KAAK8jD,YAAYn+C,OACjC86F,EAAcH,EAAWtgG,KAAK0hD,UAAUvC,WAAWgB,sBAEvD,KAAKgG,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,IACxBnmD,KAAKg9C,MAAMmJ,GAAQqW,gBAAgB72D,OAAS86F,GAC9CzgG,KAAKk/F,4BAA4Bl/F,KAAKg9C,MAAMmJ,GAIlDnmD,MAAKgnD,uBACLhnD,KAAKk9F,sBAEDl9F,KAAK8jD,YAAYn+C,QAAU43F,IAC7Bv9F,KAAK87D,gBAAkB,KAe7Bl8D,EAAQi9F,kBAAoB,SAAS/2C,GACnC,MACE5gD,MAAK2lB,IAAIi7B,EAAK/zC,EAAI/R,KAAKkkD,WAAWnyC,IAAM/R,KAAK0hD,UAAUvC,WAAWe,kBAAkBlgD,KAAKid,OAEzF/X,KAAK2lB,IAAIi7B,EAAK9zC,EAAIhS,KAAKkkD,WAAWlyC,IAAMhS,KAAK0hD,UAAUvC,WAAWe,kBAAkBlgD,KAAKid,OAU7Frd,EAAQ88F,gBAAkB,WACxB,IAAK,GAAIl3F,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAChD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACvC,IAAoB,GAAfsgD,EAAKqF,QAAkC,GAAfrF,EAAKsF,OAAkB,CAClD,GAAI3/B,GAAS,EAASzrB,KAAK8jD,YAAYn+C,OAAST,KAAK4G,IAAI,IAAIg6C,EAAKr3C,QAAQwuC,MACtEmR,EAAQ,EAAIlpD,KAAKymB,GAAKzmB,KAAKE,QACZ,IAAf0gD,EAAKqF,SAAkBrF,EAAK/zC,EAAI0Z,EAASvmB,KAAKqZ,IAAI6vC,IACnC,GAAftI,EAAKsF,SAAkBtF,EAAK9zC,EAAIyZ,EAASvmB,KAAKkZ,IAAIgwC,IACtDpuD,KAAK6+F,uBAAuB/4C,MAYlClmD,EAAQq+F,YAAc,WAMpB,IAAK,GALDyC,GAAU,EACVC,EAAiB,EACjBC,EAAa,EACbC,EAAa,EAERr7F,EAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAAK,CAEhD,GAAIsgD,GAAO9lD,KAAKg9C,MAAMh9C,KAAK8jD,YAAYt+C,GACnCsgD,GAAK+V,mBAAqBglC,IAC5BA,EAAa/6C,EAAK+V,oBAEpB6kC,GAAW56C,EAAK+V,mBAChB8kC,GAAkBz7F,KAAK4uB,IAAIgyB,EAAK+V,mBAAmB,GACnD+kC,GAAc,EAEhBF,GAAoBE,EACpBD,GAAkCC,CAElC,IAAIE,GAAWH,EAAiBz7F,KAAK4uB,IAAI4sE,EAAQ,GAE7CK,EAAoB77F,KAAKyqB,KAAKmxE,EAElC9gG,MAAKqtE,aAAenoE,KAAKC,MAAMu7F,EAAU,EAAEK,GAGvC/gG,KAAKqtE,aAAewzB,IACtB7gG,KAAKqtE,aAAewzB,IAexBjhG,EAAQo+F,sBAAwB,SAASgD,GACvChhG,KAAKqtE,aAAe,CACpB,IAAI4zB,GAAe/7F,KAAKC,MAAMnF,KAAK8jD,YAAYn+C,OAASq7F,EACxD,KAAK,GAAI76C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,IACiB,GAAzCnmD,KAAKg9C,MAAMmJ,GAAQ0V,oBAA2B77D,KAAKg9C,MAAMmJ,GAAQgJ,aAAaxpD,QAAU,GACtFs7F,EAAe,IACjBjhG,KAAKu/F,oBAAoBv/F,KAAKg9C,MAAMmJ,IAAQ,GAAK,EAAK,GACtD86C,GAAgB,IAa1BrhG,EAAQm+F,kBAAoB,WAC1B,GAAImD,GAAS,EACTC,EAAQ,CACZ,KAAK,GAAIh7C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KACiB,GAAzCnmD,KAAKg9C,MAAMmJ,GAAQ0V,oBAA2B77D,KAAKg9C,MAAMmJ,GAAQgJ,aAAaxpD,QAAU,IAC1Fu7F,GAAU,GAEZC,GAAS,EAGb,OAAOD,GAAOC,IAMZ,SAASthG,EAAQD,EAASM,GAE9B,GAAIS,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,GAgB/BN,GAAQ+nD,iBAAmB,WACzB3nD,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAW9/C,MAAQh9C,KAAKg9C,MACpDh9C,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAWh/C,MAAQ99C,KAAK89C,MACpD99C,KAAKkvD,QAAgB,OAAElvD,KAAK88F,WAAWh5C,YAAc9jD,KAAK8jD,aAa5DlkD,EAAQwhG,gBAAkB,SAASC,EAAUC,GACxB96F,SAAf86F,GAA0C,UAAdA,EAC9BthG,KAAKuhG,sBAAsBF,GAG3BrhG,KAAKwhG,sBAAsBH,IAY/BzhG,EAAQ2hG,sBAAwB,SAASF,GACvCrhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YACjErhG,KAAKg9C,MAAch9C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAC3DrhG,KAAK89C,MAAc99C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,OAU7DzhG,EAAQ6hG,uBAAyB,WAC/BzhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAiB,QAAe,YACxDlvD,KAAKg9C,MAAch9C,KAAKkvD,QAAiB,QAAS,MAClDlvD,KAAK89C,MAAc99C,KAAKkvD,QAAiB,QAAS,OAWpDtvD,EAAQ4hG,sBAAwB,SAASH,GACvCrhG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YACjErhG,KAAKg9C,MAAch9C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAC3DrhG,KAAK89C,MAAc99C,KAAKkvD,QAAgB,OAAEmyC,GAAiB,OAU7DzhG,EAAQ8hG,kBAAoB,WAC1B1hG,KAAKohG,gBAAgBphG,KAAK88F,YAU5Bl9F,EAAQk9F,QAAU,WAChB,MAAO98F,MAAKstE,aAAattE,KAAKstE,aAAa3nE,OAAO,IAUpD/F,EAAQ+hG,gBAAkB,WACxB,GAAI3hG,KAAKstE,aAAa3nE,OAAS,EAC7B,MAAO3F,MAAKstE,aAAattE,KAAKstE,aAAa3nE,OAAO,EAGlD,MAAM,IAAIU,WAAU,iEAaxBzG,EAAQgiG,iBAAmB,SAASC,GAClC7hG,KAAKstE,aAAatlE,KAAK65F,IAUzBjiG,EAAQkiG,kBAAoB,WAC1B9hG,KAAKstE,aAAajzB,OAWpBz6C,EAAQmiG,iBAAmB,SAASF,GAElC7hG,KAAKkvD,QAAgB,OAAE2yC,IAAU7kD,SACAc,SACAgG,eACAsY,eAAkBp8D,KAAKid,MACvBswD,YAAe/mE,QAGhDxG,KAAKkvD,QAAgB,OAAE2yC,GAAoB,YAAI,GAAIt+F,IAC9ClD,GAAGwhG,EACF12F,OACEgB,WAAY,UACZC,OAAQ,iBAEJpM,KAAK0hD,WACjB1hD,KAAKkvD,QAAgB,OAAE2yC,GAAoB,YAAExlC,YAAc,GAW7Dz8D,EAAQoiG,oBAAsB,SAASX,SAC9BrhG,MAAKkvD,QAAgB,OAAEmyC,IAWhCzhG,EAAQqiG,oBAAsB,SAASZ,SAC9BrhG,MAAKkvD,QAAgB,OAAEmyC,IAWhCzhG,EAAQsiG,cAAgB,SAASb,GAE/BrhG,KAAKkvD,QAAgB,OAAEmyC,GAAYrhG,KAAKkvD,QAAgB,OAAEmyC,GAG1DrhG,KAAKgiG,oBAAoBX,IAW3BzhG,EAAQuiG,gBAAkB,SAASd,GAEjCrhG,KAAKkvD,QAAgB,OAAEmyC,GAAYrhG,KAAKkvD,QAAgB,OAAEmyC,GAG1DrhG,KAAKiiG,oBAAoBZ,IAa3BzhG,EAAQwiG,qBAAuB,SAASf,GAEtC,IAAK,GAAIl7C,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAAEl7C,GAAUnmD,KAAKg9C,MAAMmJ,GAKnE,KAAK,GAAI6G,KAAUhtD,MAAK89C,MAClB99C,KAAK89C,MAAMh4C,eAAeknD,KAC5BhtD,KAAKkvD,QAAgB,OAAEmyC,GAAiB,MAAEr0C,GAAUhtD,KAAK89C,MAAMkP,GAKnE,KAAK,GAAIxnD,GAAI,EAAGA,EAAIxF,KAAK8jD,YAAYn+C,OAAQH,IAC3CxF,KAAKkvD,QAAgB,OAAEmyC,GAAuB,YAAEr5F,KAAKhI,KAAK8jD,YAAYt+C,KAW1E5F,EAAQyiG,6BAA+B,WACrCriG,KAAKm8F,aAAa,GAAE,IAUtBv8F,EAAQm9F,WAAa,SAASj3C,GAE5B,GAAIw8C,GAAStiG,KAAK88F,gBAWX98F,MAAKg9C,MAAM8I,EAAKzlD,GAEvB,IAAIkiG,GAAmB5hG,EAAKqE,YAG5BhF,MAAKkiG,cAAcI,GAGnBtiG,KAAK+hG,iBAAiBQ,GAGtBviG,KAAK4hG,iBAAiBW,GAGtBviG,KAAKohG,gBAAgBphG,KAAK88F,WAG1B98F,KAAKg9C,MAAM8I,EAAKzlD,IAAMylD,GAUxBlmD,EAAQ49F,gBAAkB,WAExB,GAAI8E,GAAStiG,KAAK88F,SAGlB,IAAc,WAAVwF,IAC8B,GAA3BtiG,KAAK8jD,YAAYn+C,QACpB3F,KAAKkvD,QAAgB,OAAEozC,GAAqB,YAAE/vF,MAAMvS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOC,aACnIxf,KAAKkvD,QAAgB,OAAEozC,GAAqB,YAAE9vF,OAAOxS,KAAKid,MAAQjd,KAAK0hD,UAAUvC,WAAWO,oBAAsB1/C,KAAKsf,MAAMC,OAAOsF,cAAe,CACnJ,GAAI29E,GAAiBxiG,KAAK2hG,iBAG1B3hG,MAAKqiG,+BAILriG,KAAKoiG,qBAAqBI,GAI1BxiG,KAAKgiG,oBAAoBM,GAGzBtiG,KAAKmiG,gBAAgBK,GAGrBxiG,KAAKohG,gBAAgBoB,GAGrBxiG,KAAK8hG,oBAGL9hG,KAAKgnD,uBAGLhnD,KAAKquD,4BAeXzuD,EAAQsxD,sBAAwB,SAASuxC,EAAYC,GACnD,GAAIC,KACJ,IAAiBn8F,SAAbk8F,EACF,IAAK,GAAIJ,KAAUtiG,MAAKkvD,QAAgB,OAClClvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,KAExCtiG,KAAKuhG,sBAAsBe,GAC3BK,EAAa36F,KAAMhI,KAAKyiG,WAK5B,KAAK,GAAIH,KAAUtiG,MAAKkvD,QAAgB,OACtC,GAAIlvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,GAAS,CAEjDtiG,KAAKuhG,sBAAsBe,EAC3B,IAAIrpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAEhDi9F,GAAa36F,KADXiR,EAAKtT,OAAS,EACG3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,IAO7C,MADA1iG,MAAK0hG,oBACEiB,GAaT/iG,EAAQuxD,mBAAqB,SAASsxC,EAAYC,GAChD,GAAIC,IAAe,CACnB,IAAiBn8F,SAAbk8F,EACF1iG,KAAKyhG,yBACLkB,EAAe3iG,KAAKyiG,SAEjB,CACHziG,KAAKyhG,wBACL,IAAIxoF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAEhDi9F,GADE1pF,EAAKtT,OAAS,EACD3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,GAKrC,MADA1iG,MAAK0hG,oBACEiB,GAaT/iG,EAAQgjG,sBAAwB,SAASH,EAAYC,GACnD,GAAiBl8F,SAAbk8F,EACF,IAAK,GAAIJ,KAAUtiG,MAAKkvD,QAAgB,OAClClvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,KAExCtiG,KAAKwhG,sBAAsBc,GAC3BtiG,KAAKyiG,UAKT,KAAK,GAAIH,KAAUtiG,MAAKkvD,QAAgB,OACtC,GAAIlvD,KAAKkvD,QAAgB,OAAEppD,eAAew8F,GAAS,CAEjDtiG,KAAKwhG,sBAAsBc,EAC3B,IAAIrpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EAC9CuT,GAAKtT,OAAS,EAChB3F,KAAKyiG,GAAaxpF,EAAK,GAAGA,EAAK,IAG/BjZ,KAAKyiG,GAAaC,GAK1B1iG,KAAK0hG,qBAaP9hG,EAAQ4vD,gBAAkB,SAASizC,EAAYC,GAC7C,GAAIzpF,GAAOhT,MAAMkN,UAAU/K,OAAO7H,KAAKmF,UAAW,EACjCc,UAAbk8F,GACF1iG,KAAKkxD,sBAAsBuxC,GAC3BziG,KAAK4iG,sBAAsBH,IAGvBxpF,EAAKtT,OAAS,GAChB3F,KAAKkxD,sBAAsBuxC,EAAYxpF,EAAK,GAAGA,EAAK,IACpDjZ,KAAK4iG,sBAAsBH,EAAYxpF,EAAK,GAAGA,EAAK,MAGpDjZ,KAAKkxD,sBAAsBuxC,EAAYC,GACvC1iG,KAAK4iG,sBAAsBH,EAAYC,KAY7C9iG,EAAQqnD,oBAAsB,WAC5B,GAAIq7C,GAAStiG,KAAK88F,SAClB98F,MAAKkvD,QAAgB,OAAEozC,GAAqB,eAC5CtiG,KAAK8jD,YAAc9jD,KAAKkvD,QAAgB,OAAEozC,GAAqB,aAWjE1iG,EAAQijG,iBAAmB,SAAS97E,EAAIu6E,GACtC,GAAsDx7C,GAAlDC,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAChD,KAAK,GAAIo8C,KAAUtiG,MAAKkvD,QAAQoyC,GAC9B,GAAIthG,KAAKkvD,QAAQoyC,GAAYx7F,eAAew8F,IACc97F,SAApDxG,KAAKkvD,QAAQoyC,GAAYgB,GAAqB,YAAiB,CAEjEtiG,KAAKohG,gBAAgBkB,EAAOhB,GAE5Bv7C,EAAO,IAAKC,EAAO,KAAMC,EAAO,IAAKC,EAAO,IAC5C,KAAK,GAAIC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKkQ,OAAOjvC,GACRk/B,EAAOH,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,QAAQ0zC,EAAOH,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,OAC9D2zC,EAAOJ,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,QAAQ2zC,EAAOJ,EAAK/zC,EAAI,GAAM+zC,EAAKvzC,OAC9DwzC,EAAOD,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,SAASuzC,EAAOD,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAC/DwzC,EAAOF,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,SAASwzC,EAAOF,EAAK9zC,EAAI,GAAM8zC,EAAKtzC,QAGvEszC,GAAO9lD,KAAKkvD,QAAQoyC,GAAYgB,GAAqB,YACrDx8C,EAAK/zC,EAAI,IAAOm0C,EAAOD,GACvBH,EAAK9zC,EAAI,IAAOg0C,EAAOD,GACvBD,EAAKvzC,MAAQ,GAAKuzC,EAAK/zC,EAAIk0C,GAC3BH,EAAKtzC,OAAS,GAAKszC,EAAK9zC,EAAI+zC,GAC5BD,EAAKr3C,QAAQgd,OAASvmB,KAAKyqB,KAAKzqB,KAAK4uB,IAAI,GAAIgyB,EAAKvzC,MAAM,GAAKrN,KAAK4uB,IAAI,GAAIgyB,EAAKtzC,OAAO,IACtFszC,EAAK9iB,SAAShjC,KAAKid,OACnB6oC,EAAKoX,YAAYn2C,KAMzBnnB,EAAQkjG,oBAAsB,SAAS/7E,GACrC/mB,KAAK6iG,iBAAiB97E,EAAI,UAC1B/mB,KAAK6iG,iBAAiB97E,EAAI,UAC1B/mB,KAAK0hG,sBAMH,SAAS7hG,EAAQD,EAASM,GAE9B,GAAIqD,GAAOrD,EAAoB,GAS/BN,GAAQmjG,yBAA2B,SAAS9+F,EAAQqpD,GAClD,GAAItQ,GAAQh9C,KAAKg9C,KACjB,KAAK,GAAImJ,KAAUnJ,GACbA,EAAMl3C,eAAeqgD,IACnBnJ,EAAMmJ,GAAQoH,kBAAkBtpD,IAClCqpD,EAAiBtlD,KAAKm+C,IAY9BvmD,EAAQojG,4BAA8B,SAAU/+F,GAC9C,GAAIqpD,KAEJ,OADAttD,MAAKkxD,sBAAsB,2BAA2BjtD,EAAOqpD,GACtDA,GAWT1tD,EAAQqjG,yBAA2B,SAASpjE,GAC1C,GAAI9tB,GAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACtCC,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,EAE1C,QACEvK,KAAQsK,EACRpK,IAAQqK,EACRqV,MAAQtV,EACRuR,OAAQtR,IAYZpS,EAAQgrD,WAAa,SAAU/qB,GAE7B,GAAIqjE,GAAiBljG,KAAKijG,yBAAyBpjE,GAC/CytB,EAAmBttD,KAAKgjG,4BAA4BE,EAIxD,OAAI51C,GAAiB3nD,OAAS,EACpB3F,KAAKg9C,MAAMsQ,EAAiBA,EAAiB3nD,OAAS,IAGvD,MAWX/F,EAAQujG,yBAA2B,SAAUl/F,EAAQwpD,GACnD,GAAI3P,GAAQ99C,KAAK89C,KACjB,KAAK,GAAIkP,KAAUlP,GACbA,EAAMh4C,eAAeknD,IACnBlP,EAAMkP,GAAQO,kBAAkBtpD,IAClCwpD,EAAiBzlD,KAAKglD,IAa9BptD,EAAQwjG,4BAA8B,SAAUn/F,GAC9C,GAAIwpD,KAEJ,OADAztD,MAAKkxD,sBAAsB,2BAA2BjtD,EAAOwpD,GACtDA,GAWT7tD,EAAQqtD,WAAa,SAASptB,GAC5B,GAAIqjE,GAAiBljG,KAAKijG,yBAAyBpjE,GAC/C4tB,EAAmBztD,KAAKojG,4BAA4BF,EAExD,OAAIz1C,GAAiB9nD,OAAS,EACrB3F,KAAK89C,MAAM2P,EAAiBA,EAAiB9nD,OAAS,IAGtD,MAWX/F,EAAQyjG,gBAAkB,SAAStgF,GAC7BA,YAAexf,GACjBvD,KAAKkrD,aAAalO,MAAMj6B,EAAI1iB,IAAM0iB,EAGlC/iB,KAAKkrD,aAAapN,MAAM/6B,EAAI1iB,IAAM0iB,GAUtCnjB,EAAQ0jG,YAAc,SAASvgF,GACzBA,YAAexf,GACjBvD,KAAK4hD,SAAS5E,MAAMj6B,EAAI1iB,IAAM0iB,EAG9B/iB,KAAK4hD,SAAS9D,MAAM/6B,EAAI1iB,IAAM0iB,GAWlCnjB,EAAQ2jG,qBAAuB,SAASxgF,GAClCA,YAAexf,SACVvD,MAAKkrD,aAAalO,MAAMj6B,EAAI1iB,UAG5BL,MAAKkrD,aAAapN,MAAM/6B,EAAI1iB,KAUvCT,EAAQ2+F,aAAe,SAASiF,GACTh9F,SAAjBg9F,IACFA,GAAe,EAEjB,KAAI,GAAIr9C,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACxCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQjV,UAGpC,KAAI,GAAI8b,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,IACxChtD,KAAKkrD,aAAapN,MAAMkP,GAAQ9b,UAIpClxC,MAAKkrD,cAAgBlO,SAASc,UAEV,GAAhB0lD,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAU7Bh3B,EAAQ6jG,kBAAoB,SAASD,GACdh9F,SAAjBg9F,IACFA,GAAe,EAGjB,KAAK,GAAIr9C,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACrCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQkW,YAAc,IAChDr8D,KAAKkrD,aAAalO,MAAMmJ,GAAQjV,WAChClxC,KAAKujG,qBAAqBvjG,KAAKkrD,aAAalO,MAAMmJ,IAKpC,IAAhBq9C,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAW7Bh3B,EAAQ8jG,sBAAwB,WAC9B,GAAI1sF,GAAQ,CACZ,KAAK,GAAImvC,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACzCnvC,GAAS,EAGb,OAAOA,IASTpX,EAAQ+jG,iBAAmB,WACzB,IAAK,GAAIx9C,KAAUnmD,MAAKkrD,aAAalO,MACnC,GAAIh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,GACzC,MAAOnmD,MAAKkrD,aAAalO,MAAMmJ,EAGnC,OAAO,OASTvmD,EAAQgkG,iBAAmB,WACzB,IAAK,GAAI52C,KAAUhtD,MAAKkrD,aAAapN,MACnC,GAAI99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,GACzC,MAAOhtD,MAAKkrD,aAAapN,MAAMkP,EAGnC,OAAO,OAUTptD,EAAQikG,sBAAwB,WAC9B,GAAI7sF,GAAQ,CACZ,KAAK,GAAIg2C,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACzCh2C,GAAS,EAGb,OAAOA,IAUTpX,EAAQkkG,wBAA0B,WAChC,GAAI9sF,GAAQ,CACZ,KAAI,GAAImvC,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACxCnvC,GAAS,EAGb,KAAI,GAAIg2C,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACxCh2C,GAAS,EAGb,OAAOA,IASTpX,EAAQmkG,kBAAoB,WAC1B,IAAI,GAAI59C,KAAUnmD,MAAKkrD,aAAalO,MAClC,GAAGh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,GACxC,OAAO,CAGX,KAAI,GAAI6G,KAAUhtD,MAAKkrD,aAAapN,MAClC,GAAG99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,GACxC,OAAO,CAGX,QAAO,GAUTptD,EAAQokG,oBAAsB,WAC5B,IAAI,GAAI79C,KAAUnmD,MAAKkrD,aAAalO,MAClC,GAAGh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACpCnmD,KAAKkrD,aAAalO,MAAMmJ,GAAQkW,YAAc,EAChD,OAAO,CAIb,QAAO,GASTz8D,EAAQqkG,sBAAwB,SAASn+C,GACvC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKvc,SACLnxC,KAAKqjG,gBAAgB31C,KAUzB9tD,EAAQskG,qBAAuB,SAASp+C,GACtC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKphD,OAAQ,EACbtM,KAAKsjG,YAAY51C,KAWrB9tD,EAAQukG,wBAA0B,SAASr+C,GACzC,IAAK,GAAItgD,GAAI,EAAGA,EAAIsgD,EAAKqJ,aAAaxpD,OAAQH,IAAK,CACjD,GAAIkoD,GAAO5H,EAAKqJ,aAAa3pD,EAC7BkoD,GAAKxc,WACLlxC,KAAKujG,qBAAqB71C,KAgB9B9tD,EAAQmrD,cAAgB,SAAS9mD,EAAQmgG,EAAQZ,EAAca,EAAgBC,GACxD99F,SAAjBg9F,IACFA,GAAe,GAEMh9F,SAAnB69F,IACFA,GAAiB,GAGa,GAA5BrkG,KAAK+jG,qBAA0C,GAAVK,GAAgD,GAA7BpkG,KAAKytE,sBAC/DztE,KAAKu+F,cAAa,GAIG,GAAnBt6F,EAAOovC,UAAmD,GAA7BrzC,KAAK0hD,UAAUzS,aAAsBq1D,EAQ1C,GAAnBrgG,EAAOovC,UACdrzC,KAAKqjG,gBAAgBp/F,GACrBu/F,GAAe,IAGfv/F,EAAOitC,WACPlxC,KAAKujG,qBAAqBt/F,KAb1BA,EAAOktC,SACPnxC,KAAKqjG,gBAAgBp/F,GACjBA,YAAkBV,IAA6C,GAArCvD,KAAKwtE,8BAA2D,GAAlB62B,GAC1ErkG,KAAKikG,sBAAsBhgG,IAaX,GAAhBu/F,GACFxjG,KAAK4tB,KAAK,SAAU5tB,KAAK42B,iBAY7Bh3B,EAAQutD,YAAc,SAASlpD,GACT,GAAhBA,EAAOqI,QACTrI,EAAOqI,OAAQ,EACftM,KAAK4tB,KAAK,YAAYk4B,KAAK7hD,EAAO5D,OAWtCT,EAAQstD,aAAe,SAASjpD,GACV,GAAhBA,EAAOqI,QACTrI,EAAOqI,OAAQ,EACftM,KAAKsjG,YAAYr/F,GACbA,YAAkBV,IACpBvD,KAAK4tB,KAAK,aAAak4B,KAAK7hD,EAAO5D,MAGnC4D,YAAkBV,IACpBvD,KAAKkkG,qBAAqBjgG,IAa9BrE,EAAQ8qD,aAAe,aAUvB9qD,EAAQgsD,WAAa,SAAS/rB,GAC5B,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EAC3B,IAAY,MAARimB,EACF9lD,KAAK+qD,cAAcjF,GAAM,OAEtB,CACH,GAAI4H,GAAO1tD,KAAKitD,WAAWptB,EACf,OAAR6tB,EACF1tD,KAAK+qD,cAAc2C,GAAM,GAGzB1tD,KAAKu+F,eAGT,GAAI5vC,GAAa3uD,KAAK42B,cACtB+3B,GAAoB,SAClB41C,KAAMxyF,EAAG8tB,EAAQ9tB,EAAGC,EAAG6tB,EAAQ7tB,GAC/BuN,QAASxN,EAAG/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAIC,EAAGhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,KAEzFhS,KAAK4tB,KAAK,QAAS+gC,GACnB3uD,KAAK6iD,WAUPjjD,EAAQisD,iBAAmB,SAAShsB,GAClC,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EACf,OAARimB,GAAyBt/C,SAATs/C,IAElB9lD,KAAKkkD,YAAenyC,EAAM/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxCC,EAAMhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAC5DhS,KAAK28F,YAAY72C,GAEnB,IAAI6I,GAAa3uD,KAAK42B,cACtB+3B,GAAoB,SAClB41C,KAAMxyF,EAAG8tB,EAAQ9tB,EAAGC,EAAG6tB,EAAQ7tB,GAC/BuN,QAASxN,EAAG/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAIC,EAAGhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,KAEzFhS,KAAK4tB,KAAK,cAAe+gC,IAU3B/uD,EAAQksD,cAAgB,SAASjsB,GAC/B,GAAIimB,GAAO9lD,KAAK4qD,WAAW/qB,EAC3B,IAAY,MAARimB,EACF9lD,KAAK+qD,cAAcjF,GAAK,OAErB,CACH,GAAI4H,GAAO1tD,KAAKitD,WAAWptB,EACf,OAAR6tB,GACF1tD,KAAK+qD,cAAc2C,GAAK,GAG5B1tD,KAAK6iD,WAUPjjD,EAAQosD,iBAAmB,SAASnsB,GAClC7/B,KAAKwkG,6BAA6B3kE,GAClC7/B,KAAKykG,2BAA2B5kE,IAGlCjgC,EAAQ4kG,6BAA+B,aACvC5kG,EAAQ6kG,2BAA6B,aAOrC7kG,EAAQg3B,aAAe,WACrB,GAAIo0B,GAAUhrD,KAAK0kG,mBACfC,EAAU3kG,KAAK4kG,kBACnB,QAAQ5nD,MAAMgO,EAASlN,MAAM6mD,IAS/B/kG,EAAQ8kG,iBAAmB,WACzB,GAAIG,KACJ,IAAiC,GAA7B7kG,KAAK0hD,UAAUzS,WACjB,IAAK,GAAIkX,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,IACzC0+C,EAAQ78F,KAAKm+C,EAInB,OAAO0+C,IASTjlG,EAAQglG,iBAAmB,WACzB,GAAIC,KACJ,IAAiC,GAA7B7kG,KAAK0hD,UAAUzS,WACjB,IAAK,GAAI+d,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,IACzC63C,EAAQ78F,KAAKglD,EAInB,OAAO63C,IASTjlG,EAAQ82B,aAAe,WACrBiC,QAAQhF,IAAI,gEAUd/zB,EAAQklG,YAAc,SAAS30D,EAAWk0D,GACxC,GAAI7+F,GAAGq7B,EAAMxgC,CAEb,KAAK8vC,GAAkC3pC,QAApB2pC,EAAUxqC,OAC3B,KAAM,qCAKR,KAFA3F,KAAKu+F,cAAa,GAEb/4F,EAAI,EAAGq7B,EAAOsP,EAAUxqC,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAClDnF,EAAK8vC,EAAU3qC,EAEf,IAAIsgD,GAAO9lD,KAAKg9C,MAAM38C,EACtB,KAAKylD,EACH,KAAM,IAAIi/C,YAAW,iBAAmB1kG,EAAK,cAE/CL,MAAK+qD,cAAcjF,GAAK,GAAK,EAAKu+C,GAAe,GAEnDrkG,KAAKyhB,UASP7hB,EAAQolG,YAAc,SAAS70D,GAC7B,GAAI3qC,GAAGq7B,EAAMxgC,CAEb,KAAK8vC,GAAkC3pC,QAApB2pC,EAAUxqC,OAC3B,KAAM,qCAKR,KAFA3F,KAAKu+F,cAAa,GAEb/4F,EAAI,EAAGq7B,EAAOsP,EAAUxqC,OAAYk7B,EAAJr7B,EAAUA,IAAK,CAClDnF,EAAK8vC,EAAU3qC,EAEf,IAAIkoD,GAAO1tD,KAAK89C,MAAMz9C,EACtB,KAAKqtD,EACH,KAAM,IAAIq3C,YAAW,iBAAmB1kG,EAAK,cAE/CL,MAAK+qD,cAAc2C,GAAK,GAAK,GAAK,GAAM,GAE1C1tD,KAAKyhB,UAOP7hB,EAAQuuD,iBAAmB,WACzB,IAAI,GAAIhI,KAAUnmD,MAAKkrD,aAAalO,MAC/Bh9C,KAAKkrD,aAAalO,MAAMl3C,eAAeqgD,KACnCnmD,KAAKg9C,MAAMl3C,eAAeqgD,UACtBnmD,MAAKkrD,aAAalO,MAAMmJ,GAIrC,KAAI,GAAI6G,KAAUhtD,MAAKkrD,aAAapN,MAC/B99C,KAAKkrD,aAAapN,MAAMh4C,eAAeknD,KACnChtD,KAAK89C,MAAMh4C,eAAeknD,UACtBhtD,MAAKkrD,aAAapN,MAAMkP,MASnC,SAASntD,EAAQD,EAASM,GAE9B,GAAIS,GAAOT,EAAoB,GAC3BqD,EAAOrD,EAAoB,IAC3BkD,EAAOlD,EAAoB,GAO/BN,GAAQqlG,qBAAuB,WAC7BjlG,KAAKqqD,oBAAoBrqD,KAAK0tE,iBAC9B1tE,KAAKklG,mBAELllG,KAAKwkG,6BAA+B,mBAC7BxkG,MAAKkvD,QAAiB,QAAS,MAAc,iBAC7ClvD,MAAKkvD,QAAiB,QAAS,MAAiB,cACvDlvD,KAAK6hD,oBAAqB,EAC1B7hD,KAAKujD,kBAAmB,GAU1B3jD,EAAQulG,4BAA8B,WACpC,IAAK,GAAIC,KAAgBplG,MAAKwjD,gBACxBxjD,KAAKwjD,gBAAgB19C,eAAes/F,KACtCplG,KAAKolG,GAAgBplG,KAAKwjD,gBAAgB4hD,SACnCplG,MAAKwjD,gBAAgB4hD,KAUlCxlG,EAAQylG,gBAAkB,WACxBrlG,KAAKkoD,UAAYloD,KAAKkoD,QACtB,IAAIo9C,GAAUtlG,KAAK0tE,gBACfE,EAAW5tE,KAAK4tE,SAChBD,EAAc3tE,KAAK2tE,WACF,IAAjB3tE,KAAKkoD,UACPo9C,EAAQr4F,MAAM26B,QAAQ,QACtBgmC,EAAS3gE,MAAM26B,QAAQ,QACvB+lC,EAAY1gE,MAAM26B,QAAQ,OAC1BgmC,EAAS57C,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,QAG7CslG,EAAQr4F,MAAM26B,QAAQ,OACtBgmC,EAAS3gE,MAAM26B,QAAQ,OACvB+lC,EAAY1gE,MAAM26B,QAAQ,QAC1BgmC,EAAS57C,QAAU,MAErBhyB,KAAKmnD,yBAQPvnD,EAAQunD,sBAAwB,WAE1BnnD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAqBnD,IAnB6B79B,SAAzBxG,KAAKwlG,kBACPxlG,KAAKwlG,gBAAgB/rC,uBACrBz5D,KAAKwlG,gBAAkBh/F,OACvBxG,KAAKylG,oBAAsB,KAC3BzlG,KAAK6hD,oBAAqB,EAC1B7hD,KAAK6iD,WAIP7iD,KAAKmlG,8BAGLnlG,KAAKujD,kBAAmB,EAGxBvjD,KAAKwtE,8BAA+B,EACpCxtE,KAAKytE,sBAAuB,EAC5BztE,KAAKklG,mBAEgB,GAAjBllG,KAAKkoD,SAAkB,CACzB,KAAOloD,KAAK0tE,gBAAgBhqD,iBAC1B1jB,KAAK0tE,gBAAgBv8D,YAAYnR,KAAK0tE,gBAAgB/pD,WAGxD3jB,MAAKklG,gBAA6B,YAAI3zF,SAASM,cAAc,QAC7D7R,KAAKklG,gBAA6B,YAAEr9F,UAAY,6BAChD7H,KAAKklG,gBAAkC,iBAAI3zF,SAASM,cAAc,QAClE7R,KAAKklG,gBAAkC,iBAAEr9F,UAAY,4BACrD7H,KAAKklG,gBAAkC,iBAAEjhF,UAAYogB,EAAgB,QACrErkC,KAAKklG,gBAA6B,YAAEzzF,YAAYzR,KAAKklG,gBAAkC,kBAEvFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA6B,YAAI3zF,SAASM,cAAc,QAC7D7R,KAAKklG,gBAA6B,YAAEr9F,UAAY,iCAChD7H,KAAKklG,gBAAkC,iBAAI3zF,SAASM,cAAc,QAClE7R,KAAKklG,gBAAkC,iBAAEr9F,UAAY,4BACrD7H,KAAKklG,gBAAkC,iBAAEjhF,UAAYogB,EAAgB,QACrErkC,KAAKklG,gBAA6B,YAAEzzF,YAAYzR,KAAKklG,gBAAkC,kBAEvFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA6B,aACnEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA6B,aAE/B,GAAhCllG,KAAK0jG,yBAAgC1jG,KAAK28C,iBAAiBC,MAC7D58C,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,8BACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAiB,SACvErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA8B,eAE7B,GAAhCllG,KAAK6jG,yBAAgE,GAAhC7jG,KAAK0jG,0BACjD1jG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,8BACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAiB,SACvErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA8B,eAEtC,GAA5BllG,KAAK+jG,sBACP/jG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAA4B,WAAI3zF,SAASM,cAAc,QAC5D7R,KAAKklG,gBAA4B,WAAEr9F,UAAY,gCAC/C7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,4BACpD7H,KAAKklG,gBAAiC,gBAAEjhF,UAAYogB,EAAY,IAChErkC,KAAKklG,gBAA4B,WAAEzzF,YAAYzR,KAAKklG,gBAAiC,iBAErFllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA4B,aAKpEllG,KAAKklG,gBAA6B,YAAElzE,QAAUhyB,KAAK0lG,sBAAsB5wE,KAAK90B,MAC9EA,KAAKklG,gBAA6B,YAAElzE,QAAUhyB,KAAK2lG,sBAAsB7wE,KAAK90B,MAC1C,GAAhCA,KAAK0jG,yBAAgC1jG,KAAK28C,iBAAiBC,KAC7D58C,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAK4lG,UAAU9wE,KAAK90B,MAE5B,GAAhCA,KAAK6jG,yBAAgE,GAAhC7jG,KAAK0jG,0BACjD1jG,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAK6lG,uBAAuB/wE,KAAK90B,OAElD,GAA5BA,KAAK+jG,sBACP/jG,KAAKklG,gBAA4B,WAAElzE,QAAUhyB,KAAKmqD,gBAAgBr1B,KAAK90B,OAEzEA,KAAK4tE,SAAS57C,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,KAElD;GAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAGgzC,sBACxBnnD,KAAKuT,GAAG,SAAUvT,KAAKulG,mBAEpB,CACH,KAAOvlG,KAAK2tE,YAAYjqD,iBACtB1jB,KAAK2tE,YAAYx8D,YAAYnR,KAAK2tE,YAAYhqD,WAGhD3jB,MAAKklG,gBAA8B,aAAI3zF,SAASM,cAAc,QAC9D7R,KAAKklG,gBAA8B,aAAEr9F,UAAY,uCACjD7H,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,QACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,4BACtD7H,KAAKklG,gBAAmC,kBAAEjhF,UAAYogB,EAAa,KACnErkC,KAAKklG,gBAA8B,aAAEzzF,YAAYzR,KAAKklG,gBAAmC,mBAEzFllG,KAAK2tE,YAAYl8D,YAAYzR,KAAKklG,gBAA8B,cAEhEllG,KAAKklG,gBAA8B,aAAElzE,QAAUhyB,KAAKqlG,gBAAgBvwE,KAAK90B,QAW7EJ,EAAQ8lG,sBAAwB,WAE9B1lG,KAAKilG,uBACDjlG,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAAuB,eAChFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,KAG3E,IAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAG2xF,SACxB9lG,KAAKuT,GAAG,SAAUvT,KAAKulG,gBASzB3lG,EAAQ+lG,sBAAwB,WAE9B3lG,KAAKilG,uBACLjlG,KAAKu+F,cAAa,GAClBv+F,KAAKujD,kBAAmB,EAEpBvjD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,cAG1B,IAAIlhE,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKu+F,eACLv+F,KAAKytE,sBAAuB,EAC5BztE,KAAKwtE,8BAA+B,EAEpCxtE,KAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAAwB,gBACjFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,KAG3E,IAAImU,GAAKnU,IACTA,MAAKulG,cAAgBpxF,EAAG4xF,eACxB/lG,KAAKuT,GAAG,SAAUvT,KAAKulG,eAGvBvlG,KAAKwjD,gBAA8B,aAAIxjD,KAAK0qD,aAC5C1qD,KAAKwjD,gBAA8C,6BAAIxjD,KAAKwkG,6BAC5DxkG,KAAKwjD,gBAAkC,iBAAIxjD,KAAK2qD,iBAChD3qD,KAAKwjD,gBAAgC,eAAIxjD,KAAK2rD,eAC9C3rD,KAAK0qD,aAAe1qD,KAAK+lG,eACzB/lG,KAAKwkG,6BAA+B,aACpCxkG,KAAK2qD,iBAAmB,aACxB3qD,KAAK2rD,eAAiB3rD,KAAKgmG,eAG3BhmG,KAAK6iD,WAQPjjD,EAAQimG,uBAAyB,WAE/B7lG,KAAKilG,uBACLjlG,KAAK6hD,oBAAqB,EAEtB7hD,KAAKulG,eACPvlG,KAAK0T,IAAI,SAAU1T,KAAKulG,eAG1BvlG,KAAKwlG,gBAAkBxlG,KAAK4jG,mBAC5B5jG,KAAKwlG,gBAAgBhsC,qBAErB,IAAIn1B,GAASrkC,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,OAEnDrkC,MAAKklG,mBACLllG,KAAKklG,gBAA0B,SAAI3zF,SAASM,cAAc,QAC1D7R,KAAKklG,gBAA0B,SAAEr9F,UAAY,8BAC7C7H,KAAKklG,gBAA+B,cAAI3zF,SAASM,cAAc,QAC/D7R,KAAKklG,gBAA+B,cAAEr9F,UAAY,4BAClD7H,KAAKklG,gBAA+B,cAAEjhF,UAAYogB,EAAa,KAC/DrkC,KAAKklG,gBAA0B,SAAEzzF,YAAYzR,KAAKklG,gBAA+B,eAEjFllG,KAAKklG,gBAAmC,kBAAI3zF,SAASM,cAAc,OACnE7R,KAAKklG,gBAAmC,kBAAEr9F,UAAY,wBAEtD7H,KAAKklG,gBAAiC,gBAAI3zF,SAASM,cAAc,QACjE7R,KAAKklG,gBAAiC,gBAAEr9F,UAAY,8BACpD7H,KAAKklG,gBAAsC,qBAAI3zF,SAASM,cAAc,QACtE7R,KAAKklG,gBAAsC,qBAAEr9F,UAAY,4BACzD7H,KAAKklG,gBAAsC,qBAAEjhF,UAAYogB,EAA4B,oBACrFrkC,KAAKklG,gBAAiC,gBAAEzzF,YAAYzR,KAAKklG,gBAAsC,sBAE/FllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAA0B,UAChEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAmC,mBACzEllG,KAAK0tE,gBAAgBj8D,YAAYzR,KAAKklG,gBAAiC,iBAGvEllG,KAAKklG,gBAA0B,SAAElzE,QAAUhyB,KAAKmnD,sBAAsBryB,KAAK90B,MAG3EA,KAAKwjD,gBAA8B,aAASxjD,KAAK0qD,aACjD1qD,KAAKwjD,gBAA8C,6BAAKxjD,KAAKwkG,6BAC7DxkG,KAAKwjD,gBAA4B,WAAWxjD,KAAK4rD,WACjD5rD,KAAKwjD,gBAAkC,iBAAKxjD,KAAK2qD,iBACjD3qD,KAAKwjD,gBAA+B,cAAQxjD,KAAKqrD,cACjDrrD,KAAK0qD,aAAmB1qD,KAAKimG,mBAC7BjmG,KAAK4rD,WAAmB,aACxB5rD,KAAKqrD,cAAmBrrD,KAAKkmG,iBAC7BlmG,KAAK2qD,iBAAmB,aACxB3qD,KAAKwkG,6BAA+BxkG,KAAKmmG,oBAGzCnmG,KAAK6iD,WAUPjjD,EAAQqmG,mBAAqB,SAASpmE,GACpC7/B,KAAKwlG,gBAAgBhxC,aAAaprC,KAAK8nB,WACvClxC,KAAKwlG,gBAAgBhxC,aAAanrC,GAAG6nB,WACrClxC,KAAKylG,oBAAsBzlG,KAAKwlG,gBAAgB9rC,wBAAwB15D,KAAKurD,qBAAqB1rB,EAAQ9tB,GAAG/R,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAC9G,OAA7BhS,KAAKylG,sBACPzlG,KAAKylG,oBAAoBt0D,SACzBnxC,KAAKujD,kBAAmB,GAE1BvjD,KAAK6iD,WAUPjjD,EAAQsmG,iBAAmB,SAAS58F,GAClC,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OACJ,QAA7BlsB,KAAKylG,qBAA6Dj/F,SAA7BxG,KAAKylG,sBAC5CzlG,KAAKylG,oBAAoB1zF,EAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GAC/D/R,KAAKylG,oBAAoBzzF,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAEjEhS,KAAK6iD,WASPjjD,EAAQumG,oBAAsB,SAAStmE,GACrC,GAAIumE,GAAUpmG,KAAK4qD,WAAW/qB,EACd,QAAZumE,GACqD,GAAnDpmG,KAAKwlG,gBAAgBhxC,aAAaprC,KAAKiqB,WACzCrzC,KAAKwlG,gBAAgB3rC,uBACrB75D,KAAKqmG,UAAUD,EAAQ/lG,GAAIL,KAAKwlG,gBAAgBn8E,GAAGhpB,IACnDL,KAAKwlG,gBAAgBhxC,aAAaprC,KAAK8nB,YAEY,GAAjDlxC,KAAKwlG,gBAAgBhxC,aAAanrC,GAAGgqB,WACvCrzC,KAAKwlG,gBAAgB3rC,uBACrB75D,KAAKqmG,UAAUrmG,KAAKwlG,gBAAgBp8E,KAAK/oB,GAAI+lG,EAAQ/lG,IACrDL,KAAKwlG,gBAAgBhxC,aAAanrC,GAAG6nB,aAIvClxC,KAAKwlG,gBAAgB3rC,uBAEvB75D,KAAKujD,kBAAmB,EACxBvjD,KAAK6iD,WASPjjD,EAAQmmG,eAAiB,SAASlmE,GAChC,GAAoC,GAAhC7/B,KAAK0jG,wBAA8B,CACrC,GAAI59C,GAAO9lD,KAAK4qD,WAAW/qB,EAE3B,IAAY,MAARimB,EACF,GAAIA,EAAKuW,YAAc,EACrBiqC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAAyB,qBAElE,CACHrkC,KAAK+qD,cAAcjF,GAAK,EACxB,IAAIygD,GAAevmG,KAAKkvD,QAAiB,QAAS,KAGlDq3C,GAAyB,WAAI,GAAIhjG,IAAMlD,GAAG,oBAAoBL,KAAK0hD,UACnE,IAAI8kD,GAAaD,EAAyB,UAC1CC,GAAWz0F,EAAI+zC,EAAK/zC,EACpBy0F,EAAWx0F,EAAI8zC,EAAK9zC,EAGpBhS,KAAK89C,MAAsB,eAAI,GAAI16C,IAAM/C,GAAG,iBAAiB+oB,KAAK08B,EAAKzlD,GAAGgpB,GAAGm9E,EAAWnmG,IAAKL,KAAMA,KAAK0hD,UACxG,IAAI+kD,GAAiBzmG,KAAK89C,MAAsB,cAChD2oD,GAAer9E,KAAO08B,EACtB2gD,EAAe94C,WAAY,EAC3B84C,EAAeh4F,QAAQqyC,cAAgBpyC,SAAS,EAC5CqyC,SAAS,EACTj6C,KAAM,aACNk6C,UAAW,IAEfylD,EAAepzD,UAAW,EAC1BozD,EAAep9E,GAAKm9E,EAEpBxmG,KAAKwjD,gBAA+B,cAAIxjD,KAAKqrD,cAC7CrrD,KAAKqrD,cAAgB,SAAS/hD,GAC5B,GAAIu2B,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,QACjCu6E,EAAiBzmG,KAAK89C,MAAsB,cAChD2oD,GAAep9E,GAAGtX,EAAI/R,KAAKurD,qBAAqB1rB,EAAQ9tB,GACxD00F,EAAep9E,GAAGrX,EAAIhS,KAAKyrD,qBAAqB5rB,EAAQ7tB,IAG1DhS,KAAK8kD,QAAS,EACd9kD,KAAK4P,WAMbhQ,EAAQomG,eAAiB,SAAS18F,GAChC,GAAoC,GAAhCtJ,KAAK0jG,wBAA8B,CACrC,GAAI7jE,GAAU7/B,KAAKuqD,YAAYjhD,EAAM4iB,OAErClsB,MAAKqrD,cAAgBrrD,KAAKwjD,gBAA+B,oBAClDxjD,MAAKwjD,gBAA+B,aAG3C,IAAIkjD,GAAgB1mG,KAAK89C,MAAsB,eAAE6V,aAG1C3zD,MAAK89C,MAAsB,qBAC3B99C,MAAKkvD,QAAiB,QAAS,MAAc,iBAC7ClvD,MAAKkvD,QAAiB,QAAS,MAAiB,aAEvD,IAAIpJ,GAAO9lD,KAAK4qD,WAAW/qB,EACf,OAARimB,IACEA,EAAKuW,YAAc,EACrBiqC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAAyB,kBAGrErkC,KAAK2mG,YAAYD,EAAc5gD,EAAKzlD,IACpCL,KAAKmnD,0BAGTnnD,KAAKu+F,iBAQT3+F,EAAQkmG,SAAW,WACjB,GAAI9lG,KAAK+jG,qBAAwC,GAAjB/jG,KAAKkoD,SAAkB,CACrD,GAAIg7C,GAAiBljG,KAAKijG,yBAAyBjjG,KAAKikD,iBACpD2iD,GAAevmG,GAAGM,EAAKqE,aAAa+M,EAAEmxF,EAAez7F,KAAKuK,EAAEkxF,EAAev7F,IAAI8gB,MAAM,MAAM+pC,gBAAe,EAAKC,gBAAe,EAClI,IAAIzyD,KAAK28C,iBAAiB1pC,IAAK,CAC7B,GAAwC,GAApCjT,KAAK28C,iBAAiB1pC,IAAItN,OAU5B,KAAM,IAAI/B,OAAM,sEAThB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiB1pC,IAAI2zF,EAAa,SAASC,GAC9C1yF,EAAGiwC,UAAUnxC,IAAI4zF,GACjB1yF,EAAGgzC,wBACHhzC,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAWP5P,MAAKokD,UAAUnxC,IAAI2zF,GACnB5mG,KAAKmnD,wBACLnnD,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAWXhQ,EAAQ+mG,YAAc,SAASG,EAAaC,GAC1C,GAAqB,GAAjB/mG,KAAKkoD,SAAkB,CACzB,GAAI0+C,IAAex9E,KAAK09E,EAAcz9E,GAAG09E,EACzC,IAAI/mG,KAAK28C,iBAAiBG,QAAS,CACjC,GAA4C,GAAxC98C,KAAK28C,iBAAiBG,QAAQn3C,OAShC,KAAM,IAAI/B,OAAM,0EARhB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBG,QAAQ8pD,EAAa,SAASC,GAClD1yF,EAAGkwC,UAAUpxC,IAAI4zF,GACjB1yF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAUP5P,MAAKqkD,UAAUpxC,IAAI2zF,GACnB5mG,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAUXhQ,EAAQymG,UAAY,SAASS,EAAaC,GACxC,GAAqB,GAAjB/mG,KAAKkoD,SAAkB,CACzB,GAAI0+C,IAAevmG,GAAIL,KAAKwlG,gBAAgBnlG,GAAI+oB,KAAK09E,EAAcz9E,GAAG09E,EACtE,IAAI/mG,KAAK28C,iBAAiBE,SAAU,CAClC,GAA6C,GAAzC78C,KAAK28C,iBAAiBE,SAASl3C,OASjC,KAAM,IAAI/B,OAAM,wEARhB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBE,SAAS+pD,EAAa,SAASC,GACnD1yF,EAAGkwC,UAAUxvC,OAAOgyF,GACpB1yF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAUP5P,MAAKqkD,UAAUxvC,OAAO+xF,GACtB5mG,KAAK8kD,QAAS,EACd9kD,KAAK4P,UAUXhQ,EAAQgmG,UAAY,WAClB,IAAI5lG,KAAK28C,iBAAiBC,MAAyB,GAAjB58C,KAAKkoD,SA4BrC,KAAM,IAAItkD,OAAM,iDA3BhB,IAAIkiD,GAAO9lD,KAAK2jG,mBACZjxF,GAAQrS,GAAGylD,EAAKzlD,GAClBooB,MAAOq9B,EAAKr9B,MACZxW,MAAO6zC,EAAKr3C,QAAQwD,MACpBmrC,MAAO0I,EAAKr3C,QAAQ2uC,MACpBjyC,OACEgB,WAAW25C,EAAKr3C,QAAQtD,MAAMgB,WAC9BC,OAAO05C,EAAKr3C,QAAQtD,MAAMiB,OAC1BC,WACEF,WAAW25C,EAAKr3C,QAAQtD,MAAMkB,UAAUF,WACxCC,OAAO05C,EAAKr3C,QAAQtD,MAAMkB,UAAUD,SAG1C,IAAyC,GAArCpM,KAAK28C,iBAAiBC,KAAKj3C,OAU7B,KAAM,IAAI/B,OAAM,wEAThB,IAAIuQ,GAAKnU,IACTA,MAAK28C,iBAAiBC,KAAKlqC,EAAM,SAAUm0F,GACzC1yF,EAAGiwC,UAAUvvC,OAAOgyF,GACpB1yF,EAAGgzC,wBACHhzC,EAAG2wC,QAAS,EACZ3wC,EAAGvE,WAoBXhQ,EAAQuqD,gBAAkB,WACxB,IAAKnqD,KAAK+jG,qBAAwC,GAAjB/jG,KAAKkoD,SACpC,GAAKloD,KAAKgkG,sBA4BRsC,MAAMtmG,KAAK0hD,UAAU7c,QAAQ7kC,KAAK0hD,UAAUrd,QAA4B,wBA5BzC,CAC/B,GAAI2iE,GAAgBhnG,KAAK0kG,mBACrBuC,EAAgBjnG,KAAK4kG,kBACzB,IAAI5kG,KAAK28C,iBAAiBI,IAAK,CAC7B,GAAI5oC,GAAKnU,KACL0S,GAAQsqC,MAAOgqD,EAAelpD,MAAOmpD,EACzC,IAAwC,GAApCjnG,KAAK28C,iBAAiBI,IAAIp3C,OAU5B,KAAM,IAAI/B,OAAM,0EAThB5D,MAAK28C,iBAAiBI,IAAIrqC,EAAM,SAAUm0F,GACxC1yF,EAAGkwC,UAAUhuC,OAAOwwF,EAAc/oD,OAClC3pC,EAAGiwC,UAAU/tC,OAAOwwF,EAAc7pD,OAClC7oC,EAAGoqF,eACHpqF,EAAG2wC,QAAS,EACZ3wC,EAAGvE,cAQP5P,MAAKqkD,UAAUhuC,OAAO4wF,GACtBjnG,KAAKokD,UAAU/tC,OAAO2wF,GACtBhnG,KAAKu+F,eACLv+F,KAAK8kD,QAAS,EACd9kD,KAAK4P,WAYT,SAAS/P,EAAQD,EAASM,GAE9B,GACIklD,IADOllD,EAAoB,GACdA,EAAoB,KACjC6D,EAAS7D,EAAoB,GAEjCN,GAAQiuE,iBAAmB,WAEzB,GAA8C,GAA1C7tE,KAAK8hD,kBAAkBC,SAASp8C,OAAa,CAC/C,IAAK,GAAIH,GAAI,EAAGA,EAAIxF,KAAK8hD,kBAAkBC,SAASp8C,OAAQH,IAC1DxF,KAAK8hD,kBAAkBC,SAASv8C,GAAG8N,SAErCtT,MAAK8hD,kBAAkBC,YAGzB/hD,KAAKykG,2BAA6B,aAG9BzkG,KAAKknG,gBAAkBlnG,KAAKknG,eAAwB,SAAKlnG,KAAKknG,eAAwB,QAAEr9F,YAC1F7J,KAAKknG,eAAwB,QAAEr9F,WAAWsH,YAAYnR,KAAKknG,eAAwB,UAYvFtnG,EAAQkuE,wBAA0B,WAChC9tE,KAAK6tE,mBAEL7tE,KAAKknG,iBACL,IAAIA,IAAkB,KAAK,OAAO,OAAO,QAAQ,SAAS,UAAU,eAChEC,GAAwB,UAAU,YAAY,YAAY,aAAa,UAAU,WAAW,cAEhGnnG,MAAKknG,eAAwB,QAAI31F,SAASM,cAAc,OACxD7R,KAAKsf,MAAM7N,YAAYzR,KAAKknG,eAAwB,QAEpD,KAAK,GAAI1hG,GAAI,EAAGA,EAAI0hG,EAAevhG,OAAQH,IAAK,CAC9CxF,KAAKknG,eAAeA,EAAe1hG,IAAM+L,SAASM,cAAc,OAChE7R,KAAKknG,eAAeA,EAAe1hG,IAAIqC,UAAY,sBAAwBq/F,EAAe1hG,GAC1FxF,KAAKknG,eAAwB,QAAEz1F,YAAYzR,KAAKknG,eAAeA,EAAe1hG,IAE9E,IAAI1B,GAAS,GAAIC,GAAO/D,KAAKknG,eAAeA,EAAe1hG,KAAMwoE,iBAAiB,GAClF5oB,GAAWkE,QAAQxlD,EAAQ9D,KAAKmnG,EAAqB3hG,IAAIsvB,KAAK90B,OAC9DolD,EAAW6hB,UAAUnjE,EAAQ9D,KAAK+rD,WAAWj3B,KAAK90B,OAElDA,KAAK8hD,kBAAkBE,KAAKh6C,KAAKlE,GAGnC9D,KAAKykG,2BAA6BzkG,KAAKonG,cAEvCpnG,KAAK8hD,kBAAkBC,SAAW/hD,KAAK8hD,kBAAkBE,MAS3DpiD,EAAQynG,YAAc,SAAS/9F,GAC7BtJ,KAAKilD,YAAYn1C,SAAS,MAC1BxG,EAAMq8B,mBAQR/lC,EAAQwnG,cAAgB,WACtBpnG,KAAK8pD,eACL9pD,KAAK2pD,eACL3pD,KAAKiqD,aAYPrqD,EAAQ8pD,QAAU,SAASpgD,GACzBtJ,KAAK+iD,WAAa/iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EAChDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQgqD,UAAY,SAAStgD,GAC3BtJ,KAAK+iD,YAAc/iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EACjDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQiqD,UAAY,SAASvgD,GAC3BtJ,KAAK8iD,WAAa9iD,KAAK0hD,UAAUrB,SAASC,MAAMvuC,EAChD/R,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQmqD,WAAa,SAASzgD,GAC5BtJ,KAAK8iD,YAAc9iD,KAAK0hD,UAAUrB,SAASC,MAAMtuC,EACjDhS,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQoqD,QAAU,SAAS1gD,GACzBtJ,KAAKgjD,cAAgBhjD,KAAK0hD,UAAUrB,SAASC,MAAMpgB,KACnDlgC,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQsqD,SAAW,SAAS5gD,GAC1BtJ,KAAKgjD,eAAiBhjD,KAAK0hD,UAAUrB,SAASC,MAAMpgB,KACpDlgC,KAAK4P,QACLtG,EAAMD,kBAQRzJ,EAAQqqD,UAAY,SAAS3gD,GAC3BtJ,KAAKgjD,cAAgB,EACrB15C,GAASA,EAAMD,kBAQjBzJ,EAAQ+pD,aAAe,SAASrgD,GAC9BtJ,KAAK+iD,WAAa,EAClBz5C,GAASA,EAAMD,kBAQjBzJ,EAAQkqD,aAAe,SAASxgD,GAC9BtJ,KAAK8iD,WAAa,EAClBx5C,GAASA,EAAMD,mBAMb,SAASxJ,EAAQD,GAErBA,EAAQgoD,aAAe,WACrB,IAAK,GAAIzB,KAAUnmD,MAAKg9C,MACtB,GAAIh9C,KAAKg9C,MAAMl3C,eAAeqgD,GAAS,CACrC,GAAIL,GAAO9lD,KAAKg9C,MAAMmJ,EACO,IAAzBL,EAAKuV,mBACPvV,EAAKlI,MAAQ,GACbkI,EAAKwV,qBAAsB,KAYnC17D,EAAQolD,yBAA2B,WACjC,GAAiD,GAA7ChlD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAmB1O,KAAK8jD,YAAYn+C,OAAS,EAAG,CAEpF,GACImgD,GAAMK,EADNmhD,EAAU,EAEVC,GAAe,EACfC,GAAiB,CAErB,KAAKrhD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACA,IAAdL,EAAKlI,MACP2pD,GAAe,EAGfC,GAAiB,EAEfF,EAAUxhD,EAAKhI,MAAMn4C,SACvB2hG,EAAUxhD,EAAKhI,MAAMn4C,QAM3B,IAAsB,GAAlB6hG,GAA0C,GAAhBD,EAC5B,KAAM,IAAI3jG,OAAM,wHAQhB5D,MAAKynG,mBAGiB,GAAlBD,IAC8C,WAA5CxnG,KAAK0hD,UAAUjB,mBAAmBG,OACpC5gD,KAAK0nG,iBAAiBJ,GAGtBtnG,KAAK2nG,0BAAyB,GAKlC,IAAIC,GAAe5nG,KAAK6nG,kBAGxB7nG,MAAK8nG,uBAAuBF,GAG5B5nG,KAAK4P,UAYXhQ,EAAQkoG,uBAAyB,SAASF,GACxC,GAAIzhD,GAAQL,CAGZ,KAAK,GAAIlI,KAASgqD,GAChB,GAAIA,EAAa9hG,eAAe83C,GAE9B,IAAKuI,IAAUyhD,GAAahqD,GAAOZ,MAC7B4qD,EAAahqD,GAAOZ,MAAMl3C,eAAeqgD,KAC3CL,EAAO8hD,EAAahqD,GAAOZ,MAAMmJ,GACkB,MAA/CnmD,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UACvF4qB,EAAKqF,SACPrF,EAAK/zC,EAAI61F,EAAahqD,GAAOmqD,OAC7BjiD,EAAKqF,QAAS,EAEdy8C,EAAahqD,GAAOmqD,QAAUH,EAAahqD,GAAO+C,aAIhDmF,EAAKsF,SACPtF,EAAK9zC,EAAI41F,EAAahqD,GAAOmqD,OAC7BjiD,EAAKsF,QAAS,EAEdw8C,EAAahqD,GAAOmqD,QAAUH,EAAahqD,GAAO+C,aAGtD3gD,KAAKgoG,kBAAkBliD,EAAKhI,MAAMgI,EAAKzlD,GAAGunG,EAAa9hD,EAAKlI,OAOpE59C,MAAK6nD,cAUPjoD,EAAQioG,iBAAmB,WACzB,GACI1hD,GAAQL,EAAMlI,EADdgqD,IAKJ,KAAKzhD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKqF,QAAS,EACdrF,EAAKsF,QAAS,EACqC,MAA/CprD,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UAC3F4qB,EAAK9zC,EAAIhS,KAAK0hD,UAAUjB,mBAAmBC,gBAAgBoF,EAAKlI,MAGhEkI,EAAK/zC,EAAI/R,KAAK0hD,UAAUjB,mBAAmBC,gBAAgBoF,EAAKlI,MAEjCp3C,SAA7BohG,EAAa9hD,EAAKlI,SACpBgqD,EAAa9hD,EAAKlI,QAAUquB,OAAQ,EAAGjvB,SAAW+qD,OAAO,EAAGpnD,YAAY,IAE1EinD,EAAa9hD,EAAKlI,OAAOquB,QAAU,EACnC27B,EAAa9hD,EAAKlI,OAAOZ,MAAMmJ,GAAUL,EAK7C,IAAImiD,GAAW,CACf,KAAKrqD,IAASgqD,GACRA,EAAa9hG,eAAe83C,IAC1BqqD,EAAWL,EAAahqD,GAAOquB,SACjCg8B,EAAWL,EAAahqD,GAAOquB,OAMrC,KAAKruB,IAASgqD,GACRA,EAAa9hG,eAAe83C,KAC9BgqD,EAAahqD,GAAO+C,aAAesnD,EAAW,GAAKjoG,KAAK0hD,UAAUjB,mBAAmBE,YACrFinD,EAAahqD,GAAO+C,aAAgBinD,EAAahqD,GAAOquB,OAAS,EACjE27B,EAAahqD,GAAOmqD,OAASH,EAAahqD,GAAO+C,YAAe,IAAOinD,EAAahqD,GAAOquB,OAAS,GAAK27B,EAAahqD,GAAO+C,YAIjI,OAAOinD,IAUThoG,EAAQ8nG,iBAAmB,SAASJ,GAClC,GAAInhD,GAAQL,CAGZ,KAAKK,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACdL,EAAKhI,MAAMn4C,QAAU2hG,IACvBxhD,EAAKlI,MAAQ,GAMnB,KAAKuI,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GACA,GAAdL,EAAKlI,OACP59C,KAAKkoG,UAAU,EAAEpiD,EAAKhI,MAAMgI,EAAKzlD,MAczCT,EAAQ+nG,yBAA2B,WACjC,GAAIxhD,GAAQL,EAAMqiD,EACd5H,EAAW,GAGf4H,GAAYnoG,KAAKg9C,MAAMh9C,KAAK8jD,YAAY,IACxCqkD,EAAUvqD,MAAQ2iD,EAClBvgG,KAAKooG,kBAAkB7H,EAAS4H,EAAUrqD,MAAMqqD,EAAU9nG,GAG1D,KAAK8lD,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBo6C,EAAWz6C,EAAKlI,MAAQ2iD,EAAWz6C,EAAKlI,MAAQ2iD,EAKpD,KAAKp6C,IAAUnmD,MAAKg9C,MACdh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BL,EAAO9lD,KAAKg9C,MAAMmJ,GAClBL,EAAKlI,OAAS2iD,IAepB3gG,EAAQ6nG,iBAAmB,WACzBznG,KAAK0hD,UAAUvC,WAAWzwC,SAAU,EACpC1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,EAC3C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAKmtE,2BACsC,GAAvCntE,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAaC,SAAU,GAExC/gD,KAAK0oD,wBAEL,IAAI8oB,GAASxxE,KAAK0hD,UAAUjB,kBAC5B+wB,GAAO9wB,gBAAkBx7C,KAAK2lB,IAAI2mD,EAAO9wB,kBACjB,MAApB8wB,EAAOt2C,WAAyC,MAApBs2C,EAAOt2C,aACrCs2C,EAAO9wB,iBAAmB,IAGJ,MAApB8wB,EAAOt2C,WAAyC,MAApBs2C,EAAOt2C,UACM,GAAvCl7B,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAah6C,KAAO,YAIM,GAAvC9G,KAAK0hD,UAAUZ,aAAapyC,UAC9B1O,KAAK0hD,UAAUZ,aAAah6C,KAAO,eAgBzClH,EAAQooG,kBAAoB,SAASlqD,EAAOuqD,EAAUT,EAAcU,GAClE,IAAK,GAAI9iG,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAAK,CACrC,GAAI64F,GAAY,IAEdA,GADEvgD,EAAMt4C,GAAGouD,MAAQy0C,EACPvqD,EAAMt4C,GAAG4jB,KAGT00B,EAAMt4C,GAAG6jB,EAIvB,IAAIk/E,IAAY,CACmC,OAA/CvoG,KAAK0hD,UAAUjB,mBAAmBvlB,WAAoE,MAA/Cl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UACvFmjE,EAAUlzC,QAAUkzC,EAAUzgD,MAAQ0qD,IACxCjK,EAAUlzC,QAAS,EACnBkzC,EAAUtsF,EAAI61F,EAAavJ,EAAUzgD,OAAOmqD,OAC5CQ,GAAY,GAIVlK,EAAUjzC,QAAUizC,EAAUzgD,MAAQ0qD,IACxCjK,EAAUjzC,QAAS,EACnBizC,EAAUrsF,EAAI41F,EAAavJ,EAAUzgD,OAAOmqD,OAC5CQ,GAAY,GAIC,GAAbA,IACFX,EAAavJ,EAAUzgD,OAAOmqD,QAAUH,EAAavJ,EAAUzgD,OAAO+C,YAClE09C,EAAUvgD,MAAMn4C,OAAS,GAC3B3F,KAAKgoG,kBAAkB3J,EAAUvgD,MAAMugD,EAAUh+F,GAAGunG,EAAavJ,EAAUzgD,UAenFh+C,EAAQsoG,UAAY,SAAStqD,EAAOE,EAAOuqD,GACzC,IAAK,GAAI7iG,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAAK,CACrC,GAAI64F,GAAY,IAEdA,GADEvgD,EAAMt4C,GAAGouD,MAAQy0C,EACPvqD,EAAMt4C,GAAG4jB,KAGT00B,EAAMt4C,GAAG6jB,IAEA,IAAnBg1E,EAAUzgD,OAAeygD,EAAUzgD,MAAQA,KAC7CygD,EAAUzgD,MAAQA,EACdygD,EAAUvgD,MAAMn4C,OAAS,GAC3B3F,KAAKkoG,UAAUtqD,EAAM,EAAGygD,EAAUvgD,MAAOugD,EAAUh+F,OAe3DT,EAAQwoG,kBAAoB,SAASxqD,EAAOE,EAAOuqD,GACjDroG,KAAKg9C,MAAMqrD,GAAU/sC,qBAAsB,CAE3C,KAAK,GADD+iC,GAAWnjE,EACN11B,EAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IAChC01B,EAAY,EACR4iB,EAAMt4C,GAAGouD,MAAQy0C,GACnBhK,EAAYvgD,EAAMt4C,GAAG4jB,KACrB8R,EAAY,IAGZmjE,EAAYvgD,EAAMt4C,GAAG6jB,GAEA,IAAnBg1E,EAAUzgD,QACZygD,EAAUzgD,MAAQA,EAAQ1iB,EAI9B,KAAK,GAAI11B,GAAI,EAAGA,EAAIs4C,EAAMn4C,OAAQH,IACA64F,EAA5BvgD,EAAMt4C,GAAGouD,MAAQy0C,EAAuBvqD,EAAMt4C,GAAG4jB,KACnC00B,EAAMt4C,GAAG6jB,GAEvBg1E,EAAUvgD,MAAMn4C,OAAS,GAAK04F,EAAU/iC,uBAAwB,GAClEt7D,KAAKooG,kBAAkB/J,EAAUzgD,MAAOygD,EAAUvgD,MAAOugD,EAAUh+F,KAWzET,EAAQ4oG,cAAgB,WACtB,IAAK,GAAIriD,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAKg9C,MAAMmJ,GAAQgF,QAAS,EAC5BnrD,KAAKg9C,MAAMmJ,GAAQiF,QAAS,KAQ9B,SAASvrD,EAAQD,EAASM,GAkgB9B,QAASuoG,KACPzoG,KAAK0hD,UAAUZ,aAAapyC,SAAW1O,KAAK0hD,UAAUZ,aAAapyC,OACnE,IAAIg6F,GAAqBn3F,SAASo3F,eAAe,qBACCD,GAAmBz7F,MAAMd,WAAhC,GAAvCnM,KAAK0hD,UAAUZ,aAAapyC,QAAwD,UACR,UAEhF1O,KAAK0oD,wBAAuB,GAO9B,QAASkgD,KACP,IAAK,GAAIziD,KAAUnmD,MAAK4jD,iBAClB5jD,KAAK4jD,iBAAiB99C,eAAeqgD,KACvCnmD,KAAK4jD,iBAAiBuC,GAAQsV,GAAK,EAAIz7D,KAAK4jD,iBAAiBuC,GAAQuV,GAAK,EAC1E17D,KAAK4jD,iBAAiBuC,GAAQoV,GAAK,EAAIv7D,KAAK4jD,iBAAiBuC,GAAQqV,GAAK,EAG7B,IAA7Cx7D,KAAK0hD,UAAUjB,mBAAmB/xC,SACpC1O,KAAKglD,2BACL6jD,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,8CAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,0BAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,0BAC7C6oG,EAAiBtoG,KAAKP,KAAM,aAAc,EAAG,wBAC7C6oG,EAAiBtoG,KAAKP,KAAM,eAAgB,EAAG,oBAG/CA,KAAK08F,kBAEP18F,KAAK8kD,QAAS,EACd9kD,KAAK4P,QAMP,QAASk5F,KACP,GAAIr6F,GAAU,gDACVs6F,KACAC,EAAez3F,SAASo3F,eAAe,wBACvCM,EAAe13F,SAASo3F,eAAe,uBAC3C,IAA4B,GAAxBK,EAAaE,QAAiB,CAMhC,GALIlpG,KAAK0hD,UAAUlD,QAAQC,UAAUE,uBAAyB3+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUE,uBAAwBoqD,EAAgB/gG,KAAK,0BAA4BhI,KAAK0hD,UAAUlD,QAAQC,UAAUE,uBAC3M3+C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUG,gBAAyCmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBAC1L5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUI,cAA2CkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACxL7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUK,gBAAyCiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBAC1L9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQC,UAAUM,SAAgDgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACzJ,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,EAAU,kBACVA,GAAW,wBACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,KAETzO,KAAK0hD,UAAUZ,aAAapyC,SAAW1O,KAAKmpG,gBAAgBroD,aAAapyC,UAC7C,GAA1Bq6F,EAAgBpjG,OAAc8I,EAAU,kBACtCA,GAAW,KACjBA,GAAW,iBAAmBzO,KAAK0hD,UAAUZ,aAAapyC,SAE7C,iDAAXD,IACFA,GAAW,UAGV,IAA4B,GAAxBw6F,EAAaC,QAAiB,CAQrC,GAPAz6F,EAAU,kBACVA,GAAW,wCACPzO,KAAK0hD,UAAUlD,QAAQQ,UAAUC,cAAgBj/C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUC,cAAgB8pD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQQ,UAAUC,cACjLj/C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUJ,gBAAwBmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBACzK5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUH,cAA0BkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACvK7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUF,gBAAwBiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBACzK9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQQ,UAAUD,SAA+BgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACxI,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,GAAW,gBACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,KAEiB,GAA1Bs6F,EAAgBpjG,SAAc8I,GAAW,KACzCzO,KAAK0hD,UAAUZ,cAAgB9gD,KAAKmpG,gBAAgBroD,eACtDryC,GAAW,mBAAqBzO,KAAK0hD,UAAUZ,cAEjDryC,GAAW,SAER,CAOH,GANAA,EAAU,kBACNzO,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,cAAgBj/C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBD,cAAgB8pD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,cACrNj/C,KAAK0hD,UAAUlD,QAAQI,gBAAkB5+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBN,gBAAwBmqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQI,gBACrL5+C,KAAK0hD,UAAUlD,QAAQK,cAAgB7+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBL,cAA0BkqD,EAAgB/gG,KAAK,iBAAmBhI,KAAK0hD,UAAUlD,QAAQK,cACnL7+C,KAAK0hD,UAAUlD,QAAQM,gBAAkB9+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBJ,gBAAwBiqD,EAAgB/gG,KAAK,mBAAqBhI,KAAK0hD,UAAUlD,QAAQM,gBACrL9+C,KAAK0hD,UAAUlD,QAAQO,SAAW/+C,KAAKmpG,gBAAgB3qD,QAAQU,sBAAsBH,SAA+BgqD,EAAgB/gG,KAAK,YAAchI,KAAK0hD,UAAUlD,QAAQO,SACpJ,GAA1BgqD,EAAgBpjG,OAAa,CAC/B8I,GAAW,oCACX,KAAK,GAAIjJ,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,MAOb,GALAA,GAAW,wBACXs6F,KACI/oG,KAAK0hD,UAAUjB,mBAAmBvlB,WAAal7B,KAAKmpG,gBAAgB1oD,mBAAmBvlB,WAAkC6tE,EAAgB/gG,KAAK,cAAgBhI,KAAK0hD,UAAUjB,mBAAmBvlB,WAChMh2B,KAAK2lB,IAAI7qB,KAAK0hD,UAAUjB,mBAAmBC,kBAAoB1gD,KAAKmpG,gBAAgB1oD,mBAAmBC,iBAAkBqoD,EAAgB/gG,KAAK,oBAAsBhI,KAAK0hD,UAAUjB,mBAAmBC,iBACtM1gD,KAAK0hD,UAAUjB,mBAAmBE,aAAe3gD,KAAKmpG,gBAAgB1oD,mBAAmBE,aAAgCooD,EAAgB/gG,KAAK,gBAAkBhI,KAAK0hD,UAAUjB,mBAAmBE,aACxK,GAA1BooD,EAAgBpjG,OAAa,CAC/B,IAAK,GAAIH,GAAI,EAAGA,EAAIujG,EAAgBpjG,OAAQH,IAC1CiJ,GAAWs6F,EAAgBvjG,GACvBA,EAAIujG,EAAgBpjG,OAAS,IAC/B8I,GAAW,KAGfA,IAAW,QAGXA,IAAW,eAEbA,IAAW,KAIbzO,KAAKopG,WAAWnlF,UAAYxV,EAO9B,QAAS46F,KACP,GAAIl0F,IAAO,iBAAkB,gBAAiB,iBAC1Cm0F,EAAc/3F,SAASg4F,cAAc,6CAA6CliG,MAClFmiG,EAAU,SAAWF,EAAc,SACnCG,EAAQl4F,SAASo3F,eAAea,EACpCC,GAAMx8F,MAAM26B,QAAU,OACtB,KAAK,GAAIpiC,GAAI,EAAGA,EAAI2P,EAAIxP,OAAQH,IAC1B2P,EAAI3P,IAAMgkG,IACZC,EAAQl4F,SAASo3F,eAAexzF,EAAI3P,IACpCikG,EAAMx8F,MAAM26B,QAAU,OAG1B5nC,MAAKwoG,gBACc,KAAfc,GACFtpG,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,GAErB,KAAf46F,EAC0C,GAA7CtpG,KAAK0hD,UAAUjB,mBAAmB/xC,UACpC1O,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,EAC3C1O,KAAK0hD,UAAUZ,aAAapyC,SAAU,EACtC1O,KAAKglD,6BAIPhlD,KAAK0hD,UAAUjB,mBAAmB/xC,SAAU,EAC5C1O,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SAAU,EACvD1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAU,GAE7C1O,KAAKmtE,0BACL,IAAIu7B,GAAqBn3F,SAASo3F,eAAe,qBACCD,GAAmBz7F,MAAMd,WAAhC,GAAvCnM,KAAK0hD,UAAUZ,aAAapyC,QAAwD,UACR,UAChF1O,KAAK8kD,QAAS,EACd9kD,KAAK4P,QAWP,QAASi5F,GAAkBxoG,EAAGgN,EAAIq8F,GAChC,GAAIC,GAAUtpG,EAAK,SACfupG,EAAar4F,SAASo3F,eAAetoG,GAAIgH,KAEzCpB,OAAMC,QAAQmH,IAChBkE,SAASo3F,eAAegB,GAAStiG,MAAQgG,EAAIzC,SAASg/F,IACtD5pG,KAAK6pG,yBAAyBH,EAAsBr8F,EAAIzC,SAASg/F,OAGjEr4F,SAASo3F,eAAegB,GAAStiG,MAAQuD,SAASyC,GAAOgY,WAAWukF,GACpE5pG,KAAK6pG,yBAAyBH,EAAuB9+F,SAASyC,GAAOgY,WAAWukF,MAGrD,gCAAzBF,GACuB,sCAAzBA,GACyB,kCAAzBA,IACA1pG,KAAKglD,2BAEPhlD,KAAK8kD,QAAS,EACd9kD,KAAK4P,QA7sBP,GAAIjP,GAAOT,EAAoB,GAC3B4pG,EAAiB5pG,EAAoB,IACrC6pG,EAA4B7pG,EAAoB,IAChD8pG,EAAiB9pG,EAAoB,GAOzCN,GAAQqqG,iBAAmB,WACzBjqG,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SAAW1O,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,QAC7E1O,KAAKmtE,2BACLntE,KAAK8kD,QAAS,EACd9kD,KAAK4P,SASPhQ,EAAQutE,yBAA2B,WAEe,GAA5CntE,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,SACnC1O,KAAKktE,YAAY48B,GACjB9pG,KAAKktE,YAAY68B,GAEjB/pG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eACzE5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aACvE7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eACzE9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAElE/+C,KAAK+sE,WAAWi9B,IAE+C,GAAxDhqG,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,SACpD1O,KAAKktE,YAAY88B,GACjBhqG,KAAKktE,YAAY48B,GAEjB9pG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eACrF5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aACnF7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eACrF9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAE9E/+C,KAAK+sE,WAAWg9B,KAGhB/pG,KAAKktE,YAAY88B,GACjBhqG,KAAKktE,YAAY68B,GACjB/pG,KAAKkqG,cAAgB1jG,OAErBxG,KAAK0hD,UAAUlD,QAAQI,eAAiB5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eACzE5+C,KAAK0hD,UAAUlD,QAAQK,aAAe7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aACvE7+C,KAAK0hD,UAAUlD,QAAQM,eAAiB9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eACzE9+C,KAAK0hD,UAAUlD,QAAQO,QAAU/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAElE/+C,KAAK+sE,WAAW+8B,KAUpBlqG,EAAQuqG,4BAA8B,WAEL,GAA3BnqG,KAAK8jD,YAAYn+C,OACnB3F,KAAKg9C,MAAMh9C,KAAK8jD,YAAY,IAAIsa,UAAU,EAAG,IAIzCp+D,KAAK8jD,YAAYn+C,OAAS3F,KAAK0hD,UAAUvC,WAAWE,kBAAyD,GAArCr/C,KAAK0hD,UAAUvC,WAAWzwC,SACpG1O,KAAKm8F,aAAan8F,KAAK0hD,UAAUvC,WAAWG,eAAe,GAI7Dt/C,KAAKoqG,qBAUTxqG,EAAQwqG,iBAAmB,WAKzBpqG,KAAKqqG,gCACLrqG,KAAKsqG,uBAEDtqG,KAAK0hD,UAAUlD,QAAQM,eAAiB,IACC,GAAvC9+C,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAC7E/gD,KAAKuqG,oCAGuD,GAAxDvqG,KAAK0hD,UAAUlD,QAAQU,sBAAsBxwC,QAC/C1O,KAAKwqG,qCAGLxqG,KAAKyqG,2BAeb7qG,EAAQyuD,wBAA0B,WAChC,GAA2C,GAAvCruD,KAAK0hD,UAAUZ,aAAapyC,SAA0D,GAAvC1O,KAAK0hD,UAAUZ,aAAaC,QAAiB,CAC9F/gD,KAAK4jD,oBACL5jD,KAAK6jD,yBAEL,KAAK,GAAIsC,KAAUnmD,MAAKg9C,MAClBh9C,KAAKg9C,MAAMl3C,eAAeqgD,KAC5BnmD,KAAK4jD,iBAAiBuC,GAAUnmD,KAAKg9C,MAAMmJ,GAG/C,IAAIogD,GAAevmG,KAAKkvD,QAAiB,QAAS,KAClD,KAAK,GAAIw7C,KAAiBnE,GACpBA,EAAazgG,eAAe4kG,KAC1B1qG,KAAK89C,MAAMh4C,eAAeygG,EAAamE,GAAez4C,cACxDjyD,KAAK4jD,iBAAiB8mD,GAAiBnE,EAAamE,GAGpDnE,EAAamE,GAAetsC,UAAU,EAAG,GAK/C,KAAK,GAAIlX,KAAOlnD,MAAK4jD,iBACf5jD,KAAK4jD,iBAAiB99C,eAAeohD,IACvClnD,KAAK6jD,uBAAuB77C,KAAKk/C,OAKrClnD,MAAK4jD,iBAAmB5jD,KAAKg9C,MAC7Bh9C,KAAK6jD,uBAAyB7jD,KAAK8jD,aAUvClkD,EAAQyqG,8BAAgC,WACtC,GAAIzrF,GAAIC,EAAI8G,EAAUmgC,EAAMtgD,EACxBw3C,EAAQh9C,KAAK4jD,iBACb+mD,EAAU3qG,KAAK0hD,UAAUlD,QAAQI,eACjCgsD,EAAe,CAEnB,KAAKplG,EAAI,EAAGA,EAAIxF,KAAK6jD,uBAAuBl+C,OAAQH,IAClDsgD,EAAO9I,EAAMh9C,KAAK6jD,uBAAuBr+C,IACzCsgD,EAAK/G,QAAU/+C,KAAK0hD,UAAUlD,QAAQO,QAEhB,WAAlB/+C,KAAK88F,WAAqC,GAAX6N,GACjC/rF,GAAMknC,EAAK/zC,EACX8M,GAAMinC,EAAK9zC,EACX2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpC+rF,EAA4B,GAAZjlF,EAAiB,EAAKglF,EAAUhlF,EAChDmgC,EAAKyV,GAAK38C,EAAKgsF,EACf9kD,EAAK0V,GAAK38C,EAAK+rF,IAGf9kD,EAAKyV,GAAK,EACVzV,EAAK0V,GAAK,IAahB57D,EAAQ6qG,uBAAyB,WAC/B,GAAII,GAAYn9C,EAAMV,EAClBpuC,EAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,EAC7Bm4B,EAAQ99C,KAAK89C,KAGjB,KAAKkP,IAAUlP,GACTA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,UACzEk3C,EAAan9C,EAAKlP,QAAQK,aAE1BgsD,IAAen9C,EAAKrkC,GAAGgzC,YAAc3O,EAAKtkC,KAAKizC,YAAc,GAAKr8D,KAAK0hD,UAAUvC,WAAWY,WAE5FnhC,EAAM8uC,EAAKtkC,KAAKrX,EAAI27C,EAAKrkC,GAAGtX,EAC5B8M,EAAM6uC,EAAKtkC,KAAKpX,EAAI07C,EAAKrkC,GAAGrX,EAC5B2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAEVp9C,EAAKtkC,KAAKmyC,IAAMA,EAChB7N,EAAKtkC,KAAKoyC,IAAMA,EAChB9N,EAAKrkC,GAAGkyC,IAAMA,EACd7N,EAAKrkC,GAAGmyC,IAAMA,KAexB57D,EAAQ2qG,kCAAoC,WAC1C,GAAIM,GAAYn9C,EAAMV,EAAQ+9C,EAC1BjtD,EAAQ99C,KAAK89C,KAGjB,KAAKkP,IAAUlP,GACb,GAAIA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,SACzD,MAAZjG,EAAKuB,KAAa,CACpB,GAAI+7C,GAAQt9C,EAAKrkC,GACb4hF,EAAQv9C,EAAKuB,IACbi8C,EAAQx9C,EAAKtkC,IAEjByhF,GAAan9C,EAAKlP,QAAQK,aAE1BksD,EAAsBC,EAAM3uC,YAAc6uC,EAAM7uC,YAAc,EAG9DwuC,GAAcE,EAAsB/qG,KAAK0hD,UAAUvC,WAAWY,WAC9D//C,KAAKmrG,sBAAsBH,EAAOC,EAAO,GAAMJ,GAC/C7qG,KAAKmrG,sBAAsBF,EAAOC,EAAO,GAAML,KAiB3DjrG,EAAQurG,sBAAwB,SAAUH,EAAOC,EAAOJ,GACtD,GAAIjsF,GAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,CAEjC/G,GAAMosF,EAAMj5F,EAAIk5F,EAAMl5F,EACtB8M,EAAMmsF,EAAMh5F,EAAIi5F,EAAMj5F,EACtB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAEVE,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,GAId57D,EAAQwqD,6BAA+B,WACrC,GAAkC5jD,SAA9BxG,KAAKorG,qBAAoC,CAC3C,KAAOprG,KAAKorG,qBAAqB1nF,iBAC/B1jB,KAAKorG,qBAAqBj6F,YAAYnR,KAAKorG,qBAAqBznF,WAGlE3jB,MAAKorG,qBAAqBvhG,WAAWsH,YAAYnR,KAAKorG,sBACtDprG,KAAKorG,qBAAuB5kG,SAQhC5G,EAAQwtE,0BAA4B,WAClC,GAAkC5mE,SAA9BxG,KAAKorG,qBAAoC,CAC3CprG,KAAKmpG,mBACLxoG,EAAK8F,WAAWzG,KAAKmpG,gBAAgBnpG,KAAK0hD,UAE1C,IAAI2pD,IAAgC,KAAM,KAAM,KAAM,KACtDrrG,MAAKorG,qBAAuB75F,SAASM,cAAc,OACnD7R,KAAKorG,qBAAqBvjG,UAAY,uBACtC7H,KAAKorG,qBAAqBnnF,UAAY,onBAW2E,GAAKjkB,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAyB,wGAA2G,GAAK3+C,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAyB,4JAGpP3+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eAAiB,wFAA0F5+C,KAAK0hD,UAAUlD,QAAQC,UAAUG,eAAiB,2JAG/L5+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aAAe,sFAAwF7+C,KAAK0hD,UAAUlD,QAAQC,UAAUI,aAAe,6JAGtL7+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eAAiB,0FAA4F9+C,KAAK0hD,UAAUlD,QAAQC,UAAUK,eAAiB,sJAGvM9+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAAU,4FAA8F/+C,KAAK0hD,UAAUlD,QAAQC,UAAUM,QAAU,sPAM/K/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAAe,kGAAoGj/C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAAe,2JAGnMj/C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eAAiB,uFAAyF5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUJ,eAAiB,0JAG9L5+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aAAe,qFAAuF7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUH,aAAe,4JAGrL7+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eAAiB,yFAA2F9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUF,eAAiB,qJAGtM9+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAAU,2FAA6F/+C,KAAK0hD,UAAUlD,QAAQQ,UAAUD,QAAU,oQAM9K/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,aAAe,kGAAoGj/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,aAAe,2JAG3Nj/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eAAiB,uFAAyF5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBN,eAAiB,0JAGtN5+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aAAe,qFAAuF7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBL,aAAe,4JAG7M7+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eAAiB,yFAA2F9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBJ,eAAiB,qJAG9N9+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAAU,2FAA6F/+C,KAAK0hD,UAAUlD,QAAQU,sBAAsBH,QAAU,uJAG3MssD,EAA6B1kG,QAAQ3G,KAAK0hD,UAAUjB,mBAAmBvlB,WAAa,0FAA4Fl7B,KAAK0hD,UAAUjB,mBAAmBvlB,UAAY,oKAGtNl7B,KAAK0hD,UAAUjB,mBAAmBC,gBAAkB,yFAA2F1gD,KAAK0hD,UAAUjB,mBAAmBC,gBAAkB,6JAGvM1gD,KAAK0hD,UAAUjB,mBAAmBE,YAAc,wFAA0F3gD,KAAK0hD,UAAUjB,mBAAmBE,YAAc,odAU9R3gD,KAAKyZ,iBAAiB6xF,cAAc15F,aAAa5R,KAAKorG,qBAAsBprG,KAAKyZ,kBACjFzZ,KAAKopG,WAAa73F,SAASM,cAAc,OACzC7R,KAAKopG,WAAWn8F,MAAMswC,SAAW,OACjCv9C,KAAKopG,WAAWn8F,MAAMszD,WAAa,UACnCvgE,KAAKyZ,iBAAiB6xF,cAAc15F,aAAa5R,KAAKopG,WAAYppG,KAAKyZ,iBAEvE;GAAI8xF,EACJA,GAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,GAAI,2CACvEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,0BACtEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,0BACtEurG,EAAeh6F,SAASo3F,eAAe,eACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,cAAe,EAAG,wBACtEurG,EAAeh6F,SAASo3F,eAAe,iBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,gBAAiB,EAAG,mBAExEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,kCACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,wBACrEurG,EAAeh6F,SAASo3F,eAAe,gBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,eAAgB,EAAG,mBAEvEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,8CACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,0BACrEurG,EAAeh6F,SAASo3F,eAAe,cACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,aAAc,EAAG,wBACrEurG,EAAeh6F,SAASo3F,eAAe,gBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,eAAgB,EAAG,mBACvEurG,EAAeh6F,SAASo3F,eAAe,qBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,oBAAqBqrG,EAA8B,gCACvGE,EAAeh6F,SAASo3F,eAAe,kBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,iBAAkB,EAAG,sCACzEurG,EAAeh6F,SAASo3F,eAAe,iBACvC4C,EAAa1iF,SAAWggF,EAAiB/zE,KAAK90B,KAAM,gBAAiB,EAAG,iCAExE,IAAIgpG,GAAez3F,SAASo3F,eAAe,wBACvCM,EAAe13F,SAASo3F,eAAe,wBACvC6C,EAAej6F,SAASo3F,eAAe,uBAC3CM,GAAaC,SAAU,EACnBlpG,KAAK0hD,UAAUlD,QAAQC,UAAU/vC,UACnCs6F,EAAaE,SAAU,GAErBlpG,KAAK0hD,UAAUjB,mBAAmB/xC,UACpC88F,EAAatC,SAAU,EAGzB,IAAIR,GAAqBn3F,SAASo3F,eAAe,sBAC7C8C,EAAwBl6F,SAASo3F,eAAe,yBAChD+C,EAAwBn6F,SAASo3F,eAAe,wBAEpDD,GAAmB12E,QAAUy2E,EAAwB3zE,KAAK90B,MAC1DyrG,EAAsBz5E,QAAU42E,EAAqB9zE,KAAK90B,MAC1D0rG,EAAsB15E,QAAU82E,EAAqBh0E,KAAK90B,MAExD0oG,EAAmBz7F,MAAMd,WADQ,GAA/BnM,KAAK0hD,UAAUZ,cAA8D,GAAtC9gD,KAAK0hD,UAAUiqD,oBAClB,UAGA,UAIxCtC,EAAqBtxF,MAAM/X,MAE3BgpG,EAAangF,SAAWwgF,EAAqBv0E,KAAK90B,MAClDipG,EAAapgF,SAAWwgF,EAAqBv0E,KAAK90B,MAClDwrG,EAAa3iF,SAAWwgF,EAAqBv0E,KAAK90B,QAWtDJ,EAAQiqG,yBAA2B,SAAUH,EAAuBriG,GAClE,GAAIukG,GAAYlC,EAAsB3hG,MAAM,IACpB,IAApB6jG,EAAUjmG,OACZ3F,KAAK0hD,UAAUkqD,EAAU,IAAMvkG,EAEJ,GAApBukG,EAAUjmG,OACjB3F,KAAK0hD,UAAUkqD,EAAU,IAAIA,EAAU,IAAMvkG,EAElB,GAApBukG,EAAUjmG,SACjB3F,KAAK0hD,UAAUkqD,EAAU,IAAIA,EAAU,IAAIA,EAAU,IAAMvkG,KA6N3D,SAASxH,GAEb,QAASgsG,GAAeC,GACvB,KAAM,IAAIloG,OAAM,uBAAyBkoG,EAAM,MAEhDD,EAAez+F,KAAO,WAAa,UACnCy+F,EAAeE,QAAUF,EACzBhsG,EAAOD,QAAUisG,EACjBA,EAAexrG,GAAK,IAKhB,SAASR,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAAI1rF,GAAIC,EAAW8G,EAAU41C,EAAIC,EAAIuvC,EACnCiB,EAAgBhB,EAAOC,EAAOzlG,EAAGomB,EAE/BoxB,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGnBooD,EAAS,GAAK,EACd7lG,EAAI,EAAI,EAGR64C,EAAej/C,KAAK0hD,UAAUlD,QAAQQ,UAAUC,aAChDitD,EAAkBjtD,CAItB,KAAKz5C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAS,EAAGH,IAEtC,IADAwlG,EAAQhuD,EAAM8G,EAAYt+C,IACrBomB,EAAIpmB,EAAI,EAAGomB,EAAIk4B,EAAYn+C,OAAQimB,IAAK,CAC3Cq/E,EAAQjuD,EAAM8G,EAAYl4B,IAC1Bm/E,EAAsBC,EAAM3uC,YAAc4uC,EAAM5uC,YAAc,EAE9Dz9C,EAAKqsF,EAAMl5F,EAAIi5F,EAAMj5F,EACrB8M,EAAKosF,EAAMj5F,EAAIg5F,EAAMh5F,EACrB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAGpB,GAAZ8G,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,GAGPumF,EAA0C,GAAvBnB,EAA4B9rD,EAAgBA,GAAgB,EAAI8rD,EAAsB/qG,KAAK0hD,UAAUvC,WAAWW,sBACnI,IAAIv6C,GAAI0mG,EAASC,CACF,GAAIA,EAAfvmF,IAEAqmF,EADa,GAAME,EAAjBvmF,EACe,EAGApgB,EAAIogB,EAAWvf,EAIlC4lG,GAA0C,GAAvBjB,EAA4B,EAAI,EAAIA,EAAsB/qG,KAAK0hD,UAAUvC,WAAWU,mBACvGmsD,GAAkC9mG,KAAKwH,IAAIiZ,EAAS,IAAKumF,GAEzD3wC,EAAK38C,EAAKotF,EACVxwC,EAAK38C,EAAKmtF,EACVhB,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,MAUhB,SAAS37D,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAAI1rF,GAAIC,EAAI8G,EAAU41C,EAAIC,EACxBwwC,EAAgBhB,EAAOC,EAAOzlG,EAAGomB,EAE/BoxB,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGnB5E,EAAej/C,KAAK0hD,UAAUlD,QAAQU,sBAAsBD,YAIhE,KAAKz5C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAS,EAAGH,IAEtC,IADAwlG,EAAQhuD,EAAM8G,EAAYt+C,IACrBomB,EAAIpmB,EAAI,EAAGomB,EAAIk4B,EAAYn+C,OAAQimB,IAItC,GAHAq/E,EAAQjuD,EAAM8G,EAAYl4B,IAGtBo/E,EAAMptD,OAASqtD,EAAMrtD,MAAO,CAE9Bh/B,EAAKqsF,EAAMl5F,EAAIi5F,EAAMj5F,EACrB8M,EAAKosF,EAAMj5F,EAAIg5F,EAAMh5F,EACrB2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,EAGpC,IAAIstF,GAAY,GAEdH,GADa/sD,EAAXt5B,GACgBzgB,KAAK4uB,IAAIq4E,EAAUxmF,EAAS,GAAKzgB,KAAK4uB,IAAIq4E,EAAUltD,EAAa,GAGlE,EAGD,GAAZt5B,EACFA,EAAW,IAGXqmF,GAAkCrmF,EAEpC41C,EAAK38C,EAAKotF,EACVxwC,EAAK38C,EAAKmtF,EAEVhB,EAAMzvC,IAAMA,EACZyvC,EAAMxvC,IAAMA,EACZyvC,EAAM1vC,IAAMA,EACZ0vC,EAAMzvC,IAAMA,IAYtB57D,EAAQ4qG,mCAAqC,WAS3C,IAAK,GARDK,GAAYn9C,EAAMV,EAClBpuC,EAAIC,EAAI08C,EAAIC,EAAIsvC,EAAanlF,EAC7Bm4B,EAAQ99C,KAAK89C,MAEbd,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBAGdr+C,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CAC3C,GAAIwlG,GAAQhuD,EAAM8G,EAAYt+C,GAC9BwlG,GAAMoB,SAAW,EACjBpB,EAAMqB,SAAW,EAKnB,IAAKr/C,IAAUlP,GACb,GAAIA,EAAMh4C,eAAeknD,KACvBU,EAAO5P,EAAMkP,GACTU,EAAKC,WAEH3tD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKkG,OAAS5zD,KAAKg9C,MAAMl3C,eAAe4nD,EAAKiG,SAqBzE,GApBAk3C,EAAan9C,EAAKlP,QAAQK,aAE1BgsD,IAAen9C,EAAKrkC,GAAGgzC,YAAc3O,EAAKtkC,KAAKizC,YAAc,GAAKr8D,KAAK0hD,UAAUvC,WAAWY,WAE5FnhC,EAAM8uC,EAAKtkC,KAAKrX,EAAI27C,EAAKrkC,GAAGtX,EAC5B8M,EAAM6uC,EAAKtkC,KAAKpX,EAAI07C,EAAKrkC,GAAGrX,EAC5B2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAEpB,GAAZ8G,IACFA,EAAW,KAIbmlF,EAAc9qG,KAAK0hD,UAAUlD,QAAQM,gBAAkB+rD,EAAallF,GAAYA,EAEhF41C,EAAK38C,EAAKksF,EACVtvC,EAAK38C,EAAKisF,EAINp9C,EAAKrkC,GAAGu0B,OAAS8P,EAAKtkC,KAAKw0B,MAC7B8P,EAAKrkC,GAAG+iF,UAAY7wC,EACpB7N,EAAKrkC,GAAGgjF,UAAY7wC,EACpB9N,EAAKtkC,KAAKgjF,UAAY7wC,EACtB7N,EAAKtkC,KAAKijF,UAAY7wC,MAEnB,CACH,GAAI7U,GAAS,EACb+G,GAAKrkC,GAAGkyC,IAAM5U,EAAO4U,EACrB7N,EAAKrkC,GAAGmyC,IAAM7U,EAAO6U,EACrB9N,EAAKtkC,KAAKmyC,IAAM5U,EAAO4U,EACvB7N,EAAKtkC,KAAKoyC,IAAM7U,EAAO6U,EAQjC,GACI4wC,GAAUC,EADVvB,EAAc,CAElB,KAAKtlG,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7B4mG,GAAWlnG,KAAK4G,IAAIg/F,EAAY5lG,KAAKwH,KAAKo+F,EAAYhlD,EAAKsmD,WAC3DC,EAAWnnG,KAAK4G,IAAIg/F,EAAY5lG,KAAKwH,KAAKo+F,EAAYhlD,EAAKumD,WAE3DvmD,EAAKyV,IAAM6wC,EACXtmD,EAAK0V,IAAM6wC,EAIb,GAAIC,GAAU,EACVC,EAAU,CACd,KAAK/mG,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7B8mG,IAAWxmD,EAAKyV,GAChBgxC,GAAWzmD,EAAK0V,GAElB,GAAIgxC,GAAeF,EAAUxoD,EAAYn+C,OACrC8mG,EAAeF,EAAUzoD,EAAYn+C,MAEzC,KAAKH,EAAI,EAAGA,EAAIs+C,EAAYn+C,OAAQH,IAAK,CACvC,GAAIsgD,GAAO9I,EAAM8G,EAAYt+C,GAC7BsgD,GAAKyV,IAAMixC,EACX1mD,EAAK0V,IAAMixC,KAOX,SAAS5sG,EAAQD,GAQrBA,EAAQ0qG,qBAAuB,WAC7B,GAA8D,GAA1DtqG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAA4B,CAC/D,GAAImH,GACA9I,EAAQh9C,KAAK4jD,iBACbE,EAAc9jD,KAAK6jD,uBACnB6oD,EAAY5oD,EAAYn+C,MAE5B3F,MAAK2sG,mBAAmB3vD,EAAM8G,EAK9B,KAAK,GAHDomD,GAAgBlqG,KAAKkqG,cAGhB1kG,EAAI,EAAOknG,EAAJlnG,EAAeA,IAC7BsgD,EAAO9I,EAAM8G,EAAYt+C,IACrBsgD,EAAKr3C,QAAQwuC,KAAO,IAEtBj9C,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASC,GAAGhnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASE,GAAGjnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASG,GAAGlnD,GAC1D9lD,KAAK4sG,sBAAsB1C,EAAcxqG,KAAKmtG,SAASI,GAAGnnD,MAelElmD,EAAQgtG,sBAAwB,SAASM,EAAapnD,GAEpD,GAAIonD,EAAaC,cAAgB,EAAG,CAClC,GAAIvuF,GAAGC,EAAG8G,CAUV,IAPA/G,EAAKsuF,EAAaE,aAAar7F,EAAI+zC,EAAK/zC,EACxC8M,EAAKquF,EAAaE,aAAap7F,EAAI8zC,EAAK9zC,EACxC2T,EAAWzgB,KAAKyqB,KAAK/Q,EAAKA,EAAKC,EAAKA,GAKhC8G,EAAWunF,EAAaG,SAAWrtG,KAAK0hD,UAAUlD,QAAQC,UAAUC,cAAe,CAErE,GAAZ/4B,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,EAEP,IAAIilF,GAAe5qG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAwBuuD,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,MAAQt3B,EAAWA,EAAWA,GACvI41C,EAAK38C,EAAKgsF,EACVpvC,EAAK38C,EAAK+rF,CACd9kD,GAAKyV,IAAMA,EACXzV,EAAK0V,IAAMA,MAIX,IAAkC,GAA9B0xC,EAAaC,cACfntG,KAAK4sG,sBAAsBM,EAAaL,SAASC,GAAGhnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASE,GAAGjnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASG,GAAGlnD,GACpD9lD,KAAK4sG,sBAAsBM,EAAaL,SAASI,GAAGnnD,OAGpD,IAAIonD,EAAaL,SAASn6F,KAAKrS,IAAMylD,EAAKzlD,GAAI,CAE5B,GAAZslB,IACFA,EAAW,GAAIzgB,KAAKE,SACpBwZ,EAAK+G,EAEP,IAAIilF,GAAe5qG,KAAK0hD,UAAUlD,QAAQC,UAAUE,sBAAwBuuD,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,MAAQt3B,EAAWA,EAAWA,GACvI41C,EAAK38C,EAAKgsF,EACVpvC,EAAK38C,EAAK+rF,CACd9kD,GAAKyV,IAAMA,EACXzV,EAAK0V,IAAMA,KAcrB57D,EAAQ+sG,mBAAqB,SAAS3vD,EAAM8G,GAU1C,IAAK,GATDgC,GACA4mD,EAAY5oD,EAAYn+C,OAExBsgD,EAAO/hD,OAAOopG,UAChBvnD,EAAO7hD,OAAOopG,UACdpnD,GAAOhiD,OAAOopG,UACdtnD,GAAO9hD,OAAOopG,UAGP9nG,EAAI,EAAOknG,EAAJlnG,EAAeA,IAAK,CAClC,GAAIuM,GAAIirC,EAAM8G,EAAYt+C,IAAIuM,EAC1BC,EAAIgrC,EAAM8G,EAAYt+C,IAAIwM,CAC1BgrC,GAAM8G,EAAYt+C,IAAIiJ,QAAQwuC,KAAO,IAC/BgJ,EAAJl0C,IAAYk0C,EAAOl0C,GACnBA,EAAIm0C,IAAQA,EAAOn0C,GACfg0C,EAAJ/zC,IAAY+zC,EAAO/zC,GACnBA,EAAIg0C,IAAQA,EAAOh0C,IAI3B,GAAIu7F,GAAWroG,KAAK2lB,IAAIq7B,EAAOD,GAAQ/gD,KAAK2lB,IAAIm7B,EAAOD,EACnDwnD,GAAW,GAAIxnD,GAAQ,GAAMwnD,EAAUvnD,GAAQ,GAAMunD,IACtCtnD,GAAQ,GAAMsnD,EAAUrnD,GAAQ,GAAMqnD,EAGzD,IAAIC,GAAkB,KAClBC,EAAWvoG,KAAKwH,IAAI8gG,EAAgBtoG,KAAK2lB,IAAIq7B,EAAOD,IACpDynD,EAAe,GAAMD,EACrBxuC,EAAU,IAAOhZ,EAAOC,GAAOgZ,EAAU,IAAOnZ,EAAOC,GAGvDkkD,GACFxqG,MACE0tG,cAAer7F,EAAE,EAAGC,EAAE,GACtBirC,KAAK,EACLxnB,OACEwwB,KAAMgZ,EAAQyuC,EAAaxnD,KAAK+Y,EAAQyuC,EACxC3nD,KAAMmZ,EAAQwuC,EAAa1nD,KAAKkZ,EAAQwuC,GAE1Cr7F,KAAMo7F,EACNJ,SAAU,EAAII,EACdZ,UAAYn6F,KAAK,MACjBgpC,SAAU,EACVkC,MAAO,EACPuvD,cAAe,GAMnB,KAHAntG,KAAK2tG,aAAazD,EAAcxqG,MAG3B8F,EAAI,EAAOknG,EAAJlnG,EAAeA,IACzBsgD,EAAO9I,EAAM8G,EAAYt+C,IACrBsgD,EAAKr3C,QAAQwuC,KAAO,GACtBj9C,KAAK4tG,aAAa1D,EAAcxqG,KAAKomD,EAKzC9lD,MAAKkqG,cAAgBA,GAWvBtqG,EAAQiuG,kBAAoB,SAASX,EAAcpnD,GACjD,GAAIgoD,GAAYZ,EAAajwD,KAAO6I,EAAKr3C,QAAQwuC,KAC7C8wD,EAAe,EAAED,CAErBZ,GAAaE,aAAar7F,EAAIm7F,EAAaE,aAAar7F,EAAIm7F,EAAajwD,KAAO6I,EAAK/zC,EAAI+zC,EAAKr3C,QAAQwuC,KACtGiwD,EAAaE,aAAar7F,GAAKg8F,EAE/Bb,EAAaE,aAAap7F,EAAIk7F,EAAaE,aAAap7F,EAAIk7F,EAAajwD,KAAO6I,EAAK9zC,EAAI8zC,EAAKr3C,QAAQwuC,KACtGiwD,EAAaE,aAAap7F,GAAK+7F,EAE/Bb,EAAajwD,KAAO6wD,CACpB,IAAIE,GAAc9oG,KAAKwH,IAAIxH,KAAKwH,IAAIo5C,EAAKtzC,OAAOszC,EAAKr6B,QAAQq6B,EAAKvzC,MAClE26F,GAAaxxD,SAAYwxD,EAAaxxD,SAAWsyD,EAAeA,EAAcd,EAAaxxD,UAa7F97C,EAAQguG,aAAe,SAASV,EAAapnD,EAAKmoD,IAC1B,GAAlBA,GAA6CznG,SAAnBynG,IAE5BjuG,KAAK6tG,kBAAkBX,EAAapnD,GAGlConD,EAAaL,SAASC,GAAGr3E,MAAMywB,KAAOJ,EAAK/zC,EACzCm7F,EAAaL,SAASC,GAAGr3E,MAAMuwB,KAAOF,EAAK9zC,EAC7ChS,KAAKkuG,eAAehB,EAAapnD,EAAK,MAGtC9lD,KAAKkuG,eAAehB,EAAapnD,EAAK,MAIpConD,EAAaL,SAASC,GAAGr3E,MAAMuwB,KAAOF,EAAK9zC,EAC7ChS,KAAKkuG,eAAehB,EAAapnD,EAAK,MAGtC9lD,KAAKkuG,eAAehB,EAAapnD,EAAK,OAc5ClmD,EAAQsuG,eAAiB,SAAShB,EAAapnD,EAAKqoD,GAClD,OAAQjB,EAAaL,SAASsB,GAAQhB,eACpC,IAAK,GACHD,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAOozC,EAC9ConD,EAAaL,SAASsB,GAAQhB,cAAgB,EAC9CntG,KAAK6tG,kBAAkBX,EAAaL,SAASsB,GAAQroD,EACrD,MACF,KAAK,GAGConD,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAKX,GAAK+zC,EAAK/zC,GACtDm7F,EAAaL,SAASsB,GAAQtB,SAASn6F,KAAKV,GAAK8zC,EAAK9zC,GACxD8zC,EAAK/zC,GAAK7M,KAAKE,SACf0gD,EAAK9zC,GAAK9M,KAAKE,WAGfpF,KAAK2tG,aAAaT,EAAaL,SAASsB,IACxCnuG,KAAK4tG,aAAaV,EAAaL,SAASsB,GAAQroD,GAElD,MACF,KAAK,GACH9lD,KAAK4tG,aAAaV,EAAaL,SAASsB,GAAQroD,KAatDlmD,EAAQ+tG,aAAe,SAAST,GAE9B,GAAIkB,GAAgB,IACc,IAA9BlB,EAAaC,gBACfiB,EAAgBlB,EAAaL,SAASn6F,KACtCw6F,EAAajwD,KAAO,EAAGiwD,EAAaE,aAAar7F,EAAI,EAAGm7F,EAAaE,aAAap7F,EAAI,GAExFk7F,EAAaC,cAAgB,EAC7BD,EAAaL,SAASn6F,KAAO,KAC7B1S,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAChCltG,KAAKquG,cAAcnB,EAAa,MAEX,MAAjBkB,GACFpuG,KAAK4tG,aAAaV,EAAakB,IAenCxuG,EAAQyuG,cAAgB,SAASnB,EAAciB,GAC7C,GAAIloD,GAAKC,EAAKH,EAAKC,EACfsoD,EAAY,GAAMpB,EAAa76F,IACnC,QAAQ87F,GACN,IAAK,KACHloD,EAAOinD,EAAaz3E,MAAMwwB,KAC1BC,EAAOgnD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCvoD,EAAOmnD,EAAaz3E,MAAMswB,KAC1BC,EAAOknD,EAAaz3E,MAAMswB,KAAOuoD,CACjC,MACF,KAAK,KACHroD,EAAOinD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCpoD,EAAOgnD,EAAaz3E,MAAMywB,KAC1BH,EAAOmnD,EAAaz3E,MAAMswB,KAC1BC,EAAOknD,EAAaz3E,MAAMswB,KAAOuoD,CACjC,MACF,KAAK,KACHroD,EAAOinD,EAAaz3E,MAAMwwB,KAC1BC,EAAOgnD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCvoD,EAAOmnD,EAAaz3E,MAAMswB,KAAOuoD,EACjCtoD,EAAOknD,EAAaz3E,MAAMuwB,IAC1B,MACF,KAAK,KACHC,EAAOinD,EAAaz3E,MAAMwwB,KAAOqoD,EACjCpoD,EAAOgnD,EAAaz3E,MAAMywB,KAC1BH,EAAOmnD,EAAaz3E,MAAMswB,KAAOuoD,EACjCtoD,EAAOknD,EAAaz3E,MAAMuwB,KAK9BknD,EAAaL,SAASsB,IACpBf,cAAcr7F,EAAE,EAAEC,EAAE,GACpBirC,KAAK,EACLxnB,OAAOwwB,KAAKA,EAAKC,KAAKA,EAAKH,KAAKA,EAAKC,KAAKA,GAC1C3zC,KAAM,GAAM66F,EAAa76F,KACzBg7F,SAAU,EAAIH,EAAaG,SAC3BR,UAAWn6F,KAAK,MAChBgpC,SAAU,EACVkC,MAAOsvD,EAAatvD,MAAM,EAC1BuvD,cAAe,IAYnBvtG,EAAQ2uG,UAAY,SAASxnF,EAAI5b,GACJ3E,SAAvBxG,KAAKkqG,gBAEPnjF,EAAIO,UAAY,EAEhBtnB,KAAKwuG,YAAYxuG,KAAKkqG,cAAcxqG,KAAKqnB,EAAI5b,KAajDvL,EAAQ4uG,YAAc,SAASC,EAAO1nF,EAAI5b,GAC1B3E,SAAV2E,IACFA,EAAQ,WAGkB,GAAxBsjG,EAAOtB,gBACTntG,KAAKwuG,YAAYC,EAAO5B,SAASC,GAAG/lF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASE,GAAGhmF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASI,GAAGlmF,GACpC/mB,KAAKwuG,YAAYC,EAAO5B,SAASG,GAAGjmF,IAEtCA,EAAIY,YAAcxc,EAClB4b,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIe,OAAO2mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIe,OAAO2mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMywB,KAAKuoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIe,OAAO2mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIlH,SAEJkH,EAAIa,YACJb,EAAIc,OAAO4mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMuwB,MAC1Cj/B,EAAIe,OAAO2mF,EAAOh5E,MAAMwwB,KAAKwoD,EAAOh5E,MAAMswB,MAC1Ch/B,EAAIlH,WAaF,SAAShgB,GAEbA,EAAOD,QAAU,SAASC,GAQzB,MAPIA,GAAO6uG,kBACV7uG,EAAOqwE,UAAY,aACnBrwE,EAAO8uG,SAEP9uG,EAAOgtG,YACPhtG,EAAO6uG,gBAAkB,GAEnB7uG,IAMJ,SAASA,EAAQD,IAEO,SAASgvG,GAA0B/uG,EAAOD,QAAUgvG,IAEnDruG,KAAKX"} \ No newline at end of file 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 new file mode 100644 index 0000000000000..a390c40d67574 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css @@ -0,0 +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 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 new file mode 100644 index 0000000000000..4af2c818c27a6 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js @@ -0,0 +1,39 @@ +/** + * vis.js + * https://github.com/almende/vis + * + * A dynamic, browser-based visualization library. + * + * @version 3.9.0 + * @date 2015-01-16 + * + * @license + * Copyright (C) 2011-2014 Almende B.V, http://almende.com + * + * Vis.js is dual licensed under both + * + * * The Apache 2.0 License + * http://www.apache.org/licenses/LICENSE-2.0 + * + * and + * + * * The MIT License + * http://opensource.org/licenses/MIT + * + * 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,{})}])}); +//# sourceMappingURL=vis.map diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 6c37cc8b98236..669ad48937c05 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -85,17 +85,13 @@ table.sortable td { filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } -span.kill-link { +a.kill-link { margin-right: 2px; margin-left: 20px; color: gray; float: right; } -span.kill-link a { - color: gray; -} - span.expand-details { font-size: 10pt; cursor: pointer; @@ -149,7 +145,7 @@ pre { border: none; } -span.expand-additional-metrics { +span.expand-additional-metrics, span.expand-dag-viz { cursor: pointer; } diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index bcf832467f00b..330df1d59a9b1 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,8 +18,6 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} -import java.util.concurrent.atomic.AtomicLong -import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -109,7 +107,7 @@ class Accumulable[R, T] ( * The typical use of this method is to directly mutate the local value, eg., to add * an element to a Set. */ - def localValue = value_ + def localValue: R = value_ /** * Set the accumulator's value; only allowed on master. @@ -137,7 +135,7 @@ class Accumulable[R, T] ( Accumulators.register(this, false) } - override def toString = if (value_ == null) "null" else value_.toString + override def toString: String = if (value_ == null) "null" else value_.toString } /** @@ -257,22 +255,22 @@ object AccumulatorParam { implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double) = 0.0 + def zero(initialValue: Double): Double = 0.0 } implicit object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int) = 0 + def zero(initialValue: Int): Int = 0 } implicit object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0L + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L } implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float) = t1 + t2 - def zero(initialValue: Float) = 0f + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f } // TODO: Add AccumulatorParams for other types, e.g. lists and strings @@ -351,6 +349,7 @@ private[spark] object Accumulators extends Logging { } } - def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue) - def stringifyValue(value: Any) = "%s".format(value) + def stringifyPartialValue(partialValue: Any): String = "%s".format(partialValue) + + def stringifyValue(value: Any): String = "%s".format(value) } diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 3b684bbeceaf2..af9765d313e9e 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -88,10 +88,7 @@ case class Aggregator[K, V, C] ( combiners.iterator } else { val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) - while (iter.hasNext) { - val pair = iter.next() - combiners.insert(pair._1, pair._2) - } + combiners.insertAll(iter) // Update task metrics if context is not null // TODO: Make context non-optional in a future release Option(context).foreach { c => diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index a96d754744a05..4d20c7369376e 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -44,10 +44,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { blockManager.get(key) match { case Some(blockResult) => // Partition is already materialized, so just return its values - val inputMetrics = blockResult.inputMetrics val existingMetrics = context.taskMetrics - .getInputMetricsForReadMethod(inputMetrics.readMethod) - existingMetrics.incBytesRead(inputMetrics.bytesRead) + .getInputMetricsForReadMethod(blockResult.readMethod) + existingMetrics.incBytesRead(blockResult.bytes) val iter = blockResult.data.asInstanceOf[Iterator[T]] new InterruptibleIterator[T](context, iter) { diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 9b05c9623b704..37198d887b07b 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue, WeakReference} import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.util.Utils /** @@ -33,6 +33,7 @@ private case class CleanRDD(rddId: Int) extends CleanupTask private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask private case class CleanAccum(accId: Long) extends CleanupTask +private case class CleanCheckpoint(rddId: Int) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -94,12 +95,12 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ - def attachListener(listener: CleanerListener) { + def attachListener(listener: CleanerListener): Unit = { listeners += listener } /** Start the cleaner. */ - def start() { + def start(): Unit = { cleaningThread.setDaemon(true) cleaningThread.setName("Spark Context Cleaner") cleaningThread.start() @@ -108,7 +109,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Stop the cleaning thread and wait until the thread has finished running its current task. */ - def stop() { + def stop(): Unit = { stopped = true // Interrupt the cleaning thread, but wait until the current task has finished before // doing so. This guards against the race condition where a cleaning thread may @@ -121,7 +122,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a RDD for cleanup when it is garbage collected. */ - def registerRDDForCleanup(rdd: RDD[_]) { + def registerRDDForCleanup(rdd: RDD[_]): Unit = { registerForCleanup(rdd, CleanRDD(rdd.id)) } @@ -130,17 +131,22 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a ShuffleDependency for cleanup when it is garbage collected. */ - def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]): Unit = { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) } /** Register a Broadcast for cleanup when it is garbage collected. */ - def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]): Unit = { registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) } + /** Register a RDDCheckpointData for cleanup when it is garbage collected. */ + def registerRDDCheckpointDataForCleanup[T](rdd: RDD[_], parentId: Int): Unit = { + registerForCleanup(rdd, CleanCheckpoint(parentId)) + } + /** Register an object for cleanup. */ - private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) } @@ -164,6 +170,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) case CleanAccum(accId) => doCleanupAccum(accId, blocking = blockOnCleanupTasks) + case CleanCheckpoint(rddId) => + doCleanCheckpoint(rddId) } } } @@ -175,7 +183,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform RDD cleanup. */ - def doCleanupRDD(rddId: Int, blocking: Boolean) { + def doCleanupRDD(rddId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning RDD " + rddId) sc.unpersistRDD(rddId, blocking) @@ -187,7 +195,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform shuffle cleanup, asynchronously. */ - def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) mapOutputTrackerMaster.unregisterShuffle(shuffleId) @@ -200,7 +208,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform broadcast cleanup. */ - def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean): Unit = { try { logDebug(s"Cleaning broadcast $broadcastId") broadcastManager.unbroadcast(broadcastId, true, blocking) @@ -212,7 +220,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform accumulator cleanup. */ - def doCleanupAccum(accId: Long, blocking: Boolean) { + def doCleanupAccum(accId: Long, blocking: Boolean): Unit = { try { logDebug("Cleaning accumulator " + accId) Accumulators.remove(accId) @@ -223,6 +231,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } + /** Perform checkpoint cleanup. */ + def doCleanCheckpoint(rddId: Int): Unit = { + try { + logDebug("Cleaning rdd checkpoint data " + rddId) + RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + listeners.foreach(_.checkpointCleaned(rddId)) + logInfo("Cleaned rdd checkpoint data " + rddId) + } + catch { + case e: Exception => logError("Error cleaning rdd checkpoint data " + rddId, e) + } + } + private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] @@ -240,4 +261,5 @@ private[spark] trait CleanerListener { def shuffleCleaned(shuffleId: Int) def broadcastCleaned(broadcastId: Long) def accumCleaned(accId: Long) + def checkpointCleaned(rddId: Long) } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 9a7cd4523e5ab..fc8cdde9348ee 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -74,7 +74,7 @@ class ShuffleDependency[K, V, C]( val mapSideCombine: Boolean = false) extends Dependency[Product2[K, V]] { - override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] + override def rdd: RDD[Product2[K, V]] = _rdd.asInstanceOf[RDD[Product2[K, V]]] val shuffleId: Int = _rdd.context.newShuffleId() @@ -91,7 +91,7 @@ class ShuffleDependency[K, V, C]( */ @DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { - override def getParents(partitionId: Int) = List(partitionId) + override def getParents(partitionId: Int): List[Int] = List(partitionId) } @@ -107,7 +107,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { - override def getParents(partitionId: Int) = { + override def getParents(partitionId: Int): List[Int] = { if (partitionId >= outStart && partitionId < outStart + length) { List(partitionId - outStart + inStart) } else { diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 21c6e6ffa6666..66bda68088502 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -17,21 +17,33 @@ package org.apache.spark +import java.util.concurrent.TimeUnit + import scala.collection.mutable +import com.codahale.metrics.{Gauge, MetricRegistry} + import org.apache.spark.scheduler._ -import org.apache.spark.util.{SystemClock, Clock} +import org.apache.spark.metrics.source.Source +import org.apache.spark.util.{ThreadUtils, Clock, SystemClock, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. * - * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If - * the scheduler queue is not drained in N seconds, then new executors are added. If the queue - * persists for another M seconds, then more executors are added and so on. The number added - * in each round increases exponentially from the previous round until an upper bound on the - * number of executors has been reached. The upper bound is based both on a configured property - * and on the number of tasks pending: the policy will never increase the number of executor - * requests past the number needed to handle all pending tasks. + * The ExecutorAllocationManager maintains a moving target number of executors which is periodically + * synced to the cluster manager. The target starts at a configured initial value and changes with + * the number of pending and running tasks. + * + * Decreasing the target number of executors happens when the current target is more than needed to + * handle the current load. The target number of executors is always truncated to the number of + * executors that could run all current running and pending tasks at once. + * + * Increasing the target number of executors happens in response to backlogged tasks waiting to be + * scheduled. If the scheduler queue is not drained in N seconds, then new executors are added. If + * the queue persists for another M seconds, then more executors are added and so on. The number + * added in each round increases exponentially from the previous round until an upper bound has been + * reached. The upper bound is based both on a configured property and on the current number of + * running and pending tasks, as described above. * * The rationale for the exponential increase is twofold: (1) Executors should be added slowly * in the beginning in case the number of extra executors needed turns out to be small. Otherwise, @@ -78,16 +90,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 5) + private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") - // Same as above, but used only after `schedulerBacklogTimeout` is exceeded - private val sustainedSchedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = conf.getLong( - "spark.dynamicAllocation.executorIdleTimeout", 600) + private val executorIdleTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -103,8 +115,10 @@ private[spark] class ExecutorAllocationManager( // Number of executors to add in the next round private var numExecutorsToAdd = 1 - // Number of executors that have been requested but have not registered yet - private var numExecutorsPending = 0 + // The desired number of executors at this moment in time. If all our executors were to die, this + // is the number of executors we would immediately want from the cluster manager. + private var numExecutorsTarget = + conf.getInt("spark.dynamicAllocation.initialExecutors", minNumExecutors) // Executors that have been requested to be removed but have not been killed yet private val executorsPendingToRemove = new mutable.HashSet[String] @@ -129,6 +143,13 @@ private[spark] class ExecutorAllocationManager( // Listener for Spark events that impact the allocation policy private val listener = new ExecutorAllocationListener + // Executor that handles the scheduling task. + private val executor = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation") + + // Metric source for ExecutorAllocationManager to expose internal status to MetricsSystem. + val executorAllocationManagerSource = new ExecutorAllocationManagerSource + /** * Verify that the settings specified through the config are valid. * If not, throw an appropriate exception. @@ -144,14 +165,14 @@ private[spark] class ExecutorAllocationManager( throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } - if (schedulerBacklogTimeout <= 0) { + if (schedulerBacklogTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") } - if (sustainedSchedulerBacklogTimeout <= 0) { + if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") } - if (executorIdleTimeout <= 0) { + if (executorIdleTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") } // Require external shuffle service for dynamic allocation @@ -173,40 +194,25 @@ private[spark] class ExecutorAllocationManager( } /** - * Register for scheduler callbacks to decide when to add and remove executors. + * Register for scheduler callbacks to decide when to add and remove executors, and start + * the scheduling task. */ def start(): Unit = { listenerBus.addListener(listener) - startPolling() - } - /** - * Start the main polling thread that keeps track of when to add and remove executors. - */ - private def startPolling(): Unit = { - val t = new Thread { - override def run(): Unit = { - while (true) { - try { - schedule() - } catch { - case e: Exception => logError("Exception in dynamic executor allocation thread!", e) - } - Thread.sleep(intervalMillis) - } - } + val scheduleTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(schedule()) } - t.setName("spark-dynamic-executor-allocation") - t.setDaemon(true) - t.start() + executor.scheduleAtFixedRate(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) } /** - * The number of executors we would have if the cluster manager were to fulfill all our existing - * requests. + * Stop the allocation manager. */ - private def targetNumExecutors(): Int = - numExecutorsPending + executorIds.size - executorsPendingToRemove.size + def stop(): Unit = { + executor.shutdown() + executor.awaitTermination(10, TimeUnit.SECONDS) + } /** * The maximum number of executors we would need under the current load to satisfy all running @@ -229,7 +235,7 @@ private[spark] class ExecutorAllocationManager( private def schedule(): Unit = synchronized { val now = clock.getTimeMillis - addOrCancelExecutorRequests(now) + updateAndSyncNumExecutorsTarget(now) removeTimes.retain { case (executorId, expireTime) => val expired = now >= expireTime @@ -241,31 +247,33 @@ private[spark] class ExecutorAllocationManager( } /** + * Updates our target number of executors and syncs the result with the cluster manager. + * * Check to see whether our existing allocation and the requests we've made previously exceed our - * current needs. If so, let the cluster manager know so that it can cancel pending requests that - * are unneeded. + * current needs. If so, truncate our target and let the cluster manager know so that it can + * cancel pending requests that are unneeded. * * If not, and the add time has expired, see if we can request new executors and refresh the add * time. * * @return the delta in the target number of executors. */ - private def addOrCancelExecutorRequests(now: Long): Int = synchronized { - val currentTarget = targetNumExecutors + private def updateAndSyncNumExecutorsTarget(now: Long): Int = synchronized { val maxNeeded = maxNumExecutorsNeeded - if (maxNeeded < currentTarget) { + if (maxNeeded < numExecutorsTarget) { // The target number exceeds the number we actually need, so stop adding new - // executors and inform the cluster manager to cancel the extra pending requests. - val newTotalExecutors = math.max(maxNeeded, minNumExecutors) - client.requestTotalExecutors(newTotalExecutors) + // executors and inform the cluster manager to cancel the extra pending requests + val oldNumExecutorsTarget = numExecutorsTarget + numExecutorsTarget = math.max(maxNeeded, minNumExecutors) + client.requestTotalExecutors(numExecutorsTarget) numExecutorsToAdd = 1 - updateNumExecutorsPending(newTotalExecutors) + numExecutorsTarget - oldNumExecutorsTarget } else if (addTime != NOT_SET && now >= addTime) { val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime += sustainedSchedulerBacklogTimeoutS * 1000 delta } else { 0 @@ -283,21 +291,30 @@ private[spark] class ExecutorAllocationManager( */ private def addExecutors(maxNumExecutorsNeeded: Int): Int = { // Do not request more executors if it would put our target over the upper bound - val currentTarget = targetNumExecutors - if (currentTarget >= maxNumExecutors) { - logDebug(s"Not adding executors because there are already ${executorIds.size} " + - s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)") + if (numExecutorsTarget >= maxNumExecutors) { + val numExecutorsPending = numExecutorsTarget - executorIds.size + logDebug(s"Not adding executors because there are already ${executorIds.size} registered " + + s"and ${numExecutorsPending} pending executor(s) (limit $maxNumExecutors)") numExecutorsToAdd = 1 return 0 } - val actualMaxNumExecutors = math.min(maxNumExecutors, maxNumExecutorsNeeded) - val newTotalExecutors = math.min(currentTarget + numExecutorsToAdd, actualMaxNumExecutors) - val addRequestAcknowledged = testing || client.requestTotalExecutors(newTotalExecutors) + val oldNumExecutorsTarget = numExecutorsTarget + // There's no point in wasting time ramping up to the number of executors we already have, so + // make sure our target is at least as much as our current allocation: + numExecutorsTarget = math.max(numExecutorsTarget, executorIds.size) + // Boost our target with the number to add for this round: + numExecutorsTarget += numExecutorsToAdd + // Ensure that our target doesn't exceed what we need at the present moment: + numExecutorsTarget = math.min(numExecutorsTarget, maxNumExecutorsNeeded) + // Ensure that our target fits within configured bounds: + numExecutorsTarget = math.max(math.min(numExecutorsTarget, maxNumExecutors), minNumExecutors) + + val addRequestAcknowledged = testing || client.requestTotalExecutors(numExecutorsTarget) if (addRequestAcknowledged) { - val delta = updateNumExecutorsPending(newTotalExecutors) + val delta = numExecutorsTarget - oldNumExecutorsTarget logInfo(s"Requesting $delta new executor(s) because tasks are backlogged" + - s" (new desired total will be $newTotalExecutors)") + s" (new desired total will be $numExecutorsTarget)") numExecutorsToAdd = if (delta == numExecutorsToAdd) { numExecutorsToAdd * 2 } else { @@ -306,23 +323,11 @@ private[spark] class ExecutorAllocationManager( delta } else { logWarning( - s"Unable to reach the cluster manager to request $newTotalExecutors total executors!") + s"Unable to reach the cluster manager to request $numExecutorsTarget total executors!") 0 } } - /** - * Given the new target number of executors, update the number of pending executor requests, - * and return the delta from the old number of pending requests. - */ - private def updateNumExecutorsPending(newTotalExecutors: Int): Int = { - val newNumExecutorsPending = - newTotalExecutors - executorIds.size + executorsPendingToRemove.size - val delta = newNumExecutorsPending - numExecutorsPending - numExecutorsPending = newNumExecutorsPending - delta - } - /** * Request the cluster manager to remove the given executor. * Return whether the request is received. @@ -353,7 +358,7 @@ private[spark] class ExecutorAllocationManager( val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + - s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") + s"$executorIdleTimeoutS seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -374,10 +379,6 @@ private[spark] class ExecutorAllocationManager( // as idle again so as not to forget that it is a candidate for removal. (see SPARK-4951) executorIds.filter(listener.isExecutorIdle).foreach(onExecutorIdle) logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") - if (numExecutorsPending > 0) { - numExecutorsPending -= 1 - logDebug(s"Decremented number of pending executors ($numExecutorsPending left)") - } } else { logWarning(s"Duplicate executor $executorId has registered") } @@ -409,8 +410,8 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerBacklogged(): Unit = synchronized { if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + - s"are building up (to expire in $schedulerBacklogTimeout seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 } } @@ -433,8 +434,8 @@ private[spark] class ExecutorAllocationManager( if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeoutS seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeoutS * 1000 } } else { logWarning(s"Attempted to mark unknown executor $executorId idle") @@ -584,6 +585,29 @@ private[spark] class ExecutorAllocationManager( } } + /** + * Metric source for ExecutorAllocationManager to expose its internal executor allocation + * status to MetricsSystem. + * Note: These metrics heavily rely on the internal implementation of + * ExecutorAllocationManager, metrics or value of metrics will be changed when internal + * implementation is changed, so these metrics are not stable across Spark version. + */ + private[spark] class ExecutorAllocationManagerSource extends Source { + val sourceName = "ExecutorAllocationManager" + val metricRegistry = new MetricRegistry() + + private def registerGauge[T](name: String, value: => T, defaultValue: T): Unit = { + metricRegistry.register(MetricRegistry.name("executors", name), new Gauge[T] { + override def getValue: T = synchronized { Option(value).getOrElse(defaultValue) } + }) + } + + registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0) + registerGauge("numberExecutorsPendingToRemove", executorsPendingToRemove.size, 0) + registerGauge("numberAllExecutors", executorIds.size, 0) + registerGauge("numberTargetExecutors", numExecutorsTarget, 0) + registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0) + } } private object ExecutorAllocationManager { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index e97a7375a267b..91f9ef8ce7185 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -168,7 +168,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } } - def jobIds = Seq(jobWaiter.jobId) + def jobIds: Seq[Int] = Seq(jobWaiter.jobId) } @@ -276,7 +276,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { override def value: Option[Try[T]] = p.future.value - def jobIds = jobs + def jobIds: Seq[Int] = jobs } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 69178da1a7773..f2b024ff6cb67 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -17,15 +17,15 @@ package org.apache.spark -import scala.concurrent.duration._ -import scala.collection.mutable +import java.util.concurrent.{ScheduledFuture, TimeUnit} -import akka.actor.{Actor, Cancellable} +import scala.collection.mutable import org.apache.spark.executor.TaskMetrics +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext} import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.{SlaveLost, TaskScheduler} -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.util.{ThreadUtils, Utils} /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -37,6 +37,12 @@ private[spark] case class Heartbeat( taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) +/** + * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is + * created. + */ +private[spark] case object TaskSchedulerIsSet + private[spark] case object ExpireDeadHosts private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) @@ -44,36 +50,73 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** * Lives in the driver to receive heartbeats from executors.. */ -private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskScheduler) - extends Actor with ActorLogReceive with Logging { +private[spark] class HeartbeatReceiver(sc: SparkContext) + extends ThreadSafeRpcEndpoint with Logging { + + override val rpcEnv: RpcEnv = sc.env.rpcEnv + + private[spark] var scheduler: TaskScheduler = null // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] + + // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses + // "milliseconds" + private val slaveTimeoutMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") + private val executorTimeoutMs = + sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 - private val executorTimeoutMs = sc.conf.getLong("spark.network.timeout", - sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120)) * 1000 - - private val checkTimeoutIntervalMs = sc.conf.getLong("spark.network.timeoutInterval", - sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60)) * 1000 - - private var timeoutCheckingTask: Cancellable = null + // "spark.network.timeoutInterval" uses "seconds", while + // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" + private val timeoutIntervalMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + private val checkTimeoutIntervalMs = + sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 - override def preStart(): Unit = { - import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, - checkTimeoutIntervalMs.milliseconds, self, ExpireDeadHosts) - super.preStart() + private var timeoutCheckingTask: ScheduledFuture[_] = null + + // "eventLoopThread" is used to run some pretty fast actions. The actions running in it should not + // block the thread for a long time. + private val eventLoopThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("heartbeat-receiver-event-loop-thread") + + private val killExecutorThread = ThreadUtils.newDaemonSingleThreadExecutor("kill-executor-thread") + + override def onStart(): Unit = { + timeoutCheckingTask = eventLoopThread.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ExpireDeadHosts)) + } + }, 0, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS) } - - override def receiveWithLogging = { - case Heartbeat(executorId, taskMetrics, blockManagerId) => - val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, taskMetrics, blockManagerId) - val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) - executorLastSeen(executorId) = System.currentTimeMillis() - sender ! response + + override def receive: PartialFunction[Any, Unit] = { case ExpireDeadHosts => expireDeadHosts() + case TaskSchedulerIsSet => + scheduler = sc.taskScheduler + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case heartbeat @ Heartbeat(executorId, taskMetrics, blockManagerId) => + if (scheduler != null) { + executorLastSeen(executorId) = System.currentTimeMillis() + eventLoopThread.submit(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + val unknownExecutor = !scheduler.executorHeartbeatReceived( + executorId, taskMetrics, blockManagerId) + val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) + context.reply(response) + } + }) + } else { + // Because Executor will sleep several seconds before sending the first "Heartbeat", this + // case rarely happens. However, if it really happens, log it and ask the executor to + // register itself again. + logWarning(s"Dropping $heartbeat because TaskScheduler is not ready yet") + context.reply(HeartbeatResponse(reregisterBlockManager = true)) + } } private def expireDeadHosts(): Unit = { @@ -84,19 +127,29 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule logWarning(s"Removing executor $executorId with no recent heartbeats: " + s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms") scheduler.executorLost(executorId, SlaveLost("Executor heartbeat " + - "timed out after ${now - lastSeenMs} ms")) + s"timed out after ${now - lastSeenMs} ms")) if (sc.supportDynamicAllocation) { - sc.killExecutor(executorId) + // Asynchronously kill the executor to avoid blocking the current thread + killExecutorThread.submit(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + sc.killExecutor(executorId) + } + }) } executorLastSeen.remove(executorId) } } } - override def postStop(): Unit = { + override def onStop(): Unit = { if (timeoutCheckingTask != null) { - timeoutCheckingTask.cancel() + timeoutCheckingTask.cancel(true) } - super.postStop() + eventLoopThread.shutdownNow() + killExecutorThread.shutdownNow() } } + +object HeartbeatReceiver { + val ENDPOINT_NAME = "HeartbeatReceiver" +} diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 09a9ccc226721..8de3a6c04df34 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -160,7 +160,7 @@ private[spark] class HttpServer( throw new ServerStateException("Server is not started") } else { val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" - s"$scheme://${Utils.localIpAddress}:$port" + s"$scheme://${Utils.localHostNameForURI()}:$port" } } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6e4edc7c80d7a..16072283edbe9 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -21,13 +21,11 @@ import java.io._ import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.{HashSet, HashMap, Map} -import scala.concurrent.Await +import scala.collection.mutable.{HashSet, Map} import scala.collection.JavaConversions._ +import scala.reflect.ClassTag -import akka.actor._ -import akka.pattern.ask - +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, RpcEndpoint} import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.BlockManagerId @@ -38,14 +36,15 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -/** Actor class for MapOutputTrackerMaster */ -private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) - extends Actor with ActorLogReceive with Logging { +/** RpcEndpoint class for MapOutputTrackerMaster */ +private[spark] class MapOutputTrackerMasterEndpoint( + override val rpcEnv: RpcEnv, tracker: MapOutputTrackerMaster, conf: SparkConf) + extends RpcEndpoint with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - override def receiveWithLogging = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case GetMapOutputStatuses(shuffleId: Int) => - val hostPort = sender.path.address.hostPort + val hostPort = context.sender.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) val mapOutputStatuses = tracker.getSerializedMapOutputStatuses(shuffleId) val serializedSize = mapOutputStatuses.size @@ -53,19 +52,19 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster val msg = s"Map output statuses were $serializedSize bytes which " + s"exceeds spark.akka.frameSize ($maxAkkaFrameSize bytes)." - /* For SPARK-1244 we'll opt for just logging an error and then throwing an exception. - * Note that on exception the actor will just restart. A bigger refactoring (SPARK-1239) - * will ultimately remove this entire code path. */ + /* 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) - throw exception + context.sendFailure(exception) + } else { + context.reply(mapOutputStatuses) } - sender ! mapOutputStatuses case StopMapOutputTracker => - logInfo("MapOutputTrackerActor stopped!") - sender ! true - context.stop(self) + logInfo("MapOutputTrackerMasterEndpoint stopped!") + context.reply(true) + stop() } } @@ -75,12 +74,9 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster * (driver and executor) use different HashMap to store its metadata. */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { - private val timeout = AkkaUtils.askTimeout(conf) - private val retryAttempts = AkkaUtils.numRetries(conf) - private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) - /** Set to the MapOutputTrackerActor living on the driver. */ - var trackerActor: ActorRef = _ + /** Set to the MapOutputTrackerMasterEndpoint living on the driver. */ + var trackerEndpoint: RpcEndpointRef = _ /** * This HashMap has different behavior for the driver and the executors. @@ -105,12 +101,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging private val fetching = new HashSet[Int] /** - * Send a message to the trackerActor and get its result within a default timeout, or + * Send a message to the trackerEndpoint and get its result within a default timeout, or * throw a SparkException if this fails. */ - protected def askTracker(message: Any): Any = { + protected def askTracker[T: ClassTag](message: Any): T = { try { - AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout) + trackerEndpoint.askWithRetry[T](message) } catch { case e: Exception => logError("Error communicating with MapOutputTracker", e) @@ -118,9 +114,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } } - /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ + /** Send a one-way message to the trackerEndpoint, to which we expect it to reply with true. */ protected def sendTracker(message: Any) { - val response = askTracker(message) + val response = askTracker[Boolean](message) if (response != true) { throw new SparkException( "Error reply received from MapOutputTracker. Expecting true, got " + response.toString) @@ -157,11 +153,10 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging if (fetchedStatuses == null) { // We won the race to fetch the output locs; do so - logInfo("Doing the fetch; tracker actor = " + trackerActor) + logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint) // This try-finally prevents hangs due to timeouts: try { - val fetchedBytes = - askTracker(GetMapOutputStatuses(shuffleId)).asInstanceOf[Array[Byte]] + val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId)) fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes) logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) @@ -328,7 +323,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) override def stop() { sendTracker(StopMapOutputTracker) mapStatuses.clear() - trackerActor = null + trackerEndpoint = null metadataCleaner.cancel() cachedSerializedStatuses.clear() } @@ -350,17 +345,22 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private[spark] object MapOutputTracker extends Logging { + val ENDPOINT_NAME = "MapOutputTracker" + // 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] = { val out = new ByteArrayOutputStream val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) - // Since statuses can be modified in parallel, sync on it - statuses.synchronized { - objOut.writeObject(statuses) + Utils.tryWithSafeFinally { + // Since statuses can be modified in parallel, sync on it + statuses.synchronized { + objOut.writeObject(statuses) + } + } { + objOut.close() } - objOut.close() out.toByteArray } diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index e53a78ead2c0e..b8d244408bc5b 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -76,7 +76,7 @@ object Partitioner { * produce an unexpected or incorrect result. */ class HashPartitioner(partitions: Int) extends Partitioner { - def numPartitions = partitions + def numPartitions: Int = partitions def getPartition(key: Any): Int = key match { case null => 0 @@ -154,7 +154,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } - def numPartitions = rangeBounds.length + 1 + def numPartitions: Int = rangeBounds.length + 1 private var binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3653f724ba192..8aed1e20e0686 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -150,8 +150,13 @@ import org.apache.spark.util.Utils * authorization. If not filter is in place the user is generally null and no authorization * can take place. * - * Connection encryption (SSL) configuration is organized hierarchically. The user can configure - * the default SSL settings which will be used for all the supported communication protocols unless + * When authentication is being used, encryption can also be enabled by setting the option + * spark.authenticate.enableSaslEncryption to true. This is only supported by communication + * channels that use the network-common library, and can be used as an alternative to SSL in those + * cases. + * + * SSL can be used for encryption for certain communication channels. The user can configure the + * default SSL settings which will be used for all the supported communication protocols unless * they are overwritten by protocol specific settings. This way the user can easily provide the * common settings for all the protocols without disabling the ability to configure each one * individually. @@ -412,6 +417,14 @@ private[spark] class SecurityManager(sparkConf: SparkConf) */ def isAuthenticationEnabled(): Boolean = authOn + /** + * Checks whether SASL encryption should be enabled. + * @return Whether to enable SASL encryption when connecting to services that support it. + */ + def isSaslEncryptionEnabled(): Boolean = { + sparkConf.getBoolean("spark.authenticate.enableSaslEncryption", false) + } + /** * Gets the user used for authenticating HTTP connections. * For now use a single hardcoded user. diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index 55cb25946c2ad..cb2cae185256a 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -28,8 +28,10 @@ import org.apache.spark.util.Utils @DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { - def value = t - override def toString = t.toString + + def value: T = t + + override def toString: String = t.toString private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/SizeEstimator.scala new file mode 100644 index 0000000000000..54fc3a856adfa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SizeEstimator.scala @@ -0,0 +1,44 @@ +/* + * 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 org.apache.spark.annotation.DeveloperApi + +/** + * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in + * memory-aware caches. + * + * Based on the following JavaWorld article: + * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html + */ +@DeveloperApi +object SizeEstimator { + /** + * :: DeveloperApi :: + * Estimate the number of bytes that the given object takes up on the JVM heap. The estimate + * includes space taken up by objects referenced by the given object, their references, and so on + * and so forth. + * + * This is useful for determining the amount of heap space a broadcast variable will occupy on + * each executor or the amount of space each object will take when caching objects in + * deserialized form. This is not the same as the serialized size of the object, which will + * typically be much smaller. + */ + @DeveloperApi + def estimate(obj: AnyRef): Long = org.apache.spark.util.SizeEstimator.estimate(obj) +} diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 2ca19f53d2f07..a8fc90ad2050e 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -68,6 +68,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (value == null) { throw new NullPointerException("null value for " + key) } + logDeprecationWarning(key) settings.put(key, value) this } @@ -133,14 +134,16 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } /** Set multiple parameters together */ - def setAll(settings: Traversable[(String, String)]) = { - this.settings.putAll(settings.toMap.asJava) + def setAll(settings: Traversable[(String, String)]): SparkConf = { + settings.foreach { case (k, v) => set(k, v) } this } /** Set a parameter if it isn't already configured */ def setIfMissing(key: String, value: String): SparkConf = { - settings.putIfAbsent(key, value) + if (settings.putIfAbsent(key, value) == null) { + logDeprecationWarning(key) + } this } @@ -174,9 +177,111 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsSeconds(key: String): Long = { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsMs(key: String): Long = { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String, defaultValue: String): Long = { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** + * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then bytes are assumed. + * @throws NoSuchElementException + */ + def getSizeAsBytes(key: String): Long = { + Utils.byteStringAsBytes(get(key)) + } + + /** + * Get a size parameter as bytes, falling back to a default if not set. If no + * suffix is provided then bytes are assumed. + */ + def getSizeAsBytes(key: String, defaultValue: String): Long = { + Utils.byteStringAsBytes(get(key, defaultValue)) + } + + /** + * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Kibibytes are assumed. + * @throws NoSuchElementException + */ + def getSizeAsKb(key: String): Long = { + Utils.byteStringAsKb(get(key)) + } + + /** + * Get a size parameter as Kibibytes, falling back to a default if not set. If no + * suffix is provided then Kibibytes are assumed. + */ + def getSizeAsKb(key: String, defaultValue: String): Long = { + Utils.byteStringAsKb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Mebibytes are assumed. + * @throws NoSuchElementException + */ + def getSizeAsMb(key: String): Long = { + Utils.byteStringAsMb(get(key)) + } + + /** + * Get a size parameter as Mebibytes, falling back to a default if not set. If no + * suffix is provided then Mebibytes are assumed. + */ + def getSizeAsMb(key: String, defaultValue: String): Long = { + Utils.byteStringAsMb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Gibibytes are assumed. + * @throws NoSuchElementException + */ + def getSizeAsGb(key: String): Long = { + Utils.byteStringAsGb(get(key)) + } + + /** + * Get a size parameter as Gibibytes, falling back to a default if not set. If no + * suffix is provided then Gibibytes are assumed. + */ + def getSizeAsGb(key: String, defaultValue: String): Long = { + Utils.byteStringAsGb(get(key, defaultValue)) + } + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { - Option(settings.get(key)) + Option(settings.get(key)).orElse(getDeprecatedConfig(key, this)) } /** Get all parameters as a list of pairs */ @@ -343,13 +448,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } } - - // Warn against the use of deprecated configs - deprecatedConfigs.values.foreach { dc => - if (contains(dc.oldName)) { - dc.warn() - } - } } /** @@ -364,19 +462,84 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { private[spark] object SparkConf extends Logging { + /** + * Maps deprecated config keys to information about the deprecation. + * + * The extra information is logged as a warning when the config is present in the user's + * configuration. + */ private val deprecatedConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( - DeprecatedConfig("spark.files.userClassPathFirst", "spark.executor.userClassPathFirst", - "1.3"), - DeprecatedConfig("spark.yarn.user.classpath.first", null, "1.3", - "Use spark.{driver,executor}.userClassPathFirst instead."), - DeprecatedConfig("spark.history.fs.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead"), - DeprecatedConfig("spark.history.updateInterval", - "spark.history.fs.update.interval.seconds", - "1.3", "Use spark.history.fs.update.interval.seconds instead")) - configs.map { x => (x.oldName, x) }.toMap + DeprecatedConfig("spark.cache.class", "0.8", + "The spark.cache.class property is no longer being used! Specify storage levels using " + + "the RDD.persist() method instead."), + DeprecatedConfig("spark.yarn.user.classpath.first", "1.3", + "Please use spark.{driver,executor}.userClassPathFirst instead."), + DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4", + "Please use spark.kryoserializer.buffer instead. The default value for " + + "spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " + + "are no longer accepted. To specify the equivalent now, one may use '64k'.") + ) + + Map(configs.map { cfg => (cfg.key -> cfg) }:_*) + } + + /** + * Maps a current config key to alternate keys that were used in previous version of Spark. + * + * The alternates are used in the order defined in this map. If deprecated configs are + * present in the user's configuration, a warning is logged. + */ + private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( + "spark.executor.userClassPathFirst" -> Seq( + AlternateConfig("spark.files.userClassPathFirst", "1.3")), + "spark.history.fs.update.interval" -> Seq( + AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), + AlternateConfig("spark.history.fs.updateInterval", "1.3"), + AlternateConfig("spark.history.updateInterval", "1.3")), + "spark.history.fs.cleaner.interval" -> Seq( + AlternateConfig("spark.history.fs.cleaner.interval.seconds", "1.4")), + "spark.history.fs.cleaner.maxAge" -> Seq( + AlternateConfig("spark.history.fs.cleaner.maxAge.seconds", "1.4")), + "spark.yarn.am.waitTime" -> Seq( + AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", + // Translate old value to a duration, with 10s wait time per try. + translation = s => s"${s.toLong * 10}s")), + "spark.reducer.maxSizeInFlight" -> Seq( + AlternateConfig("spark.reducer.maxMbInFlight", "1.4")), + "spark.kryoserializer.buffer" -> + Seq(AlternateConfig("spark.kryoserializer.buffer.mb", "1.4", + translation = s => s"${s.toDouble * 1000}k")), + "spark.kryoserializer.buffer.max" -> Seq( + AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")), + "spark.shuffle.file.buffer" -> Seq( + AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), + "spark.executor.logs.rolling.maxSize" -> Seq( + AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), + "spark.io.compression.snappy.blockSize" -> Seq( + AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), + "spark.io.compression.lz4.blockSize" -> Seq( + AlternateConfig("spark.io.compression.lz4.block.size", "1.4")), + "spark.rpc.numRetries" -> Seq( + AlternateConfig("spark.akka.num.retries", "1.4")), + "spark.rpc.retry.wait" -> Seq( + AlternateConfig("spark.akka.retry.wait", "1.4")), + "spark.rpc.askTimeout" -> Seq( + AlternateConfig("spark.akka.askTimeout", "1.4")), + "spark.rpc.lookupTimeout" -> Seq( + AlternateConfig("spark.akka.lookupTimeout", "1.4")) + ) + + /** + * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated + * config keys. + * + * Maps the deprecated config name to a 2-tuple (new config name, alternate config info). + */ + private val allAlternatives: Map[String, (String, AlternateConfig)] = { + configsWithAlternatives.keys.flatMap { key => + configsWithAlternatives(key).map { cfg => (cfg.key -> (key -> cfg)) } + }.toMap } /** @@ -407,61 +570,57 @@ private[spark] object SparkConf extends Logging { } /** - * Translate the configuration key if it is deprecated and has a replacement, otherwise just - * returns the provided key. - * - * @param userKey Configuration key from the user / caller. - * @param warn Whether to print a warning if the key is deprecated. Warnings will be printed - * only once for each key. + * Looks for available deprecated keys for the given config option, and return the first + * value available. */ - private def translateConfKey(userKey: String, warn: Boolean = false): String = { - deprecatedConfigs.get(userKey) - .map { deprecatedKey => - if (warn) { - deprecatedKey.warn() - } - deprecatedKey.newName.getOrElse(userKey) - }.getOrElse(userKey) + def getDeprecatedConfig(key: String, conf: SparkConf): Option[String] = { + configsWithAlternatives.get(key).flatMap { alts => + alts.collectFirst { case alt if conf.contains(alt.key) => + val value = conf.get(alt.key) + if (alt.translation != null) alt.translation(value) else value + } + } + } + + /** + * Logs a warning message if the given config key is deprecated. + */ + def logDeprecationWarning(key: String): Unit = { + deprecatedConfigs.get(key).foreach { cfg => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"may be removed in the future. ${cfg.deprecationMessage}") + } + + allAlternatives.get(key).foreach { case (newKey, cfg) => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"and may be removed in the future. Please use the new key '$newKey' instead.") + } } /** - * Holds information about keys that have been deprecated or renamed. + * Holds information about keys that have been deprecated and do not have a replacement. * - * @param oldName Old configuration key. - * @param newName New configuration key, or `null` if key has no replacement, in which case the - * deprecated key will be used (but the warning message will still be printed). + * @param key The deprecated key. * @param version Version of Spark where key was deprecated. - * @param deprecationMessage Message to include in the deprecation warning; mandatory when - * `newName` is not provided. + * @param deprecationMessage Message to include in the deprecation warning. */ private case class DeprecatedConfig( - oldName: String, - _newName: String, + key: String, version: String, - deprecationMessage: String = null) { - - private val warned = new AtomicBoolean(false) - val newName = Option(_newName) - - if (newName == null && (deprecationMessage == null || deprecationMessage.isEmpty())) { - throw new IllegalArgumentException("Need new config name or deprecation message.") - } + deprecationMessage: String) - def warn(): Unit = { - if (warned.compareAndSet(false, true)) { - if (newName != null) { - val message = Option(deprecationMessage).getOrElse( - s"Please use the alternative '$newName' instead.") - logWarning( - s"The configuration option '$oldName' has been replaced as of Spark $version and " + - s"may be removed in the future. $message") - } else { - logWarning( - s"The configuration option '$oldName' has been deprecated as of Spark $version and " + - s"may be removed in the future. $deprecationMessage") - } - } - } + /** + * Information about an alternate configuration key that has been deprecated. + * + * @param key The deprecated config key. + * @param version The Spark version in which the key was deprecated. + * @param translation A translation function for converting old config values into new ones. + */ + private case class AlternateConfig( + key: String, + version: String, + translation: String => String = null) - } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 228ff715fe7cb..2ca6882c8d890 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -23,7 +23,7 @@ import java.io._ import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID import scala.collection.{Map, Set} @@ -31,8 +31,7 @@ import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} - -import akka.actor.Props +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -48,12 +47,14 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} -import org.apache.spark.executor.TriggerThreadDump +import org.apache.spark.executor.{ExecutorEndpoint, TriggerThreadDump} import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.io.CompressionCodec +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} @@ -95,10 +96,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val startTime = System.currentTimeMillis() - @volatile private var stopped: Boolean = false + private val stopped: AtomicBoolean = new AtomicBoolean(false) private def assertNotStopped(): Unit = { - if (stopped) { + if (stopped.get()) { throw new IllegalStateException("Cannot call methods on a stopped SparkContext") } } @@ -193,8 +194,44 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - private[spark] val conf = config.clone() - conf.validateSettings() + /* ------------------------------------------------------------------------------------- * + | Private variables. These variables keep the internal state of the context, and are | + | not accessible by the outside world. They're mutable since we want to initialize all | + | of them to some neutral value ahead of time, so that calling "stop()" while the | + | constructor is still running is safe. | + * ------------------------------------------------------------------------------------- */ + + private var _conf: SparkConf = _ + private var _eventLogDir: Option[URI] = None + private var _eventLogCodec: Option[String] = None + private var _env: SparkEnv = _ + private var _metadataCleaner: MetadataCleaner = _ + private var _jobProgressListener: JobProgressListener = _ + private var _statusTracker: SparkStatusTracker = _ + private var _progressBar: Option[ConsoleProgressBar] = None + private var _ui: Option[SparkUI] = None + private var _hadoopConfiguration: Configuration = _ + private var _executorMemory: Int = _ + private var _schedulerBackend: SchedulerBackend = _ + private var _taskScheduler: TaskScheduler = _ + private var _heartbeatReceiver: RpcEndpointRef = _ + @volatile private var _dagScheduler: DAGScheduler = _ + private var _applicationId: String = _ + private var _applicationAttemptId: Option[String] = None + private var _eventLogger: Option[EventLoggingListener] = None + private var _executorAllocationManager: Option[ExecutorAllocationManager] = None + private var _cleaner: Option[ContextCleaner] = None + private var _listenerBusStarted: Boolean = false + private var _jars: Seq[String] = _ + private var _files: Seq[String] = _ + private var _shutdownHookRef: AnyRef = _ + + /* ------------------------------------------------------------------------------------- * + | Accessors and public fields. These provide access to the internal state of the | + | context. | + * ------------------------------------------------------------------------------------- */ + + private[spark] def conf: SparkConf = _conf /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -202,63 +239,26 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def getConf: SparkConf = conf.clone() - if (!conf.contains("spark.master")) { - throw new SparkException("A master URL must be set in your configuration") - } - if (!conf.contains("spark.app.name")) { - throw new SparkException("An application name must be set in your configuration") - } - - if (conf.getBoolean("spark.logConf", false)) { - logInfo("Spark configuration:\n" + conf.toDebugString) - } - - // Set Spark driver host and port system properties - conf.setIfMissing("spark.driver.host", Utils.localHostName()) - conf.setIfMissing("spark.driver.port", "0") + def jars: Seq[String] = _jars + def files: Seq[String] = _files + def master: String = _conf.get("spark.master") + def appName: String = _conf.get("spark.app.name") - val jars: Seq[String] = - conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val files: Seq[String] = - conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val master = conf.get("spark.master") - val appName = conf.get("spark.app.name") - - private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - private[spark] val eventLogDir: Option[String] = { - if (isEventLogEnabled) { - Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) - } else { - None - } - } - private[spark] val eventLogCodec: Option[String] = { - val compress = conf.getBoolean("spark.eventLog.compress", false) - if (compress && isEventLogEnabled) { - Some(CompressionCodec.getCodecName(conf)).map(CompressionCodec.getShortName) - } else { - None - } - } + private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def eventLogDir: Option[URI] = _eventLogDir + private[spark] def eventLogCodec: Option[String] = _eventLogCodec - // Generate the random name for a temp folder in Tachyon + // Generate the random name for a temp folder in external block store. // Add a timestamp as the suffix here to make it more safe - val tachyonFolderName = "spark-" + randomUUID.toString() - conf.set("spark.tachyonStore.folderName", tachyonFolderName) + val externalBlockStoreFolderName = "spark-" + randomUUID.toString() + @deprecated("Use externalBlockStoreFolderName instead.", "1.4.0") + val tachyonFolderName = externalBlockStoreFolderName - val isLocal = (master == "local" || master.startsWith("local[")) - - if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + def isLocal: Boolean = (master == "local" || master.startsWith("local[")) // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - - // Create the Spark execution environment (cache, map output tracker, etc) - // This function allows components created by SparkEnv to be mocked in unit tests: private[spark] def createSparkEnv( conf: SparkConf, @@ -267,8 +267,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli SparkEnv.createDriverEnv(conf, isLocal, listenerBus) } - private[spark] val env = createSparkEnv(conf, isLocal, listenerBus) - SparkEnv.set(env) + private[spark] def env: SparkEnv = _env // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() @@ -276,35 +275,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - - - private[spark] val jobProgressListener = new JobProgressListener(conf) - listenerBus.addListener(jobProgressListener) + private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner + private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener - val statusTracker = new SparkStatusTracker(this) + def statusTracker: SparkStatusTracker = _statusTracker - private[spark] val progressBar: Option[ConsoleProgressBar] = - if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { - Some(new ConsoleProgressBar(this)) - } else { - None - } + private[spark] def progressBar: Option[ConsoleProgressBar] = _progressBar - // Initialize the Spark UI - private[spark] val ui: Option[SparkUI] = - if (conf.getBoolean("spark.ui.enabled", true)) { - Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, - env.securityManager,appName)) - } else { - // For tests, do not enable the UI - None - } - - // Bind the UI before starting the task scheduler to communicate - // the bound port to the cluster manager properly - ui.foreach(_.bind()) + private[spark] def ui: Option[SparkUI] = _ui /** * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. @@ -312,127 +290,275 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you * plan to set some global configurations for all Hadoop RDDs. */ - val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) + def hadoopConfiguration: Configuration = _hadoopConfiguration + + private[spark] def executorMemory: Int = _executorMemory + + // Environment variables to pass to our executors. + private[spark] val executorEnvs = HashMap[String, String]() - // Add each JAR given through the constructor - if (jars != null) { - jars.foreach(addJar) + // Set SPARK_USER for user who is running SparkContext. + val sparkUser = Utils.getCurrentUserName() + + private[spark] def schedulerBackend: SchedulerBackend = _schedulerBackend + private[spark] def schedulerBackend_=(sb: SchedulerBackend): Unit = { + _schedulerBackend = sb } - if (files != null) { - files.foreach(addFile) + private[spark] def taskScheduler: TaskScheduler = _taskScheduler + private[spark] def taskScheduler_=(ts: TaskScheduler): Unit = { + _taskScheduler = ts + } + + private[spark] def dagScheduler: DAGScheduler = _dagScheduler + private[spark] def dagScheduler_=(ds: DAGScheduler): Unit = { + _dagScheduler = ds + } + + def applicationId: String = _applicationId + def applicationAttemptId: Option[String] = _applicationAttemptId + + def metricsSystem: MetricsSystem = if (_env != null) _env.metricsSystem else null + + private[spark] def eventLogger: Option[EventLoggingListener] = _eventLogger + + private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = + _executorAllocationManager + + private[spark] def cleaner: Option[ContextCleaner] = _cleaner + + private[spark] var checkpointDir: Option[String] = None + + // Thread Local variable that can be used by users to pass information down the stack + private val localProperties = new InheritableThreadLocal[Properties] { + override protected def childValue(parent: Properties): Properties = new Properties(parent) + override protected def initialValue(): Properties = new Properties() } + /* ------------------------------------------------------------------------------------- * + | Initialization. This code initializes the context in a manner that is exception-safe. | + | All internal fields holding state are initialized here, and any error prompts the | + | stop() method to be called. | + * ------------------------------------------------------------------------------------- */ + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value } - private[spark] val executorMemory = conf.getOption("spark.executor.memory") - .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) - .orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem)) - .map(Utils.memoryStringToMb) - .getOrElse(512) - - // Environment variables to pass to our executors. - private[spark] val executorEnvs = HashMap[String, String]() - - // Convert java options to env vars as a work around - // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) - value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value - } - Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => - executorEnvs("SPARK_PREPEND_CLASSES") = v + /** Control our logLevel. This overrides any user-defined log settings. + * @param logLevel The desired log level as a string. + * Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN + */ + def setLogLevel(logLevel: String) { + val validLevels = Seq("ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN") + if (!validLevels.contains(logLevel)) { + throw new IllegalArgumentException( + s"Supplied level $logLevel did not match one of: ${validLevels.mkString(",")}") + } + Utils.setLogLevel(org.apache.log4j.Level.toLevel(logLevel)) } - // The Mesos scheduler backend relies on this environment variable to set executor memory. - // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv - // Set SPARK_USER for user who is running SparkContext. - val sparkUser = Utils.getCurrentUserName() - executorEnvs("SPARK_USER") = sparkUser - - // Create and start the scheduler - private[spark] var (schedulerBackend, taskScheduler) = - SparkContext.createTaskScheduler(this, master) - private val heartbeatReceiver = env.actorSystem.actorOf( - Props(new HeartbeatReceiver(this, taskScheduler)), "HeartbeatReceiver") - @volatile private[spark] var dagScheduler: DAGScheduler = _ try { - dagScheduler = new DAGScheduler(this) - } catch { - case e: Exception => { - try { - stop() - } finally { - throw new SparkException("Error while constructing DAGScheduler", e) + _conf = config.clone() + _conf.validateSettings() + + if (!_conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!_conf.contains("spark.app.name")) { + throw new SparkException("An application name must be set in your configuration") + } + + if (_conf.getBoolean("spark.logConf", false)) { + logInfo("Spark configuration:\n" + _conf.toDebugString) + } + + // Set Spark driver host and port system properties + _conf.setIfMissing("spark.driver.host", Utils.localHostName()) + _conf.setIfMissing("spark.driver.port", "0") + + _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) + + _jars =_conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)) + .toSeq.flatten + + _eventLogDir = + if (isEventLogEnabled) { + val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) + .stripSuffix("/") + Some(Utils.resolveURI(unresolvedDir)) + } else { + None + } + + _eventLogCodec = { + val compress = _conf.getBoolean("spark.eventLog.compress", false) + if (compress && isEventLogEnabled) { + Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) + } else { + None } } - } - // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's - // constructor - taskScheduler.start() + _conf.set("spark.externalBlockStore.folderName", externalBlockStoreFolderName) - val applicationId: String = taskScheduler.applicationId() - conf.set("spark.app.id", applicationId) + if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - env.blockManager.initialize(applicationId) + // Create the Spark execution environment (cache, map output tracker, etc) + _env = createSparkEnv(_conf, isLocal, listenerBus) + SparkEnv.set(_env) - val metricsSystem = env.metricsSystem + _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - // The metrics system for Driver need to be set spark.app.id to app ID. - // So it should start after we get app ID from the task scheduler and set spark.app.id. - metricsSystem.start() - // Attach the driver metrics servlet handler to the web ui after the metrics system is started. - metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (isEventLogEnabled) { - val logger = - new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } + _statusTracker = new SparkStatusTracker(this) - // Optionally scale number of executors dynamically based on workload. Exposed for testing. - private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) - private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) - private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (dynamicAllocationEnabled) { - assert(supportDynamicAllocation, - "Dynamic allocation of executors is currently only supported in YARN mode") - Some(new ExecutorAllocationManager(this, listenerBus, conf)) - } else { - None + _progressBar = + if (_conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { + Some(new ConsoleProgressBar(this)) + } else { + None + } + + _ui = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, + _env.securityManager,appName, startTime = startTime)) + } else { + // For tests, do not enable the UI + None + } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + _ui.foreach(_.bind()) + + _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) + + // Add each JAR given through the constructor + if (jars != null) { + jars.foreach(addJar) } - executorAllocationManager.foreach(_.start()) - private[spark] val cleaner: Option[ContextCleaner] = { - if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { - Some(new ContextCleaner(this)) - } else { - None + if (files != null) { + files.foreach(addFile) } - } - cleaner.foreach(_.start()) - setupAndStartListenerBus() - postEnvironmentUpdate() - postApplicationStart() + _executorMemory = _conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) + .orElse(Option(System.getenv("SPARK_MEM")) + .map(warnSparkMem)) + .map(Utils.memoryStringToMb) + .getOrElse(512) + + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + executorEnvs("SPARK_PREPEND_CLASSES") = v + } + // The Mesos scheduler backend relies on this environment variable to set executor memory. + // TODO: Set this only in the Mesos scheduler. + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= _conf.getExecutorEnv + executorEnvs("SPARK_USER") = sparkUser + + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will + // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) + _heartbeatReceiver = env.rpcEnv.setupEndpoint( + HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + + // Create and start the scheduler + val (sched, ts) = SparkContext.createTaskScheduler(this, master) + _schedulerBackend = sched + _taskScheduler = ts + _dagScheduler = new DAGScheduler(this) + _heartbeatReceiver.send(TaskSchedulerIsSet) + + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + _taskScheduler.start() + + _applicationId = _taskScheduler.applicationId() + _applicationAttemptId = taskScheduler.applicationAttemptId() + _conf.set("spark.app.id", _applicationId) + _env.blockManager.initialize(_applicationId) + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. + metricsSystem.start() + // Attach the driver metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + + _eventLogger = + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(_applicationId, _applicationAttemptId, _eventLogDir.get, + _conf, _hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else { + None + } - private[spark] var checkpointDir: Option[String] = None + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + val dynamicAllocationEnabled = _conf.getBoolean("spark.dynamicAllocation.enabled", false) + _executorAllocationManager = + if (dynamicAllocationEnabled) { + assert(supportDynamicAllocation, + "Dynamic allocation of executors is currently only supported in YARN mode") + Some(new ExecutorAllocationManager(this, listenerBus, _conf)) + } else { + None + } + _executorAllocationManager.foreach(_.start()) - // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { - override protected def childValue(parent: Properties): Properties = new Properties(parent) + _cleaner = + if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + + setupAndStartListenerBus() + postEnvironmentUpdate() + postApplicationStart() + + // Post init + _taskScheduler.postStartHook() + _env.metricsSystem.registerSource(new DAGSchedulerSource(dagScheduler)) + _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) + _executorAllocationManager.foreach { e => + _env.metricsSystem.registerSource(e.executorAllocationManagerSource) + } + + // Make sure the context is stopped if the user forgets about it. This avoids leaving + // unfinished event logs around after the JVM exits cleanly. It doesn't help if the JVM + // is killed, though. + _shutdownHookRef = Utils.addShutdownHook(Utils.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () => + logInfo("Invoking stop() from shutdown hook") + stop() + } + } catch { + case NonFatal(e) => + logError("Error initializing SparkContext.", e) + try { + stop() + } catch { + case NonFatal(inner) => + logError("Error stopping SparkContext after init error.", inner) + } finally { + throw e + } } /** @@ -446,10 +572,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (executorId == SparkContext.DRIVER_IDENTIFIER) { Some(Utils.getThreadDump()) } else { - val (host, port) = env.blockManager.master.getActorSystemHostPortForExecutor(executorId).get - val actorRef = AkkaUtils.makeExecutorRef("ExecutorActor", conf, host, port, env.actorSystem) - Some(AkkaUtils.askWithReply[Array[ThreadStackTrace]](TriggerThreadDump, actorRef, - AkkaUtils.numRetries(conf), AkkaUtils.retryWaitMs(conf), AkkaUtils.askTimeout(conf))) + val (host, port) = env.blockManager.master.getRpcHostPortForExecutor(executorId).get + val endpointRef = env.rpcEnv.setupEndpointRef( + SparkEnv.executorActorSystemName, + RpcAddress(host, port), + ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME) + Some(endpointRef.askWithRetry[Array[ThreadStackTrace]](TriggerThreadDump)) } } catch { case e: Exception => @@ -474,9 +602,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Spark fair scheduler pool. */ def setLocalProperty(key: String, value: String) { - if (localProperties.get() == null) { - localProperties.set(new Properties()) - } if (value == null) { localProperties.get.remove(key) } else { @@ -537,18 +662,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } - // Post init - taskScheduler.postStartHook() - - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - - private def initDriverMetrics() { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) - } - - initDriverMetrics() + /** + * Execute a block of code in a scope such that all new RDDs created in this body will + * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}. + * + * Note: Return statements are NOT allowed in the given body. + */ + private def withScope[U](body: => U): U = RDDOperationScope.withScope[U](this)(body) // Methods for creating RDDs @@ -560,7 +680,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @note avoid using `parallelize(Seq())` to create an empty `RDD`. Consider `emptyRDD` for an * RDD with no partitions, or `parallelize(Seq[T]())` for an RDD of `T` with empty partitions. */ - def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { + def parallelize[T: ClassTag]( + seq: Seq[T], + numSlices: Int = defaultParallelism): RDD[T] = withScope { assertNotStopped() new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) } @@ -569,14 +691,16 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * * This method is identical to `parallelize`. */ - def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { + def makeRDD[T: ClassTag]( + seq: Seq[T], + numSlices: Int = defaultParallelism): RDD[T] = withScope { parallelize(seq, numSlices) } /** Distribute a local Scala collection to form an RDD, with one or more * location preferences (hostnames of Spark nodes) for each object. * Create a new partition for each collection item. */ - def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { + def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) @@ -586,10 +710,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Read a text file from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI, and return it as an RDD of Strings. */ - def textFile(path: String, minPartitions: Int = defaultMinPartitions): RDD[String] = { + def textFile( + path: String, + minPartitions: Int = defaultMinPartitions): RDD[String] = withScope { assertNotStopped() hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], - minPartitions).map(pair => pair._2.toString).setName(path) + minPartitions).map(pair => pair._2.toString) } /** @@ -619,11 +745,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * * @param minPartitions A suggestion value of the minimal splitting number for input data. */ - def wholeTextFiles(path: String, minPartitions: Int = defaultMinPartitions): - RDD[(String, String)] = { + def wholeTextFiles( + path: String, + minPartitions: Int = defaultMinPartitions): RDD[(String, String)] = withScope { assertNotStopped() val job = new NewHadoopJob(hadoopConfiguration) - NewFileInputFormat.addInputPath(job, new Path(path)) + // Use setInputPaths so that wholeTextFiles aligns with hadoopFile/textFile in taking + // comma separated files as input. (see SPARK-7155) + NewFileInputFormat.setInputPaths(job, path) val updateConf = job.getConfiguration new WholeTextFileRDD( this, @@ -665,11 +794,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @note Small files are preferred; very large files may cause bad performance. */ @Experimental - def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): - RDD[(String, PortableDataStream)] = { + def binaryFiles( + path: String, + minPartitions: Int = defaultMinPartitions): RDD[(String, PortableDataStream)] = withScope { assertNotStopped() val job = new NewHadoopJob(hadoopConfiguration) - NewFileInputFormat.addInputPath(job, new Path(path)) + // Use setInputPaths so that binaryFiles aligns with hadoopFile/textFile in taking + // comma separated files as input. (see SPARK-7155) + NewFileInputFormat.setInputPaths(job, path) val updateConf = job.getConfiguration new BinaryFileRDD( this, @@ -693,8 +825,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @return An RDD of data with values, represented as byte arrays */ @Experimental - def binaryRecords(path: String, recordLength: Int, conf: Configuration = hadoopConfiguration) - : RDD[Array[Byte]] = { + def binaryRecords( + path: String, + recordLength: Int, + conf: Configuration = hadoopConfiguration): RDD[Array[Byte]] = withScope { assertNotStopped() conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path, @@ -735,8 +869,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minPartitions: Int = defaultMinPartitions - ): RDD[(K, V)] = { + minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope { assertNotStopped() // Add necessary security credentials to the JobConf before broadcasting it. SparkHadoopUtil.get.addCredentials(conf) @@ -756,8 +889,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], valueClass: Class[V], - minPartitions: Int = defaultMinPartitions - ): RDD[(K, V)] = { + minPartitions: Int = defaultMinPartitions): RDD[(K, V)] = withScope { assertNotStopped() // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration)) @@ -788,7 +920,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def hadoopFile[K, V, F <: InputFormat[K, V]] (path: String, minPartitions: Int) - (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope { hadoopFile(path, fm.runtimeClass.asInstanceOf[Class[F]], km.runtimeClass.asInstanceOf[Class[K]], @@ -811,13 +943,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * copy them using a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) - (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope { hadoopFile[K, V, F](path, defaultMinPartitions) + } /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */ def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]] (path: String) - (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = { + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = withScope { newAPIHadoopFile( path, fm.runtimeClass.asInstanceOf[Class[F]], @@ -840,12 +973,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli fClass: Class[F], kClass: Class[K], vClass: Class[V], - conf: Configuration = hadoopConfiguration): RDD[(K, V)] = { + conf: Configuration = hadoopConfiguration): RDD[(K, V)] = withScope { assertNotStopped() // The call to new NewHadoopJob automatically adds security credentials to conf, // so we don't need to explicitly add them ourselves val job = new NewHadoopJob(conf) - NewFileInputFormat.addInputPath(job, new Path(path)) + // Use setInputPaths so that newAPIHadoopFile aligns with hadoopFile/textFile in taking + // comma separated files as input. (see SPARK-7155) + NewFileInputFormat.setInputPaths(job, path) val updatedConf = job.getConfiguration new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf).setName(path) } @@ -872,7 +1007,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli conf: Configuration = hadoopConfiguration, fClass: Class[F], kClass: Class[K], - vClass: Class[V]): RDD[(K, V)] = { + vClass: Class[V]): RDD[(K, V)] = withScope { assertNotStopped() // Add necessary security credentials to the JobConf. Required to access secure HDFS. val jconf = new JobConf(conf) @@ -892,7 +1027,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli keyClass: Class[K], valueClass: Class[V], minPartitions: Int - ): RDD[(K, V)] = { + ): RDD[(K, V)] = withScope { assertNotStopped() val inputFormatClass = classOf[SequenceFileInputFormat[K, V]] hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions) @@ -906,7 +1041,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first * copy them using a `map` function. * */ - def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] = { + def sequenceFile[K, V]( + path: String, + keyClass: Class[K], + valueClass: Class[V]): RDD[(K, V)] = withScope { assertNotStopped() sequenceFile(path, keyClass, valueClass, defaultMinPartitions) } @@ -936,16 +1074,17 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def sequenceFile[K, V] (path: String, minPartitions: Int = defaultMinPartitions) (implicit km: ClassTag[K], vm: ClassTag[V], - kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]) - : RDD[(K, V)] = { - assertNotStopped() - val kc = kcf() - val vc = vcf() - val format = classOf[SequenceFileInputFormat[Writable, Writable]] - val writables = hadoopFile(path, format, + kcf: () => WritableConverter[K], vcf: () => WritableConverter[V]): RDD[(K, V)] = { + withScope { + assertNotStopped() + val kc = kcf() + val vc = vcf() + val format = classOf[SequenceFileInputFormat[Writable, Writable]] + val writables = hadoopFile(path, format, kc.writableClass(km).asInstanceOf[Class[Writable]], vc.writableClass(vm).asInstanceOf[Class[Writable]], minPartitions) - writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) } + writables.map { case (k, v) => (kc.convert(k), vc.convert(v)) } + } } /** @@ -958,23 +1097,20 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def objectFile[T: ClassTag]( path: String, - minPartitions: Int = defaultMinPartitions - ): RDD[T] = { + minPartitions: Int = defaultMinPartitions): RDD[T] = withScope { assertNotStopped() sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minPartitions) .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes, Utils.getContextOrSparkClassLoader)) } - protected[spark] def checkpointFile[T: ClassTag]( - path: String - ): RDD[T] = { + protected[spark] def checkpointFile[T: ClassTag](path: String): RDD[T] = withScope { new CheckpointRDD[T](this, path) } /** Build the union of a list of RDDs. */ - def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = { + def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = withScope { val partitioners = rdds.flatMap(_.partitioner).toSet - if (partitioners.size == 1) { + if (rdds.forall(_.partitioner.isDefined) && partitioners.size == 1) { new PartitionerAwareUnionRDD(this, rdds) } else { new UnionRDD(this, rdds) @@ -982,11 +1118,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** Build the union of a list of RDDs passed as variable-length arguments. */ - def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = + def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = withScope { union(Seq(first) ++ rest) + } /** Get an RDD that has no partitions or elements. */ - def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + def emptyRDD[T: ClassTag]: EmptyRDD[T] = new EmptyRDD[T](this) // Methods for creating shared variables @@ -994,7 +1131,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`. */ - def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = + def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { val acc = new Accumulator(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) @@ -1006,7 +1143,8 @@ 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`. */ - def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = { + def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) + : Accumulator[T] = { val acc = new Accumulator(initialValue, param, Some(name)) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1018,7 +1156,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = { + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1031,7 +1170,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @tparam R accumulator result type * @tparam T type that can be added to the accumulator */ - def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = { + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { val acc = new Accumulable(initialValue, param, Some(name)) cleaner.foreach(_.registerAccumulatorForCleanup(acc)) acc @@ -1135,8 +1275,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Return whether dynamically adjusting the amount of resources allocated to * this application is supported. This is currently only available for YARN. */ - private[spark] def supportDynamicAllocation = - master.contains("yarn") || dynamicAllocationTesting + private[spark] def supportDynamicAllocation = + master.contains("yarn") || _conf.getBoolean("spark.dynamicAllocation.testing", false) /** * :: DeveloperApi :: @@ -1153,7 +1293,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * This is currently only supported in YARN mode. Return whether the request is received. */ private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => @@ -1209,7 +1349,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId) /** The version of Spark on which this application is running. */ - def version = SPARK_VERSION + def version: String = SPARK_VERSION /** * Return a map from the slave to the max memory available for caching and the remaining @@ -1303,6 +1443,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register an RDD to be persisted in memory and/or disk storage */ private[spark] def persistRDD(rdd: RDD[_]) { + _executorAllocationManager.foreach { _ => + logWarning( + s"Dynamic allocation currently does not support cached RDDs. Cached data for RDD " + + s"${rdd.id} will be lost when executors are removed.") + } persistentRdds(rdd.id) = rdd } @@ -1389,32 +1534,49 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli addedJars.clear() } - /** Shut down the SparkContext. */ + // Shut down the SparkContext. def stop() { - SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - if (!stopped) { - stopped = true - postApplicationEnd() - ui.foreach(_.stop()) - env.metricsSystem.report() - metadataCleaner.cancel() - cleaner.foreach(_.stop()) - dagScheduler.stop() - dagScheduler = null - listenerBus.stop() - eventLogger.foreach(_.stop()) - env.actorSystem.stop(heartbeatReceiver) - progressBar.foreach(_.stop()) - taskScheduler = null - // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) - logInfo("Successfully stopped SparkContext") - SparkContext.clearActiveContext() - } else { - logInfo("SparkContext already stopped") - } + // Use the stopping variable to ensure no contention for the stop scenario. + // Still track the stopped variable for use elsewhere in the code. + if (!stopped.compareAndSet(false, true)) { + logInfo("SparkContext already stopped.") + return + } + if (_shutdownHookRef != null) { + Utils.removeShutdownHook(_shutdownHookRef) } + + postApplicationEnd() + _ui.foreach(_.stop()) + if (env != null) { + env.metricsSystem.report() + } + if (metadataCleaner != null) { + metadataCleaner.cancel() + } + _cleaner.foreach(_.stop()) + _executorAllocationManager.foreach(_.stop()) + if (_dagScheduler != null) { + _dagScheduler.stop() + _dagScheduler = null + } + if (_listenerBusStarted) { + listenerBus.stop() + _listenerBusStarted = false + } + _eventLogger.foreach(_.stop()) + if (env != null && _heartbeatReceiver != null) { + env.rpcEnv.stop(_heartbeatReceiver) + } + _progressBar.foreach(_.stop()) + _taskScheduler = null + // TODO: Cache.stop()? + if (_env != null) { + _env.stop() + SparkEnv.set(null) + } + SparkContext.clearActiveContext() + logInfo("Successfully stopped SparkContext") } @@ -1476,7 +1638,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - if (stopped) { + if (stopped.get()) { throw new IllegalStateException("SparkContext has been shutdown") } val callSite = getCallSite @@ -1517,7 +1679,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli partitions: Seq[Int], allowLocal: Boolean ): Array[U] = { - runJob(rdd, (context: TaskContext, iter: Iterator[T]) => func(iter), partitions, allowLocal) + val cleanedFunc = clean(func) + runJob(rdd, (ctx: TaskContext, it: Iterator[T]) => cleanedFunc(it), partitions, allowLocal) } /** @@ -1571,7 +1734,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val callSite = getCallSite logInfo("Starting job: " + callSite.shortForm) val start = System.nanoTime - val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, + val cleanedFunc = clean(func) + val result = dagScheduler.runApproximateJob(rdd, cleanedFunc, evaluator, callSite, timeout, localProperties.get) logInfo( "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") @@ -1659,7 +1823,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - def getCheckpointDir = checkpointDir + def getCheckpointDir: Option[String] = checkpointDir /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: Int = { @@ -1737,6 +1901,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } listenerBus.start(this) + _listenerBusStarted = true } /** Post the application start event */ @@ -1744,7 +1909,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser)) + startTime, sparkUser, applicationAttemptId)) } /** Post the application end event */ @@ -1788,11 +1953,12 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. */ - private var activeContext: Option[SparkContext] = None + private val activeContext: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null) /** * Points to a partially-constructed SparkContext if some thread is in the SparkContext @@ -1827,7 +1993,8 @@ object SparkContext extends Logging { logWarning(warnMsg) } - activeContext.foreach { ctx => + if (activeContext.get() != null) { + val ctx = activeContext.get() val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + " To ignore this error, set spark.driver.allowMultipleContexts = true. " + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" @@ -1842,6 +2009,39 @@ object SparkContext extends Logging { } } + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(config: SparkConf): SparkContext = { + // Synchronize to ensure that multiple create requests don't trigger an exception + // from assertNoOtherContextIsRunning within setActiveContext + SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + if (activeContext.get() == null) { + setActiveContext(new SparkContext(config), allowMultipleContexts = false) + } + activeContext.get() + } + } + + /** + * This function may be used to get or instantiate a SparkContext and register it as a + * singleton object. Because we can only have one active SparkContext per JVM, + * this is useful when applications may wish to share a SparkContext. + * + * This method allows not passing a SparkConf (useful if just retrieving). + * + * Note: This function cannot be used to create multiple SparkContext instances + * even if multiple contexts are allowed. + */ + def getOrCreate(): SparkContext = { + getOrCreate(new SparkConf()) + } + /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is * running. Throws an exception if a running context is detected and logs a warning if another @@ -1868,7 +2068,7 @@ object SparkContext extends Logging { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { assertNoOtherContextIsRunning(sc, allowMultipleContexts) contextBeingConstructed = None - activeContext = Some(sc) + activeContext.set(sc) } } @@ -1879,17 +2079,27 @@ object SparkContext extends Logging { */ private[spark] def clearActiveContext(): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - activeContext = None + activeContext.set(null) } } private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" - private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" - private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" + private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" + private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" + + /** + * Executor id for the driver. In earlier versions of Spark, this was ``, but this was + * changed to `driver` because the angle brackets caused escaping issues in URLs and XML (see + * SPARK-6716 for more details). + */ + private[spark] val DRIVER_IDENTIFIER = "driver" - private[spark] val DRIVER_IDENTIFIER = "" + /** + * Legacy version of DRIVER_IDENTIFIER, retained for backwards-compatibility. + */ + private[spark] val LEGACY_DRIVER_IDENTIFIER = "" // The following deprecated objects have already been copied to `object AccumulatorParam` to // make the compiler find them automatically. They are duplicate codes only for backward @@ -1900,28 +2110,28 @@ object SparkContext extends Logging { "backward compatibility.", "1.3.0") object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double) = 0.0 + def zero(initialValue: Double): Double = 0.0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int) = 0 + def zero(initialValue: Int): Int = 0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0L + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + "backward compatibility.", "1.3.0") object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float) = t1 + t2 - def zero(initialValue: Float) = 0f + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f } // The following deprecated functions have already been moved to `object RDD` to @@ -1931,18 +2141,18 @@ object SparkContext extends Logging { @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): PairRDDFunctions[K, V] = RDD.rddToPairRDDFunctions(rdd) - } @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) + def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]): AsyncRDDActions[T] = + RDD.rddToAsyncRDDActions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = { + rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { val kf = implicitly[K => Writable] val vf = implicitly[V => Writable] // Set the Writable class to null and `SequenceFileRDDFunctions` will use Reflection to get it @@ -1954,16 +2164,17 @@ object SparkContext extends Logging { @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(K, V)]): OrderedRDDFunctions[K, V, (K, V)] = RDD.rddToOrderedRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) + def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions = + RDD.doubleRDDToDoubleRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") - def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]): DoubleRDDFunctions = RDD.numericRDDToDoubleRDDFunctions(rdd) // The following deprecated functions have already been moved to `object WritableFactory` to @@ -2129,29 +2340,29 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, 1) + val backend = new LocalBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) case LOCAL_N_REGEX(threads) => - def localCpuCount = Runtime.getRuntime.availableProcessors() + def localCpuCount: Int = Runtime.getRuntime.availableProcessors() // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads. val threadCount = if (threads == "*") localCpuCount else threads.toInt if (threadCount <= 0) { throw new SparkException(s"Asked to run locally with $threadCount threads") } val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => - def localCpuCount = Runtime.getRuntime.availableProcessors() + def localCpuCount: Int = Runtime.getRuntime.availableProcessors() // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2a0c7e756dd3a..0c4d28f786edd 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -24,7 +24,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.util.Properties -import akka.actor._ import com.google.common.collect.MapMaker import org.apache.spark.annotation.DeveloperApi @@ -34,12 +33,15 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.rpc.{RpcEndpointRef, RpcEndpoint, RpcEnv} +import org.apache.spark.rpc.akka.AkkaRpcEnv import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus} -import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor +import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, MemoryAllocator} +import org.apache.spark.util.{RpcUtils, Utils} /** * :: DeveloperApi :: @@ -54,7 +56,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} @DeveloperApi class SparkEnv ( val executorId: String, - val actorSystem: ActorSystem, + private[spark] val rpcEnv: RpcEnv, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -68,9 +70,13 @@ class SparkEnv ( val sparkFilesDir: String, val metricsSystem: MetricsSystem, val shuffleMemoryManager: ShuffleMemoryManager, + val executorMemoryManager: ExecutorMemoryManager, val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { + // TODO Remove actorSystem + val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem + private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -91,14 +97,15 @@ class SparkEnv ( blockManager.master.stop() metricsSystem.stop() outputCommitCoordinator.stop() - actorSystem.shutdown() + rpcEnv.shutdown() + // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. // actorSystem.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 @@ -236,16 +243,15 @@ object SparkEnv extends Logging { val securityManager = new SecurityManager(conf) // Create the ActorSystem for Akka and get the port it binds to. - val (actorSystem, boundPort) = { - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) - } + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + val rpcEnv = RpcEnv.create(actorSystemName, hostname, port, conf, securityManager) + val actorSystem = rpcEnv.asInstanceOf[AkkaRpcEnv].actorSystem // Figure out which port Akka actually bound to in case the original port is 0 or occupied. if (isDriver) { - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) } else { - conf.set("spark.executor.port", boundPort.toString) + conf.set("spark.executor.port", rpcEnv.address.port.toString) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -281,12 +287,14 @@ object SparkEnv extends Logging { val closureSerializer = instantiateClassFromConf[Serializer]( "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") - def registerOrLookup(name: String, newActor: => Actor): ActorRef = { + def registerOrLookupEndpoint( + name: String, endpointCreator: => RpcEndpoint): + RpcEndpointRef = { if (isDriver) { logInfo("Registering " + name) - actorSystem.actorOf(Props(newActor), name = name) + rpcEnv.setupEndpoint(name, endpointCreator) } else { - AkkaUtils.makeDriverRef(name, conf, actorSystem) + RpcUtils.makeDriverRef(name, conf, rpcEnv) } } @@ -298,9 +306,9 @@ object SparkEnv extends Logging { // Have to assign trackerActor after initialization as MapOutputTrackerActor // requires the MapOutputTracker itself - mapOutputTracker.trackerActor = registerOrLookup( - "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + mapOutputTracker.trackerEndpoint = registerOrLookupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint( + rpcEnv, mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) // Let the user specify short names for shuffle managers val shortShuffleMgrNames = Map( @@ -320,12 +328,13 @@ object SparkEnv extends Logging { new NioBlockTransferService(conf, securityManager) } - val blockManagerMaster = new BlockManagerMaster(registerOrLookup( - "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) + val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( + BlockManagerMaster.DRIVER_ENDPOINT_NAME, + new BlockManagerMasterEndpoint(rpcEnv, isLocal, conf, listenerBus)), + conf, isDriver) // NB: blockManager is not valid until initialize() is called later. - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + val blockManager = new BlockManager(executorId, rpcEnv, blockManagerMaster, serializer, conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager, numUsableCores) @@ -368,22 +377,25 @@ object SparkEnv extends Logging { "." } - // Warn about deprecated spark.cache.class property - if (conf.contains("spark.cache.class")) { - logWarning("The spark.cache.class property is no longer being used! Specify storage " + - "levels using the RDD.persist() method instead.") - } - val outputCommitCoordinator = mockOutputCommitCoordinator.getOrElse { new OutputCommitCoordinator(conf) } - val outputCommitCoordinatorActor = registerOrLookup("OutputCommitCoordinator", - new OutputCommitCoordinatorActor(outputCommitCoordinator)) - outputCommitCoordinator.coordinatorActor = Some(outputCommitCoordinatorActor) + val outputCommitCoordinatorRef = registerOrLookupEndpoint("OutputCommitCoordinator", + new OutputCommitCoordinatorEndpoint(rpcEnv, outputCommitCoordinator)) + outputCommitCoordinator.coordinatorRef = Some(outputCommitCoordinatorRef) + + val executorMemoryManager: ExecutorMemoryManager = { + val allocator = if (conf.getBoolean("spark.unsafe.offHeap", false)) { + MemoryAllocator.UNSAFE + } else { + MemoryAllocator.HEAP + } + new ExecutorMemoryManager(allocator) + } val envInstance = new SparkEnv( executorId, - actorSystem, + rpcEnv, serializer, closureSerializer, cacheManager, @@ -397,9 +409,10 @@ object SparkEnv extends Logging { sparkFilesDir, metricsSystem, shuffleMemoryManager, + executorMemoryManager, outputCommitCoordinator, conf) - + // Add a reference to tmp dir created by driver, we will delete this tmp dir when stop() is // 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. diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 6eb4537d10477..2ec42d3aea169 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.executor.CommitDeniedException import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD @@ -104,55 +103,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } def commit() { - val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() - - // Called after we have decided to commit - def performCommit(): Unit = { - try { - cmtr.commitTask(taCtxt) - logInfo (s"$taID: Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e - } - } - - // First, check whether the task's output has already been committed by some other attempt - if (cmtr.needsTaskCommit(taCtxt)) { - // The task output needs to be committed, but we don't know whether some other task attempt - // might be racing to commit the same output partition. Therefore, coordinate with the driver - // in order to determine whether this attempt can commit (see SPARK-4879). - val shouldCoordinateWithDriver: Boolean = { - val sparkConf = SparkEnv.get.conf - // We only need to coordinate with the driver if there are multiple concurrent task - // attempts, which should only occur if speculation is enabled - val speculationEnabled = sparkConf.getBoolean("spark.speculation", false) - // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs - sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) - } - if (shouldCoordinateWithDriver) { - val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator - val canCommit = outputCommitCoordinator.canCommit(jobID, splitID, attemptID) - if (canCommit) { - performCommit() - } else { - val msg = s"$taID: Not committed because the driver did not authorize commit" - logInfo(msg) - // We need to abort the task so that the driver can reschedule new attempts, if necessary - cmtr.abortTask(taCtxt) - throw new CommitDeniedException(msg, jobID, splitID, attemptID) - } - } else { - // Speculation is disabled or a user has chosen to manually bypass the commit coordination - performCommit() - } - } else { - // Some other attempt committed the output, so we do nothing and signal success - logInfo(s"No need to commit output of task because needsTaskCommit=false: ${taID.value}") - } + SparkHadoopMapRedUtil.commitTask( + getOutputCommitter(), getTaskContext(), jobID, splitID, attemptID) } def commitJob() { diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index edbdda8a0bcb6..34ee3a48f8e74 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -45,8 +45,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext) { */ def getJobIdsForGroup(jobGroup: String): Array[Int] = { jobProgressListener.synchronized { - val jobData = jobProgressListener.jobIdToData.valuesIterator - jobData.filter(_.jobGroup.orNull == jobGroup).map(_.jobId).toArray + jobProgressListener.jobGroupToJobIds.getOrElse(jobGroup, Seq.empty).toArray } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 7d7fe1a446313..d09e17dea0911 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,6 +21,7 @@ import java.io.Serializable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.TaskCompletionListener @@ -133,4 +134,9 @@ abstract class TaskContext extends Serializable { /** ::DeveloperApi:: */ @DeveloperApi def taskMetrics(): TaskMetrics + + /** + * Returns the manager for this task's managed memory. + */ + private[spark] def taskMemoryManager(): TaskMemoryManager } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 337c8e4ebebcd..b4d572cb52313 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.apache.spark.executor.TaskMetrics +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} import scala.collection.mutable.ArrayBuffer @@ -27,6 +28,7 @@ private[spark] class TaskContextImpl( val partitionId: Int, override val taskAttemptId: Long, override val attemptNumber: Int, + override val taskMemoryManager: TaskMemoryManager, val runningLocally: Boolean = false, val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index c415fe99b105e..fe19f07e32d1b 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -27,9 +27,9 @@ private[spark] object TaskState extends Enumeration { type TaskState = Value - def isFailed(state: TaskState) = (LOST == state) || (FAILED == state) + def isFailed(state: TaskState): Boolean = (LOST == state) || (FAILED == state) - def isFinished(state: TaskState) = FINISHED_STATES.contains(state) + def isFinished(state: TaskState): Boolean = FINISHED_STATES.contains(state) def toMesos(state: TaskState): MesosTaskState = state match { case LAUNCHING => MesosTaskState.TASK_STARTING diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 35b324ba6f573..398ca41e16151 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -107,7 +107,7 @@ private[spark] object TestUtils { private class JavaSourceFromString(val name: String, val code: String) extends SimpleJavaFileObject(createURI(name), SOURCE) { - override def getCharContent(ignoreEncodingErrors: Boolean) = code + override def getCharContent(ignoreEncodingErrors: Boolean): String = code } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ diff --git a/core/src/main/scala/org/apache/spark/annotation/Private.java b/core/src/main/scala/org/apache/spark/annotation/Private.java new file mode 100644 index 0000000000000..9082fcf0c84bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Private.java @@ -0,0 +1,41 @@ +/* + * 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.annotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * A class that is considered private to the internals of Spark -- there is a high-likelihood + * they will be changed in future versions of Spark. + * + * This should be used only when the standard Scala / Java means of protecting classes are + * insufficient. In particular, Java has no equivalent of private[spark], so we use this annotation + * in its place. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Private ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Private {} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 79e4ebf2db578..61af867b11b9c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -163,6 +163,20 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) /** Add up the elements in this RDD. */ def sum(): JDouble = srdd.sum() + /** + * Returns the minimum element from this RDD as defined by + * the default comparator natural order. + * @return the minimum of the RDD + */ + def min(): JDouble = min(com.google.common.collect.Ordering.natural()) + + /** + * Returns the maximum element from this RDD as defined by + * the default comparator natural order. + * @return the maximum of the RDD + */ + def max(): JDouble = max(com.google.common.collect.Ordering.natural()) + /** * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and * count of the RDD's elements in one operation. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a023712be1166..8441bb3a3047e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -661,7 +661,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ - def fn = (x: V) => f.call(x).asScala + def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 3e9beb670f7ad..db4e996feb31c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -179,7 +179,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) - override def toString = rdd.toString + override def toString: String = rdd.toString /** Assign a name to this RDD */ def setName(name: String): JavaRDD[T] = { @@ -192,7 +192,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) */ def sortBy[S](f: JFunction[T, S], ascending: Boolean, numPartitions: Int): JavaRDD[T] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x) + def fn: (T) => S = (x: T) => f.call(x) import com.google.common.collect.Ordering // shadows scala.math.Ordering implicit val ordering = Ordering.natural().asInstanceOf[Ordering[S]] implicit val ctag: ClassTag[S] = fakeClassTag 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 8da42934a7d96..8bf0627fc420d 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 @@ -17,8 +17,9 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList, Iterator => JIterator} +import java.{lang => jl} import java.lang.{Iterable => JIterable, Long => JLong} +import java.util.{Comparator, List => JList, Iterator => JIterator} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -93,7 +94,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * of the original partition. */ def mapPartitionsWithIndex[R]( - f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]], + f: JFunction2[jl.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), preservesPartitioning)(fakeClassTag))(fakeClassTag) @@ -109,7 +110,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to all elements of this RDD. */ def mapToPair[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - def cm = implicitly[ClassTag[(K2, V2)]] + def cm: ClassTag[(K2, V2)] = implicitly[ClassTag[(K2, V2)]] new JavaPairRDD(rdd.map[(K2, V2)](f)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -119,7 +120,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala JavaRDD.fromRDD(rdd.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -129,8 +130,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMapToDouble(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala - new JavaDoubleRDD(rdd.flatMap(fn).map((x: java.lang.Double) => x.doubleValue())) + def fn: (T) => Iterable[jl.Double] = (x: T) => f.call(x).asScala + new JavaDoubleRDD(rdd.flatMap(fn).map((x: jl.Double) => x.doubleValue())) } /** @@ -139,8 +140,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala - def cm = implicitly[ClassTag[(K2, V2)]] + def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala + def cm: ClassTag[(K2, V2)] = implicitly[ClassTag[(K2, V2)]] JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -148,7 +149,9 @@ 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 fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaRDD.fromRDD(rdd.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -157,7 +160,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -166,8 +171,10 @@ 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 fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) - new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: java.lang.Double) => x.doubleValue())) + def fn: (Iterator[T]) => Iterator[jl.Double] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } + new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: jl.Double) => x.doubleValue())) } /** @@ -175,7 +182,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -184,7 +193,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[jl.Double] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) .map(x => x.doubleValue())) } @@ -194,7 +205,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } JavaPairRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -277,8 +290,10 @@ 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] = { - def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator( - f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) + def fn: (Iterator[T], Iterator[U]) => Iterator[V] = { + (x: Iterator[T], y: Iterator[U]) => asScalaIterator( + f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) + } JavaRDD.fromRDD( rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } @@ -441,8 +456,8 @@ 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, java.lang.Long] = - mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) + def countByValue(): java.util.Map[T, jl.Long] = + mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new jl.Long(x._2))))) /** * (Experimental) Approximate version of countByValue(). 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 6d6ed693be752..02e49a853c5f7 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 @@ -108,7 +108,7 @@ class JavaSparkContext(val sc: SparkContext) private[spark] val env = sc.env - def statusTracker = new JavaSparkStatusTracker(sc) + def statusTracker: JavaSparkStatusTracker = new JavaSparkStatusTracker(sc) def isLocal: java.lang.Boolean = sc.isLocal @@ -755,6 +755,14 @@ class JavaSparkContext(val sc: SparkContext) */ def getLocalProperty(key: String): String = sc.getLocalProperty(key) + /** Control our logLevel. This overrides any user-defined log settings. + * @param logLevel The desired log level as a string. + * Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN + */ + def setLogLevel(logLevel: String) { + sc.setLogLevel(logLevel) + } + /** * Assigns a group ID to all the jobs started by this thread until the group ID is set to a * different value or cleared. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 71b26737b8c02..8f9647eea9e25 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.api.java +import java.util.Map.Entry + import com.google.common.base.Optional import java.{util => ju} @@ -30,8 +32,8 @@ private[spark] object JavaUtils { } // Workaround for SPARK-3926 / SI-8911 - def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = - new SerializableMapWrapper(underlying) + def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]): SerializableMapWrapper[A, B] + = new SerializableMapWrapper(underlying) // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper, // but implements java.io.Serializable. It can't just be subclassed to make it @@ -40,36 +42,33 @@ private[spark] object JavaUtils { class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) extends ju.AbstractMap[A, B] with java.io.Serializable { self => - override def size = underlying.size + override def size: Int = underlying.size override def get(key: AnyRef): B = try { - underlying get key.asInstanceOf[A] match { - case None => null.asInstanceOf[B] - case Some(v) => v - } + underlying.getOrElse(key.asInstanceOf[A], null.asInstanceOf[B]) } catch { case ex: ClassCastException => null.asInstanceOf[B] } override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] { - def size = self.size + override def size: Int = self.size - def iterator = new ju.Iterator[ju.Map.Entry[A, B]] { + override def iterator: ju.Iterator[ju.Map.Entry[A, B]] = new ju.Iterator[ju.Map.Entry[A, B]] { val ui = underlying.iterator var prev : Option[A] = None - def hasNext = ui.hasNext + def hasNext: Boolean = ui.hasNext - def next() = { - val (k, v) = ui.next + def next(): Entry[A, B] = { + val (k, v) = ui.next() prev = Some(k) new ju.Map.Entry[A, B] { import scala.util.hashing.byteswap32 - def getKey = k - def getValue = v - def setValue(v1 : B) = self.put(k, v1) - override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) - override def equals(other: Any) = other match { + override def getKey: A = k + override def getValue: B = v + override def setValue(v1 : B): B = self.put(k, v1) + override def hashCode: Int = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) + override def equals(other: Any): Boolean = other match { case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue case _ => false } @@ -81,7 +80,7 @@ private[spark] object JavaUtils { case Some(k) => underlying match { case mm: mutable.Map[A, _] => - mm remove k + mm.remove(k) prev = None case _ => throw new UnsupportedOperationException("remove") diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 4c71b69069eb3..7409dc2d866f6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -54,9 +54,11 @@ private[spark] class PythonRDD( val bufferSize = conf.getInt("spark.buffer.size", 65536) val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true) - override def getPartitions = firstParent.partitions + override def getPartitions: Array[Partition] = firstParent.partitions - override val partitioner = if (preservePartitoning) firstParent.partitioner else None + override val partitioner: Option[Partitioner] = { + if (preservePartitoning) firstParent.partitioner else None + } override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis @@ -92,7 +94,7 @@ private[spark] class PythonRDD( // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = new Iterator[Array[Byte]] { - def next(): Array[Byte] = { + override def next(): Array[Byte] = { val obj = _nextObj if (hasNext) { _nextObj = read() @@ -175,7 +177,7 @@ private[spark] class PythonRDD( var _nextObj = read() - def hasNext = _nextObj != null + override def hasNext: Boolean = _nextObj != null } new InterruptibleIterator(context, stdoutIterator) } @@ -303,11 +305,10 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. * This is used by PySpark's shuffle operations. */ -private class PairwiseRDD(prev: RDD[Array[Byte]]) extends - RDD[(Long, Array[Byte])](prev) { - override def getPartitions = prev.partitions - override val partitioner = prev.partitioner - override def compute(split: Partition, context: TaskContext) = +private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Long, Array[Byte])](prev) { + override def getPartitions: Array[Partition] = prev.partitions + override val partitioner: Option[Partitioner] = prev.partitioner + override def compute(split: Partition, context: TaskContext): Iterator[(Long, Array[Byte])] = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (Utils.deserializeLongValue(a), b) case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) @@ -435,7 +436,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, minSplits: Int, - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] @@ -462,7 +463,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, @@ -488,7 +489,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, @@ -505,7 +506,7 @@ private[spark] object PythonRDD extends Logging { inputFormatClass: String, keyClass: String, valueClass: String, - conf: Configuration) = { + conf: Configuration): RDD[(K, V)] = { val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] @@ -531,7 +532,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = hadoopRDDFromClassNames[K, V, F](sc, @@ -557,7 +558,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - batchSize: Int) = { + batchSize: Int): JavaRDD[Array[Byte]] = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = hadoopRDDFromClassNames[K, V, F](sc, @@ -603,8 +604,7 @@ private[spark] object PythonRDD extends Logging { * The thread will terminate after all the data are sent or any exceptions happen. */ private def serveIterator[T](items: Iterator[T], threadName: String): Int = { - val serverSocket = new ServerSocket(0, 1) - serverSocket.setReuseAddress(true) + val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) // Close the socket if no connection in 3 seconds serverSocket.setSoTimeout(3000) @@ -614,9 +614,9 @@ private[spark] object PythonRDD extends Logging { try { val sock = serverSocket.accept() val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) - try { + Utils.tryWithSafeFinally { writeIteratorToStream(items, out) - } finally { + } { out.close() } } catch { @@ -686,7 +686,7 @@ private[spark] object PythonRDD extends Logging { pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, path: String, - compressionCodecClass: String) = { + compressionCodecClass: String): Unit = { saveAsHadoopFile( pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat", null, null, null, null, new java.util.HashMap(), compressionCodecClass) @@ -711,7 +711,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, confAsMap: java.util.HashMap[String, String], - compressionCodecClass: String) = { + compressionCodecClass: String): Unit = { val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) @@ -741,7 +741,7 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String]): Unit = { val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) @@ -766,7 +766,7 @@ private[spark] object PythonRDD extends Logging { confAsMap: java.util.HashMap[String, String], keyConverterClass: String, valueConverterClass: String, - useNewAPI: Boolean) = { + useNewAPI: Boolean): Unit = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized), keyConverterClass, valueConverterClass, new JavaToWritableConverter) @@ -862,9 +862,9 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial val file = File.createTempFile("broadcast", "", dir) path = file.getAbsolutePath val out = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { Utils.copyStream(in, out) - } finally { + } { out.close() } } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index fb52a960e0765..1f1debcf84ad4 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -56,16 +56,13 @@ private[spark] object SerDeUtil extends Logging { // {'\0', 0, 0, 0} /* Sentinel */ // }; // TODO: support Py_UNICODE with 2 bytes - // FIXME: unpickle array of float is wrong in Pyrolite, so we reverse the - // machine code for float/double here to workaround it. - // we should fix this after Pyrolite fix them val machineCodes: Map[Char, Int] = if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 3, 'h' -> 5, 'I' -> 7, 'i' -> 9, - 'L' -> 11, 'l' -> 13, 'f' -> 14, 'd' -> 16, 'u' -> 21 + 'L' -> 11, 'l' -> 13, 'f' -> 15, 'd' -> 17, 'u' -> 21 ) } else { Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 2, 'h' -> 4, 'I' -> 6, 'i' -> 8, - 'L' -> 10, 'l' -> 12, 'f' -> 15, 'd' -> 17, 'u' -> 20 + 'L' -> 10, 'l' -> 12, 'f' -> 14, 'd' -> 16, 'u' -> 20 ) } override def construct(args: Array[Object]): Object = { @@ -84,7 +81,7 @@ private[spark] object SerDeUtil extends Logging { private var initialized = false // This should be called before trying to unpickle array.array from Python // In cluster mode, this should be put in closure - def initialize() = { + def initialize(): Unit = { synchronized{ if (!initialized) { Unpickler.registerConstructor("array", "array", new ArrayConstructor()) diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index cf289fb3ae39f..8f30ff9202c83 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -18,38 +18,37 @@ package org.apache.spark.api.python import java.io.{DataOutput, DataInput} +import java.{util => ju} import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat + +import org.apache.spark.SparkException import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.{SparkContext, SparkException} /** * A class to test Pyrolite serialization on the Scala side, that will be deserialized * in Python - * @param str - * @param int - * @param double */ case class TestWritable(var str: String, var int: Int, var double: Double) extends Writable { def this() = this("", 0, 0.0) - def getStr = str + def getStr: String = str def setStr(str: String) { this.str = str } - def getInt = int + def getInt: Int = int def setInt(int: Int) { this.int = int } - def getDouble = double + def getDouble: Double = double def setDouble(double: Double) { this.double = double } - def write(out: DataOutput) = { + def write(out: DataOutput): Unit = { out.writeUTF(str) out.writeInt(int) out.writeDouble(double) } - def readFields(in: DataInput) = { + def readFields(in: DataInput): Unit = { str = in.readUTF() int = in.readInt() double = in.readDouble() @@ -57,28 +56,28 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten } private[python] class TestInputKeyConverter extends Converter[Any, Any] { - override def convert(obj: Any) = { + override def convert(obj: Any): Char = { obj.asInstanceOf[IntWritable].get().toChar } } private[python] class TestInputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ - override def convert(obj: Any) = { + override def convert(obj: Any): ju.List[Double] = { val m = obj.asInstanceOf[MapWritable] seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq) } } private[python] class TestOutputKeyConverter extends Converter[Any, Any] { - override def convert(obj: Any) = { + override def convert(obj: Any): Text = { new Text(obj.asInstanceOf[Int].toString) } } private[python] class TestOutputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ - override def convert(obj: Any) = { + override def convert(obj: Any): DoubleWritable = { new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head) } } @@ -86,7 +85,7 @@ private[python] class TestOutputValueConverter extends Converter[Any, Any] { private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable]) private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] { - override def convert(obj: Any) = obj match { + override def convert(obj: Any): DoubleArrayWritable = obj match { case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] => val daw = new DoubleArrayWritable daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_))) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala new file mode 100644 index 0000000000000..3a2c94bd9d875 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -0,0 +1,145 @@ +/* + * 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.api.r + +import java.io.{DataOutputStream, File, FileOutputStream, IOException} +import java.net.{InetSocketAddress, ServerSocket} +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelInitializer, EventLoopGroup} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} + +import org.apache.spark.Logging + +/** + * Netty-based backend server that is used to communicate between R and Java. + */ +private[spark] class RBackend { + + private[this] var channelFuture: ChannelFuture = null + private[this] var bootstrap: ServerBootstrap = null + private[this] var bossGroup: EventLoopGroup = null + + def init(): Int = { + bossGroup = new NioEventLoopGroup(2) + val workerGroup = bossGroup + val handler = new RBackendHandler(this) + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(classOf[NioServerSocketChannel]) + + bootstrap.childHandler(new ChannelInitializer[SocketChannel]() { + def initChannel(ch: SocketChannel): Unit = { + ch.pipeline() + .addLast("encoder", new ByteArrayEncoder()) + .addLast("frameDecoder", + // maxFrameLength = 2G + // lengthFieldOffset = 0 + // lengthFieldLength = 4 + // lengthAdjustment = 0 + // initialBytesToStrip = 4, i.e. strip out the length field itself + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)) + .addLast("decoder", new ByteArrayDecoder()) + .addLast("handler", handler) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(0)) + channelFuture.syncUninterruptibly() + channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort() + } + + def run(): Unit = { + channelFuture.channel.closeFuture().syncUninterruptibly() + } + + def close(): Unit = { + if (channelFuture != null) { + // close is a local operation and should finish within milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS) + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } + +} + +private[spark] object RBackend extends Logging { + def main(args: Array[String]): Unit = { + if (args.length < 1) { + System.err.println("Usage: RBackend ") + System.exit(-1) + } + val sparkRBackend = new RBackend() + try { + // bind to random port + val boundPort = sparkRBackend.init() + val serverSocket = new ServerSocket(0, 1) + val listenPort = serverSocket.getLocalPort() + + // tell the R process via temporary file + val path = args(0) + val f = new File(path + ".tmp") + val dos = new DataOutputStream(new FileOutputStream(f)) + dos.writeInt(boundPort) + dos.writeInt(listenPort) + dos.close() + f.renameTo(new File(path)) + + // wait for the end of stdin, then exit + new Thread("wait for socket to close") { + setDaemon(true) + override def run(): Unit = { + // any un-catched exception will also shutdown JVM + val buf = new Array[Byte](1024) + // shutdown JVM if R does not connect back in 10 seconds + serverSocket.setSoTimeout(10000) + try { + val inSocket = serverSocket.accept() + serverSocket.close() + // wait for the end of socket, closed if R process die + inSocket.getInputStream().read(buf) + } finally { + sparkRBackend.close() + System.exit(0) + } + } + }.start() + + sparkRBackend.run() + } catch { + case e: IOException => + logError("Server shutting down: failed with exception ", e) + sparkRBackend.close() + System.exit(1) + } + System.exit(0) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala new file mode 100644 index 0000000000000..0075d963711f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -0,0 +1,223 @@ +/* + * 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.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.collection.mutable.HashMap + +import io.netty.channel.ChannelHandler.Sharable +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.api.r.SerDe._ + +/** + * Handler for RBackend + * TODO: This is marked as sharable to get a handle to RBackend. Is it safe to re-use + * this across connections ? + */ +@Sharable +private[r] class RBackendHandler(server: RBackend) + extends SimpleChannelInboundHandler[Array[Byte]] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, msg: Array[Byte]): Unit = { + val bis = new ByteArrayInputStream(msg) + val dis = new DataInputStream(bis) + + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + // First bit is isStatic + val isStatic = readBoolean(dis) + val objId = readString(dis) + val methodName = readString(dis) + val numArgs = readInt(dis) + + if (objId == "SparkRHandler") { + methodName match { + case "stopBackend" => + writeInt(dos, 0) + writeType(dos, "void") + server.close() + case "rm" => + try { + val t = readObjectType(dis) + assert(t == 'c') + val objToRemove = readString(dis) + JVMObjectTracker.remove(objToRemove) + writeInt(dos, 0) + writeObject(dos, null) + } catch { + case e: Exception => + logError(s"Removing $objId failed", e) + writeInt(dos, -1) + } + case _ => dos.writeInt(-1) + } + } else { + handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos) + } + + val reply = bos.toByteArray + ctx.write(reply) + } + + override def channelReadComplete(ctx: ChannelHandlerContext): Unit = { + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + // Close the connection when an exception is raised. + cause.printStackTrace() + ctx.close() + } + + def handleMethodCall( + isStatic: Boolean, + objId: String, + methodName: String, + numArgs: Int, + dis: DataInputStream, + dos: DataOutputStream): Unit = { + var obj: Object = null + try { + val cls = if (isStatic) { + Class.forName(objId) + } else { + JVMObjectTracker.get(objId) match { + case None => throw new IllegalArgumentException("Object not found " + objId) + case Some(o) => + obj = o + o.getClass + } + } + + val args = readArgs(numArgs, dis) + + val methods = cls.getMethods + val selectedMethods = methods.filter(m => m.getName == methodName) + if (selectedMethods.length > 0) { + val methods = selectedMethods.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + } + if (methods.isEmpty) { + logWarning(s"cannot find matching method ${cls}.$methodName. " + + s"Candidates are:") + selectedMethods.foreach { method => + logWarning(s"$methodName(${method.getParameterTypes.mkString(",")})") + } + throw new Exception(s"No matched method found for $cls.$methodName") + } + val ret = methods.head.invoke(obj, args:_*) + + // Write status bit + writeInt(dos, 0) + writeObject(dos, ret.asInstanceOf[AnyRef]) + } else if (methodName == "") { + // methodName should be "" for constructor + val ctor = cls.getConstructors.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + }.head + + val obj = ctor.newInstance(args:_*) + + writeInt(dos, 0) + writeObject(dos, obj.asInstanceOf[AnyRef]) + } else { + throw new IllegalArgumentException("invalid method " + methodName + " for object " + objId) + } + } catch { + case e: Exception => + logError(s"$methodName on $objId failed", e) + writeInt(dos, -1) + } + } + + // Read a number of arguments from the data input stream + def readArgs(numArgs: Int, dis: DataInputStream): Array[java.lang.Object] = { + (0 until numArgs).map { arg => + readObject(dis) + }.toArray + } + + // Checks if the arguments passed in args matches the parameter types. + // NOTE: Currently we do exact match. We may add type conversions later. + def matchMethod( + numArgs: Int, + args: Array[java.lang.Object], + parameterTypes: Array[Class[_]]): Boolean = { + if (parameterTypes.length != numArgs) { + return false + } + + for (i <- 0 to numArgs - 1) { + val parameterType = parameterTypes(i) + var parameterWrapperType = parameterType + + // Convert native parameters to Object types as args is Array[Object] here + if (parameterType.isPrimitive) { + parameterWrapperType = parameterType match { + case java.lang.Integer.TYPE => classOf[java.lang.Integer] + case java.lang.Double.TYPE => classOf[java.lang.Double] + case java.lang.Boolean.TYPE => classOf[java.lang.Boolean] + case _ => parameterType + } + } + if (!parameterWrapperType.isInstance(args(i))) { + return false + } + } + true + } +} + +/** + * Helper singleton that tracks JVM objects returned to R. + * This is useful for referencing these objects in RPC calls. + */ +private[r] object JVMObjectTracker { + + // TODO: This map should be thread-safe if we want to support multiple + // connections at the same time + private[this] val objMap = new HashMap[String, Object] + + // TODO: We support only one connection now, so an integer is fine. + // Investigate using use atomic integer in the future. + private[this] var objCounter: Int = 0 + + def getObject(id: String): Object = { + objMap(id) + } + + def get(id: String): Option[Object] = { + objMap.get(id) + } + + def put(obj: Object): String = { + val objId = objCounter.toString + objCounter = objCounter + 1 + objMap.put(objId, obj) + objId + } + + def remove(id: String): Option[Object] = { + objMap.remove(id) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala new file mode 100644 index 0000000000000..6fea5e1144f2f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -0,0 +1,453 @@ +/* + * 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.api.r + +import java.io._ +import java.net.ServerSocket +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ +import scala.io.Source +import scala.reflect.ClassTag +import scala.util.Try + +import org.apache.spark._ +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( + parent: RDD[T], + numPartitions: Int, + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Broadcast[Object]]) + extends RDD[U](parent) with Logging { + protected var dataStream: DataInputStream = _ + private var bootTime: Double = _ + override def getPartitions: Array[Partition] = parent.partitions + + override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + + // The parent may be also an RRDD, so we should launch it first. + val parentIterator = firstParent[T].iterator(partition, context) + + // we expect two connections + val serverSocket = new ServerSocket(0, 2) + val listenPort = serverSocket.getLocalPort() + + // The stdout/stderr is shared by multiple tasks, because we use one daemon + // to launch child process as worker. + val errThread = RRDD.createRWorker(rLibDir, listenPort) + + // We use two sockets to separate input and output, then it's easy to manage + // the lifecycle of them to avoid deadlock. + // TODO: optimize it to use one socket + + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val inSocket = serverSocket.accept() + startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index) + + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + val inputStream = new BufferedInputStream(outSocket.getInputStream) + dataStream = new DataInputStream(inputStream) + serverSocket.close() + + try { + + return new Iterator[U] { + def next(): U = { + val obj = _nextObj + if (hasNext) { + _nextObj = read() + } + obj + } + + var _nextObj = read() + + def hasNext(): Boolean = { + val hasMore = (_nextObj != null) + if (!hasMore) { + dataStream.close() + } + hasMore + } + } + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines()) + } + } + + /** + * Start a thread to write RDD data to the R process. + */ + private def startStdinThread[T]( + output: OutputStream, + iter: Iterator[T], + partition: Int): Unit = { + + val env = SparkEnv.get + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val stream = new BufferedOutputStream(output, bufferSize) + + new Thread("writer for R") { + override def run(): Unit = { + try { + SparkEnv.set(env) + val dataOut = new DataOutputStream(stream) + dataOut.writeInt(partition) + + SerDe.writeString(dataOut, deserializer) + SerDe.writeString(dataOut, serializer) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames) + + dataOut.writeInt(func.length) + dataOut.write(func) + + dataOut.writeInt(broadcastVars.length) + broadcastVars.foreach { broadcast => + // TODO(shivaram): Read a Long in R to avoid this cast + dataOut.writeInt(broadcast.id.toInt) + // TODO: Pass a byte array from R to avoid this cast ? + val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] + dataOut.writeInt(broadcastByteArr.length) + dataOut.write(broadcastByteArr) + } + + dataOut.writeInt(numPartitions) + + if (!iter.hasNext) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + + val printOut = new PrintStream(stream) + + def writeElem(elem: Any): Unit = { + if (deserializer == SerializationFormats.BYTE) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr) + } else if (deserializer == SerializationFormats.ROW) { + dataOut.write(elem.asInstanceOf[Array[Byte]]) + } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) + printOut.println(elem) + } + } + + for (elem <- iter) { + elem match { + case (key, value) => + writeElem(key) + writeElem(value) + case _ => + writeElem(elem) + } + } + stream.flush() + } catch { + // TODO: We should propogate this error to the task thread + case e: Exception => + logError("R Writer thread got an exception", e) + } finally { + Try(output.close()) + } + } + }.start() + } + + protected def readData(length: Int): U + + protected def read(): U = { + try { + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length >= 0 => + readData(length) + } + } catch { + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) + } + } +} + +/** + * Form an RDD[(Int, Array[Byte])] from key-value pairs returned from R. + * This is used by SparkR's shuffle operations. + */ +private class PairwiseRRDD[T: ClassTag]( + parent: RDD[T], + numPartitions: Int, + hashFunc: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, (Int, Array[Byte])]( + parent, numPartitions, hashFunc, deserializer, + SerializationFormats.BYTE, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + override protected def readData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null + } + } + + lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) +} + +/** + * An RDD that stores serialized R objects as Array[Byte]. + */ +private class RRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, Array[Byte]]( + parent, -1, func, deserializer, serializer, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + override protected def readData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null + } + } + + lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) +} + +/** + * An RDD that stores R objects as Array[String]. + */ +private class StringRRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, String]( + parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + override protected def readData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null + } + } + + lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) +} + +private object SpecialLengths { + val TIMING_DATA = -1 +} + +private[r] class BufferedStreamThread( + in: InputStream, + name: String, + errBufferSize: Int) extends Thread(name) with Logging { + val lines = new Array[String](errBufferSize) + var lineIdx = 0 + override def run() { + for (line <- Source.fromInputStream(in).getLines) { + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } + logInfo(line) + } + } + + def getLines(): String = synchronized { + (0 until errBufferSize).filter { x => + lines((x + lineIdx) % errBufferSize) != null + }.map { x => + lines((x + lineIdx) % errBufferSize) + }.mkString("\n") + } +} + +private[r] object RRDD { + // Because forking processes from Java is expensive, we prefer to launch + // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. + // This daemon currently only works on UNIX-based systems now, so we should + // also fall back to launching workers (worker.R) directly. + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonChannel: DataOutputStream = _ + + def createSparkContext( + master: String, + appName: String, + sparkHome: String, + jars: Array[String], + sparkEnvirMap: JMap[Object, Object], + sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = { + + val sparkConf = new SparkConf().setAppName(appName) + .setSparkHome(sparkHome) + .setJars(jars) + + // Override `master` if we have a user-specified value + if (master != "") { + sparkConf.setMaster(master) + } else { + // If conf has no master set it to "local" to maintain + // backwards compatibility + sparkConf.setIfMissing("spark.master", "local") + } + + for ((name, value) <- sparkEnvirMap) { + sparkConf.set(name.asInstanceOf[String], value.asInstanceOf[String]) + } + for ((name, value) <- sparkExecutorEnvMap) { + sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String]) + } + + new JavaSparkContext(sparkConf) + } + + /** + * Start a thread to print the process's stderr to ours + */ + private def startStdoutThread(proc: Process): BufferedStreamThread = { + val BUFFER_SIZE = 100 + val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) + thread.setDaemon(true) + thread.start() + thread + } + + private def createRProcess(rLibDir: String, port: Int, script: String): BufferedStreamThread = { + val rCommand = "Rscript" + val rOptions = "--vanilla" + val rExecScript = rLibDir + "/SparkR/worker/" + script + val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript)) + // Unset the R_TESTS environment variable for workers. + // This is set by R CMD check as startup.Rs + // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) + // and confuses worker script which tries to load a non-existent file + pb.environment().put("R_TESTS", "") + pb.environment().put("SPARKR_RLIBDIR", rLibDir) + pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.redirectErrorStream(true) // redirect stderr into stdout + val proc = pb.start() + val errThread = startStdoutThread(proc) + errThread + } + + /** + * ProcessBuilder used to launch worker R processes. + */ + def createRWorker(rLibDir: String, port: Int): BufferedStreamThread = { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!Utils.isWindows && useDaemon) { + synchronized { + if (daemonChannel == null) { + // we expect one connections + val serverSocket = new ServerSocket(0, 1) + val daemonPort = serverSocket.getLocalPort + errThread = createRProcess(rLibDir, daemonPort, "daemon.R") + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val sock = serverSocket.accept() + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + serverSocket.close() + } + try { + daemonChannel.writeInt(port) + daemonChannel.flush() + } catch { + case e: IOException => + // daemon process died + daemonChannel.close() + daemonChannel = null + errThread = null + // fail the current task, retry by scheduler + throw e + } + errThread + } + } else { + createRProcess(rLibDir, port, "worker.R") + } + } + + /** + * Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is + * called from R. + */ + def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { + JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala new file mode 100644 index 0000000000000..371dfe454d1a2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -0,0 +1,344 @@ +/* + * 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.api.r + +import java.io.{DataInputStream, DataOutputStream} +import java.sql.{Date, Time} + +import scala.collection.JavaConversions._ + +/** + * Utility functions to serialize, deserialize objects to / from R + */ +private[spark] object SerDe { + + // Type mapping from R to Java + // + // NULL -> void + // integer -> Int + // character -> String + // logical -> Boolean + // double, numeric -> Double + // raw -> Array[Byte] + // Date -> Date + // POSIXlt/POSIXct -> Time + // + // list[T] -> Array[T], where T is one of above mentioned types + // environment -> Map[String, T], where T is a native type + // jobj -> Object, where jobj is an object created in the backend + + def readObjectType(dis: DataInputStream): Char = { + dis.readByte().toChar + } + + def readObject(dis: DataInputStream): Object = { + val dataType = readObjectType(dis) + readTypedObject(dis, dataType) + } + + def readTypedObject( + dis: DataInputStream, + dataType: Char): Object = { + dataType match { + case 'n' => null + case 'i' => new java.lang.Integer(readInt(dis)) + case 'd' => new java.lang.Double(readDouble(dis)) + case 'b' => new java.lang.Boolean(readBoolean(dis)) + case 'c' => readString(dis) + case 'e' => readMap(dis) + case 'r' => readBytes(dis) + case 'l' => readList(dis) + case 'D' => readDate(dis) + case 't' => readTime(dis) + case 'j' => JVMObjectTracker.getObject(readString(dis)) + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") + } + } + + def readBytes(in: DataInputStream): Array[Byte] = { + val len = readInt(in) + val out = new Array[Byte](len) + val bytesRead = in.readFully(out) + out + } + + def readInt(in: DataInputStream): Int = { + in.readInt() + } + + def readDouble(in: DataInputStream): Double = { + in.readDouble() + } + + def readStringBytes(in: DataInputStream, len: Int): String = { + val bytes = new Array[Byte](len) + in.readFully(bytes) + assert(bytes(len - 1) == 0) + val str = new String(bytes.dropRight(1), "UTF-8") + str + } + + def readString(in: DataInputStream): String = { + val len = in.readInt() + readStringBytes(in, len) + } + + def readBoolean(in: DataInputStream): Boolean = { + val intVal = in.readInt() + if (intVal == 0) false else true + } + + def readDate(in: DataInputStream): Date = { + Date.valueOf(readString(in)) + } + + def readTime(in: DataInputStream): Time = { + val t = in.readDouble() + new Time((t * 1000L).toLong) + } + + def readBytesArr(in: DataInputStream): Array[Array[Byte]] = { + val len = readInt(in) + (0 until len).map(_ => readBytes(in)).toArray + } + + def readIntArr(in: DataInputStream): Array[Int] = { + val len = readInt(in) + (0 until len).map(_ => readInt(in)).toArray + } + + def readDoubleArr(in: DataInputStream): Array[Double] = { + val len = readInt(in) + (0 until len).map(_ => readDouble(in)).toArray + } + + def readBooleanArr(in: DataInputStream): Array[Boolean] = { + val len = readInt(in) + (0 until len).map(_ => readBoolean(in)).toArray + } + + def readStringArr(in: DataInputStream): Array[String] = { + val len = readInt(in) + (0 until len).map(_ => readString(in)).toArray + } + + def readList(dis: DataInputStream): Array[_] = { + val arrType = readObjectType(dis) + arrType match { + case 'i' => readIntArr(dis) + case 'c' => readStringArr(dis) + case 'd' => readDoubleArr(dis) + case 'b' => readBooleanArr(dis) + case 'j' => readStringArr(dis).map(x => JVMObjectTracker.getObject(x)) + case 'r' => readBytesArr(dis) + case _ => throw new IllegalArgumentException(s"Invalid array type $arrType") + } + } + + def readMap(in: DataInputStream): java.util.Map[Object, Object] = { + val len = readInt(in) + if (len > 0) { + val keysType = readObjectType(in) + val keysLen = readInt(in) + val keys = (0 until keysLen).map(_ => readTypedObject(in, keysType)) + + val valuesType = readObjectType(in) + val valuesLen = readInt(in) + val values = (0 until valuesLen).map(_ => readTypedObject(in, valuesType)) + mapAsJavaMap(keys.zip(values).toMap) + } else { + new java.util.HashMap[Object, Object]() + } + } + + // Methods to write out data from Java to R + // + // Type mapping from Java to R + // + // void -> NULL + // Int -> integer + // String -> character + // Boolean -> logical + // Double -> double + // Long -> double + // Array[Byte] -> raw + // Date -> Date + // Time -> POSIXct + // + // Array[T] -> list() + // Object -> jobj + + def writeType(dos: DataOutputStream, typeStr: String): Unit = { + typeStr match { + case "void" => dos.writeByte('n') + case "character" => dos.writeByte('c') + case "double" => dos.writeByte('d') + case "integer" => dos.writeByte('i') + case "logical" => dos.writeByte('b') + case "date" => dos.writeByte('D') + case "time" => dos.writeByte('t') + case "raw" => dos.writeByte('r') + case "list" => dos.writeByte('l') + case "jobj" => dos.writeByte('j') + case _ => throw new IllegalArgumentException(s"Invalid type $typeStr") + } + } + + def writeObject(dos: DataOutputStream, value: Object): Unit = { + if (value == null) { + writeType(dos, "void") + } else { + value.getClass.getName match { + case "java.lang.String" => + writeType(dos, "character") + writeString(dos, value.asInstanceOf[String]) + case "long" | "java.lang.Long" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Long].toDouble) + case "double" | "java.lang.Double" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Double]) + case "int" | "java.lang.Integer" => + writeType(dos, "integer") + writeInt(dos, value.asInstanceOf[Int]) + case "boolean" | "java.lang.Boolean" => + writeType(dos, "logical") + writeBoolean(dos, value.asInstanceOf[Boolean]) + case "java.sql.Date" => + writeType(dos, "date") + writeDate(dos, value.asInstanceOf[Date]) + case "java.sql.Time" => + writeType(dos, "time") + writeTime(dos, value.asInstanceOf[Time]) + case "[B" => + writeType(dos, "raw") + writeBytes(dos, value.asInstanceOf[Array[Byte]]) + // TODO: Types not handled right now include + // byte, char, short, float + + // Handle arrays + case "[Ljava.lang.String;" => + writeType(dos, "list") + writeStringArr(dos, value.asInstanceOf[Array[String]]) + case "[I" => + writeType(dos, "list") + writeIntArr(dos, value.asInstanceOf[Array[Int]]) + case "[J" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Long]].map(_.toDouble)) + case "[D" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Double]]) + case "[Z" => + writeType(dos, "list") + writeBooleanArr(dos, value.asInstanceOf[Array[Boolean]]) + case "[[B" => + writeType(dos, "list") + writeBytesArr(dos, value.asInstanceOf[Array[Array[Byte]]]) + case otherName => + // Handle array of objects + if (otherName.startsWith("[L")) { + val objArr = value.asInstanceOf[Array[Object]] + writeType(dos, "list") + writeType(dos, "jobj") + dos.writeInt(objArr.length) + objArr.foreach(o => writeJObj(dos, o)) + } else { + writeType(dos, "jobj") + writeJObj(dos, value) + } + } + } + } + + def writeInt(out: DataOutputStream, value: Int): Unit = { + out.writeInt(value) + } + + def writeDouble(out: DataOutputStream, value: Double): Unit = { + out.writeDouble(value) + } + + def writeBoolean(out: DataOutputStream, value: Boolean): Unit = { + val intValue = if (value) 1 else 0 + out.writeInt(intValue) + } + + def writeDate(out: DataOutputStream, value: Date): Unit = { + writeString(out, value.toString) + } + + def writeTime(out: DataOutputStream, value: Time): Unit = { + out.writeDouble(value.getTime.toDouble / 1000.0) + } + + + // NOTE: Only works for ASCII right now + def writeString(out: DataOutputStream, value: String): Unit = { + val len = value.length + out.writeInt(len + 1) // For the \0 + out.writeBytes(value) + out.writeByte(0) + } + + def writeBytes(out: DataOutputStream, value: Array[Byte]): Unit = { + out.writeInt(value.length) + out.write(value) + } + + def writeJObj(out: DataOutputStream, value: Object): Unit = { + val objId = JVMObjectTracker.put(value) + writeString(out, objId) + } + + def writeIntArr(out: DataOutputStream, value: Array[Int]): Unit = { + writeType(out, "integer") + out.writeInt(value.length) + value.foreach(v => out.writeInt(v)) + } + + def writeDoubleArr(out: DataOutputStream, value: Array[Double]): Unit = { + writeType(out, "double") + out.writeInt(value.length) + value.foreach(v => out.writeDouble(v)) + } + + def writeBooleanArr(out: DataOutputStream, value: Array[Boolean]): Unit = { + writeType(out, "logical") + out.writeInt(value.length) + value.foreach(v => writeBoolean(out, v)) + } + + def writeStringArr(out: DataOutputStream, value: Array[String]): Unit = { + writeType(out, "character") + out.writeInt(value.length) + value.foreach(v => writeString(out, v)) + } + + def writeBytesArr(out: DataOutputStream, value: Array[Array[Byte]]): Unit = { + writeType(out, "raw") + out.writeInt(value.length) + value.foreach(v => writeBytes(out, v)) + } +} + +private[r] object SerializationFormats { + val BYTE = "byte" + val STRING = "string" + val ROW = "row" +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index a5ea478f231d7..12d79f6ed311b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -146,5 +146,5 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo } } - override def toString = "Broadcast(" + id + ")" + override def toString: String = "Broadcast(" + id + ")" } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index 8f8a0b11f9f2e..685313ac009ba 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -58,7 +58,7 @@ private[spark] class BroadcastManager( private val nextBroadcastId = new AtomicLong(0) - def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = { + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = { broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 1444c0dd3d2d6..4457c75e8b0fc 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -160,12 +160,12 @@ private[broadcast] object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } - def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) + def getFile(id: Long): File = new File(broadcastDir, BroadcastBlockId(id).name) private def write(id: Long, value: Any) { val file = getFile(id) val fileOutputStream = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { val out: OutputStream = { if (compress) { compressionCodec.compressedOutputStream(fileOutputStream) @@ -175,10 +175,13 @@ private[broadcast] object HttpBroadcast extends Logging { } val ser = SparkEnv.get.serializer.newInstance() val serOut = ser.serializeStream(out) - serOut.writeObject(value) - serOut.close() + Utils.tryWithSafeFinally { + serOut.writeObject(value) + } { + serOut.close() + } files += file - } finally { + } { fileOutputStream.close() } } @@ -212,9 +215,11 @@ private[broadcast] object HttpBroadcast extends Logging { } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) - val obj = serIn.readObject[T]() - serIn.close() - obj + Utils.tryWithSafeFinally { + serIn.readObject[T]() + } { + serIn.close() + } } /** @@ -222,7 +227,7 @@ private[broadcast] object HttpBroadcast extends Logging { * If removeFromDriver is true, also remove these persisted blocks on the driver * and delete the associated broadcast file. */ - def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = synchronized { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) if (removeFromDriver) { val file = getFile(id) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala index c7ef02d572a19..cf3ae36f27949 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -31,7 +31,7 @@ class HttpBroadcastFactory extends BroadcastFactory { HttpBroadcast.initialize(isDriver, conf, securityMgr) } - override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] = new HttpBroadcast[T](value_, isLocal, id) override def stop() { HttpBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 94142d33369c7..a0c9b5e63c744 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -74,7 +74,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } else { None } - blockSize = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 + // Note: use getSizeAsKb (not bytes) to maintain compatiblity if no units are provided + blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024 } setConf(SparkEnv.get.conf) @@ -222,7 +223,7 @@ private object TorrentBroadcast extends Logging { * Remove all persisted blocks associated with this torrent broadcast on the executors. * If removeFromDriver is true, also remove these persisted blocks on the driver. */ - def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = { logDebug(s"Unpersisting TorrentBroadcast $id") SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index fb024c12094f2..96d8dd79908c8 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -30,7 +30,7 @@ class TorrentBroadcastFactory extends BroadcastFactory { override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { } - override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = { + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] = { new TorrentBroadcast[T](value_, id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 3d0d68de8f495..ae99432f5ce86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,15 +17,18 @@ package org.apache.spark.deploy +import java.net.URI + private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], - val memoryPerSlave: Int, + val memoryPerExecutorMB: Int, val command: Command, var appUiUrl: String, - val eventLogDir: Option[String] = None, + val eventLogDir: Option[URI] = None, // short name of compression codec used when writing event logs, if any (e.g. lzf) - val eventLogCodec: Option[String] = None) + val eventLogCodec: Option[String] = None, + val coresPerExecutor: Option[Int] = None) extends Serializable { val user = System.getProperty("user.name", "") @@ -33,13 +36,13 @@ private[spark] class ApplicationDescription( def copy( name: String = name, maxCores: Option[Int] = maxCores, - memoryPerSlave: Int = memoryPerSlave, + memoryPerExecutorMB: Int = memoryPerExecutorMB, command: Command = command, appUiUrl: String = appUiUrl, - eventLogDir: Option[String] = eventLogDir, + eventLogDir: Option[URI] = eventLogDir, eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = new ApplicationDescription( - name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) + name, maxCores, memoryPerExecutorMB, command, appUiUrl, eventLogDir, eventLogCodec) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 237d26fc6bd0e..848b62f9de71b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import scala.collection.mutable.HashSet import scala.concurrent._ import akka.actor._ @@ -27,25 +28,28 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, Utils} /** * Proxy that relays messages to the driver. + * + * We currently don't support retry if submission fails. In HA mode, client will submit request to + * all masters and see which one could handle it. */ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with ActorLogReceive with Logging { - var masterActor: ActorSelection = _ - val timeout = AkkaUtils.askTimeout(conf) + private val masterActors = driverArgs.masters.map { m => + context.actorSelection(Master.toAkkaUrl(m, AkkaUtils.protocol(context.system))) + } + private val lostMasters = new HashSet[Address] + private var activeMasterActor: ActorSelection = null - override def preStart() = { - masterActor = context.actorSelection( - Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system))) + val timeout = RpcUtils.askTimeout(conf) + override def preStart(): Unit = { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}") - driverArgs.cmd match { case "launch" => // TODO: We could add an env variable here and intercept it in `sc.addJar` that would @@ -79,23 +83,28 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) driverArgs.supervise, command) - masterActor ! RequestSubmitDriver(driverDescription) + // This assumes only one Master is active at a time + for (masterActor <- masterActors) { + masterActor ! RequestSubmitDriver(driverDescription) + } case "kill" => val driverId = driverArgs.driverId - masterActor ! RequestKillDriver(driverId) + // This assumes only one Master is active at a time + for (masterActor <- masterActors) { + masterActor ! RequestKillDriver(driverId) + } } } /* Find out driver status then exit the JVM */ def pollAndReportStatus(driverId: String) { - println(s"... waiting before polling master for driver state") + println("... waiting before polling master for driver state") Thread.sleep(5000) println("... polling master for driver state") - val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout) + val statusFuture = (activeMasterActor ? RequestDriverStatus(driverId))(timeout) .mapTo[DriverStatusResponse] val statusResponse = Await.result(statusFuture, timeout) - statusResponse.found match { case false => println(s"ERROR: Cluster master did not recognize $driverId") @@ -118,24 +127,50 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) } } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case SubmitDriverResponse(success, driverId, message) => println(message) - if (success) pollAndReportStatus(driverId.get) else System.exit(-1) + if (success) { + activeMasterActor = context.actorSelection(sender.path) + pollAndReportStatus(driverId.get) + } else if (!Utils.responseFromBackup(message)) { + System.exit(-1) + } + case KillDriverResponse(driverId, success, message) => println(message) - if (success) pollAndReportStatus(driverId) else System.exit(-1) + if (success) { + activeMasterActor = context.actorSelection(sender.path) + pollAndReportStatus(driverId) + } else if (!Utils.responseFromBackup(message)) { + System.exit(-1) + } case DisassociatedEvent(_, remoteAddress, _) => - println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") - System.exit(-1) + if (!lostMasters.contains(remoteAddress)) { + println(s"Error connecting to master $remoteAddress.") + lostMasters += remoteAddress + // Note that this heuristic does not account for the fact that a Master can recover within + // the lifetime of this client. Thus, once a Master is lost it is lost to us forever. This + // is not currently a concern, however, because this client does not retry submissions. + if (lostMasters.size >= masterActors.size) { + println("No master is available, exiting.") + System.exit(-1) + } + } case AssociationErrorEvent(cause, _, remoteAddress, _, _) => - println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") - println(s"Cause was: $cause") - System.exit(-1) + if (!lostMasters.contains(remoteAddress)) { + println(s"Error connecting to master ($remoteAddress).") + println(s"Cause was: $cause") + lostMasters += remoteAddress + if (lostMasters.size >= masterActors.size) { + println("No master is available, exiting.") + System.exit(-1) + } + } } } @@ -155,7 +190,7 @@ object Client { if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) { conf.set("spark.akka.logLifecycleEvents", "true") } - conf.set("spark.akka.askTimeout", "10") + conf.set("spark.rpc.askTimeout", "10") conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING")) Logger.getRootLogger.setLevel(driverArgs.logLevel) @@ -163,7 +198,9 @@ object Client { "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) // Verify driverArgs.master is a valid url so that we can use it in ClientActor safely - Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(actorSystem)) + for (m <- driverArgs.masters) { + Master.toAkkaUrl(m, AkkaUtils.protocol(actorSystem)) + } actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) actorSystem.awaitTermination() diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 53bc62aff7395..316e2d59f01b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -22,8 +22,7 @@ import java.net.{URI, URISyntaxException} import scala.collection.mutable.ListBuffer import org.apache.log4j.Level - -import org.apache.spark.util.{IntParam, MemoryParam} +import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** * Command-line parser for the driver client. @@ -35,14 +34,14 @@ private[deploy] class ClientArguments(args: Array[String]) { var logLevel = Level.WARN // launch parameters - var master: String = "" + var masters: Array[String] = null var jarUrl: String = "" var mainClass: String = "" var supervise: Boolean = DEFAULT_SUPERVISE var memory: Int = DEFAULT_MEMORY var cores: Int = DEFAULT_CORES private var _driverOptions = ListBuffer[String]() - def driverOptions = _driverOptions.toSeq + def driverOptions: Seq[String] = _driverOptions.toSeq // kill parameters var driverId: String = "" @@ -80,13 +79,13 @@ private[deploy] class ClientArguments(args: Array[String]) { } jarUrl = _jarUrl - master = _master + masters = Utils.parseStandaloneMasterUrls(_master) mainClass = _mainClass _driverOptions ++= tail case "kill" :: _master :: _driverId :: tail => cmd = "kill" - master = _master + masters = Utils.parseStandaloneMasterUrls(_master) driverId = _driverId case _ => diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 7f600d89604a2..9db6fd1ac4dbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -101,6 +101,8 @@ private[deploy] object DeployMessages { case class RegisterApplication(appDescription: ApplicationDescription) extends DeployMessage + case class UnregisterApplication(appId: String) + case class MasterChangeAcknowledged(appId: String) // Master to AppClient @@ -162,7 +164,7 @@ private[deploy] object DeployMessages { Utils.checkHost(host, "Required hostname") assert (port > 0) - def uri = "spark://" + host + ":" + port + def uri: String = "spark://" + host + ":" + port def restUri: Option[String] = restPort.map { p => "spark://" + host + ":" + p } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala similarity index 51% rename from core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala rename to core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index b9798963bab0a..09973a0a2c998 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -15,14 +15,19 @@ * limitations under the License. */ -package org.apache.spark.deploy.worker +package org.apache.spark.deploy + +import java.util.concurrent.CountDownLatch + +import scala.collection.JavaConversions._ import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.sasl.SaslRpcHandler +import org.apache.spark.network.sasl.SaslServerBootstrap import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler +import org.apache.spark.util.Utils /** * Provides a server from which Executors can read shuffle files (rather than reading directly from @@ -31,8 +36,8 @@ import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler * * Optionally requires SASL authentication in order to read. See [[SecurityManager]]. */ -private[worker] -class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: SecurityManager) +private[deploy] +class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityManager) extends Logging { private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false) @@ -41,26 +46,71 @@ class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: Secu private val transportConf = SparkTransportConf.fromSparkConf(sparkConf, numUsableCores = 0) private val blockHandler = new ExternalShuffleBlockHandler(transportConf) - private val transportContext: TransportContext = { - val handler = if (useSasl) new SaslRpcHandler(blockHandler, securityManager) else blockHandler - new TransportContext(transportConf, handler) - } + private val transportContext: TransportContext = new TransportContext(transportConf, blockHandler) private var server: TransportServer = _ /** Starts the external shuffle service if the user has configured us to. */ def startIfEnabled() { if (enabled) { - require(server == null, "Shuffle server already started") - logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl") - server = transportContext.createServer(port) + start() } } + /** Start the external shuffle service */ + def start() { + require(server == null, "Shuffle server already started") + logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl") + val bootstraps = + if (useSasl) { + Seq(new SaslServerBootstrap(transportConf, securityManager)) + } else { + Nil + } + server = transportContext.createServer(port, bootstraps) + } + def stop() { - if (enabled && server != null) { + if (server != null) { server.close() server = null } } } + +/** + * A main class for running the external shuffle service. + */ +object ExternalShuffleService extends Logging { + @volatile + private var server: ExternalShuffleService = _ + + private val barrier = new CountDownLatch(1) + + def main(args: Array[String]): Unit = { + val sparkConf = new SparkConf + Utils.loadDefaultSparkProperties(sparkConf) + val securityManager = new SecurityManager(sparkConf) + + // we override this value since this service is started from the command line + // and we assume the user really wants it to be running + sparkConf.set("spark.shuffle.service.enabled", "true") + server = new ExternalShuffleService(sparkConf, securityManager) + server.start() + + installShutdownHook() + + // keep running until the process is terminated + barrier.await() + } + + private def installShutdownHook(): Unit = { + Runtime.getRuntime.addShutdownHook(new Thread("External Shuffle Service shutdown thread") { + override def run() { + logInfo("Shutting down shuffle service.") + server.stop() + barrier.countDown() + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 5668b53fc6f4f..c048b78910f38 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -32,7 +32,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkConf, SparkContext} -import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} +import org.apache.spark.deploy.master.RecoveryState import org.apache.spark.util.Utils /** @@ -426,7 +426,7 @@ private object SparkDocker { } private class DockerId(val id: String) { - override def toString = id + override def toString: String = id } private object Docker extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 458a7c3a455de..2954f932b4f41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import org.json4s.JsonAST.JObject import org.json4s.JsonDSL._ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -24,7 +25,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner private[deploy] object JsonProtocol { - def writeWorkerInfo(obj: WorkerInfo) = { + def writeWorkerInfo(obj: WorkerInfo): JObject = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ ("port" -> obj.port) ~ @@ -39,34 +40,34 @@ private[deploy] object JsonProtocol { ("lastheartbeat" -> obj.lastHeartbeat) } - def writeApplicationInfo(obj: ApplicationInfo) = { + def writeApplicationInfo(obj: ApplicationInfo): JObject = { ("starttime" -> obj.startTime) ~ ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ - ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~ ("submitdate" -> obj.submitDate.toString) ~ ("state" -> obj.state.toString) ~ ("duration" -> obj.duration) } - def writeApplicationDescription(obj: ApplicationDescription) = { + def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ - ("memoryperslave" -> obj.memoryPerSlave) ~ + ("memoryperslave" -> obj.memoryPerExecutorMB) ~ ("user" -> obj.user) ~ ("command" -> obj.command.toString) } - def writeExecutorRunner(obj: ExecutorRunner) = { + def writeExecutorRunner(obj: ExecutorRunner): JObject = { ("id" -> obj.execId) ~ ("memory" -> obj.memory) ~ ("appid" -> obj.appId) ~ ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - def writeDriverInfo(obj: DriverInfo) = { + def writeDriverInfo(obj: DriverInfo): JObject = { ("id" -> obj.id) ~ ("starttime" -> obj.startTime.toString) ~ ("state" -> obj.state.toString) ~ @@ -74,7 +75,7 @@ private[deploy] object JsonProtocol { ("memory" -> obj.desc.mem) } - def writeMasterState(obj: MasterStateResponse) = { + def writeMasterState(obj: MasterStateResponse): JObject = { ("url" -> obj.uri) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ ("cores" -> obj.workers.map(_.cores).sum) ~ @@ -87,7 +88,7 @@ private[deploy] object JsonProtocol { ("status" -> obj.status.toString) } - def writeWorkerState(obj: WorkerStateResponse) = { + def writeWorkerState(obj: WorkerStateResponse): JObject = { ("id" -> obj.workerId) ~ ("masterurl" -> obj.masterUrl) ~ ("masterwebuiurl" -> obj.masterWebUiUrl) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 3ab425aab84c8..f0e77c2ba982b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -53,7 +53,7 @@ class LocalSparkCluster( /* Start the Master */ val (masterSystem, masterPort, _, _) = Master.startSystemAndActor(localHostname, 0, 0, _conf) masterActorSystems += masterSystem - val masterUrl = "spark://" + localHostname + ":" + masterPort + val masterUrl = "spark://" + Utils.localHostNameForURI() + ":" + masterPort val masters = Array(masterUrl) /* Start the Workers */ diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala new file mode 100644 index 0000000000000..e99779f299785 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -0,0 +1,92 @@ +/* + * 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 + +import java.io._ +import java.util.concurrent.{Semaphore, TimeUnit} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.api.r.RBackend +import org.apache.spark.util.RedirectThread + +/** + * Main class used to launch SparkR applications using spark-submit. It executes R as a + * subprocess and then has it connect back to the JVM to access system properties etc. + */ +object RRunner { + def main(args: Array[String]): Unit = { + val rFile = PythonRunner.formatPath(args(0)) + + val otherArgs = args.slice(1, args.length) + + // Time to wait for SparkR backend to initialize in seconds + val backendTimeout = sys.env.getOrElse("SPARKR_BACKEND_TIMEOUT", "120").toInt + val rCommand = "Rscript" + + // Check if the file path exists. + // If not, change directory to current working directory for YARN cluster mode + val rF = new File(rFile) + val rFileNormalized = if (!rF.exists()) { + new Path(rFile).getName + } else { + rFile + } + + // Launch a SparkR backend server for the R process to connect to; this will let it see our + // Java system properties etc. + val sparkRBackend = new RBackend() + @volatile var sparkRBackendPort = 0 + val initialized = new Semaphore(0) + val sparkRBackendThread = new Thread("SparkR backend") { + override def run() { + sparkRBackendPort = sparkRBackend.init() + initialized.release() + sparkRBackend.run() + } + } + + sparkRBackendThread.start() + // Wait for RBackend initialization to finish + if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { + // Launch R + val returnCode = try { + val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) + val env = builder.environment() + env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) + val sparkHome = System.getenv("SPARK_HOME") + env.put("R_PROFILE_USER", + Seq(sparkHome, "R", "lib", "SparkR", "profile", "general.R").mkString(File.separator)) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect R output").start() + + process.waitFor() + } finally { + sparkRBackend.close() + } + System.exit(returnCode) + } else { + System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + System.exit(-1) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala similarity index 89% rename from core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala rename to core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 5b22481ea8c5f..b8d3993540220 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.master +package org.apache.spark.deploy import scala.collection.JavaConversions._ @@ -25,15 +25,17 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.{Logging, SparkConf} -private[deploy] object SparkCuratorUtil extends Logging { +private[spark] object SparkCuratorUtil extends Logging { private val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 private val ZK_SESSION_TIMEOUT_MILLIS = 60000 private val RETRY_WAIT_MILLIS = 5000 private val MAX_RECONNECT_ATTEMPTS = 3 - def newClient(conf: SparkConf): CuratorFramework = { - val ZK_URL = conf.get("spark.deploy.zookeeper.url") + def newClient( + conf: SparkConf, + zkUrlConf: String = "spark.deploy.zookeeper.url"): CuratorFramework = { + val ZK_URL = conf.get(zkUrlConf) val zk = CuratorFrameworkFactory.newClient(ZK_URL, ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, new ExponentialBackoffRetry(RETRY_WAIT_MILLIS, MAX_RECONNECT_ATTEMPTS)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e0a32fb65cd51..b563034457a91 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,22 +17,27 @@ package org.apache.spark.deploy +import java.io.{ByteArrayInputStream, DataInputStream} import java.lang.reflect.Method import java.security.PrivilegedExceptionAction +import java.util.{Arrays, Comparator} +import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.fs.FileSystem.Statistics +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils import scala.collection.JavaConversions._ +import scala.concurrent.duration._ +import scala.language.postfixOps /** * :: DeveloperApi :: @@ -40,7 +45,8 @@ import scala.collection.JavaConversions._ */ @DeveloperApi class SparkHadoopUtil extends Logging { - val conf: Configuration = newConfiguration(new SparkConf()) + private val sparkConf = new SparkConf() + val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) /** @@ -193,7 +199,7 @@ class SparkHadoopUtil extends Logging { * that file. */ def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { - def recurse(path: Path) = { + def recurse(path: Path): Array[FileStatus] = { val (directories, leaves) = fs.listStatus(path).partition(_.isDir) leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath)) } @@ -201,6 +207,103 @@ class SparkHadoopUtil extends Logging { val baseStatus = fs.getFileStatus(basePath) if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) } + + /** + * Lists all the files in a directory with the specified prefix, and does not end with the + * given suffix. The returned {{FileStatus}} instances are sorted by the modification times of + * the respective files. + */ + def listFilesSorted( + remoteFs: FileSystem, + dir: Path, + prefix: String, + exclusionSuffix: String): Array[FileStatus] = { + val fileStatuses = remoteFs.listStatus(dir, + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + name.startsWith(prefix) && !name.endsWith(exclusionSuffix) + } + }) + Arrays.sort(fileStatuses, new Comparator[FileStatus] { + override def compare(o1: FileStatus, o2: FileStatus): Int = { + Longs.compare(o1.getModificationTime, o2.getModificationTime) + } + }) + fileStatuses + } + + /** + * How much time is remaining (in millis) from now to (fraction * renewal time for the token that + * is valid the latest)? + * This will return -ve (or 0) value if the fraction of validity has already expired. + */ + def getTimeFromNowToRenewal( + sparkConf: SparkConf, + fraction: Double, + credentials: Credentials): Long = { + val now = System.currentTimeMillis() + + val renewalInterval = + sparkConf.getLong("spark.yarn.token.renewal.interval", (24 hours).toMillis) + + credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + .map { t => + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + (identifier.getIssueDate + fraction * renewalInterval).toLong - now + }.foldLeft(0L)(math.max) + } + + + private[spark] def getSuffixForCredentialsPath(credentialsPath: Path): Int = { + val fileName = credentialsPath.getName + fileName.substring( + fileName.lastIndexOf(SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM) + 1).toInt + } + + + private val HADOOP_CONF_PATTERN = "(\\$\\{hadoopconf-[^\\}\\$\\s]+\\})".r.unanchored + + /** + * Substitute variables by looking them up in Hadoop configs. Only variables that match the + * ${hadoopconf- .. } pattern are substituted. + */ + def substituteHadoopVariables(text: String, hadoopConf: Configuration): String = { + text match { + case HADOOP_CONF_PATTERN(matched) => { + logDebug(text + " matched " + HADOOP_CONF_PATTERN) + val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } + val eval = Option[String](hadoopConf.get(key)) + .map { value => + logDebug("Substituted " + matched + " with " + value) + text.replace(matched, value) + } + if (eval.isEmpty) { + // The variable was not found in Hadoop configs, so return text as is. + text + } else { + // Continue to substitute more variables. + substituteHadoopVariables(eval.get, hadoopConf) + } + } + case _ => { + logDebug(text + " didn't match " + HADOOP_CONF_PATTERN) + text + } + } + } + + /** + * Start a thread to periodically update the current user's credentials with new delegation + * tokens so that writes to HDFS do not fail. + */ + private[spark] def startExecutorDelegationTokenRenewer(conf: SparkConf) {} + + /** + * Stop the thread that does the delegation token updates. + */ + private[spark] def stopExecutorDelegationTokenRenewer() {} } object SparkHadoopUtil { @@ -221,6 +324,10 @@ object SparkHadoopUtil { } } + val SPARK_YARN_CREDS_TEMP_EXTENSION = ".tmp" + + val SPARK_YARN_CREDS_COUNTER_DELIM = "-" + def get: SparkHadoopUtil = { hadoop } 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 4f506be63fe59..8a0327984e195 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -36,11 +36,11 @@ import org.apache.ivy.core.retrieve.RetrieveOptions import org.apache.ivy.core.settings.IvySettings import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.resolver.{ChainResolver, IBiblioResolver} - import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} + /** * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone cluster mode. @@ -77,6 +77,7 @@ object SparkSubmit { // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val SPARKR_SHELL = "sparkr-shell" private val CLASS_NOT_FOUND_EXIT_STATUS = 101 @@ -113,19 +114,21 @@ object SparkSubmit { } } - /** Kill an existing submission using the REST protocol. Standalone cluster mode only. */ + /** + * Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only. + */ private def kill(args: SparkSubmitArguments): Unit = { - new StandaloneRestClient() - .killSubmission(args.master, args.submissionToKill) + new RestSubmissionClient(args.master) + .killSubmission(args.submissionToKill) } /** * Request the status of an existing submission using the REST protocol. - * Standalone cluster mode only. + * Standalone and Mesos cluster mode only. */ private def requestStatus(args: SparkSubmitArguments): Unit = { - new StandaloneRestClient() - .requestSubmissionStatus(args.master, args.submissionToRequestStatusFor) + new RestSubmissionClient(args.master) + .requestSubmissionStatus(args.submissionToRequestStatusFor) } /** @@ -251,6 +254,7 @@ object SparkSubmit { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -284,13 +288,24 @@ object SparkSubmit { } } + // Require all R files to be local + if (args.isR && !isYarnCluster) { + if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { + printErrorAndExit(s"Only local R files are supported: $args.primaryResource") + } + } + // The following modes are not supported or applicable (clusterManager, deployMode) match { - case (MESOS, CLUSTER) => - printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") + case (MESOS, CLUSTER) if args.isPython => + printErrorAndExit("Cluster deploy mode is currently not supported for python " + + "applications on Mesos clusters.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") + case (STANDALONE, CLUSTER) if args.isR => + printErrorAndExit("Cluster deploy mode is currently not supported for R " + + "applications on standalone clusters.") case (_, CLUSTER) if isShell(args.primaryResource) => printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") case (_, CLUSTER) if isSqlShell(args.mainClass) => @@ -317,11 +332,32 @@ object SparkSubmit { } } - // In yarn-cluster mode for a python app, add primary resource and pyFiles to files - // that can be distributed with the job - if (args.isPython && isYarnCluster) { - args.files = mergeFileLists(args.files, args.primaryResource) - args.files = mergeFileLists(args.files, args.pyFiles) + // If we're running a R app, set the main class to our specific R runner + if (args.isR && deployMode == CLIENT) { + if (args.primaryResource == SPARKR_SHELL) { + args.mainClass = "org.apache.spark.api.r.RBackend" + } else { + // If a R file is provided, add it to the child arguments and list of files to deploy. + // Usage: RRunner
[app arguments] + args.mainClass = "org.apache.spark.deploy.RRunner" + args.childArgs = ArrayBuffer(args.primaryResource) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } + } + + if (isYarnCluster) { + // In yarn-cluster mode for a python app, add primary resource and pyFiles to files + // that can be distributed with the job + if (args.isPython) { + args.files = mergeFileLists(args.files, args.primaryResource) + args.files = mergeFileLists(args.files, args.pyFiles) + } + + // In yarn-cluster mode for a R app, add primary resource to files + // that can be distributed with the job + if (args.isR) { + args.files = mergeFileLists(args.files, args.primaryResource) + } } // Special flag to avoid deprecation warnings at the client @@ -345,15 +381,6 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.driver.extraLibraryPath"), - // Standalone cluster only - // Do not set CL arguments here because there are multiple possibilities for the main class - OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), - OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), - OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, sysProp = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE, CLUSTER, sysProp = "spark.driver.cores"), - OptionAssigner(args.supervise.toString, STANDALONE, CLUSTER, - sysProp = "spark.driver.supervise"), - // Yarn client only OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, CLIENT, sysProp = "spark.executor.instances"), @@ -373,13 +400,27 @@ object SparkSubmit { OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"), OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), + // Yarn client or cluster + OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, clOption = "--principal"), + OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, clOption = "--keytab"), + // Other options + OptionAssigner(args.executorCores, STANDALONE, ALL_DEPLOY_MODES, + sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, - sysProp = "spark.files") + sysProp = "spark.files"), + OptionAssigner(args.jars, STANDALONE | MESOS, CLUSTER, sysProp = "spark.jars"), + OptionAssigner(args.driverMemory, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.memory"), + OptionAssigner(args.driverCores, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.cores"), + OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, + sysProp = "spark.driver.supervise"), + OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy") ) // In client mode, launch the application main class directly @@ -405,8 +446,8 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" - // For python files, the primary resource is already distributed as a regular file - if (!isYarnCluster && !args.isPython) { + // For python and R files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !args.isPython && !args.isR) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) @@ -418,7 +459,7 @@ object SparkSubmit { // All Spark parameters are expected to be passed to the client through system properties. if (args.isStandaloneCluster) { if (args.useRest) { - childMainClass = "org.apache.spark.deploy.rest.StandaloneRestClient" + childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" childArgs += (args.primaryResource, args.mainClass) } else { // In legacy standalone cluster mode, use Client as a wrapper around the user class @@ -447,6 +488,10 @@ object SparkSubmit { childArgs += ("--py-files", pyFilesNames) } childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else if (args.isR) { + val mainFile = new Path(args.primaryResource).getName + childArgs += ("--primary-r-file", mainFile) + childArgs += ("--class", "org.apache.spark.deploy.RRunner") } else { if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) @@ -458,6 +503,15 @@ object SparkSubmit { } } + if (isMesosCluster) { + assert(args.useRest, "Mesos cluster mode is only supported through the REST submission API") + childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" + childArgs += (args.primaryResource, args.mainClass) + if (args.childArgs != null) { + childArgs ++= args.childArgs + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) @@ -591,15 +645,15 @@ object SparkSubmit { /** * Return whether the given primary resource represents a user jar. */ - private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + private[deploy] def isUserJar(res: String): Boolean = { + !isShell(res) && !isPython(res) && !isInternal(res) && !isR(res) } /** * Return whether the given primary resource represents a shell. */ - private[deploy] def isShell(primaryResource: String): Boolean = { - primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + private[deploy] def isShell(res: String): Boolean = { + (res == SPARK_SHELL || res == PYSPARK_SHELL || res == SPARKR_SHELL) } /** @@ -619,12 +673,19 @@ object SparkSubmit { /** * Return whether the given primary resource requires running python. */ - private[deploy] def isPython(primaryResource: String): Boolean = { - primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL + private[deploy] def isPython(res: String): Boolean = { + res != null && res.endsWith(".py") || res == PYSPARK_SHELL } - private[deploy] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL + /** + * Return whether the given primary resource requires running R. + */ + private[deploy] def isR(res: String): Boolean = { + res != null && res.endsWith(".R") || res == SPARKR_SHELL + } + + private[deploy] def isInternal(res: String): Boolean = { + res == SPARK_INTERNAL } /** @@ -640,7 +701,7 @@ object SparkSubmit { } /** Provides utility functions to be used inside SparkSubmit. */ -private[deploy] object SparkSubmitUtils { +private[spark] object SparkSubmitUtils { // Exposed for testing var printStream = SparkSubmit.printStream @@ -677,13 +738,31 @@ private[deploy] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string * @param remoteRepos Comma-delimited string of remote repositories + * @param ivySettings The Ivy settings for this session * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ - def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { + def createRepoResolvers(remoteRepos: Option[String], ivySettings: IvySettings): ChainResolver = { // We need a chain resolver if we want to check multiple repositories val cr = new ChainResolver cr.setName("list") + val localM2 = new IBiblioResolver + localM2.setM2compatible(true) + val m2Path = ".m2" + File.separator + "repository" + File.separator + localM2.setRoot(new File(System.getProperty("user.home"), m2Path).toURI.toString) + localM2.setUsepoms(true) + localM2.setName("local-m2-cache") + cr.add(localM2) + + val localIvy = new IBiblioResolver + localIvy.setRoot(new File(ivySettings.getDefaultIvyUserDir, + "local" + File.separator).toURI.toString) + val ivyPattern = Seq("[organisation]", "[module]", "[revision]", "[type]s", + "[artifact](-[classifier]).[ext]").mkString(File.separator) + localIvy.setPattern(ivyPattern) + localIvy.setName("local-ivy-cache") + cr.add(localIvy) + // the biblio resolver resolves POM declared dependencies val br: IBiblioResolver = new IBiblioResolver br.setM2compatible(true) @@ -716,8 +795,7 @@ private[deploy] object SparkSubmitUtils { /** * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath - * (will append to jars in SparkSubmit). The name of the jar is given - * after a '!' by Ivy. It also sometimes contains '(bundle)' after '.jar'. Remove that as well. + * (will append to jars in SparkSubmit). * @param artifacts Sequence of dependencies that were resolved and retrieved * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies @@ -726,10 +804,9 @@ private[deploy] object SparkSubmitUtils { artifacts: Array[AnyRef], cacheDirectory: File): String = { artifacts.map { artifactInfo => - val artifactString = artifactInfo.toString - val jarName = artifactString.drop(artifactString.lastIndexOf("!") + 1) + val artifact = artifactInfo.asInstanceOf[Artifact].getModuleRevisionId cacheDirectory.getAbsolutePath + File.separator + - jarName.substring(0, jarName.lastIndexOf(".jar") + 4) + s"${artifact.getOrganisation}_${artifact.getName}-${artifact.getRevision}.jar" }.mkString(",") } @@ -777,7 +854,7 @@ private[deploy] object SparkSubmitUtils { } /** A nice function to use in tests as well. Values are dummy strings. */ - def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + def getModuleDescriptor: DefaultModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) /** @@ -811,6 +888,7 @@ private[deploy] object SparkSubmitUtils { if (alternateIvyCache.trim.isEmpty) { new File(ivySettings.getDefaultIvyUserDir, "jars") } else { + ivySettings.setDefaultIvyUserDir(new File(alternateIvyCache)) ivySettings.setDefaultCache(new File(alternateIvyCache, "cache")) new File(alternateIvyCache, "jars") } @@ -820,7 +898,7 @@ private[deploy] object SparkSubmitUtils { // create a pattern matcher ivySettings.addMatcher(new GlobPatternMatcher) // create the dependency resolvers - val repoResolver = createRepoResolvers(remoteRepos) + val repoResolver = createRepoResolvers(remoteRepos, ivySettings) ivySettings.addResolver(repoResolver) ivySettings.setDefaultResolver(repoResolver.getName) @@ -854,7 +932,8 @@ private[deploy] object SparkSubmitUtils { } // retrieve all resolved dependencies ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId, - packagesDirectory.getAbsolutePath + File.separator + "[artifact](-[classifier]).[ext]", + packagesDirectory.getAbsolutePath + File.separator + + "[organization]_[artifact]-[revision].[ext]", retrieveOptions.setConfs(Array(ivyConfName))) System.setOut(sysOut) resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory) 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 2250d5a28e4ef..c0e4c771908b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -59,9 +59,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + var isR: Boolean = false var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() var proxyUser: String = null + var principal: String = null + var keytab: String = null // Standalone cluster mode only var supervise: Boolean = false @@ -76,12 +79,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => Utils.getPropertiesFromFile(filename).foreach { case (k, v) => - if (k.startsWith("spark.")) { - defaultProperties(k) = v - if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") - } else { - SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") - } + defaultProperties(k) = v + if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") } } defaultProperties @@ -96,6 +95,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } // Populate `sparkProperties` map from properties file mergeDefaultSparkProperties() + // Remove keys that don't start with "spark." from `sparkProperties`. + ignoreNonSparkProperties() // Use `sparkProperties` map along with env vars to fill in any missing parameters loadEnvironmentArguments() @@ -116,6 +117,18 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } } + /** + * Remove keys that don't start with "spark." from `sparkProperties`. + */ + private def ignoreNonSparkProperties(): Unit = { + sparkProperties.foreach { case (k, v) => + if (!k.startsWith("spark.")) { + sparkProperties -= k + SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + /** * Load arguments from environment variables, Spark properties etc. */ @@ -158,7 +171,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given - if (mainClass == null && !isPython && primaryResource != null) { + if (mainClass == null && !isPython && !isR && primaryResource != null) { val uri = new URI(primaryResource) val uriScheme = uri.getScheme() @@ -211,9 +224,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S printUsageAndExit(-1) } if (primaryResource == null) { - SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python or R file)") } - if (mainClass == null && !isPython) { + if (mainClass == null && SparkSubmit.isUserJar(primaryResource)) { SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") } if (pyFiles != null && !isPython) { @@ -230,8 +243,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://")) { - SparkSubmit.printErrorAndExit("Killing submissions is only supported in standalone mode!") + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + SparkSubmit.printErrorAndExit( + "Killing submissions is only supported in standalone or Mesos mode!") } if (submissionToKill == null) { SparkSubmit.printErrorAndExit("Please specify a submission to kill.") @@ -239,9 +253,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://")) { + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { SparkSubmit.printErrorAndExit( - "Requesting submission statuses is only supported in standalone mode!") + "Requesting submission statuses is only supported in standalone or Mesos mode!") } if (submissionToRequestStatusFor == null) { SparkSubmit.printErrorAndExit("Please specify a submission to request status for.") @@ -252,7 +266,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S master.startsWith("spark://") && deployMode == "cluster" } - override def toString = { + override def toString: String = { s"""Parsed arguments: | master $master | deployMode $deployMode @@ -381,6 +395,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case PROXY_USER => proxyUser = value + case PRINCIPAL => + principal = value + + case KEYTAB => + keytab = value + case HELP => printUsageAndExit(0) @@ -414,6 +434,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S opt } isPython = SparkSubmit.isPython(opt) + isR = SparkSubmit.isR(opt) false } @@ -473,6 +494,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). + | + | Spark standalone or Mesos with cluster deploy mode only: | --supervise If given, restarts the driver on failure. | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. @@ -480,14 +503,24 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | + | Spark standalone and YARN only: + | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, + | or all available cores on the worker in standalone mode) + | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). - | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. + | --principal PRINCIPAL Principal to be used to login to KDC, while running on + | secure HDFS. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. This keytab will be copied to + | the node running the Application Master via the Secure + | Distributed Cache, for renewing the login tickets and the + | delegation tokens periodically. """.stripMargin ) SparkSubmit.exitFn() diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 2d24083a77b73..43c8a934c311a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.{ActorLogReceive, Utils, AkkaUtils} +import org.apache.spark.util.{ActorLogReceive, RpcUtils, Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -116,7 +116,7 @@ private[spark] class AppClient( masterAkkaUrls.map(AddressFromURIString(_).hostPort).contains(remoteUrl.hostPort) } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ registered = true @@ -157,6 +157,7 @@ private[spark] class AppClient( case StopAppClient => markDead("Application has been stopped.") + master ! UnregisterApplication(appId) sender ! true context.stop(self) } @@ -192,7 +193,7 @@ private[spark] class AppClient( def stop() { if (actor != null) { try { - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) val future = actor.ask(StopAppClient)(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index c1c4812f17fbe..40835b9550586 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -46,7 +46,7 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val conf = new SparkConf - val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localHostName(), 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription("TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index ea6c85ee511d5..298a8201960d1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,15 +19,19 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[history] case class ApplicationHistoryInfo( - id: String, - name: String, +private[spark] case class ApplicationAttemptInfo( + attemptId: Option[String], startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, completed: Boolean = false) +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + attempts: List[ApplicationAttemptInfo]) + private[history] abstract class ApplicationHistoryProvider { /** @@ -41,9 +45,10 @@ private[history] abstract class ApplicationHistoryProvider { * Returns the Spark UI for a specific application. * * @param appId The application ID. + * @param attemptId The application attempt ID (or None if there is no attempt ID). * @return The application's UI, or None if application is not found. */ - def getAppUI(appId: String): Option[SparkUI] + def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] /** * Called when the server is shutting down. 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 7fde02040927d..45c2be34c8680 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 @@ -17,47 +17,43 @@ package org.apache.spark.deploy.history -import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream} +import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream} import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import scala.collection.mutable -import scala.concurrent.duration.Duration -import com.google.common.util.concurrent.ThreadFactoryBuilder - -import com.google.common.util.concurrent.MoreExecutors -import org.apache.hadoop.fs.permission.AccessControlException +import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.permission.AccessControlException + +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SecurityManager, SparkConf} - +import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** * A class that provides application history from event logs stored in the file system. * This provider checks for new finished applications in the background periodically and * renders the history application UI by parsing the associated event logs. */ -private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider - with Logging { +private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) + extends ApplicationHistoryProvider with Logging { + + def this(conf: SparkConf) = { + this(conf, new SystemClock()) + } import FsHistoryProvider._ private val NOT_STARTED = "" // Interval between each check for event log updates - private val UPDATE_INTERVAL_MS = conf.getOption("spark.history.fs.update.interval.seconds") - .orElse(conf.getOption("spark.history.fs.updateInterval")) - .orElse(conf.getOption("spark.history.updateInterval")) - .map(_.toInt) - .getOrElse(10) * 1000 + private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") // Interval between each cleaner checks for event logs to delete - private val CLEAN_INTERVAL_MS = conf.getLong("spark.history.fs.cleaner.interval.seconds", - DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S) * 1000 + private val CLEAN_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.cleaner.interval", "1d") private val logDir = conf.getOption("spark.history.fs.logDirectory") .map { d => Utils.resolveURI(d).toString } @@ -81,6 +77,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] = new mutable.LinkedHashMap() + // List of application logs to be deleted by event log cleaner. + private var attemptsToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] + // Constants used to parse Spark 1.0.0 log directories. private[history] val LOG_PREFIX = "EVENT_LOG_" private[history] val SPARK_VERSION_PREFIX = EventLoggingListener.SPARK_VERSION_KEY + "_" @@ -93,7 +92,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def getRunner(operateFun: () => Unit): Runnable = { new Runnable() { - override def run() = Utils.tryOrExit { + override def run(): Unit = Utils.tryOrExit { operateFun() } } @@ -104,7 +103,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private val replayExecutor: ExecutorService = { if (!conf.contains("spark.testing")) { - Executors.newSingleThreadExecutor(Utils.namedThreadFactory("log-replay-executor")) + ThreadUtils.newDaemonSingleThreadExecutor("log-replay-executor") } else { MoreExecutors.sameThreadExecutor() } @@ -118,7 +117,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (!fs.exists(path)) { var msg = s"Log directory specified does not exist: $logDir." if (logDir == DEFAULT_LOG_DIR) { - msg += " Did you configure the correct one through spark.fs.history.logDirectory?" + msg += " Did you configure the correct one through spark.history.fs.logDirectory?" } throw new IllegalArgumentException(msg) } @@ -130,44 +129,44 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // Disable the background thread during tests. if (!conf.contains("spark.testing")) { // A task that periodically checks for event log updates on disk. - pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(checkForLogs), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS) if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { // A task that periodically cleans event logs on disk. - pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_MS, - TimeUnit.MILLISECONDS) + pool.scheduleAtFixedRate(getRunner(cleanLogs), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS) } } } - override def getListing() = applications.values + override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values - override def getAppUI(appId: String): Option[SparkUI] = { + override def getAppUI(appId: String, attemptId: Option[String]): Option[SparkUI] = { try { - applications.get(appId).map { info => - val replayBus = new ReplayListenerBus() - val ui = { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, - s"${HistoryServer.UI_PATH_PREFIX}/$appId") - // Do not call ui.bind() to avoid creating a new server for each application - } + applications.get(appId).flatMap { appInfo => + appInfo.attempts.find(_.attemptId == attemptId).map { attempt => + val replayBus = new ReplayListenerBus() + val ui = { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, + HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime) + // Do not call ui.bind() to avoid creating a new server for each application + } - val appListener = new ApplicationEventListener() - replayBus.addListener(appListener) - val appInfo = replay(fs.getFileStatus(new Path(logDir, info.logPath)), replayBus) + val appListener = new ApplicationEventListener() + replayBus.addListener(appListener) + val appInfo = replay(fs.getFileStatus(new Path(logDir, attempt.logPath)), replayBus) - ui.setAppName(s"${appInfo.name} ($appId)") + ui.setAppName(s"${appInfo.name} ($appId)") - val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setAcls(uiAclsEnabled) - // make sure to set admin acls before view acls so they are properly picked up - ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) - ui.getSecurityManager.setViewAcls(appInfo.sparkUser, - appListener.viewAcls.getOrElse("")) - ui + val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so they are properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) + ui.getSecurityManager.setViewAcls(attempt.sparkUser, + appListener.viewAcls.getOrElse("")) + ui + } } } catch { case e: FileNotFoundException => None @@ -225,7 +224,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def mergeApplicationListing(logs: Seq[FileStatus]): Unit = { val bus = new ReplayListenerBus() - val newApps = logs.flatMap { fileStatus => + val newAttempts = logs.flatMap { fileStatus => try { val res = replay(fileStatus, bus) logInfo(s"Application log ${res.logPath} loaded successfully.") @@ -233,74 +232,108 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } catch { case e: Exception => logError( - s"Exception encountered when attempting to load application log ${fileStatus.getPath}") + s"Exception encountered when attempting to load application log ${fileStatus.getPath}", + e) None } - }.toSeq.sortWith(compareAppInfo) - - // When there are new logs, merge the new list with the existing one, maintaining - // the expected ordering (descending end time). Maintaining the order is important - // to avoid having to sort the list every time there is a request for the log list. - if (newApps.nonEmpty) { - val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { - if (!mergedApps.contains(info.id) || - mergedApps(info.id).logPath.endsWith(EventLoggingListener.IN_PROGRESS) && - !info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - mergedApps += (info.id -> info) - } - } + } + + if (newAttempts.isEmpty) { + return + } - val newIterator = newApps.iterator.buffered - val oldIterator = applications.values.iterator.buffered - while (newIterator.hasNext && oldIterator.hasNext) { - if (compareAppInfo(newIterator.head, oldIterator.head)) { - addIfAbsent(newIterator.next()) - } else { - addIfAbsent(oldIterator.next()) + // Build a map containing all apps that contain new attempts. The app information in this map + // contains both the new app attempt, and those that were already loaded in the existing apps + // map. If an attempt has been updated, it replaces the old attempt in the list. + val newAppMap = new mutable.HashMap[String, FsApplicationHistoryInfo]() + newAttempts.foreach { attempt => + val appInfo = newAppMap.get(attempt.appId) + .orElse(applications.get(attempt.appId)) + .map { app => + val attempts = + app.attempts.filter(_.attemptId != attempt.attemptId).toList ++ List(attempt) + new FsApplicationHistoryInfo(attempt.appId, attempt.name, + attempts.sortWith(compareAttemptInfo)) } + .getOrElse(new FsApplicationHistoryInfo(attempt.appId, attempt.name, List(attempt))) + newAppMap(attempt.appId) = appInfo + } + + // Merge the new app list with the existing one, maintaining the expected ordering (descending + // end time). Maintaining the order is important to avoid having to sort the list every time + // there is a request for the log list. + val newApps = newAppMap.values.toSeq.sortWith(compareAppInfo) + val mergedApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def addIfAbsent(info: FsApplicationHistoryInfo): Unit = { + if (!mergedApps.contains(info.id)) { + mergedApps += (info.id -> info) } - newIterator.foreach(addIfAbsent) - oldIterator.foreach(addIfAbsent) + } - applications = mergedApps + val newIterator = newApps.iterator.buffered + val oldIterator = applications.values.iterator.buffered + while (newIterator.hasNext && oldIterator.hasNext) { + if (newAppMap.contains(oldIterator.head.id)) { + oldIterator.next() + } else if (compareAppInfo(newIterator.head, oldIterator.head)) { + addIfAbsent(newIterator.next()) + } else { + addIfAbsent(oldIterator.next()) + } } + newIterator.foreach(addIfAbsent) + oldIterator.foreach(addIfAbsent) + + applications = mergedApps } /** * Delete event logs from the log directory according to the clean policy defined by the user. */ - private def cleanLogs(): Unit = { + private[history] def cleanLogs(): Unit = { try { - val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) - .getOrElse(Seq[FileStatus]()) - val maxAge = conf.getLong("spark.history.fs.cleaner.maxAge.seconds", - DEFAULT_SPARK_HISTORY_FS_MAXAGE_S) * 1000 + val maxAge = conf.getTimeAsSeconds("spark.history.fs.cleaner.maxAge", "7d") * 1000 - val now = System.currentTimeMillis() + val now = clock.getTimeMillis() val appsToRetain = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() - applications.values.foreach { info => - if (now - info.lastUpdated <= maxAge) { - appsToRetain += (info.id -> info) + def shouldClean(attempt: FsApplicationAttemptInfo): Boolean = { + now - attempt.lastUpdated > maxAge && attempt.completed + } + + // Scan all logs from the log directory. + // Only completed applications older than the specified max age will be deleted. + applications.values.foreach { app => + val (toClean, toRetain) = app.attempts.partition(shouldClean) + attemptsToClean ++= toClean + + if (toClean.isEmpty) { + appsToRetain += (app.id -> app) + } else if (toRetain.nonEmpty) { + appsToRetain += (app.id -> + new FsApplicationHistoryInfo(app.id, app.name, toRetain.toList)) } } applications = appsToRetain - // Scan all logs from the log directory. - // Only directories older than the specified max age will be deleted - statusList.foreach { dir => + val leftToClean = new mutable.ListBuffer[FsApplicationAttemptInfo] + attemptsToClean.foreach { attempt => try { - if (now - dir.getModificationTime() > maxAge) { - // if path is a directory and set to true, - // the directory is deleted else throws an exception - fs.delete(dir.getPath, true) + val path = new Path(logDir, attempt.logPath) + if (fs.exists(path)) { + fs.delete(path, true) } } catch { - case t: IOException => logError(s"IOException in cleaning logs of $dir", t) + case e: AccessControlException => + logInfo(s"No permission to delete ${attempt.logPath}, ignoring.") + case t: IOException => + logError(s"IOException in cleaning ${attempt.logPath}", t) + leftToClean += attempt } } + + attemptsToClean = leftToClean } catch { case t: Exception => logError("Exception in cleaning logs", t) } @@ -314,14 +347,36 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def compareAppInfo( i1: FsApplicationHistoryInfo, i2: FsApplicationHistoryInfo): Boolean = { - if (i1.endTime != i2.endTime) i1.endTime >= i2.endTime else i1.startTime >= i2.startTime + val a1 = i1.attempts.head + val a2 = i2.attempts.head + if (a1.endTime != a2.endTime) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime + } + + /** + * Comparison function that defines the sort order for application attempts within the same + * application. Order is: running attempts before complete attempts, running attempts sorted + * by start time, completed attempts sorted by end time. + * + * Normally applications should have a single running attempt; but failure to call sc.stop() + * may cause multiple running attempts to show up. + * + * @return Whether `a1` should precede `a2`. + */ + private def compareAttemptInfo( + a1: FsApplicationAttemptInfo, + a2: FsApplicationAttemptInfo): Boolean = { + if (a1.completed == a2.completed) { + if (a1.completed) a1.endTime >= a2.endTime else a1.startTime >= a2.startTime + } else { + !a1.completed + } } /** * Replays the events in the specified log file and returns information about the associated * application. */ - private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationHistoryInfo = { + private def replay(eventLog: FileStatus, bus: ReplayListenerBus): FsApplicationAttemptInfo = { val logPath = eventLog.getPath() logInfo(s"Replaying log path: $logPath") val logInput = @@ -332,17 +387,19 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } try { val appListener = new ApplicationEventListener + val appCompleted = isApplicationCompleted(eventLog) bus.addListener(appListener) - bus.replay(logInput, logPath.toString) - new FsApplicationHistoryInfo( + bus.replay(logInput, logPath.toString, !appCompleted) + new FsApplicationAttemptInfo( logPath.getName(), - appListener.appId.getOrElse(logPath.getName()), appListener.appName.getOrElse(NOT_STARTED), + appListener.appId.getOrElse(logPath.getName()), + appListener.appAttemptId, appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), getModificationTime(eventLog).get, appListener.sparkUser.getOrElse(NOT_STARTED), - isApplicationCompleted(eventLog)) + appCompleted) } finally { logInput.close() } @@ -421,21 +478,23 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private object FsHistoryProvider { val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - - // One day - val DEFAULT_SPARK_HISTORY_FS_CLEANER_INTERVAL_S = Duration(1, TimeUnit.DAYS).toSeconds - - // One week - val DEFAULT_SPARK_HISTORY_FS_MAXAGE_S = Duration(7, TimeUnit.DAYS).toSeconds } -private class FsApplicationHistoryInfo( +private class FsApplicationAttemptInfo( val logPath: String, - id: String, - name: String, + val name: String, + val appId: String, + attemptId: Option[String], startTime: Long, endTime: Long, lastUpdated: Long, sparkUser: String, completed: Boolean = true) - extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser, completed) + extends ApplicationAttemptInfo( + attemptId, startTime, endTime, lastUpdated, sparkUser, completed) + +private class FsApplicationHistoryInfo( + id: String, + override val name: String, + override val attempts: List[FsApplicationAttemptInfo]) + extends ApplicationHistoryInfo(id, name, attempts) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 6e432d63c6b5a..0830cc1ba1245 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -34,18 +34,28 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") val requestedIncomplete = Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean - val allApps = parent.getApplicationList().filter(_.completed != requestedIncomplete) - val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0 - val apps = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allApps.size)) + val allApps = parent.getApplicationList() + .filter(_.attempts.head.completed != requestedIncomplete) + val allAppsSize = allApps.size + + val actualFirst = if (requestedFirst < allAppsSize) requestedFirst else 0 + val appsToShow = allApps.slice(actualFirst, actualFirst + pageSize) val actualPage = (actualFirst / pageSize) + 1 - val last = Math.min(actualFirst + pageSize, allApps.size) - 1 - val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + val last = Math.min(actualFirst + pageSize, allAppsSize) - 1 + val pageCount = allAppsSize / pageSize + (if (allAppsSize % pageSize > 0) 1 else 0) val secondPageFromLeft = 2 val secondPageFromRight = pageCount - 1 - val appTable = UIUtils.listingTable(appHeader, appRow, apps) + val hasMultipleAttempts = appsToShow.exists(_.attempts.size > 1) + val appTable = + if (hasMultipleAttempts) { + UIUtils.listingTable(appWithAttemptHeader, appWithAttemptRow, appsToShow) + } else { + UIUtils.listingTable(appHeader, appRow, appsToShow) + } + val providerConfig = parent.getProviderConfig() val content =
@@ -59,7 +69,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") // to the first and last page. If the current page +/- `plusOrMinus` is greater // than the 2nd page from the first page or less than the 2nd page from the last // page, `...` will be displayed. - if (allApps.size > 0) { + if (allAppsSize > 0) { val leftSideIndices = rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _, requestedIncomplete) val rightSideIndices = @@ -67,7 +77,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") requestedIncomplete)

- Showing {actualFirst + 1}-{last + 1} of {allApps.size} + Showing {actualFirst + 1}-{last + 1} of {allAppsSize} {if (requestedIncomplete) "(Incomplete applications)"} { @@ -90,6 +100,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("")

++ appTable + } else if (requestedIncomplete) { +

No incomplete applications found!

} else {

No completed applications found!

++

Did you specify the correct logging directory? @@ -123,30 +135,85 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") "Spark User", "Last Updated") - private def rangeIndices(range: Seq[Int], condition: Int => Boolean, showIncomplete: Boolean): - Seq[Node] = { + private val appWithAttemptHeader = Seq( + "App ID", + "App Name", + "Attempt ID", + "Started", + "Completed", + "Duration", + "Spark User", + "Last Updated") + + private def rangeIndices( + range: Seq[Int], + condition: Int => Boolean, + showIncomplete: Boolean): Seq[Node] = { range.filter(condition).map(nextPage => {nextPage} ) } - private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" - val startTime = UIUtils.formatDate(info.startTime) - val endTime = if (info.endTime > 0) UIUtils.formatDate(info.endTime) else "-" + private def attemptRow( + renderAttemptIdColumn: Boolean, + info: ApplicationHistoryInfo, + attempt: ApplicationAttemptInfo, + isFirst: Boolean): Seq[Node] = { + val uiAddress = HistoryServer.getAttemptURI(info.id, attempt.attemptId) + val startTime = UIUtils.formatDate(attempt.startTime) + val endTime = if (attempt.endTime > 0) UIUtils.formatDate(attempt.endTime) else "-" val duration = - if (info.endTime > 0) UIUtils.formatDuration(info.endTime - info.startTime) else "-" - val lastUpdated = UIUtils.formatDate(info.lastUpdated) + if (attempt.endTime > 0) { + UIUtils.formatDuration(attempt.endTime - attempt.startTime) + } else { + "-" + } + val lastUpdated = UIUtils.formatDate(attempt.lastUpdated) - {info.id} - {info.name} - {startTime} - {endTime} - {duration} - {info.sparkUser} - {lastUpdated} + { + if (isFirst) { + if (info.attempts.size > 1 || renderAttemptIdColumn) { + + {info.id} + + {info.name} + } else { + {info.id} + {info.name} + } + } else { + Nil + } + } + { + if (renderAttemptIdColumn) { + if (info.attempts.size > 1 && attempt.attemptId.isDefined) { + + {attempt.attemptId.get} + } else { +   + } + } else { + Nil + } + } + {startTime} + {endTime} + + {duration} + {attempt.sparkUser} + {lastUpdated} } + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + attemptRow(false, info, info.attempts.head, true) + } + + private def appWithAttemptRow(info: ApplicationHistoryInfo): Seq[Node] = { + attemptRow(true, info, info.attempts.head, true) ++ + info.attempts.drop(1).flatMap(attemptRow(true, info, _, false)) + } + private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { "/?" + Array( "page=" + linkPage, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index af483d560b33e..50522e69dc519 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,9 +25,10 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.SignalLogger +import org.apache.spark.util.{SignalLogger, Utils} /** * A web server that renders SparkUIs of completed applications. @@ -45,14 +46,18 @@ class HistoryServer( provider: ApplicationHistoryProvider, securityManager: SecurityManager, port: Int) - extends WebUI(securityManager, port, conf) with Logging { + extends WebUI(securityManager, port, conf) with Logging with UIRoot { // How many applications to retain private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { - val ui = provider.getAppUI(key).getOrElse(throw new NoSuchElementException()) + val parts = key.split("/") + require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") + val ui = provider + .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) + .getOrElse(throw new NoSuchElementException(s"no app with key $key")) attachSparkUI(ui) ui } @@ -61,7 +66,7 @@ class HistoryServer( private val appCache = CacheBuilder.newBuilder() .maximumSize(retainedApplications) .removalListener(new RemovalListener[String, SparkUI] { - override def onRemoval(rm: RemovalNotification[String, SparkUI]) = { + override def onRemoval(rm: RemovalNotification[String, SparkUI]): Unit = { detachSparkUI(rm.getValue()) } }) @@ -69,6 +74,8 @@ class HistoryServer( private val loaderServlet = new HttpServlet { protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + // Parse the URI created by getAttemptURI(). It contains an app ID and an optional + // attempt ID (separated by a slash). val parts = Option(req.getPathInfo()).getOrElse("").split("/") if (parts.length < 2) { res.sendError(HttpServletResponse.SC_BAD_REQUEST, @@ -76,18 +83,23 @@ class HistoryServer( return } - val appId = parts(1) + val appKey = + if (parts.length == 3) { + s"${parts(1)}/${parts(2)}" + } else { + parts(1) + } // Note we don't use the UI retrieved from the cache; the cache loader above will register // the app's UI, and all we need to do is redirect the user to the same URI that was // requested, and the proper data should be served at that point. try { - appCache.get(appId) + appCache.get(appKey) res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) } catch { case e: Exception => e.getCause() match { case nsee: NoSuchElementException => - val msg =

Application {appId} not found.
+ val msg =
Application {appKey} not found.
res.setStatus(HttpServletResponse.SC_NOT_FOUND) UIUtils.basicSparkPage(msg, "Not Found").foreach( n => res.getWriter().write(n.toString)) @@ -102,6 +114,10 @@ class HistoryServer( } } + def getSparkUI(appKey: String): Option[SparkUI] = { + Option(appCache.get(appKey)) + } + initialize() /** @@ -112,6 +128,9 @@ class HistoryServer( */ def initialize() { attachPage(new HistoryPage(this)) + + attachHandler(JsonRootResource.getJsonServlet(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) val contextHandler = new ServletContextHandler @@ -149,14 +168,20 @@ class HistoryServer( * * @return List of all known applications. */ - def getApplicationList() = provider.getListing() + def getApplicationList(): Iterable[ApplicationHistoryInfo] = { + provider.getListing() + } + + def getApplicationInfoList: Iterator[ApplicationInfo] = { + getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) + } /** * Returns the provider configuration to show in the listing page. * * @return A map with the provider's configuration. */ - def getProviderConfig() = provider.getConfig() + def getProviderConfig(): Map[String, String] = provider.getConfig() } @@ -194,11 +219,7 @@ object HistoryServer extends Logging { val server = new HistoryServer(conf, provider, securityManager, port) server.bind() - Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") { - override def run() = { - server.stop() - } - }) + Utils.addShutdownHook { () => server.stop() } // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } @@ -217,4 +238,9 @@ object HistoryServer extends Logging { } } + private[history] def getAttemptURI(appId: String, attemptId: Option[String]): String = { + val attemptSuffix = attemptId.map { id => s"/$id" }.getOrElse("") + s"${HistoryServer.UI_PATH_PREFIX}/${appId}${attemptSuffix}" + } + } 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 536aedb6f9fe9..1620e95bea218 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 @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[deploy] class ApplicationInfo( +private[spark] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, @@ -75,9 +75,11 @@ private[deploy] class ApplicationInfo( } } - private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): - ExecutorDesc = { - val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + private[master] def addExecutor( + worker: WorkerInfo, + cores: Int, + useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerExecutorMB) executors(exec.id) = exec coresGranted += cores exec @@ -91,7 +93,7 @@ private[deploy] class ApplicationInfo( } } - private[master] val requestedCores = desc.maxCores.getOrElse(defaultCores) + private val requestedCores = desc.maxCores.getOrElse(defaultCores) private[master] def coresLeft: Int = requestedCores - coresGranted @@ -111,6 +113,10 @@ private[deploy] class ApplicationInfo( endTime = System.currentTimeMillis() } + private[master] def isFinished: Boolean = { + state != ApplicationState.WAITING && state != ApplicationState.RUNNING + } + def duration: Long = { if (endTime != -1) { endTime - startTime diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index d2d30bfd7fcba..f459ed5b3a1a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import akka.serialization.Serialization import org.apache.spark.Logging +import org.apache.spark.util.Utils /** @@ -48,7 +49,7 @@ private[master] class FileSystemPersistenceEngine( new File(dir + File.separator + name).delete() } - override def read[T: ClassTag](prefix: String) = { + override def read[T: ClassTag](prefix: String): Seq[T] = { val files = new File(dir).listFiles().filter(_.getName.startsWith(prefix)) files.map(deserializeFromFile[T]) } @@ -59,9 +60,9 @@ private[master] class FileSystemPersistenceEngine( val serializer = serialization.findSerializerFor(value) val serialized = serializer.toBinary(value) val out = new FileOutputStream(file) - try { + Utils.tryWithSafeFinally { out.write(serialized) - } finally { + } { out.close() } } 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 1b42121c8db05..53e1903a3d125 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 @@ -47,7 +47,7 @@ import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, RpcUtils, SignalLogger, Utils} private[master] class Master( host: String, @@ -62,7 +62,7 @@ private[master] class Master( private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - + private val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) @@ -86,7 +86,7 @@ private[master] class Master( private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling - private val waitingDrivers = new ArrayBuffer[DriverInfo] + private val waitingDrivers = new ArrayBuffer[DriverInfo] private var nextDriverNumber = 0 Utils.checkHost(host, "Expected hostname") @@ -130,7 +130,7 @@ private[master] class Master( private val restServer = if (restServerEnabled) { val port = conf.getInt("spark.master.rest.port", 6066) - Some(new StandaloneRestServer(host, port, self, masterUrl, conf)) + Some(new StandaloneRestServer(host, port, conf, self, masterUrl)) } else { None } @@ -204,7 +204,7 @@ private[master] class Master( self ! RevokedLeadership } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { @@ -254,7 +254,8 @@ private[master] class Master( case RequestSubmitDriver(description) => { if (state != RecoveryState.ALIVE) { - val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state." + val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + + "Can only accept driver submissions in ALIVE state." sender ! SubmitDriverResponse(false, None, msg) } else { logInfo("Driver submitted " + description.command.mainClass) @@ -274,7 +275,8 @@ private[master] class Master( case RequestKillDriver(driverId) => { if (state != RecoveryState.ALIVE) { - val msg = s"Can only kill drivers in ALIVE state. Current state: $state." + val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + + s"Can only kill drivers in ALIVE state." sender ! KillDriverResponse(driverId, success = false, msg) } else { logInfo("Asked to kill driver " + driverId) @@ -305,12 +307,18 @@ private[master] class Master( } case RequestDriverStatus(driverId) => { - (drivers ++ completedDrivers).find(_.id == driverId) match { - case Some(driver) => - sender ! DriverStatusResponse(found = true, Some(driver.state), - driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception) - case None => - sender ! DriverStatusResponse(found = false, None, None, None, None) + if (state != RecoveryState.ALIVE) { + val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + + "Can only request driver status in ALIVE state." + sender ! DriverStatusResponse(found = false, None, None, None, Some(new Exception(msg))) + } else { + (drivers ++ completedDrivers).find(_.id == driverId) match { + case Some(driver) => + sender ! DriverStatusResponse(found = true, Some(driver.state), + driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception) + case None => + sender ! DriverStatusResponse(found = false, None, None, None, None) + } } } @@ -339,7 +347,11 @@ private[master] class Master( if (ExecutorState.isFinished(state)) { // Remove this executor from the worker and app logInfo(s"Removing executor ${exec.fullId} because it is $state") - appInfo.removeExecutor(exec) + // If an application has already finished, preserve its + // state to display its information properly on the UI + if (!appInfo.isFinished) { + appInfo.removeExecutor(exec) + } exec.worker.removeExecutor(exec) val normalExit = exitStatus == Some(0) @@ -428,6 +440,10 @@ private[master] class Master( if (canCompleteRecovery) { completeRecovery() } } + case UnregisterApplication(applicationId) => + logInfo(s"Received unregister request from application $applicationId") + idToApp.get(applicationId).foreach(finishApplication) + case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was logInfo(s"$address got disassociated, removing it.") @@ -516,52 +532,28 @@ private[master] class Master( } /** - * Can an app use the given worker? True if the worker has enough memory and we haven't already - * launched an executor for the app on it (right now the standalone backend doesn't like having - * two executors on the same worker). - */ - private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { - worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) - } - - /** - * Schedule the currently available resources among waiting apps. This method will be called - * every time a new app joins or resource availability changes. + * Schedule executors to be launched on the workers. + * + * There are two modes of launching executors. The first attempts to spread out an application's + * executors on as many workers as possible, while the second does the opposite (i.e. launch them + * on as few workers as possible). The former is usually better for data locality purposes and is + * the default. + * + * The number of cores assigned to each executor is configurable. When this is explicitly set, + * multiple executors from the same application may be launched on the same worker if the worker + * has enough cores and memory. Otherwise, each executor grabs all the cores available on the + * worker by default, in which case only one executor may be launched on each worker. */ - private def schedule() { - if (state != RecoveryState.ALIVE) { return } - - // First schedule drivers, they take strict precedence over applications - // Randomization helps balance drivers - val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val numWorkersAlive = shuffledAliveWorkers.size - var curPos = 0 - - for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers - // We assign workers to each waiting driver in a round-robin fashion. For each driver, we - // start from the last worker that was assigned a driver, and continue onwards until we have - // explored all alive workers. - var launched = false - var numWorkersVisited = 0 - while (numWorkersVisited < numWorkersAlive && !launched) { - val worker = shuffledAliveWorkers(curPos) - numWorkersVisited += 1 - if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { - launchDriver(worker, driver) - waitingDrivers -= driver - launched = true - } - curPos = (curPos + 1) % numWorkersAlive - } - } - + private def startExecutorsOnWorkers(): Unit = { // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. if (spreadOutApps) { - // Try to spread out each app among all the nodes, until it has all its cores + // Try to spread out each app among all the workers, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && + worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) + .sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) @@ -574,32 +566,61 @@ private[master] class Master( pos = (pos + 1) % numUsable } // Now that we've decided how many cores to give on each node, let's actually give them - for (pos <- 0 until numUsable) { - if (assigned(pos) > 0) { - val exec = app.addExecutor(usableWorkers(pos), assigned(pos)) - launchExecutor(usableWorkers(pos), exec) - app.state = ApplicationState.RUNNING - } + for (pos <- 0 until numUsable if assigned(pos) > 0) { + allocateWorkerResourceToExecutors(app, assigned(pos), usableWorkers(pos)) } } } else { - // Pack each app into as few nodes as possible until we've assigned all its cores + // Pack each app into as few workers as possible until we've assigned all its cores for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) { for (app <- waitingApps if app.coresLeft > 0) { - if (canUse(app, worker)) { - val coresToUse = math.min(worker.coresFree, app.coresLeft) - if (coresToUse > 0) { - val exec = app.addExecutor(worker, coresToUse) - launchExecutor(worker, exec) - app.state = ApplicationState.RUNNING - } - } + allocateWorkerResourceToExecutors(app, app.coresLeft, worker) } } } } - private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + /** + * Allocate a worker's resources to one or more executors. + * @param app the info of the application which the executors belong to + * @param coresToAllocate cores on this worker to be allocated to this application + * @param worker the worker info + */ + private def allocateWorkerResourceToExecutors( + app: ApplicationInfo, + coresToAllocate: Int, + worker: WorkerInfo): Unit = { + val memoryPerExecutor = app.desc.memoryPerExecutorMB + val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(coresToAllocate) + var coresLeft = coresToAllocate + while (coresLeft >= coresPerExecutor && worker.memoryFree >= memoryPerExecutor) { + val exec = app.addExecutor(worker, coresPerExecutor) + coresLeft -= coresPerExecutor + launchExecutor(worker, exec) + app.state = ApplicationState.RUNNING + } + } + + /** + * Schedule the currently available resources among waiting apps. This method will be called + * every time a new app joins or resource availability changes. + */ + private def schedule(): Unit = { + if (state != RecoveryState.ALIVE) { return } + // Drivers take strict precedence over executors + val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers + for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { + for (driver <- waitingDrivers) { + if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { + launchDriver(worker, driver) + waitingDrivers -= driver + } + } + } + startExecutorsOnWorkers() + } + + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc): Unit = { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, @@ -733,9 +754,9 @@ private[master] class Master( /** * Rebuild a new SparkUI from the given application's event logs. - * Return whether this is successful. + * Return the UI if successful, else None */ - private def rebuildSparkUI(app: ApplicationInfo): Boolean = { + private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -743,30 +764,30 @@ private[master] class Master( .getOrElse { // Event logging is not enabled for this application app.desc.appUiUrl = notFoundBasePath - return false + return None } - + val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, app.desc.eventLogCodec) + eventLogDir, app.id, None, app.desc.eventLogCodec) val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + + val inProgressExists = fs.exists(new Path(eventLogFilePrefix + EventLoggingListener.IN_PROGRESS)) - + 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.") } - + val (eventLogFile, status) = if (inProgressExists) { (eventLogFilePrefix + EventLoggingListener.IN_PROGRESS, " (in progress)") } else { (eventLogFilePrefix, " (completed)") } - + val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { replayBus.replay(logInput, eventLogFile, maybeTruncated) @@ -777,7 +798,7 @@ private[master] class Master( webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it app.desc.appUiUrl = ui.basePath - true + Some(ui) } catch { case fnf: FileNotFoundException => // Event logging is enabled for this application, but no event logs are found @@ -787,7 +808,7 @@ private[master] class Master( msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - false + None case e: Exception => // Relay exception message to application UI page val title = s"Application history load error (${app.id})" @@ -796,7 +817,7 @@ private[master] class Master( logError(msg, e) msg = URLEncoder.encode(msg, "UTF-8") app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title" - false + None } } @@ -846,8 +867,8 @@ private[master] class Master( } private def removeDriver( - driverId: String, - finalState: DriverState, + driverId: String, + finalState: DriverState, exception: Option[Exception]) { drivers.find(d => d.id == driverId) match { case Some(driver) => @@ -918,7 +939,7 @@ private[deploy] object Master extends Logging { securityManager = securityMgr) val actor = actorSystem.actorOf( Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName) - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) val portsRequest = actor.ask(BoundPortsRequest)(timeout) val portsResponse = Await.result(portsRequest, timeout).asInstanceOf[BoundPortsResponse] (actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index da5060778edeb..a03d460509e03 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -33,7 +33,7 @@ import scala.reflect.ClassTag * The implementation of this trait defines how name-object pairs are stored or retrieved. */ @DeveloperApi -trait PersistenceEngine { +abstract class PersistenceEngine { /** * Defines how the object is serialized and persisted. Implementation will diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 1583bf1f60032..351db8fab2041 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -51,20 +51,27 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial */ private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { + val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") - def createPersistenceEngine() = { + def createPersistenceEngine(): PersistenceEngine = { logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) } - def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) + def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { + new MonarchyLeaderAgent(master) + } } private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) { - def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) - def createLeaderElectionAgent(master: LeaderElectable) = + def createPersistenceEngine(): PersistenceEngine = { + new ZooKeeperPersistenceEngine(conf, serializer) + } + + def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { new ZooKeeperLeaderElectionAgent(master, conf) + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index e94aae93e4495..9b3d48c6edc84 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -104,7 +104,7 @@ private[spark] class WorkerInfo( "http://" + this.publicAddress + ":" + this.webUiPort } - def setState(state: WorkerState.Value) = { + def setState(state: WorkerState.Value): Unit = { this.state = state } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 4823fd7cac0cb..52758d6a7c4be 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -23,6 +23,7 @@ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} +import org.apache.spark.deploy.SparkCuratorUtil private[master] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 1ac6677ad2b6d..80db6d474b5c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -26,6 +26,7 @@ import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkCuratorUtil private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) @@ -46,7 +47,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializat zk.delete().forPath(WORKING_DIR + "/" + name) } - override def read[T: ClassTag](prefix: String) = { + override def read[T: ClassTag](prefix: String): Seq[T] = { val file = zk.getChildren.forPath(WORKING_DIR).filter(_.startsWith(prefix)) file.map(deserializeFromFile[T]).flatten } 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 761aa8f7b1ef6..06e265f99e231 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 @@ -23,10 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import org.json4s.JValue -import org.json4s.JsonAST.JNothing -import org.apache.spark.deploy.{ExecutorState, JsonProtocol} +import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} @@ -37,21 +35,6 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") private val master = parent.masterActorRef private val timeout = parent.timeout - /** Executor details for a particular application */ - override def renderJson(request: HttpServletRequest): JValue = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - if (app == null) { - JNothing - } else { - JsonProtocol.writeApplicationInfo(app) - } - } - /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") @@ -94,7 +77,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • 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 dee2e4a447c6e..756927682cd24 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 @@ -35,10 +35,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout - override def renderJson(request: HttpServletRequest): JValue = { + def getMasterState: MasterStateResponse = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) - JsonProtocol.writeMasterState(state) + Await.result(stateFuture, timeout) + } + + override def renderJson(request: HttpServletRequest): JValue = { + JsonProtocol.writeMasterState(getMasterState) } def handleAppKillRequest(request: HttpServletRequest): Unit = { @@ -68,23 +71,18 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, timeout) + val state = getMasterState val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val activeAppHeaders = Seq("Application ID", "Name", "Cores in Use", - "Cores Requested", "Memory per Node", "Submitted Time", "User", "State", "Duration") + val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per Node", "Submitted Time", + "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse - val activeAppsTable = UIUtils.listingTable(activeAppHeaders, activeAppRow, activeApps) - - val completedAppHeaders = Seq("Application ID", "Name", "Cores Requested", "Memory per Node", - "Submitted Time", "User", "State", "Duration") + val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse - val completedAppsTable = UIUtils.listingTable(completedAppHeaders, completeAppRow, - completedApps) + val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class") @@ -95,7 +93,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { // For now we only show driver information if the user has submitted drivers to the cluster. // This is until we integrate the notion of drivers and applications in the UI. - def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0 + def hasDrivers: Boolean = activeDrivers.length > 0 || completedDrivers.length > 0 val content =
    @@ -191,17 +189,18 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def appRow(app: ApplicationInfo, active: Boolean): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (app.state == ApplicationState.RUNNING || app.state == ApplicationState.WAITING)) { - val killLinkUri = s"app/kill?id=${app.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill application ${app.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill application ${app.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) +
    } - {app.id} @@ -210,18 +209,11 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.desc.name} - { - if (active) { - - {app.coresGranted} - - } - } - {if (app.requestedCores == Int.MaxValue) "*" else app.requestedCores} + {app.coresGranted} - - {Utils.megabytesToString(app.desc.memoryPerSlave)} + + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)} {UIUtils.formatDate(app.submitDate)} {app.desc.user} @@ -230,25 +222,19 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def activeAppRow(app: ApplicationInfo): Seq[Node] = { - appRow(app, active = true) - } - - private def completeAppRow(app: ApplicationInfo): Seq[Node] = { - appRow(app, active = false) - } - private def driverRow(driver: DriverInfo): Seq[Node] = { val killLink = if (parent.killEnabled && (driver.state == DriverState.RUNNING || driver.state == DriverState.SUBMITTED || driver.state == DriverState.RELAUNCHING)) { - val killLinkUri = s"driver/kill?id=${driver.id}&terminate=true" - val confirm = "return window.confirm(" + - s"'Are you sure you want to kill driver ${driver.id} ?');" - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill driver ${driver.id} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" +
    + + + (kill) +
    } {driver.id} {killLink} 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 1b670418ab1ff..eb26e9f99c70b 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,21 +19,25 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.Logging import org.apache.spark.deploy.master.Master +import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils /** * Web UI server for the standalone master. */ private[master] class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging + with UIRoot { val masterActorRef = master.self - val timeout = AkkaUtils.askTimeout(master.conf) + val timeout = RpcUtils.askTimeout(master.conf) val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) + val masterPage = new MasterPage(this) + initialize() /** Initialize all components of the server. */ @@ -43,10 +47,11 @@ class MasterWebUI(val master: Master, requestedPort: Int) attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler( - createRedirectHandler("/app/kill", "/", masterPage.handleAppKillRequest)) - attachHandler( - createRedirectHandler("/driver/kill", "/", masterPage.handleDriverKillRequest)) + attachHandler(JsonRootResource.getJsonServlet(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(). */ @@ -60,6 +65,23 @@ class MasterWebUI(val master: Master, requestedPort: Int) 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/mesos/MesosClusterDispatcher.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala new file mode 100644 index 0000000000000..5d4e5b899dfdc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -0,0 +1,116 @@ +/* + * 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.mesos + +import java.util.concurrent.CountDownLatch + +import org.apache.spark.deploy.mesos.ui.MesosClusterUI +import org.apache.spark.deploy.rest.mesos.MesosRestServer +import org.apache.spark.scheduler.cluster.mesos._ +import org.apache.spark.util.SignalLogger +import org.apache.spark.{Logging, SecurityManager, SparkConf} + +/* + * A dispatcher that is responsible for managing and launching drivers, and is intended to be + * used for Mesos cluster mode. The dispatcher is a long-running process started by the user in + * the cluster independently of Spark applications. + * It contains a [[MesosRestServer]] that listens for requests to submit drivers and a + * [[MesosClusterScheduler]] that processes these requests by negotiating with the Mesos master + * for resources. + * + * A typical new driver lifecycle is the following: + * - Driver submitted via spark-submit talking to the [[MesosRestServer]] + * - [[MesosRestServer]] queues the driver request to [[MesosClusterScheduler]] + * - [[MesosClusterScheduler]] gets resource offers and launches the drivers that are in queue + * + * This dispatcher supports both Mesos fine-grain or coarse-grain mode as the mode is configurable + * per driver launched. + * This class is needed since Mesos doesn't manage frameworks, so the dispatcher acts as + * a daemon to launch drivers as Mesos frameworks upon request. The dispatcher is also started and + * stopped by sbin/start-mesos-dispatcher and sbin/stop-mesos-dispatcher respectively. + */ +private[mesos] class MesosClusterDispatcher( + args: MesosClusterDispatcherArguments, + conf: SparkConf) + extends Logging { + + private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host) + private val recoveryMode = conf.get("spark.mesos.deploy.recoveryMode", "NONE").toUpperCase() + logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) + + private val engineFactory = recoveryMode match { + case "NONE" => new BlackHoleMesosClusterPersistenceEngineFactory + case "ZOOKEEPER" => new ZookeeperMesosClusterPersistenceEngineFactory(conf) + case _ => throw new IllegalArgumentException("Unsupported recovery mode: " + recoveryMode) + } + + private val scheduler = new MesosClusterScheduler(engineFactory, conf) + + private val server = new MesosRestServer(args.host, args.port, conf, scheduler) + private val webUi = new MesosClusterUI( + new SecurityManager(conf), + args.webUiPort, + conf, + publicAddress, + scheduler) + + private val shutdownLatch = new CountDownLatch(1) + + def start(): Unit = { + webUi.bind() + scheduler.frameworkUrl = webUi.activeWebUiUrl + scheduler.start() + server.start() + } + + def awaitShutdown(): Unit = { + shutdownLatch.await() + } + + def stop(): Unit = { + webUi.stop() + server.stop() + scheduler.stop() + shutdownLatch.countDown() + } +} + +private[mesos] object MesosClusterDispatcher extends Logging { + def main(args: Array[String]) { + SignalLogger.register(log) + val conf = new SparkConf + val dispatcherArgs = new MesosClusterDispatcherArguments(args, conf) + conf.setMaster(dispatcherArgs.masterUrl) + conf.setAppName(dispatcherArgs.name) + dispatcherArgs.zookeeperUrl.foreach { z => + conf.set("spark.mesos.deploy.recoveryMode", "ZOOKEEPER") + conf.set("spark.mesos.deploy.zookeeper.url", z) + } + val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf) + dispatcher.start() + val shutdownHook = new Thread() { + override def run() { + logInfo("Shutdown hook is shutting down dispatcher") + dispatcher.stop() + dispatcher.awaitShutdown() + } + } + Runtime.getRuntime.addShutdownHook(shutdownHook) + dispatcher.awaitShutdown() + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala new file mode 100644 index 0000000000000..894cb78d8591a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala @@ -0,0 +1,101 @@ +/* + * 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.mesos + +import org.apache.spark.SparkConf +import org.apache.spark.util.{IntParam, Utils} + + +private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) { + var host = Utils.localHostName() + var port = 7077 + var name = "Spark Cluster" + var webUiPort = 8081 + var masterUrl: String = _ + var zookeeperUrl: Option[String] = None + var propertiesFile: String = _ + + parse(args.toList) + + propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) + + private def parse(args: List[String]): Unit = args match { + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value, "Please use hostname " + value) + host = value + parse(tail) + + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--webui-port" | "-p") :: IntParam(value) :: tail => + webUiPort = value + parse(tail) + + case ("--zk" | "-z") :: value :: tail => + zookeeperUrl = Some(value) + parse(tail) + + case ("--master" | "-m") :: value :: tail => + if (!value.startsWith("mesos://")) { + System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") + System.exit(1) + } + masterUrl = value.stripPrefix("mesos://") + parse(tail) + + case ("--name") :: value :: tail => + name = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--help") :: tail => + printUsageAndExit(0) + + case Nil => { + if (masterUrl == null) { + System.err.println("--master is required") + printUsageAndExit(1) + } + } + + case _ => + printUsageAndExit(1) + } + + private def printUsageAndExit(exitCode: Int): Unit = { + System.err.println( + "Usage: MesosClusterDispatcher [options]\n" + + "\n" + + "Options:\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + + " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + + " --name NAME Framework name to show in Mesos UI\n" + + " -m --master MASTER URI for connecting to Mesos master\n" + + " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + + " Zookeeper for persistence\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.scala new file mode 100644 index 0000000000000..1948226800afe --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosDriverDescription.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.deploy.mesos + +import java.util.Date + +import org.apache.spark.deploy.Command +import org.apache.spark.scheduler.cluster.mesos.MesosClusterRetryState + +/** + * Describes a Spark driver that is submitted from the + * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]], to be launched by + * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. + * @param jarUrl URL to the application jar + * @param mem Amount of memory for the driver + * @param cores Number of cores for the driver + * @param supervise Supervise the driver for long running app + * @param command The command to launch the driver. + * @param schedulerProperties Extra properties to pass the Mesos scheduler + */ +private[spark] class MesosDriverDescription( + val name: String, + val jarUrl: String, + val mem: Int, + val cores: Double, + val supervise: Boolean, + val command: Command, + val schedulerProperties: Map[String, String], + val submissionId: String, + val submissionDate: Date, + val retryState: Option[MesosClusterRetryState] = None) + extends Serializable { + + def copy( + name: String = name, + jarUrl: String = jarUrl, + mem: Int = mem, + cores: Double = cores, + supervise: Boolean = supervise, + command: Command = command, + schedulerProperties: Map[String, String] = schedulerProperties, + submissionId: String = submissionId, + submissionDate: Date = submissionDate, + retryState: Option[MesosClusterRetryState] = retryState): MesosDriverDescription = { + new MesosDriverDescription(name, jarUrl, mem, cores, supervise, command, schedulerProperties, + submissionId, submissionDate, retryState) + } + + override def toString: String = s"MesosDriverDescription (${command.mainClass})" +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala new file mode 100644 index 0000000000000..be8560d10fc62 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala @@ -0,0 +1,180 @@ +/* + * 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.mesos.ui + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.deploy.Command +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.scheduler.cluster.mesos.{MesosClusterSubmissionState, MesosClusterRetryState} +import org.apache.spark.ui.{UIUtils, WebUIPage} + + +private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") { + + override def render(request: HttpServletRequest): Seq[Node] = { + val driverId = request.getParameter("id") + require(driverId != null && driverId.nonEmpty, "Missing id parameter") + + val state = parent.scheduler.getDriverState(driverId) + if (state.isEmpty) { + val content = +
    +

    Cannot find driver {driverId}

    +
    + return UIUtils.basicSparkPage(content, s"Details for Job $driverId") + } + val driverState = state.get + val driverHeaders = Seq("Driver property", "Value") + val schedulerHeaders = Seq("Scheduler property", "Value") + val commandEnvHeaders = Seq("Command environment variable", "Value") + val launchedHeaders = Seq("Launched property", "Value") + val commandHeaders = Seq("Comamnd property", "Value") + val retryHeaders = Seq("Last failed status", "Next retry time", "Retry count") + val driverDescription = Iterable.apply(driverState.description) + val submissionState = Iterable.apply(driverState.submissionState) + val command = Iterable.apply(driverState.description.command) + val schedulerProperties = Iterable.apply(driverState.description.schedulerProperties) + val commandEnv = Iterable.apply(driverState.description.command.environment) + val driverTable = + UIUtils.listingTable(driverHeaders, driverRow, driverDescription) + val commandTable = + UIUtils.listingTable(commandHeaders, commandRow, command) + val commandEnvTable = + UIUtils.listingTable(commandEnvHeaders, propertiesRow, commandEnv) + val schedulerTable = + UIUtils.listingTable(schedulerHeaders, propertiesRow, schedulerProperties) + val launchedTable = + UIUtils.listingTable(launchedHeaders, launchedRow, submissionState) + val retryTable = + UIUtils.listingTable( + retryHeaders, retryRow, Iterable.apply(driverState.description.retryState)) + val content = +

    Driver state information for driver id {driverId}

    + Back to Drivers +
    +
    +

    Driver state: {driverState.state}

    +

    Driver properties

    + {driverTable} +

    Driver command

    + {commandTable} +

    Driver command environment

    + {commandEnvTable} +

    Scheduler properties

    + {schedulerTable} +

    Launched state

    + {launchedTable} +

    Retry state

    + {retryTable} +
    +
    ; + + UIUtils.basicSparkPage(content, s"Details for Job $driverId") + } + + private def launchedRow(submissionState: Option[MesosClusterSubmissionState]): Seq[Node] = { + submissionState.map { state => + + Mesos Slave ID + {state.slaveId.getValue} + + + Mesos Task ID + {state.taskId.getValue} + + + Launch Time + {state.startDate} + + + Finish Time + {state.finishDate.map(_.toString).getOrElse("")} + + + Last Task Status + {state.mesosTaskStatus.map(_.toString).getOrElse("")} + + }.getOrElse(Seq[Node]()) + } + + private def propertiesRow(properties: collection.Map[String, String]): Seq[Node] = { + properties.map { case (k, v) => + + {k}{v} + + }.toSeq + } + + private def commandRow(command: Command): Seq[Node] = { + + Main class{command.mainClass} + + + Arguments{command.arguments.mkString(" ")} + + + Class path entries{command.classPathEntries.mkString(" ")} + + + Java options{command.javaOpts.mkString((" "))} + + + Library path entries{command.libraryPathEntries.mkString((" "))} + + } + + private def driverRow(driver: MesosDriverDescription): Seq[Node] = { + + Name{driver.name} + + + Id{driver.submissionId} + + + Cores{driver.cores} + + + Memory{driver.mem} + + + Submitted{driver.submissionDate} + + + Supervise{driver.supervise} + + } + + private def retryRow(retryState: Option[MesosClusterRetryState]): Seq[Node] = { + retryState.map { state => + + + {state.lastFailureStatus} + + + {state.nextRetry} + + + {state.retries} + + + }.getOrElse(Seq[Node]()) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala new file mode 100644 index 0000000000000..7419fa9699648 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -0,0 +1,117 @@ +/* + * 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.mesos.ui + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.mesos.Protos.TaskStatus +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState +import org.apache.spark.ui.{UIUtils, WebUIPage} + +private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + val state = parent.scheduler.getSchedulerState() + val queuedHeaders = Seq("Driver ID", "Submit Date", "Main Class", "Driver Resources") + val driverHeaders = queuedHeaders ++ + Seq("Start Date", "Mesos Slave ID", "State") + val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++ + Seq("Last Failed Status", "Next Retry Time", "Attempt Count") + val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) + val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) + val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) + val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers) + val content = +

    Mesos Framework ID: {state.frameworkId}

    +
    +
    +

    Queued Drivers:

    + {queuedTable} +

    Launched Drivers:

    + {launchedTable} +

    Finished Drivers:

    + {finishedTable} +

    Supervise drivers waiting for retry:

    + {retryTable} +
    +
    ; + UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") + } + + private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { + val id = submission.submissionId + + {id} + {submission.submissionDate} + {submission.command.mainClass} + cpus: {submission.cores}, mem: {submission.mem} + + } + + private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = { + val id = state.driverDescription.submissionId + + {id} + {state.driverDescription.submissionDate} + {state.driverDescription.command.mainClass} + cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem} + {state.startDate} + {state.slaveId.getValue} + {stateString(state.mesosTaskStatus)} + + } + + private def retryRow(submission: MesosDriverDescription): Seq[Node] = { + val id = submission.submissionId + + {id} + {submission.submissionDate} + {submission.command.mainClass} + {submission.retryState.get.lastFailureStatus} + {submission.retryState.get.nextRetry} + {submission.retryState.get.retries} + + } + + private def stateString(status: Option[TaskStatus]): String = { + if (status.isEmpty) { + return "" + } + val sb = new StringBuilder + val s = status.get + sb.append(s"State: ${s.getState}") + if (status.get.hasMessage) { + sb.append(s", Message: ${s.getMessage}") + } + if (status.get.hasHealthy) { + sb.append(s", Healthy: ${s.getHealthy}") + } + if (status.get.hasSource) { + sb.append(s", Source: ${s.getSource}") + } + if (status.get.hasReason) { + sb.append(s", Reason: ${s.getReason}") + } + if (status.get.hasTimestamp) { + sb.append(s", Time: ${s.getTimestamp}") + } + sb.toString() + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala new file mode 100644 index 0000000000000..3f693545a0349 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterUI.scala @@ -0,0 +1,49 @@ +/* + * 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.mesos.ui + +import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.{SparkUI, WebUI} + +/** + * UI that displays driver results from the [[org.apache.spark.deploy.mesos.MesosClusterDispatcher]] + */ +private[spark] class MesosClusterUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + dispatcherPublicAddress: String, + val scheduler: MesosClusterScheduler) + extends WebUI(securityManager, port, conf) { + + initialize() + + def activeWebUiUrl: String = "http://" + dispatcherPublicAddress + ":" + boundPort + + override def initialize() { + attachPage(new MesosClusterPage(this)) + attachPage(new DriverPage(this)) + attachHandler(createStaticHandler(MesosClusterUI.STATIC_RESOURCE_DIR, "/static")) + } +} + +private object MesosClusterUI { + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR +} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala similarity index 68% rename from core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala rename to core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 420442f7564cc..6078f50518ba4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -18,20 +18,20 @@ package org.apache.spark.deploy.rest import java.io.{DataOutputStream, FileNotFoundException} -import java.net.{HttpURLConnection, SocketException, URL} +import java.net.{ConnectException, HttpURLConnection, SocketException, URL} import javax.servlet.http.HttpServletResponse +import scala.collection.mutable import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException import com.google.common.base.Charsets import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.Utils /** - * A client that submits applications to the standalone Master using a REST protocol. - * This client is intended to communicate with the [[StandaloneRestServer]] and is - * currently used for cluster mode only. + * A client that submits applications to a [[RestSubmissionServer]]. * * In protocol version v1, the REST URL takes the form http://[host:port]/v1/submissions/[action], * where [action] can be one of create, kill, or status. Each type of request is represented in @@ -52,8 +52,16 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} * implementation of this client can use that information to retry using the version specified * by the server. */ -private[deploy] class StandaloneRestClient extends Logging { - import StandaloneRestClient._ +private[spark] class RestSubmissionClient(master: String) extends Logging { + import RestSubmissionClient._ + + private val supportedMasterPrefixes = Seq("spark://", "mesos://") + + private val masters: Array[String] = Utils.parseStandaloneMasterUrls(master) + + // Set of masters that lost contact with us, used to keep track of + // whether there are masters still alive for us to communicate with + private val lostMasters = new mutable.HashSet[String] /** * Submit an application specified by the parameters in the provided request. @@ -61,52 +69,98 @@ private[deploy] class StandaloneRestClient extends Logging { * If the submission was successful, poll the status of the submission and report * it to the user. Otherwise, report the error message provided by the server. */ - private[rest] def createSubmission( - master: String, - request: CreateSubmissionRequest): SubmitRestProtocolResponse = { + def createSubmission(request: CreateSubmissionRequest): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to launch an application in $master.") - validateMaster(master) - val url = getSubmitUrl(master) - val response = postJson(url, request.toJson) - response match { - case s: CreateSubmissionResponse => - reportSubmissionStatus(master, s) - handleRestResponse(s) - case unexpected => - handleUnexpectedRestResponse(unexpected) + var handled: Boolean = false + var response: SubmitRestProtocolResponse = null + for (m <- masters if !handled) { + validateMaster(m) + val url = getSubmitUrl(m) + try { + response = postJson(url, request.toJson) + response match { + case s: CreateSubmissionResponse => + if (s.success) { + reportSubmissionStatus(s) + handleRestResponse(s) + handled = true + } + case unexpected => + handleUnexpectedRestResponse(unexpected) + } + } catch { + case e: SubmitRestConnectionException => + if (handleConnectionException(m)) { + throw new SubmitRestConnectionException("Unable to connect to server", e) + } + } } response } /** Request that the server kill the specified submission. */ - def killSubmission(master: String, submissionId: String): SubmitRestProtocolResponse = { + def killSubmission(submissionId: String): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to kill submission $submissionId in $master.") - validateMaster(master) - val response = post(getKillUrl(master, submissionId)) - response match { - case k: KillSubmissionResponse => handleRestResponse(k) - case unexpected => handleUnexpectedRestResponse(unexpected) + var handled: Boolean = false + var response: SubmitRestProtocolResponse = null + for (m <- masters if !handled) { + validateMaster(m) + val url = getKillUrl(m, submissionId) + try { + response = post(url) + response match { + case k: KillSubmissionResponse => + if (!Utils.responseFromBackup(k.message)) { + handleRestResponse(k) + handled = true + } + case unexpected => + handleUnexpectedRestResponse(unexpected) + } + } catch { + case e: SubmitRestConnectionException => + if (handleConnectionException(m)) { + throw new SubmitRestConnectionException("Unable to connect to server", e) + } + } } response } /** Request the status of a submission from the server. */ def requestSubmissionStatus( - master: String, submissionId: String, quiet: Boolean = false): SubmitRestProtocolResponse = { logInfo(s"Submitting a request for the status of submission $submissionId in $master.") - validateMaster(master) - val response = get(getStatusUrl(master, submissionId)) - response match { - case s: SubmissionStatusResponse => if (!quiet) { handleRestResponse(s) } - case unexpected => handleUnexpectedRestResponse(unexpected) + + var handled: Boolean = false + var response: SubmitRestProtocolResponse = null + for (m <- masters if !handled) { + validateMaster(m) + val url = getStatusUrl(m, submissionId) + try { + response = get(url) + response match { + case s: SubmissionStatusResponse if s.success => + if (!quiet) { + handleRestResponse(s) + } + handled = true + case unexpected => + handleUnexpectedRestResponse(unexpected) + } + } catch { + case e: SubmitRestConnectionException => + if (handleConnectionException(m)) { + throw new SubmitRestConnectionException("Unable to connect to server", e) + } + } } response } /** Construct a message that captures the specified parameters for submitting an application. */ - private[rest] def constructSubmitRequest( + def constructSubmitRequest( appResource: String, mainClass: String, appArgs: Array[String], @@ -147,9 +201,17 @@ private[deploy] class StandaloneRestClient extends Logging { conn.setRequestProperty("Content-Type", "application/json") conn.setRequestProperty("charset", "utf-8") conn.setDoOutput(true) - val out = new DataOutputStream(conn.getOutputStream) - out.write(json.getBytes(Charsets.UTF_8)) - out.close() + try { + val out = new DataOutputStream(conn.getOutputStream) + Utils.tryWithSafeFinally { + out.write(json.getBytes(Charsets.UTF_8)) + } { + out.close() + } + } catch { + case e: ConnectException => + throw new SubmitRestConnectionException("Connect Exception when connect to server", e) + } readResponse(conn) } @@ -187,11 +249,9 @@ private[deploy] class StandaloneRestClient extends Logging { } } catch { case unreachable @ (_: FileNotFoundException | _: SocketException) => - throw new SubmitRestConnectionException( - s"Unable to connect to server ${connection.getURL}", unreachable) + throw new SubmitRestConnectionException("Unable to connect to server", unreachable) case malformed @ (_: JsonProcessingException | _: SubmitRestProtocolException) => - throw new SubmitRestProtocolException( - "Malformed response received from server", malformed) + throw new SubmitRestProtocolException("Malformed response received from server", malformed) } } @@ -215,33 +275,41 @@ private[deploy] class StandaloneRestClient extends Logging { /** Return the base URL for communicating with the server, including the protocol version. */ private def getBaseUrl(master: String): String = { - val masterUrl = master.stripPrefix("spark://").stripSuffix("/") + var masterUrl = master + supportedMasterPrefixes.foreach { prefix => + if (master.startsWith(prefix)) { + masterUrl = master.stripPrefix(prefix) + } + } + masterUrl = masterUrl.stripSuffix("/") s"http://$masterUrl/$PROTOCOL_VERSION/submissions" } /** Throw an exception if this is not standalone mode. */ private def validateMaster(master: String): Unit = { - if (!master.startsWith("spark://")) { - throw new IllegalArgumentException("This REST client is only supported in standalone mode.") + val valid = supportedMasterPrefixes.exists { prefix => master.startsWith(prefix) } + if (!valid) { + throw new IllegalArgumentException( + "This REST client only supports master URLs that start with " + + "one of the following: " + supportedMasterPrefixes.mkString(",")) } } /** Report the status of a newly created submission. */ private def reportSubmissionStatus( - master: String, submitResponse: CreateSubmissionResponse): Unit = { if (submitResponse.success) { val submissionId = submitResponse.submissionId if (submissionId != null) { logInfo(s"Submission successfully created as $submissionId. Polling submission state...") - pollSubmissionStatus(master, submissionId) + pollSubmissionStatus(submissionId) } else { // should never happen logError("Application successfully submitted, but submission ID was not provided!") } } else { val failMessage = Option(submitResponse.message).map { ": " + _ }.getOrElse("") - logError("Application submission failed" + failMessage) + logError(s"Application submission failed$failMessage") } } @@ -249,9 +317,9 @@ private[deploy] class StandaloneRestClient extends Logging { * Poll the status of the specified submission and log it. * This retries up to a fixed number of times before giving up. */ - private def pollSubmissionStatus(master: String, submissionId: String): Unit = { + private def pollSubmissionStatus(submissionId: String): Unit = { (1 to REPORT_DRIVER_STATUS_MAX_TRIES).foreach { _ => - val response = requestSubmissionStatus(master, submissionId, quiet = true) + val response = requestSubmissionStatus(submissionId, quiet = true) val statusResponse = response match { case s: SubmissionStatusResponse => s case _ => return // unexpected type, let upstream caller handle it @@ -289,9 +357,24 @@ private[deploy] class StandaloneRestClient extends Logging { private def handleUnexpectedRestResponse(unexpected: SubmitRestProtocolResponse): Unit = { logError(s"Error: Server responded with message of unexpected type ${unexpected.messageType}.") } + + /** + * When a connection exception is caught, return true if all masters are lost. + * Note that the heuristic used here does not take into account that masters + * can recover during the lifetime of this client. This assumption should be + * harmless because this client currently does not support retrying submission + * on failure yet (SPARK-6443). + */ + private def handleConnectionException(masterUrl: String): Boolean = { + if (!lostMasters.contains(masterUrl)) { + logWarning(s"Unable to connect to server ${masterUrl}.") + lostMasters += masterUrl + } + lostMasters.size >= masters.size + } } -private[rest] object StandaloneRestClient { +private[spark] object RestSubmissionClient { private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -311,15 +394,15 @@ private[rest] object StandaloneRestClient { } val sparkProperties = conf.getAll.toMap val environmentVariables = env.filter { case (k, _) => k.startsWith("SPARK_") } - val client = new StandaloneRestClient + val client = new RestSubmissionClient(master) val submitRequest = client.constructSubmitRequest( appResource, mainClass, appArgs, sparkProperties, environmentVariables) - client.createSubmission(master, submitRequest) + client.createSubmission(submitRequest) } def main(args: Array[String]): Unit = { if (args.size < 2) { - sys.error("Usage: StandaloneRestClient [app resource] [main class] [app args*]") + sys.error("Usage: RestSubmissionClient [app resource] [main class] [app args*]") sys.exit(1) } val appResource = args(0) 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 new file mode 100644 index 0000000000000..2e78d03e5c0cc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -0,0 +1,318 @@ +/* + * 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.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.servlet.{ServletHolder, ServletContextHandler} +import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.Utils + +/** + * A server that responds to requests submitted by the [[RestSubmissionClient]]. + * + * This server responds with different HTTP codes depending on the situation: + * 200 OK - Request was processed successfully + * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type + * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand + * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request + * + * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]] + * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]] + * instead of the one expected by the client. If the construction of this error response itself + * fails, the response will consist of an empty body with a response code that indicates internal + * server error. + */ +private[spark] abstract class RestSubmissionServer( + val host: String, + val requestedPort: Int, + val masterConf: SparkConf) extends Logging { + protected val submitRequestServlet: SubmitRequestServlet + protected val killRequestServlet: KillRequestServlet + protected val statusRequestServlet: StatusRequestServlet + + private var _server: Option[Server] = None + + // A mapping from URL prefixes to servlets that serve them. Exposed for testing. + protected val baseContext = s"/${RestSubmissionServer.PROTOCOL_VERSION}/submissions" + protected lazy val contextToServlet = Map[String, RestServlet]( + s"$baseContext/create/*" -> submitRequestServlet, + s"$baseContext/kill/*" -> killRequestServlet, + s"$baseContext/status/*" -> statusRequestServlet, + "/*" -> new ErrorServlet // default handler + ) + + /** Start the server and return the bound port. */ + def start(): Int = { + val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf) + _server = Some(server) + logInfo(s"Started REST server for submitting applications on port $boundPort") + boundPort + } + + /** + * Map the servlets to their corresponding contexts and attach them to a server. + * 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 mainHandler = new ServletContextHandler + mainHandler.setContextPath("/") + contextToServlet.foreach { case (prefix, servlet) => + mainHandler.addServlet(new ServletHolder(servlet), prefix) + } + server.setHandler(mainHandler) + server.start() + val boundPort = server.getConnectors()(0).getLocalPort + (server, boundPort) + } + + def stop(): Unit = { + _server.foreach(_.stop()) + } +} + +private[rest] object RestSubmissionServer { + val PROTOCOL_VERSION = RestSubmissionClient.PROTOCOL_VERSION + val SC_UNKNOWN_PROTOCOL_VERSION = 468 +} + +/** + * An abstract servlet for handling requests passed to the [[RestSubmissionServer]]. + */ +private[rest] abstract class RestServlet extends HttpServlet with Logging { + + /** + * Serialize the given response message to JSON and send it through the response servlet. + * This validates the response before sending it to ensure it is properly constructed. + */ + protected def sendResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): Unit = { + val message = validateResponse(responseMessage, responseServlet) + responseServlet.setContentType("application/json") + responseServlet.setCharacterEncoding("utf-8") + responseServlet.getWriter.write(message.toJson) + } + + /** + * Return any fields in the client request message that the server does not know about. + * + * The mechanism for this is to reconstruct the JSON on the server side and compare the + * diff between this JSON and the one generated on the client side. Any fields that are + * only in the client JSON are treated as unexpected. + */ + protected def findUnknownFields( + requestJson: String, + requestMessage: SubmitRestProtocolMessage): Array[String] = { + val clientSideJson = parse(requestJson) + val serverSideJson = parse(requestMessage.toJson) + val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson) + unknown match { + case j: JObject => j.obj.map { case (k, _) => k }.toArray + case _ => Array.empty[String] // No difference + } + } + + /** Return a human readable String representation of the exception. */ + protected def formatException(e: Throwable): String = { + val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n") + s"$e\n$stackTraceString" + } + + /** Construct an error message to signal the fact that an exception has been thrown. */ + protected def handleError(message: String): ErrorResponse = { + val e = new ErrorResponse + e.serverSparkVersion = sparkVersion + e.message = message + e + } + + /** + * Parse a submission ID from the relative path, assuming it is the first part of the path. + * For instance, we expect the path to take the form /[submission ID]/maybe/something/else. + * The returned submission ID cannot be empty. If the path is unexpected, return None. + */ + protected def parseSubmissionId(path: String): Option[String] = { + if (path == null || path.isEmpty) { + None + } else { + path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty) + } + } + + /** + * Validate the response to ensure that it is correctly constructed. + * + * If it is, simply return the message as is. Otherwise, return an error response instead + * to propagate the exception back to the client and set the appropriate error code. + */ + private def validateResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + try { + responseMessage.validate() + responseMessage + } catch { + case e: Exception => + responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + handleError("Internal server error: " + formatException(e)) + } + } +} + +/** + * A servlet for handling kill requests passed to the [[RestSubmissionServer]]. + */ +private[rest] abstract class KillRequestServlet extends RestServlet { + + /** + * If a submission ID is specified in the URL, have the Master kill the corresponding + * driver and return an appropriate response to the client. Otherwise, return error. + */ + protected override def doPost( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleKill).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in kill request.") + } + sendResponse(responseMessage, response) + } + + protected def handleKill(submissionId: String): KillSubmissionResponse +} + +/** + * A servlet for handling status requests passed to the [[RestSubmissionServer]]. + */ +private[rest] abstract class StatusRequestServlet extends RestServlet { + + /** + * If a submission ID is specified in the URL, request the status of the corresponding + * driver from the Master and include it in the response. Otherwise, return error. + */ + protected override def doGet( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val submissionId = parseSubmissionId(request.getPathInfo) + val responseMessage = submissionId.map(handleStatus).getOrElse { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Submission ID is missing in status request.") + } + sendResponse(responseMessage, response) + } + + protected def handleStatus(submissionId: String): SubmissionStatusResponse +} + +/** + * A servlet for handling submit requests passed to the [[RestSubmissionServer]]. + */ +private[rest] abstract class SubmitRequestServlet extends RestServlet { + + /** + * Submit an application to the Master with parameters specified in the request. + * + * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON. + * If the request is successfully processed, return an appropriate response to the + * client indicating so. Otherwise, return error instead. + */ + protected override def doPost( + requestServlet: HttpServletRequest, + responseServlet: HttpServletResponse): Unit = { + val responseMessage = + try { + val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString + val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson) + // The response should have already been validated on the client. + // In case this is not true, validate it ourselves to avoid potential NPEs. + requestMessage.validate() + handleSubmit(requestMessageJson, requestMessage, responseServlet) + } catch { + // The client failed to provide a valid JSON, so this is not our fault + case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Malformed request: " + formatException(e)) + } + sendResponse(responseMessage, responseServlet) + } + + protected def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse +} + +/** + * A default servlet that handles error cases that are not captured by other servlets. + */ +private class ErrorServlet extends RestServlet { + private val serverVersion = RestSubmissionServer.PROTOCOL_VERSION + + /** Service a faulty request by returning an appropriate error message to the client. */ + protected override def service( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val path = request.getPathInfo + val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList + var versionMismatch = false + var msg = + parts match { + case Nil => + // http://host:port/ + "Missing protocol version." + case `serverVersion` :: Nil => + // http://host:port/correct-version + "Missing the /submissions prefix." + case `serverVersion` :: "submissions" :: tail => + // http://host:port/correct-version/submissions/* + "Missing an action: please specify one of /create, /kill, or /status." + case unknownVersion :: tail => + // http://host:port/unknown-version/* + versionMismatch = true + s"Unknown protocol version '$unknownVersion'." + case _ => + // never reached + s"Malformed path $path." + } + msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." + val error = handleError(msg) + // If there is a version mismatch, include the highest protocol version that + // this server supports in case the client wants to retry with our version + if (versionMismatch) { + error.highestProtocolVersion = serverVersion + response.setStatus(RestSubmissionServer.SC_UNKNOWN_PROTOCOL_VERSION) + } else { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + } + sendResponse(error, response) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 4f19af59f409f..502b9bb701ccf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -18,26 +18,16 @@ package org.apache.spark.deploy.rest import java.io.File -import java.net.InetSocketAddress -import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} - -import scala.io.Source +import javax.servlet.http.HttpServletResponse import akka.actor.ActorRef -import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler} -import org.eclipse.jetty.util.thread.QueuedThreadPool -import org.json4s._ -import org.json4s.jackson.JsonMethods._ - -import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} -import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} +import org.apache.spark.util.{AkkaUtils, RpcUtils, Utils} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} /** - * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * A server that responds to requests submitted by the [[RestSubmissionClient]]. * This is intended to be embedded in the standalone Master and used in cluster mode only. * * This server responds with different HTTP codes depending on the situation: @@ -54,176 +44,34 @@ import org.apache.spark.deploy.ClientArguments._ * * @param host the address this server should bind to * @param requestedPort the port this server will attempt to bind to + * @param masterConf the conf used by the Master * @param masterActor reference to the Master actor to which requests can be sent * @param masterUrl the URL of the Master new drivers will attempt to connect to - * @param masterConf the conf used by the Master */ private[deploy] class StandaloneRestServer( host: String, requestedPort: Int, + masterConf: SparkConf, masterActor: ActorRef, - masterUrl: String, - masterConf: SparkConf) - extends Logging { - - import StandaloneRestServer._ - - private var _server: Option[Server] = None - - // A mapping from URL prefixes to servlets that serve them. Exposed for testing. - protected val baseContext = s"/$PROTOCOL_VERSION/submissions" - protected val contextToServlet = Map[String, StandaloneRestServlet]( - s"$baseContext/create/*" -> new SubmitRequestServlet(masterActor, masterUrl, masterConf), - s"$baseContext/kill/*" -> new KillRequestServlet(masterActor, masterConf), - s"$baseContext/status/*" -> new StatusRequestServlet(masterActor, masterConf), - "/*" -> new ErrorServlet // default handler - ) - - /** Start the server and return the bound port. */ - def start(): Int = { - val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf) - _server = Some(server) - logInfo(s"Started REST server for submitting applications on port $boundPort") - boundPort - } - - /** - * Map the servlets to their corresponding contexts and attach them to a server. - * 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 mainHandler = new ServletContextHandler - mainHandler.setContextPath("/") - contextToServlet.foreach { case (prefix, servlet) => - mainHandler.addServlet(new ServletHolder(servlet), prefix) - } - server.setHandler(mainHandler) - server.start() - val boundPort = server.getConnectors()(0).getLocalPort - (server, boundPort) - } - - def stop(): Unit = { - _server.foreach(_.stop()) - } -} - -private[rest] object StandaloneRestServer { - val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION - val SC_UNKNOWN_PROTOCOL_VERSION = 468 -} - -/** - * An abstract servlet for handling requests passed to the [[StandaloneRestServer]]. - */ -private[rest] abstract class StandaloneRestServlet extends HttpServlet with Logging { - - /** - * Serialize the given response message to JSON and send it through the response servlet. - * This validates the response before sending it to ensure it is properly constructed. - */ - protected def sendResponse( - responseMessage: SubmitRestProtocolResponse, - responseServlet: HttpServletResponse): Unit = { - val message = validateResponse(responseMessage, responseServlet) - responseServlet.setContentType("application/json") - responseServlet.setCharacterEncoding("utf-8") - responseServlet.getWriter.write(message.toJson) - } - - /** - * Return any fields in the client request message that the server does not know about. - * - * The mechanism for this is to reconstruct the JSON on the server side and compare the - * diff between this JSON and the one generated on the client side. Any fields that are - * only in the client JSON are treated as unexpected. - */ - protected def findUnknownFields( - requestJson: String, - requestMessage: SubmitRestProtocolMessage): Array[String] = { - val clientSideJson = parse(requestJson) - val serverSideJson = parse(requestMessage.toJson) - val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson) - unknown match { - case j: JObject => j.obj.map { case (k, _) => k }.toArray - case _ => Array.empty[String] // No difference - } - } - - /** Return a human readable String representation of the exception. */ - protected def formatException(e: Throwable): String = { - val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n") - s"$e\n$stackTraceString" - } - - /** Construct an error message to signal the fact that an exception has been thrown. */ - protected def handleError(message: String): ErrorResponse = { - val e = new ErrorResponse - e.serverSparkVersion = sparkVersion - e.message = message - e - } - - /** - * Parse a submission ID from the relative path, assuming it is the first part of the path. - * For instance, we expect the path to take the form /[submission ID]/maybe/something/else. - * The returned submission ID cannot be empty. If the path is unexpected, return None. - */ - protected def parseSubmissionId(path: String): Option[String] = { - if (path == null || path.isEmpty) { - None - } else { - path.stripPrefix("/").split("/").headOption.filter(_.nonEmpty) - } - } - - /** - * Validate the response to ensure that it is correctly constructed. - * - * If it is, simply return the message as is. Otherwise, return an error response instead - * to propagate the exception back to the client and set the appropriate error code. - */ - private def validateResponse( - responseMessage: SubmitRestProtocolResponse, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - try { - responseMessage.validate() - responseMessage - } catch { - case e: Exception => - responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) - handleError("Internal server error: " + formatException(e)) - } - } + masterUrl: String) + extends RestSubmissionServer(host, requestedPort, masterConf) { + + protected override val submitRequestServlet = + new StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) + protected override val killRequestServlet = + new StandaloneKillRequestServlet(masterActor, masterConf) + protected override val statusRequestServlet = + new StandaloneStatusRequestServlet(masterActor, masterConf) } /** * A servlet for handling kill requests passed to the [[StandaloneRestServer]]. */ -private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * If a submission ID is specified in the URL, have the Master kill the corresponding - * driver and return an appropriate response to the client. Otherwise, return error. - */ - protected override def doPost( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val submissionId = parseSubmissionId(request.getPathInfo) - val responseMessage = submissionId.map(handleKill).getOrElse { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in kill request.") - } - sendResponse(responseMessage, response) - } +private[rest] class StandaloneKillRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends KillRequestServlet { protected def handleKill(submissionId: String): KillSubmissionResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout) val k = new KillSubmissionResponse @@ -238,26 +86,11 @@ private[rest] class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling status requests passed to the [[StandaloneRestServer]]. */ -private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * If a submission ID is specified in the URL, request the status of the corresponding - * driver from the Master and include it in the response. Otherwise, return error. - */ - protected override def doGet( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val submissionId = parseSubmissionId(request.getPathInfo) - val responseMessage = submissionId.map(handleStatus).getOrElse { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Submission ID is missing in status request.") - } - sendResponse(responseMessage, response) - } +private[rest] class StandaloneStatusRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends StatusRequestServlet { protected def handleStatus(submissionId: String): SubmissionStatusResponse = { - val askTimeout = AkkaUtils.askTimeout(conf) + val askTimeout = RpcUtils.askTimeout(conf) val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout) val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } @@ -276,71 +109,11 @@ private[rest] class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) /** * A servlet for handling submit requests passed to the [[StandaloneRestServer]]. */ -private[rest] class SubmitRequestServlet( +private[rest] class StandaloneSubmitRequestServlet( masterActor: ActorRef, masterUrl: String, conf: SparkConf) - extends StandaloneRestServlet { - - /** - * Submit an application to the Master with parameters specified in the request. - * - * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON. - * If the request is successfully processed, return an appropriate response to the - * client indicating so. Otherwise, return error instead. - */ - protected override def doPost( - requestServlet: HttpServletRequest, - responseServlet: HttpServletResponse): Unit = { - val responseMessage = - try { - val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString - val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson) - // The response should have already been validated on the client. - // In case this is not true, validate it ourselves to avoid potential NPEs. - requestMessage.validate() - handleSubmit(requestMessageJson, requestMessage, responseServlet) - } catch { - // The client failed to provide a valid JSON, so this is not our fault - case e @ (_: JsonProcessingException | _: SubmitRestProtocolException) => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError("Malformed request: " + formatException(e)) - } - sendResponse(responseMessage, responseServlet) - } - - /** - * Handle the submit request and construct an appropriate response to return to the client. - * - * This assumes that the request message is already successfully validated. - * If the request message is not of the expected type, return error to the client. - */ - private def handleSubmit( - requestMessageJson: String, - requestMessage: SubmitRestProtocolMessage, - responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { - requestMessage match { - case submitRequest: CreateSubmissionRequest => - val askTimeout = AkkaUtils.askTimeout(conf) - val driverDescription = buildDriverDescription(submitRequest) - val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( - DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) - val submitResponse = new CreateSubmissionResponse - submitResponse.serverSparkVersion = sparkVersion - submitResponse.message = response.message - submitResponse.success = response.success - submitResponse.submissionId = response.driverId.orNull - val unknownFields = findUnknownFields(requestMessageJson, requestMessage) - if (unknownFields.nonEmpty) { - // If there are fields that the server does not know about, warn the client - submitResponse.unknownFields = unknownFields - } - submitResponse - case unexpected => - responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) - handleError(s"Received message of unexpected type ${unexpected.messageType}.") - } - } + extends SubmitRequestServlet { /** * Build a driver description from the fields specified in the submit request. @@ -389,50 +162,37 @@ private[rest] class SubmitRequestServlet( new DriverDescription( appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) } -} - -/** - * A default servlet that handles error cases that are not captured by other servlets. - */ -private class ErrorServlet extends StandaloneRestServlet { - private val serverVersion = StandaloneRestServer.PROTOCOL_VERSION - /** Service a faulty request by returning an appropriate error message to the client. */ - protected override def service( - request: HttpServletRequest, - response: HttpServletResponse): Unit = { - val path = request.getPathInfo - val parts = path.stripPrefix("/").split("/").filter(_.nonEmpty).toList - var versionMismatch = false - var msg = - parts match { - case Nil => - // http://host:port/ - "Missing protocol version." - case `serverVersion` :: Nil => - // http://host:port/correct-version - "Missing the /submissions prefix." - case `serverVersion` :: "submissions" :: tail => - // http://host:port/correct-version/submissions/* - "Missing an action: please specify one of /create, /kill, or /status." - case unknownVersion :: tail => - // http://host:port/unknown-version/* - versionMismatch = true - s"Unknown protocol version '$unknownVersion'." - case _ => - // never reached - s"Malformed path $path." - } - msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." - val error = handleError(msg) - // If there is a version mismatch, include the highest protocol version that - // this server supports in case the client wants to retry with our version - if (versionMismatch) { - error.highestProtocolVersion = serverVersion - response.setStatus(StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) - } else { - response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + /** + * Handle the submit request and construct an appropriate response to return to the client. + * + * This assumes that the request message is already successfully validated. + * If the request message is not of the expected type, return error to the client. + */ + protected override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case submitRequest: CreateSubmissionRequest => + val askTimeout = RpcUtils.askTimeout(conf) + val driverDescription = buildDriverDescription(submitRequest) + val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( + DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) + val submitResponse = new CreateSubmissionResponse + submitResponse.serverSparkVersion = sparkVersion + submitResponse.message = response.message + submitResponse.success = response.success + submitResponse.submissionId = response.driverId.orNull + val unknownFields = findUnknownFields(requestMessageJson, requestMessage) + if (unknownFields.nonEmpty) { + // If there are fields that the server does not know about, warn the client + submitResponse.unknownFields = unknownFields + } + submitResponse + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") } - sendResponse(error, response) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index d80abdf15fb34..0d50a768942ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -61,7 +61,7 @@ private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { assertProperty[Boolean](key, "boolean", _.toBoolean) private def assertPropertyIsNumeric(key: String): Unit = - assertProperty[Int](key, "numeric", _.toInt) + assertProperty[Double](key, "numeric", _.toDouble) private def assertPropertyIsMemory(key: String): Unit = assertProperty[Int](key, "memory", Utils.memoryStringToMb) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala index 8fde8c142a4c1..0e226ee294cab 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala @@ -35,7 +35,7 @@ private[rest] abstract class SubmitRestProtocolResponse extends SubmitRestProtoc /** * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. */ -private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse { +private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -46,7 +46,7 @@ private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse /** * A response to a kill request in the REST application submission protocol. */ -private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { +private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -58,7 +58,7 @@ private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { /** * A response to a status request in the REST application submission protocol. */ -private[rest] class SubmissionStatusResponse extends SubmitRestProtocolResponse { +private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { var submissionId: String = null var driverState: String = null var workerId: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala new file mode 100644 index 0000000000000..8198296eeb341 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -0,0 +1,158 @@ +/* + * 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.rest.mesos + +import java.io.File +import java.text.SimpleDateFormat +import java.util.Date +import java.util.concurrent.atomic.AtomicLong +import javax.servlet.http.HttpServletResponse + +import org.apache.spark.deploy.Command +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest._ +import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler +import org.apache.spark.util.Utils +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} + + +/** + * A server that responds to requests submitted by the [[RestSubmissionClient]]. + * All requests are forwarded to + * [[org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler]]. + * This is intended to be used in Mesos cluster mode only. + * For more details about the REST submission please refer to [[RestSubmissionServer]] javadocs. + */ +private[spark] class MesosRestServer( + host: String, + requestedPort: Int, + masterConf: SparkConf, + scheduler: MesosClusterScheduler) + extends RestSubmissionServer(host, requestedPort, masterConf) { + + protected override val submitRequestServlet = + new MesosSubmitRequestServlet(scheduler, masterConf) + protected override val killRequestServlet = + new MesosKillRequestServlet(scheduler, masterConf) + protected override val statusRequestServlet = + new MesosStatusRequestServlet(scheduler, masterConf) +} + +private[mesos] class MesosSubmitRequestServlet( + scheduler: MesosClusterScheduler, + conf: SparkConf) + extends SubmitRequestServlet { + + private val DEFAULT_SUPERVISE = false + private val DEFAULT_MEMORY = 512 // mb + private val DEFAULT_CORES = 1.0 + + private val nextDriverNumber = new AtomicLong(0) + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + private def newDriverId(submitDate: Date): String = { + "driver-%s-%04d".format( + createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) + } + + /** + * Build a driver description from the fields specified in the submit request. + * + * This involves constructing a command that launches a mesos framework for the job. + * This does not currently consider fields used by python applications since python + * is not supported in mesos cluster mode yet. + */ + private def buildDriverDescription(request: CreateSubmissionRequest): MesosDriverDescription = { + // Required fields, including the main class because python is not yet supported + val appResource = Option(request.appResource).getOrElse { + throw new SubmitRestMissingFieldException("Application jar is missing.") + } + val mainClass = Option(request.mainClass).getOrElse { + throw new SubmitRestMissingFieldException("Main class is missing.") + } + + // Optional fields + val sparkProperties = request.sparkProperties + val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") + val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") + val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") + val superviseDriver = sparkProperties.get("spark.driver.supervise") + val driverMemory = sparkProperties.get("spark.driver.memory") + val driverCores = sparkProperties.get("spark.driver.cores") + val appArgs = request.appArgs + val environmentVariables = request.environmentVariables + val name = request.sparkProperties.get("spark.app.name").getOrElse(mainClass) + + // Construct driver description + val conf = new SparkConf(false).setAll(sparkProperties) + val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = new Command( + mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) + val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) + val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) + val submitDate = new Date() + val submissionId = newDriverId(submitDate) + + new MesosDriverDescription( + name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, + command, request.sparkProperties, submissionId, submitDate) + } + + protected override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case submitRequest: CreateSubmissionRequest => + val driverDescription = buildDriverDescription(submitRequest) + val s = scheduler.submitDriver(driverDescription) + s.serverSparkVersion = sparkVersion + val unknownFields = findUnknownFields(requestMessageJson, requestMessage) + if (unknownFields.nonEmpty) { + // If there are fields that the server does not know about, warn the client + s.unknownFields = unknownFields + } + s + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } + } +} + +private[mesos] class MesosKillRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) + extends KillRequestServlet { + protected override def handleKill(submissionId: String): KillSubmissionResponse = { + val k = scheduler.killDriver(submissionId) + k.serverSparkVersion = sparkVersion + k + } +} + +private[mesos] class MesosStatusRequestServlet(scheduler: MesosClusterScheduler, conf: SparkConf) + extends StatusRequestServlet { + protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { + val d = scheduler.getDriverStatus(submissionId) + d.serverSparkVersion = sparkVersion + d + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 27a9eabb1ede7..ef7a703bffe67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -24,14 +24,14 @@ import scala.collection.JavaConversions._ import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.util.{Utils, Clock, SystemClock} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -44,7 +44,8 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String) + val workerUrl: String, + val securityManager: SecurityManager) extends Logging { @volatile private var process: Option[Process] = None @@ -56,8 +57,14 @@ private[deploy] class DriverRunner( private var finalExitCode: Option[Int] = None // Decoupled for testing - def setClock(_clock: Clock) = clock = _clock - def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + def setClock(_clock: Clock): Unit = { + clock = _clock + } + + def setSleeper(_sleeper: Sleeper): Unit = { + sleeper = _sleeper + } + private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) @@ -130,12 +137,9 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val jarFileSystem = jarPath.getFileSystem(hadoopConf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) @@ -143,7 +147,14 @@ private[deploy] class DriverRunner( if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destPath, false, hadoopConf) + Utils.fetchFile( + driverDesc.jarUrl, + driverDir, + conf, + securityManager, + hadoopConf, + System.currentTimeMillis(), + useCache = false) } if (!localJarFile.exists()) { // Verify copy succeeded @@ -155,7 +166,7 @@ private[deploy] class DriverRunner( private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) { builder.directory(baseDir) - def initialize(process: Process) = { + def initialize(process: Process): Unit = { // Redirect stdout and stderr to files val stdout = new File(baseDir, "stdout") CommandUtils.redirectStream(process.getInputStream, stdout) @@ -169,8 +180,8 @@ private[deploy] class DriverRunner( runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, - supervise: Boolean) { + def runCommandWithRetry( + command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Unit = { // Time to wait between submission retries. var waitSeconds = 1 // A run of this many seconds resets the exponential back-off. @@ -216,8 +227,8 @@ private[deploy] trait ProcessBuilderLike { } private[deploy] object ProcessBuilderLike { - def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike { - def start() = processBuilder.start() - def command = processBuilder.command() + def apply(processBuilder: ProcessBuilder): ProcessBuilderLike = new ProcessBuilderLike { + override def start(): Process = processBuilder.start() + override def command: Seq[String] = processBuilder.command() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index deef6ef9043c6..d1a12b01e78f7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -19,10 +19,9 @@ package org.apache.spark.deploy.worker import java.io.File -import akka.actor._ - import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, Utils} +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} /** * Utility object for launching driver programs such that they share fate with the Worker process. @@ -39,9 +38,9 @@ object DriverWrapper { */ case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() - val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", + val rpcEnv = RpcEnv.create("Driver", Utils.localHostName(), 0, conf, new SecurityManager(conf)) - actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") + rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) val currentLoader = Thread.currentThread.getContextClassLoader val userJarUrl = new File(userJar).toURI().toURL() @@ -58,7 +57,7 @@ object DriverWrapper { val mainMethod = clazz.getMethod("main", classOf[Array[String]]) mainMethod.invoke(null, extraArgs.toArray[String]) - actorSystem.shutdown() + rpcEnv.shutdown() case _ => System.err.println("Usage: DriverWrapper [options]") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 83e24a7a1f80c..7aa85b732fc87 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -28,6 +28,7 @@ import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.Utils import org.apache.spark.util.logging.FileAppender /** @@ -50,7 +51,7 @@ private[deploy] class ExecutorRunner( val workerUrl: String, conf: SparkConf, val appLocalDirs: Seq[String], - var state: ExecutorState.Value) + @volatile var state: ExecutorState.Value) extends Logging { private val fullId = appId + "/" + execId @@ -61,7 +62,7 @@ private[deploy] class ExecutorRunner( // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. - private var shutdownHook: Thread = null + private var shutdownHook: AnyRef = null private[worker] def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { @@ -69,12 +70,7 @@ private[deploy] class ExecutorRunner( } workerThread.start() // Shutdown hook that kills actors on shutdown. - shutdownHook = new Thread() { - override def run() { - killProcess(Some("Worker shutting down")) - } - } - Runtime.getRuntime.addShutdownHook(shutdownHook) + shutdownHook = Utils.addShutdownHook { () => killProcess(Some("Worker shutting down")) } } /** @@ -106,7 +102,7 @@ private[deploy] class ExecutorRunner( workerThread = null state = ExecutorState.KILLED try { - Runtime.getRuntime.removeShutdownHook(shutdownHook) + Utils.removeShutdownHook(shutdownHook) } catch { case e: IllegalStateException => None } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c1b0a295f9f74..8f3cc54051048 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -34,6 +34,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem @@ -61,7 +62,7 @@ private[worker] class Worker( assert (port > 0) // For worker and executor IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 @@ -85,10 +86,10 @@ private[worker] class Worker( private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = + private val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECS = + private val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) private val testing: Boolean = sys.props.contains("spark.testing") @@ -112,7 +113,7 @@ private[worker] class Worker( } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } - + var workDir: File = null val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] @@ -122,7 +123,7 @@ private[worker] class Worker( val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. - private val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) + private val shuffleService = new ExternalShuffleService(conf, securityMgr) private val publicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") @@ -134,7 +135,7 @@ private[worker] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) - + private var registrationRetryTimer: Option[Cancellable] = None var coresUsed = 0 @@ -275,7 +276,7 @@ private[worker] class Worker( } } - override def receiveWithLogging = { + override def receiveWithLogging: PartialFunction[Any, Unit] = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true @@ -436,7 +437,8 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl) + akkaUrl, + securityMgr) drivers(driverId) = driver driver.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 88f9d880ac209..9678631da9f6f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -105,7 +105,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { if (masters != null) { // Two positional arguments were given printUsageAndExit(1) } - masters = value.stripPrefix("spark://").split(",").map("spark://" + _) + masters = Utils.parseStandaloneMasterUrls(value) parse(tail) case Nil => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 09d866fb0cd90..83fb991891a41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -17,58 +17,63 @@ package org.apache.spark.deploy.worker -import akka.actor.{Actor, Address, AddressFromURIString} -import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, DisassociatedEvent, RemotingLifecycleEvent} - import org.apache.spark.Logging import org.apache.spark.deploy.DeployMessages.SendHeartbeat -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.rpc._ /** * Actor which connects to a worker process and terminates the JVM if the connection is severed. * Provides fate sharing between a worker and its associated child processes. */ -private[spark] class WorkerWatcher(workerUrl: String) - extends Actor with ActorLogReceive with Logging { - - override def preStart() { - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) +private[spark] class WorkerWatcher(override val rpcEnv: RpcEnv, workerUrl: String) + extends RpcEndpoint with Logging { + override def onStart() { logInfo(s"Connecting to worker $workerUrl") - val worker = context.actorSelection(workerUrl) - worker ! SendHeartbeat // need to send a message here to initiate connection + if (!isTesting) { + rpcEnv.asyncSetupEndpointRefByURI(workerUrl) + } } // Used to avoid shutting down JVM during tests + // In the normal case, exitNonZero will call `System.exit(-1)` to shutdown the JVM. In the unit + // test, the user should call `setTesting(true)` so that `exitNonZero` will set `isShutDown` to + // true rather than calling `System.exit`. The user can check `isShutDown` to know if + // `exitNonZero` is called. private[deploy] var isShutDown = false private[deploy] def setTesting(testing: Boolean) = isTesting = testing private var isTesting = false // Lets us filter events only from the worker's actor system - private val expectedHostPort = AddressFromURIString(workerUrl).hostPort - private def isWorker(address: Address) = address.hostPort == expectedHostPort + private val expectedAddress = RpcAddress.fromURIString(workerUrl) + private def isWorker(address: RpcAddress) = expectedAddress == address private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) - override def receiveWithLogging = { - case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => - logInfo(s"Successfully connected to $workerUrl") + override def receive: PartialFunction[Any, Unit] = { + case e => logWarning(s"Received unexpected message: $e") + } - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) - if isWorker(remoteAddress) => - // These logs may not be seen if the worker (and associated pipe) has died - logError(s"Could not initialize connection to worker $workerUrl. Exiting.") - logError(s"Error was: $cause") - exitNonZero() + override def onConnected(remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { + logInfo(s"Successfully connected to $workerUrl") + } + } - case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { // This log message will never be seen logError(s"Lost connection to worker actor $workerUrl. Exiting.") exitNonZero() + } + } - case e: AssociationEvent => - // pass through association events relating to other remote actor systems - - case e => logWarning(s"Received unexpected actor system event: $e") + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + if (isWorker(remoteAddress)) { + // These logs may not be seen if the worker (and associated pipe) has died + logError(s"Could not initialize connection to worker $workerUrl. Exiting.") + logError(s"Error was: $cause") + exitNonZero() + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index de6423beb543e..b3bb5f911dbd7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -25,7 +25,7 @@ import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.worker.ui.WorkerWebUI._ import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils /** * Web UI server for the standalone worker. @@ -38,7 +38,7 @@ class WorkerWebUI( extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { - private[ui] val timeout = AkkaUtils.askTimeout(worker.conf) + private[ui] val timeout = RpcUtils.askTimeout(worker.conf) initialize() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dd19e4947db1e..ed159dec4f998 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -20,40 +20,53 @@ package org.apache.spark.executor import java.net.URL import java.nio.ByteBuffer -import scala.collection.mutable -import scala.concurrent.Await +import org.apache.hadoop.conf.Configuration -import akka.actor.{Actor, ActorSelection, Props} -import akka.pattern.Patterns -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} +import scala.collection.mutable +import scala.util.{Failure, Success} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.rpc._ +import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} +import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.util.{SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( + override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, hostPort: String, cores: Int, userClassPath: Seq[URL], env: SparkEnv) - extends Actor with ActorLogReceive with ExecutorBackend with Logging { + extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") var executor: Executor = null - var driver: ActorSelection = null + @volatile var driver: Option[RpcEndpointRef] = None - override def preStart() { + // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need + // to be changed so that we don't share the serializer instance across threads + private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + + override def onStart() { + import scala.concurrent.ExecutionContext.Implicits.global logInfo("Connecting to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - driver ! RegisterExecutor(executorId, hostPort, cores, extractLogUrls) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => + driver = Some(ref) + ref.ask[RegisteredExecutor.type]( + RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls)) + } onComplete { + case Success(msg) => Utils.tryLogNonFatalError { + Option(self).foreach(_.send(msg)) // msg must be RegisteredExecutor + } + case Failure(e) => logError(s"Cannot register with driver: $driverUrl", e) + } } def extractLogUrls: Map[String, String] = { @@ -62,7 +75,7 @@ private[spark] class CoarseGrainedExecutorBackend( .map(e => (e._1.substring(prefix.length).toLowerCase, e._2)) } - override def receiveWithLogging = { + override def receive: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") val (hostname, _) = Utils.parseHostPort(hostPort) @@ -77,7 +90,6 @@ private[spark] class CoarseGrainedExecutorBackend( logError("Received LaunchTask command but executor was null") System.exit(1) } else { - val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber, @@ -92,23 +104,28 @@ private[spark] class CoarseGrainedExecutorBackend( executor.killTask(taskId, interruptThread) } - case x: DisassociatedEvent => - if (x.remoteAddress == driver.anchorPath.address) { - logError(s"Driver $x disassociated! Shutting down.") - System.exit(1) - } else { - logWarning(s"Received irrelevant DisassociatedEvent $x") - } - case StopExecutor => logInfo("Driver commanded a shutdown") executor.stop() - context.stop(self) - context.system.shutdown() + stop() + rpcEnv.shutdown() + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (driver.exists(_.address == remoteAddress)) { + logError(s"Driver $remoteAddress disassociated! Shutting down.") + System.exit(1) + } else { + logWarning(s"An unknown ($remoteAddress) driver disconnected.") + } } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - driver ! StatusUpdate(executorId, taskId, state, data) + val msg = StatusUpdate(executorId, taskId, state, data) + driver match { + case Some(driverRef) => driverRef.send(msg) + case None => logWarning(s"Drop $msg because has not yet connected to driver") + } } } @@ -132,16 +149,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Bootstrap to fetch the driver's Spark properties. val executorConf = new SparkConf val port = executorConf.getInt("spark.executor.port", 0) - val (fetcher, _) = AkkaUtils.createActorSystem( + val fetcher = RpcEnv.create( "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf)) - val driver = fetcher.actorSelection(driverUrl) - val timeout = AkkaUtils.askTimeout(executorConf) - val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) - val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] ++ + val driver = fetcher.setupEndpointRefByURI(driverUrl) + val props = driver.askWithRetry[Seq[(String, String)]](RetrieveSparkProps) ++ Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() @@ -155,6 +170,12 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { driverConf.set(key, value) } } + if (driverConf.contains("spark.yarn.credentials.file")) { + logInfo("Will periodically update credentials from: " + + driverConf.get("spark.yarn.credentials.file")) + SparkHadoopUtil.get.startExecutorDelegationTokenRenewer(driverConf) + } + val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, isLocal = false) @@ -162,16 +183,15 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val boundPort = env.conf.getInt("spark.executor.port", 0) assert(boundPort != 0) - // Start the CoarseGrainedExecutorBackend actor. + // Start the CoarseGrainedExecutorBackend endpoint. val sparkHostPort = hostname + ":" + boundPort - env.actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, userClassPath, env), - name = "Executor") + env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend( + env.rpcEnv, driverUrl, executorId, sparkHostPort, cores, userClassPath, env)) workerUrl.foreach { url => - env.actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } - env.actorSystem.awaitTermination() + env.rpcEnv.awaitTermination() + SparkHadoopUtil.get.stopExecutorDelegationTokenRenewer() } } 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 bf3135ef081c1..8f916e0502ecb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,19 +21,18 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import akka.actor.Props - import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util._ /** @@ -62,8 +61,6 @@ private[spark] class Executor( private val conf = env.conf - @volatile private var isStopped = false - // No ip or host:port - just hostname Utils.checkHost(executorHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -80,7 +77,7 @@ private[spark] class Executor( } // Start worker thread pool - private val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") + private val threadPool = ThreadUtils.newDaemonCachedThreadPool("Executor task launch worker") private val executorSource = new ExecutorSource(threadPool, executorId) if (!isLocal) { @@ -88,15 +85,12 @@ private[spark] class Executor( env.blockManager.initialize(conf.getAppId) } - // Create an actor for receiving RPCs from the driver - private val executorActor = env.actorSystem.actorOf( - Props(new ExecutorActor(executorId)), "ExecutorActor") + // Create an RpcEndpoint for receiving RPCs from the driver + private val executorEndpoint = env.rpcEnv.setupEndpoint( + ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) // Whether to load classes in user jars before those in Spark jars - private val userClassPathFirst: Boolean = { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) - } + private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager @@ -116,6 +110,9 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + // Executor for the heartbeat task. + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + startDriverHeartbeater() def launchTask( @@ -139,8 +136,9 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - env.actorSystem.stop(executorActor) - isStopped = true + env.rpcEnv.stop(executorEndpoint) + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() if (!isLocal) { env.stop() @@ -181,6 +179,7 @@ private[spark] class Executor( } override def run(): Unit = { + val taskMemoryManager = new TaskMemoryManager(env.executorMemoryManager) val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) val ser = env.closureSerializer.newInstance() @@ -193,6 +192,7 @@ private[spark] class Executor( val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) + task.setTaskMemoryManager(taskMemoryManager) // If this task has been killed before we deserialized it, let's quit now. Otherwise, // continue executing the task. @@ -209,7 +209,21 @@ private[spark] class Executor( // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() - val value = task.run(taskAttemptId = taskId, attemptNumber = attemptNumber) + val value = try { + task.run(taskAttemptId = taskId, attemptNumber = attemptNumber) + } finally { + // Note: this memory freeing logic is duplicated in DAGScheduler.runLocallyWithinThread; + // when changing this, make sure to update both copies. + val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() + if (freedMemory > 0) { + val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" + if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { + throw new SparkException(errMsg) + } else { + logError(errMsg) + } + } + } val taskFinish = System.currentTimeMillis() // If the task has been killed, let's fail it. @@ -223,8 +237,12 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.setExecutorDeserializeTime(taskStart - deserializeStartTime) - m.setExecutorRunTime(taskFinish - taskStart) + // Deserialization happens in two parts: first, we deserialize a Task object, which + // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. + m.setExecutorDeserializeTime( + (taskStart - deserializeStartTime) + task.executorDeserializeTime) + // We need to subtract Task.run()'s deserialization time to avoid double-counting + m.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) m.setJvmGCTime(computeTotalGcTime() - startGCTime) m.setResultSerializationTime(afterSerialization - beforeSerialization) } @@ -391,11 +409,8 @@ private[spark] class Executor( } } - private val timeout = AkkaUtils.lookupTimeout(conf) - private val retryAttempts = AkkaUtils.numRetries(conf) - private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) private val heartbeatReceiverRef = - AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + RpcUtils.makeDriverRef(HeartbeatReceiver.ENDPOINT_NAME, conf, env.rpcEnv) /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { @@ -426,8 +441,7 @@ private[spark] class Executor( val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) try { - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) + val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse](message) if (response.reregisterBlockManager) { logWarning("Told to re-register on heartbeat") env.blockManager.reregister() @@ -438,23 +452,17 @@ private[spark] class Executor( } /** - * Starts a thread to report heartbeat and partial metrics for active tasks to driver. - * This thread stops running when the executor is stopped. + * Schedules a task to report heartbeat and partial metrics for active tasks to driver. */ private def startDriverHeartbeater(): Unit = { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) - val thread = new Thread() { - override def run() { - // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) - while (!isStopped) { - reportHeartBeat() - Thread.sleep(interval) - } - } + val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") + + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) } - thread.setDaemon(true) - thread.setName("driver-heartbeater") - thread.start() + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala index 41925f7e97e84..cf362f8464735 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala @@ -17,10 +17,8 @@ package org.apache.spark.executor -import akka.actor.Actor -import org.apache.spark.Logging - -import org.apache.spark.util.{Utils, ActorLogReceive} +import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} +import org.apache.spark.util.Utils /** * Driver -> Executor message to trigger a thread dump. @@ -28,14 +26,18 @@ import org.apache.spark.util.{Utils, ActorLogReceive} private[spark] case object TriggerThreadDump /** - * Actor that runs inside of executors to enable driver -> executor RPC. + * [[RpcEndpoint]] that runs inside of executors to enable driver -> executor RPC. */ private[spark] -class ExecutorActor(executorId: String) extends Actor with ActorLogReceive with Logging { +class ExecutorEndpoint(override val rpcEnv: RpcEnv, executorId: String) extends RpcEndpoint { - override def receiveWithLogging = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case TriggerThreadDump => - sender ! Utils.getThreadDump() + context.reply(Utils.getThreadDump()) } } + +object ExecutorEndpoint { + val EXECUTOR_ENDPOINT_NAME = "ExecutorEndpoint" +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 52862ae0ca5e4..ea36fb60bd540 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -33,11 +33,11 @@ object ExecutorExitCode { /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 - /** TachyonStore failed to initialize after many attempts. */ - val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 + /** ExternalBlockStore failed to initialize after many attempts. */ + val EXTERNAL_BLOCK_STORE_FAILED_TO_INITIALIZE = 54 - /** TachyonStore failed to create a local temporary directory after many attempts. */ - val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 + /** ExternalBlockStore failed to create a local temporary directory after many attempts. */ + val EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR = 55 def explainExitCode(exitCode: Int): String = { exitCode match { @@ -46,9 +46,11 @@ object ExecutorExitCode { case OOM => "OutOfMemoryError" case DISK_STORE_FAILED_TO_CREATE_DIR => "Failed to create local directory (bad spark.local.dir?)" - case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." - case TACHYON_STORE_FAILED_TO_CREATE_DIR => - "TachyonStore failed to create a local temporary directory." + // TODO: replace external block store with concrete implementation name + case EXTERNAL_BLOCK_STORE_FAILED_TO_INITIALIZE => "ExternalBlockStore failed to initialize." + // TODO: replace external block store with concrete implementation name + case EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR => + "ExternalBlockStore failed to create a local temporary directory." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { 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 07b152651dedf..06152f16ae618 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,13 +17,10 @@ package org.apache.spark.executor -import java.util.concurrent.atomic.AtomicLong - -import org.apache.spark.executor.DataReadMethod.DataReadMethod - import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} /** @@ -44,14 +41,14 @@ class TaskMetrics extends Serializable { * Host's name the task runs on */ private var _hostname: String = _ - def hostname = _hostname + def hostname: String = _hostname private[spark] def setHostname(value: String) = _hostname = value /** * Time taken on the executor to deserialize this task */ private var _executorDeserializeTime: Long = _ - def executorDeserializeTime = _executorDeserializeTime + def executorDeserializeTime: Long = _executorDeserializeTime private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value @@ -59,14 +56,14 @@ class TaskMetrics extends Serializable { * Time the executor spends actually running the task (including fetching shuffle data) */ private var _executorRunTime: Long = _ - def executorRunTime = _executorRunTime + def executorRunTime: Long = _executorRunTime private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value /** * The number of bytes this task transmitted back to the driver as the TaskResult */ private var _resultSize: Long = _ - def resultSize = _resultSize + def resultSize: Long = _resultSize private[spark] def setResultSize(value: Long) = _resultSize = value @@ -74,31 +71,31 @@ class TaskMetrics extends Serializable { * Amount of time the JVM spent in garbage collection while executing this task */ private var _jvmGCTime: Long = _ - def jvmGCTime = _jvmGCTime + def jvmGCTime: Long = _jvmGCTime private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value /** * Amount of time spent serializing the task result */ private var _resultSerializationTime: Long = _ - def resultSerializationTime = _resultSerializationTime + def resultSerializationTime: Long = _resultSerializationTime private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value /** * The number of in-memory bytes spilled by this task */ private var _memoryBytesSpilled: Long = _ - def memoryBytesSpilled = _memoryBytesSpilled - private[spark] def incMemoryBytesSpilled(value: Long) = _memoryBytesSpilled += value - private[spark] def decMemoryBytesSpilled(value: Long) = _memoryBytesSpilled -= value + def memoryBytesSpilled: Long = _memoryBytesSpilled + private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value + private[spark] def decMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled -= value /** * The number of on-disk bytes spilled by this task */ private var _diskBytesSpilled: Long = _ - def diskBytesSpilled = _diskBytesSpilled - def incDiskBytesSpilled(value: Long) = _diskBytesSpilled += value - def decDiskBytesSpilled(value: Long) = _diskBytesSpilled -= value + def diskBytesSpilled: Long = _diskBytesSpilled + def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value + def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read @@ -106,7 +103,7 @@ class TaskMetrics extends Serializable { */ private var _inputMetrics: Option[InputMetrics] = None - def inputMetrics = _inputMetrics + def inputMetrics: Option[InputMetrics] = _inputMetrics /** * This should only be used when recreating TaskMetrics, not when updating input metrics in @@ -128,7 +125,7 @@ class TaskMetrics extends Serializable { */ private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None - def shuffleReadMetrics = _shuffleReadMetrics + def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics /** * This should only be used when recreating TaskMetrics, not when updating read metrics in @@ -177,17 +174,18 @@ class TaskMetrics extends Serializable { * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, * we can store all the different inputMetrics (one per readMethod). */ - private[spark] def getInputMetricsForReadMethod( - readMethod: DataReadMethod): InputMetrics = synchronized { - _inputMetrics match { - case None => - val metrics = new InputMetrics(readMethod) - _inputMetrics = Some(metrics) - metrics - case Some(metrics @ InputMetrics(method)) if method == readMethod => - metrics - case Some(InputMetrics(method)) => - new InputMetrics(readMethod) + private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): InputMetrics = { + synchronized { + _inputMetrics match { + case None => + val metrics = new InputMetrics(readMethod) + _inputMetrics = Some(metrics) + metrics + case Some(metrics @ InputMetrics(method)) if method == readMethod => + metrics + case Some(InputMetrics(method)) => + new InputMetrics(readMethod) + } } } @@ -256,14 +254,14 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { */ private var _bytesRead: Long = _ def bytesRead: Long = _bytesRead - def incBytesRead(bytes: Long) = _bytesRead += bytes + def incBytesRead(bytes: Long): Unit = _bytesRead += bytes /** * Total records read. */ private var _recordsRead: Long = _ def recordsRead: Long = _recordsRead - def incRecordsRead(records: Long) = _recordsRead += records + def incRecordsRead(records: Long): Unit = _recordsRead += records /** * Invoke the bytesReadCallback and mutate bytesRead. @@ -293,15 +291,15 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { * Total bytes written */ private var _bytesWritten: Long = _ - def bytesWritten = _bytesWritten - private[spark] def setBytesWritten(value : Long) = _bytesWritten = value + def bytesWritten: Long = _bytesWritten + private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value /** * Total records written */ private var _recordsWritten: Long = 0L - def recordsWritten = _recordsWritten - private[spark] def setRecordsWritten(value: Long) = _recordsWritten = value + def recordsWritten: Long = _recordsWritten + private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value } /** @@ -314,7 +312,7 @@ class ShuffleReadMetrics extends Serializable { * Number of remote blocks fetched in this shuffle by this task */ private var _remoteBlocksFetched: Int = _ - def remoteBlocksFetched = _remoteBlocksFetched + def remoteBlocksFetched: Int = _remoteBlocksFetched private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value @@ -322,7 +320,7 @@ class ShuffleReadMetrics extends Serializable { * Number of local blocks fetched in this shuffle by this task */ private var _localBlocksFetched: Int = _ - def localBlocksFetched = _localBlocksFetched + def localBlocksFetched: Int = _localBlocksFetched private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value @@ -332,7 +330,7 @@ class ShuffleReadMetrics extends Serializable { * still not finished processing block A, it is not considered to be blocking on block B. */ private var _fetchWaitTime: Long = _ - def fetchWaitTime = _fetchWaitTime + def fetchWaitTime: Long = _fetchWaitTime private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value @@ -340,7 +338,7 @@ class ShuffleReadMetrics extends Serializable { * Total number of remote bytes read from the shuffle by this task */ private var _remoteBytesRead: Long = _ - def remoteBytesRead = _remoteBytesRead + def remoteBytesRead: Long = _remoteBytesRead private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value @@ -348,24 +346,24 @@ class ShuffleReadMetrics extends Serializable { * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ private var _localBytesRead: Long = _ - def localBytesRead = _localBytesRead + def localBytesRead: Long = _localBytesRead private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value /** * Total bytes fetched in the shuffle by this task (both remote and local). */ - def totalBytesRead = _remoteBytesRead + _localBytesRead + def totalBytesRead: Long = _remoteBytesRead + _localBytesRead /** * Number of blocks fetched in this shuffle by this task (remote or local) */ - def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched + def totalBlocksFetched: Int = _remoteBlocksFetched + _localBlocksFetched /** * Total number of records read from the shuffle by this task */ private var _recordsRead: Long = _ - def recordsRead = _recordsRead + def recordsRead: Long = _recordsRead private[spark] def incRecordsRead(value: Long) = _recordsRead += value private[spark] def decRecordsRead(value: Long) = _recordsRead -= value } @@ -380,7 +378,7 @@ class ShuffleWriteMetrics extends Serializable { * Number of bytes written for the shuffle by this task */ @volatile private var _shuffleBytesWritten: Long = _ - def shuffleBytesWritten = _shuffleBytesWritten + def shuffleBytesWritten: Long = _shuffleBytesWritten private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value @@ -388,7 +386,7 @@ class ShuffleWriteMetrics extends Serializable { * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ @volatile private var _shuffleWriteTime: Long = _ - def shuffleWriteTime= _shuffleWriteTime + def shuffleWriteTime: Long = _shuffleWriteTime private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value @@ -396,7 +394,7 @@ class ShuffleWriteMetrics extends Serializable { * Total number of records written to the shuffle by this task */ @volatile private var _shuffleRecordsWritten: Long = _ - def shuffleRecordsWritten = _shuffleRecordsWritten + def shuffleRecordsWritten: Long = _shuffleRecordsWritten private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 593a62b3e3b32..6cda7772f77bc 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -73,16 +73,16 @@ private[spark] abstract class StreamBasedRecordReader[T]( private var key = "" private var value: T = null.asInstanceOf[T] - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} - override def close() = {} + override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {} + override def close(): Unit = {} - override def getProgress = if (processed) 1.0f else 0.0f + override def getProgress: Float = if (processed) 1.0f else 0.0f - override def getCurrentKey = key + override def getCurrentKey: String = key - override def getCurrentValue = value + override def getCurrentValue: T = value - override def nextKeyValue = { + override def nextKeyValue: Boolean = { if (!processed) { val fileIn = new PortableDataStream(split, context, index) value = parseStream(fileIn) @@ -119,7 +119,8 @@ private[spark] class StreamRecordReader( * The format for the PortableDataStream files */ private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { - override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) + : CombineFileRecordReader[String, PortableDataStream] = { new CombineFileRecordReader[String, PortableDataStream]( split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) } @@ -204,7 +205,7 @@ class PortableDataStream( /** * Close the file (if it is currently open) */ - def close() = { + def close(): Unit = { if (isOpen) { try { fileIn.close() diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0709b6d689e86..0756cdb2ed8e6 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -97,7 +97,7 @@ private[spark] object CompressionCodec { /** * :: DeveloperApi :: * LZ4 implementation of [[org.apache.spark.io.CompressionCodec]]. - * Block size can be configured by `spark.io.compression.lz4.block.size`. + * Block size can be configured by `spark.io.compression.lz4.blockSize`. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark @@ -107,7 +107,7 @@ private[spark] object CompressionCodec { class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getInt("spark.io.compression.lz4.block.size", 32768) + val blockSize = conf.getSizeAsBytes("spark.io.compression.lz4.blockSize", "32k").toInt new LZ4BlockOutputStream(s, blockSize) } @@ -137,7 +137,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. - * Block size can be configured by `spark.io.compression.snappy.block.size`. + * Block size can be configured by `spark.io.compression.snappy.blockSize`. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark @@ -153,7 +153,7 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { } override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768) + val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt new SnappyOutputStream(s, blockSize) } diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 21b782edd2a9e..818f7a4c8d422 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -17,9 +17,15 @@ package org.apache.spark.mapred +import java.io.IOException import java.lang.reflect.Modifier -import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{TaskAttemptContext => MapReduceTaskAttemptContext} +import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} + +import org.apache.spark.executor.CommitDeniedException +import org.apache.spark.{Logging, SparkEnv, TaskContext} private[spark] trait SparkHadoopMapRedUtil { @@ -52,7 +58,7 @@ trait SparkHadoopMapRedUtil { jobId: Int, isMap: Boolean, taskId: Int, - attemptId: Int) = { + attemptId: Int): TaskAttemptID = { new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) } @@ -65,3 +71,86 @@ trait SparkHadoopMapRedUtil { } } } + +object SparkHadoopMapRedUtil extends Logging { + /** + * Commits a task output. Before committing the task output, we need to know whether some other + * task attempt might be racing to commit the same output partition. Therefore, coordinate with + * the driver in order to determine whether this attempt can commit (please see SPARK-4879 for + * details). + * + * Output commit coordinator is only contacted when the following two configurations are both set + * to `true`: + * + * - `spark.speculation` + * - `spark.hadoop.outputCommitCoordination.enabled` + */ + def commitTask( + committer: MapReduceOutputCommitter, + mrTaskContext: MapReduceTaskAttemptContext, + jobId: Int, + splitId: Int, + attemptId: Int): Unit = { + + val mrTaskAttemptID = mrTaskContext.getTaskAttemptID + + // Called after we have decided to commit + def performCommit(): Unit = { + try { + committer.commitTask(mrTaskContext) + logInfo(s"$mrTaskAttemptID: Committed") + } catch { + case cause: IOException => + logError(s"Error committing the output of task: $mrTaskAttemptID", cause) + committer.abortTask(mrTaskContext) + throw cause + } + } + + // First, check whether the task's output has already been committed by some other attempt + if (committer.needsTaskCommit(mrTaskContext)) { + val shouldCoordinateWithDriver: Boolean = { + val sparkConf = SparkEnv.get.conf + // We only need to coordinate with the driver if there are multiple concurrent task + // attempts, which should only occur if speculation is enabled + val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false) + // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs + sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) + } + + if (shouldCoordinateWithDriver) { + val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator + val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, attemptId) + + if (canCommit) { + performCommit() + } else { + val message = + s"$mrTaskAttemptID: Not committed because the driver did not authorize commit" + logInfo(message) + // We need to abort the task so that the driver can reschedule new attempts, if necessary + committer.abortTask(mrTaskContext) + throw new CommitDeniedException(message, jobId, splitId, attemptId) + } + } else { + // Speculation is disabled or a user has chosen to manually bypass the commit coordination + performCommit() + } + } else { + // Some other attempt committed the output, so we do nothing and signal success + logInfo(s"No need to commit output of task because needsTaskCommit=false: $mrTaskAttemptID") + } + } + + def commitTask( + committer: MapReduceOutputCommitter, + mrTaskContext: MapReduceTaskAttemptContext, + sparkTaskContext: TaskContext): Unit = { + commitTask( + committer, + mrTaskContext, + sparkTaskContext.stageId(), + sparkTaskContext.partitionId(), + sparkTaskContext.attemptNumber()) + } +} diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 3340673f91156..cfd20392d12f1 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -45,7 +45,7 @@ trait SparkHadoopMapReduceUtil { jobId: Int, isMap: Boolean, taskId: Int, - attemptId: Int) = { + attemptId: Int): TaskAttemptID = { val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID") try { // First, attempt to use the old-style constructor that takes a boolean isMap diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 345db36630fd5..9150ad35712a1 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} @@ -84,7 +85,7 @@ private[spark] class MetricsSystem private ( /** * Get any UI handlers used by this metrics system; can only be called after start(). */ - def getServletHandlers = { + def getServletHandlers: Array[ServletContextHandler] = { require(running, "Can only call getServletHandlers on a running MetricsSystem") metricsServlet.map(_.getHandlers).getOrElse(Array()) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 2f65bc8b46609..0c2e212a33074 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,8 +30,12 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, - securityMgr: SecurityManager) extends Sink { +private[spark] class MetricsServlet( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink { + val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" @@ -45,10 +49,12 @@ private[spark] class MetricsServlet(val property: Properties, val registry: Metr val mapper = new ObjectMapper().registerModule( new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample)) - def getHandlers = Array[ServletContextHandler]( - createServletHandler(servletPath, - new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) - ) + def getHandlers: Array[ServletContextHandler] = { + Array[ServletContextHandler]( + createServletHandler(servletPath, + new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) + ) + } def getMetricsSnapshot(request: HttpServletRequest): String = { mapper.writeValueAsString(registry) diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 0d83d8c425ca4..9fad4e7deacb6 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -18,7 +18,7 @@ package org.apache.spark.metrics.sink private[spark] trait Sink { - def start: Unit - def stop: Unit + def start(): Unit + def stop(): Unit def report(): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 3f0950dae1f24..6181c0ee9fa2b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -24,7 +24,7 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCallback, TransportClientFactory} -import org.apache.spark.network.sasl.{SaslRpcHandler, SaslClientBootstrap} +import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.network.shuffle.protocol.UploadBlock @@ -49,18 +49,18 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage private[this] var appId: String = _ override def init(blockDataManager: BlockDataManager): Unit = { - val (rpcHandler: RpcHandler, bootstrap: Option[TransportClientBootstrap]) = { - val nettyRpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) - if (!authEnabled) { - (nettyRpcHandler, None) - } else { - (new SaslRpcHandler(nettyRpcHandler, securityManager), - Some(new SaslClientBootstrap(transportConf, conf.getAppId, securityManager))) - } + val rpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) + var serverBootstrap: Option[TransportServerBootstrap] = None + var clientBootstrap: Option[TransportClientBootstrap] = None + if (authEnabled) { + serverBootstrap = Some(new SaslServerBootstrap(transportConf, securityManager)) + clientBootstrap = Some(new SaslClientBootstrap(transportConf, conf.getAppId, securityManager, + securityManager.isSaslEncryptionEnabled())) } transportContext = new TransportContext(transportConf, rpcHandler) - clientFactory = transportContext.createClientFactory(bootstrap.toList) - server = transportContext.createServer(conf.getInt("spark.blockManager.port", 0)) + clientFactory = transportContext.createClientFactory(clientBootstrap.toList) + server = transportContext.createServer(conf.getInt("spark.blockManager.port", 0), + serverBootstrap.toList) appId = conf.getAppId logInfo("Server created on " + server.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index a1a2c00ed1542..1ba25aa74aa02 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -32,11 +32,11 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int): BlockMessage = blockMessages(i) - def iterator = blockMessages.iterator + def iterator: Iterator[BlockMessage] = blockMessages.iterator - def length = blockMessages.length + def length: Int = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala index 3b245c5c7a4f3..9a9e22b0c2366 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala @@ -31,9 +31,9 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val initialSize = currentSize() var gotChunkForSendingOnce = false - def size = initialSize + def size: Int = initialSize - def currentSize() = { + def currentSize(): Int = { if (buffers == null || buffers.isEmpty) { 0 } else { @@ -100,11 +100,11 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: buffers.foreach(_.flip) } - def hasAckId() = (ackId != 0) + def hasAckId(): Boolean = ackId != 0 - def isCompletelyReceived() = !buffers(0).hasRemaining + def isCompletelyReceived: Boolean = !buffers(0).hasRemaining - override def toString = { + override def toString: String = { if (hasAckId) { "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" } else { diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index c2d9578be7ebb..6b898bd4bfc1b 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -101,9 +101,11 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, socketRemoteConnectionManagerId } - def key() = channel.keyFor(selector) + def key(): SelectionKey = channel.keyFor(selector) - def getRemoteAddress() = channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + def getRemoteAddress(): InetSocketAddress = { + channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] + } // Returns whether we have to register for further reads or not. def read(): Boolean = { @@ -179,7 +181,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, buffer.get(bytes) bytes.foreach(x => print(x + " ")) buffer.position(curPosition) - print(" (" + bytes.size + ")") + print(" (" + bytes.length + ")") } def printBuffer(buffer: ByteBuffer, position: Int, length: Int) { @@ -280,7 +282,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, /* channel.socket.setSendBufferSize(256 * 1024) */ - override def getRemoteAddress() = address + override def getRemoteAddress(): InetSocketAddress = address val DEFAULT_INTEREST = SelectionKey.OP_READ diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala index 764dc5e5503ed..b3b281ff465f1 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala @@ -18,7 +18,9 @@ package org.apache.spark.network.nio private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString: String = { + connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + } } private[nio] object ConnectionId { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index ee22c6656e69e..497871ed6d5e5 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -36,7 +36,7 @@ import io.netty.util.{Timeout, TimerTask, HashedWheelTimer} import org.apache.spark._ import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} import scala.util.Try import scala.util.control.NonFatal @@ -79,17 +79,18 @@ private[nio] class ConnectionManager( private val selector = SelectorProvider.provider.openSelector() private val ackTimeoutMonitor = - new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) + new HashedWheelTimer(ThreadUtils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) + conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. - // + // // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, // we only query for the minimum value because we are using LinkedBlockingDeque. - // - // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" // parameter is necessary. private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) @@ -101,7 +102,7 @@ private[nio] class ConnectionManager( handlerThreadCount, conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-message-executor")) { + ThreadUtils.namedThreadFactory("handle-message-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -116,7 +117,7 @@ private[nio] class ConnectionManager( ioThreadCount, conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-read-write-executor")) { + ThreadUtils.namedThreadFactory("handle-read-write-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -133,7 +134,7 @@ private[nio] class ConnectionManager( connectThreadCount, conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-connect-executor")) { + ThreadUtils.namedThreadFactory("handle-connect-executor")) { override def afterExecute(r: Runnable, t: Throwable): Unit = { super.afterExecute(r, t) @@ -159,7 +160,7 @@ private[nio] class ConnectionManager( private val registerRequests = new SynchronizedQueue[SendingConnection] implicit val futureExecContext = ExecutionContext.fromExecutor( - Utils.newDaemonCachedThreadPool("Connection manager future execution context")) + ThreadUtils.newDaemonCachedThreadPool("Connection manager future execution context")) @volatile private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message] = null @@ -187,8 +188,9 @@ private[nio] class ConnectionManager( private val writeRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]() + @volatile private var isActive = true private val selectorThread = new Thread("connection-manager-thread") { - override def run() = ConnectionManager.this.run() + override def run(): Unit = ConnectionManager.this.run() } selectorThread.setDaemon(true) // start this thread last, since it invokes run(), which accesses members above @@ -341,7 +343,7 @@ private[nio] class ConnectionManager( def run() { try { - while(!selectorThread.isInterrupted) { + while (isActive) { while (!registerRequests.isEmpty) { val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) @@ -397,7 +399,7 @@ private[nio] class ConnectionManager( } catch { // Explicitly only dealing with CancelledKeyException here since other exceptions // should be dealt with differently. - case e: CancelledKeyException => { + case e: CancelledKeyException => // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -419,8 +421,11 @@ private[nio] class ConnectionManager( } } } - } - 0 + 0 + + case e: ClosedSelectorException => + logDebug("Failed select() as selector is closed.", e) + return } if (selectedKeysCount == 0) { @@ -651,7 +656,7 @@ private[nio] class ConnectionManager( connection.synchronized { if (connection.sparkSaslServer == null) { logDebug("Creating sasl Server") - connection.sparkSaslServer = new SparkSaslServer(conf.getAppId, securityManager) + connection.sparkSaslServer = new SparkSaslServer(conf.getAppId, securityManager, false) } } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) @@ -795,7 +800,7 @@ private[nio] class ConnectionManager( if (!conn.isSaslComplete()) { conn.synchronized { if (conn.sparkSaslClient == null) { - conn.sparkSaslClient = new SparkSaslClient(conf.getAppId, securityManager) + conn.sparkSaslClient = new SparkSaslClient(conf.getAppId, securityManager, false) var firstResponse: Array[Byte] = null try { firstResponse = conn.sparkSaslClient.firstToken() @@ -987,10 +992,11 @@ private[nio] class ConnectionManager( } def stop() { + isActive = false ackTimeoutMonitor.stop() + selector.close() selectorThread.interrupt() selectorThread.join() - selector.close() val connections = connectionsByKey.values connections.foreach(_.close()) if (connectionsByKey.size != 0) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala index cbb37ec5ced1f..1cd13d887c6f6 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala @@ -26,7 +26,7 @@ private[nio] case class ConnectionManagerId(host: String, port: Int) { Utils.checkHost(host) assert (port > 0) - def toSocketAddress() = new InetSocketAddress(host, port) + def toSocketAddress(): InetSocketAddress = new InetSocketAddress(host, port) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala index fb4a979b824c3..85d2fe2bf9c20 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -42,7 +42,9 @@ private[nio] abstract class Message(val typ: Long, val id: Int) { def timeTaken(): String = (finishTime - startTime).toString + " ms" - override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" + override def toString: String = { + this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" + } } @@ -51,7 +53,7 @@ private[nio] object Message { var lastId = 1 - def getNewId() = synchronized { + def getNewId(): Int = synchronized { lastId += 1 if (lastId == 0) { lastId += 1 diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala index 278c5ac356ef2..a4568e849fa13 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala @@ -24,9 +24,9 @@ import scala.collection.mutable.ArrayBuffer private[nio] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { - val size = if (buffer == null) 0 else buffer.remaining + val size: Int = if (buffer == null) 0 else buffer.remaining - lazy val buffers = { + lazy val buffers: ArrayBuffer[ByteBuffer] = { val ab = new ArrayBuffer[ByteBuffer]() ab += header.buffer if (buffer != null) { @@ -35,7 +35,7 @@ class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { ab } - override def toString = { + override def toString: String = { "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala index 6e20f291c5cec..7b3da4bb9d5ee 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala @@ -50,8 +50,10 @@ private[nio] class MessageChunkHeader( flip.asInstanceOf[ByteBuffer] } - override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + + override def toString: String = { + "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + " and sizes " + totalSize + " / " + chunkSize + " bytes, securityNeg: " + securityNeg + } } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index cadd0c7ed19ba..53c4b32c95ab3 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -99,7 +99,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { case None => "(partial: " + initialValue + ")" } } - def getFinalValueInternal() = PartialResult.this.getFinalValueInternal().map(f) + def getFinalValueInternal(): Option[T] = PartialResult.this.getFinalValueInternal().map(f) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 646df283ac069..ec185340c3a2d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -33,7 +33,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi /** * Returns a future for counting the number of elements in the RDD. */ - def countAsync(): FutureAction[Long] = { + def countAsync(): FutureAction[Long] = self.withScope { val totalCount = new AtomicLong self.context.submitJob( self, @@ -45,7 +45,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } result }, - Range(0, self.partitions.size), + Range(0, self.partitions.length), (index: Int, data: Long) => totalCount.addAndGet(data), totalCount.get()) } @@ -53,16 +53,16 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi /** * Returns a future for retrieving all elements of this RDD. */ - def collectAsync(): FutureAction[Seq[T]] = { - val results = new Array[Array[T]](self.partitions.size) - self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size), + def collectAsync(): FutureAction[Seq[T]] = self.withScope { + val results = new Array[Array[T]](self.partitions.length) + self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.length), (index, data) => results(index) = data, results.flatten.toSeq) } /** * Returns a future for retrieving the first num elements of the RDD. */ - def takeAsync(num: Int): FutureAction[Seq[T]] = { + def takeAsync(num: Int): FutureAction[Seq[T]] = self.withScope { val f = new ComplexFutureAction[Seq[T]] f.run { @@ -109,17 +109,17 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi /** * Applies a function f to all elements of this RDD. */ - def foreachAsync(f: T => Unit): FutureAction[Unit] = { + def foreachAsync(f: T => Unit): FutureAction[Unit] = self.withScope { val cleanF = self.context.clean(f) - self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size), + self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.length), (index, data) => Unit, Unit) } /** * Applies a function f to each partition of this RDD. */ - def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = { - self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size), + def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = self.withScope { + self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.length), (index, data) => Unit, Unit) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index fffa1911f5bc2..922030263756b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -31,12 +31,12 @@ private[spark] class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds: Array[BlockId]) extends RDD[T](sc, Nil) { - @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) + @transient lazy val _locations = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) @volatile private var _isValid = true override def getPartitions: Array[Partition] = { assertValid() - (0 until blockIds.size).map(i => { + (0 until blockIds.length).map(i => { new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] }).toArray } @@ -54,7 +54,7 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds override def getPreferredLocations(split: Partition): Seq[String] = { assertValid() - locations_(split.asInstanceOf[BlockRDDPartition].blockId) + _locations(split.asInstanceOf[BlockRDDPartition].blockId) } /** @@ -79,14 +79,14 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds /** Check if this BlockRDD is valid. If not valid, exception is thrown. */ private[spark] def assertValid() { - if (!_isValid) { + if (!isValid) { throw new SparkException( "Attempted to use %s after its blocks have been removed!".format(toString)) } } protected def getBlockIdLocations(): Map[BlockId, Seq[String]] = { - locations_ + _locations } } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 1cbd684224b7c..c1d6971787572 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -53,11 +53,11 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( extends RDD[Pair[T, U]](sc, Nil) with Serializable { - val numPartitionsInRdd2 = rdd2.partitions.size + val numPartitionsInRdd2 = rdd2.partitions.length override def getPartitions: Array[Partition] = { // create the cross product split - val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size) + val array = new Array[Partition](rdd1.partitions.length * rdd2.partitions.length) for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) { val idx = s1.index * numPartitionsInRdd2 + s2.index array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index) @@ -70,7 +70,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( (rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)).distinct } - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[(T, U)] = { val currSplit = split.asInstanceOf[CartesianPartition] for (x <- rdd1.iterator(currSplit.s1, context); y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 1c13e2c372845..0d130dd4c7a60 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -48,7 +49,7 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) if (fs.exists(cpath)) { val dirContents = fs.listStatus(cpath).map(_.getPath) val partitionFiles = dirContents.filter(_.getName.startsWith("part-")).map(_.toString).sorted - val numPart = partitionFiles.size + val numPart = partitionFiles.length if (numPart > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) || ! partitionFiles(numPart-1).endsWith(CheckpointRDD.splitIdToFile(numPart-1)))) { throw new SparkException("Invalid checkpoint directory: " + checkpointPath) @@ -112,8 +113,11 @@ private[spark] object CheckpointRDD extends Logging { } val serializer = env.serializer.newInstance() val serializeStream = serializer.serializeStream(fileOutputStream) - serializeStream.writeAll(iterator) - serializeStream.close() + Utils.tryWithSafeFinally { + serializeStream.writeAll(iterator) + } { + serializeStream.close() + } if (!fs.rename(tempOutputPath, finalOutputPath)) { if (!fs.exists(finalOutputPath)) { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 07398a6fa62f6..658e8c8b89318 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -29,15 +29,16 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} import org.apache.spark.util.Utils import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.ShuffleHandle - -private[spark] sealed trait CoGroupSplitDep extends Serializable +/** The references to rdd and splitIndex are transient because redundant information is stored + * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from + * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the + * task closure. */ private[spark] case class NarrowCoGroupSplitDep( - rdd: RDD[_], - splitIndex: Int, + @transient rdd: RDD[_], + @transient splitIndex: Int, var split: Partition - ) extends CoGroupSplitDep { + ) extends Serializable { @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { @@ -47,9 +48,16 @@ private[spark] case class NarrowCoGroupSplitDep( } } -private[spark] case class ShuffleCoGroupSplitDep(handle: ShuffleHandle) extends CoGroupSplitDep - -private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) +/** + * Stores information about the narrow dependencies used by a CoGroupedRdd. + * + * @param narrowDeps maps to the dependencies variable in the parent RDD: for each one to one + * dependency in dependencies, narrowDeps has a NarrowCoGroupSplitDep (describing + * the partition for that dependency) at the corresponding index. The size of + * narrowDeps should always be equal to the number of parents. + */ +private[spark] class CoGroupPartition( + idx: Int, val narrowDeps: Array[Option[NarrowCoGroupSplitDep]]) extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx @@ -99,15 +107,15 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def getPartitions: Array[Partition] = { val array = new Array[Partition](part.numPartitions) - for (i <- 0 until array.size) { + for (i <- 0 until array.length) { // Each CoGroupPartition will have a dependency per contributing RDD array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) => // Assume each RDD contributed a single dependency, and get it dependencies(j) match { case s: ShuffleDependency[_, _, _] => - new ShuffleCoGroupSplitDep(s.shuffleHandle) + None case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + Some(new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))) } }.toArray) } @@ -120,20 +128,21 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] - val numRdds = split.deps.size + val numRdds = dependencies.length // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] - for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => + for ((dep, depNum) <- dependencies.zipWithIndex) dep match { + case oneToOneDependency: OneToOneDependency[Product2[K, Any]] => + val dependencyPartition = split.narrowDeps(depNum).get.split // Read them from the parent - val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] + val it = oneToOneDependency.rdd.iterator(dependencyPartition, context) rddIterators += ((it, depNum)) - case ShuffleCoGroupSplitDep(handle) => + case shuffleDependency: ShuffleDependency[_, _, _] => // Read map outputs of shuffle val it = SparkEnv.get.shuffleManager - .getReader(handle, split.index, split.index + 1, context) + .getReader(shuffleDependency.shuffleHandle, split.index, split.index + 1, context) .read() rddIterators += ((it, depNum)) } 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 b073eba8a1574..0c1b02c07d09f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -166,7 +166,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: // determines the tradeoff between load-balancing the partitions sizes and their locality // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * prev.partitions.size).toInt + val slack = (balanceSlack * prev.partitions.length).toInt var noLocality = true // if true if no preferredLocations exists for parent RDD @@ -186,7 +186,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: override val isEmpty = !it.hasNext // initializes/resets to start iterating from the beginning - def resetIterator() = { + def resetIterator(): Iterator[(String, Partition)] = { val iterators = (0 to 2).map( x => prev.partitions.iterator.flatMap(p => { if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None @@ -196,10 +196,10 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - def hasNext(): Boolean = { !isEmpty } + override def hasNext: Boolean = { !isEmpty } // return the next preferredLocation of some partition of the RDD - def next(): (String, Partition) = { + override def next(): (String, Partition) = { if (it.hasNext) { it.next() } else { @@ -237,7 +237,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: val rotIt = new LocationIterator(prev) // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext()) { + if (!rotIt.hasNext) { (1 to targetLen).foreach(x => groupArr += PartitionGroup()) return } @@ -343,7 +343,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: private case class PartitionGroup(prefLoc: Option[String] = None) { var arr = mutable.ArrayBuffer[Partition]() - def size = arr.size + def size: Int = arr.size } private object PartitionGroup { diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 03afc289736bb..926bce6f15a2a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -30,47 +30,59 @@ import org.apache.spark.util.StatCounter */ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ - def sum(): Double = { - self.reduce(_ + _) + def sum(): Double = self.withScope { + self.fold(0.0)(_ + _) } /** * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and * count of the RDD's elements in one operation. */ - def stats(): StatCounter = { + def stats(): StatCounter = self.withScope { self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b)) } /** Compute the mean of this RDD's elements. */ - def mean(): Double = stats().mean + def mean(): Double = self.withScope { + stats().mean + } /** Compute the variance of this RDD's elements. */ - def variance(): Double = stats().variance + def variance(): Double = self.withScope { + stats().variance + } /** Compute the standard deviation of this RDD's elements. */ - def stdev(): Double = stats().stdev + def stdev(): Double = self.withScope { + stats().stdev + } /** * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev(): Double = stats().sampleStdev + def sampleStdev(): Double = self.withScope { + stats().sampleStdev + } /** * Compute the sample variance of this RDD's elements (which corrects for bias in * estimating the variance by dividing by N-1 instead of N). */ - def sampleVariance(): Double = stats().sampleVariance + def sampleVariance(): Double = self.withScope { + stats().sampleVariance + } /** * :: Experimental :: * Approximate operation to return the mean within a timeout. */ @Experimental - def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + def meanApprox( + timeout: Long, + confidence: Double = 0.95): PartialResult[BoundedDouble] = self.withScope { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new MeanEvaluator(self.partitions.size, confidence) + val evaluator = new MeanEvaluator(self.partitions.length, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } @@ -79,9 +91,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * Approximate operation to return the sum within a timeout. */ @Experimental - def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + def sumApprox( + timeout: Long, + confidence: Double = 0.95): PartialResult[BoundedDouble] = self.withScope { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new SumEvaluator(self.partitions.size, confidence) + val evaluator = new SumEvaluator(self.partitions.length, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } @@ -93,7 +107,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * If the RDD contains infinity, NaN throws an exception * If the elements in RDD do not vary (max == min) always returns a single bucket. */ - def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { + def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = self.withScope { // Scala's built-in range has issues. See #SI-8782 def customRange(min: Double, max: Double, steps: Int): IndexedSeq[Double] = { val span = max - min @@ -140,7 +154,9 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * the maximum value of the last position and all NaN entries will be counted * in that bucket. */ - def histogram(buckets: Array[Double], evenBuckets: Boolean = false): Array[Long] = { + def histogram( + buckets: Array[Double], + evenBuckets: Boolean = false): Array[Long] = self.withScope { if (buckets.length < 2) { throw new IllegalArgumentException("buckets array must have at least two elements") } @@ -191,25 +207,23 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } } // Determine the bucket function in constant time. Requires that buckets are evenly spaced - def fastBucketFunction(min: Double, increment: Double, count: Int)(e: Double): Option[Int] = { + def fastBucketFunction(min: Double, max: Double, count: Int)(e: Double): Option[Int] = { // If our input is not a number unless the increment is also NaN then we fail fast - if (e.isNaN()) { - return None - } - val bucketNumber = (e - min)/(increment) - // We do this rather than buckets.lengthCompare(bucketNumber) - // because Array[Double] fails to override it (for now). - if (bucketNumber > count || bucketNumber < 0) { + if (e.isNaN || e < min || e > max) { None } else { - Some(bucketNumber.toInt.min(count - 1)) + // Compute ratio of e's distance along range to total range first, for better precision + val bucketNumber = (((e - min) / (max - min)) * count).toInt + // should be less than count, but will equal count if e == max, in which case + // it's part of the last end-range-inclusive bucket, so return count-1 + Some(math.min(bucketNumber, count - 1)) } } // Decide which bucket function to pass to histogramPartition. We decide here - // rather than having a general function so that the decission need only be made + // rather than having a general function so that the decision need only be made // once rather than once per shard val bucketFunction = if (evenBuckets) { - fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _ + fastBucketFunction(buckets.head, buckets.last, buckets.length - 1) _ } else { basicBucketFunction _ } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 486e86ce1bb19..2cefe63d44b20 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -99,7 +99,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp */ @DeveloperApi class HadoopRDD[K, V]( - sc: SparkContext, + @transient sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], initLocalJobConfFuncOpt: Option[JobConf => Unit], inputFormatClass: Class[_ <: InputFormat[K, V]], @@ -108,6 +108,10 @@ class HadoopRDD[K, V]( minPartitions: Int) extends RDD[(K, V)](sc, Nil) with Logging { + if (initLocalJobConfFuncOpt.isDefined) { + sc.clean(initLocalJobConfFuncOpt.get) + } + def this( sc: SparkContext, conf: JobConf, @@ -215,8 +219,7 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics.getInputMetricsForReadMethod(DataReadMethod.Hadoop) // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes @@ -240,7 +243,7 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - override def getNext() = { + override def getNext(): (K, V) = { try { finished = !reader.next(key, value) } catch { @@ -337,11 +340,11 @@ private[spark] object HadoopRDD extends Logging { * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. */ - def getCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.get(key) + def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key) - def containsCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.containsKey(key) + def containsCachedMetadata(key: String): Boolean = SparkEnv.get.hadoopJobMetadata.containsKey(key) - def putCachedMetadata(key: String, value: Any) = + private def putCachedMetadata(key: String, value: Any): Unit = SparkEnv.get.hadoopJobMetadata.put(key, value) /** Add Hadoop configuration specific to a single partition and attempt. */ @@ -371,7 +374,7 @@ private[spark] object HadoopRDD extends Logging { override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[U] = { val partition = split.asInstanceOf[HadoopPartition] val inputSplit = partition.inputSplit.value f(inputSplit, firstParent[T].iterator(split, context)) diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index e2267861e79df..0c28f045e46e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.sql.{Connection, ResultSet} +import java.sql.{PreparedStatement, Connection, ResultSet} import scala.reflect.ClassTag @@ -28,8 +28,9 @@ import org.apache.spark.util.NextIterator import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { - override def index = idx + override def index: Int = idx } + // TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. @@ -70,7 +71,8 @@ class JdbcRDD[T: ClassTag]( }).toArray } - override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + override def compute(thePart: Partition, context: TaskContext): Iterator[T] = new NextIterator[T] + { context.addTaskCompletionListener{ context => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() @@ -88,7 +90,7 @@ class JdbcRDD[T: ClassTag]( stmt.setLong(2, part.upper) val rs = stmt.executeQuery() - override def getNext: T = { + override def getNext(): T = { if (rs.next()) { mapRow(rs) } else { diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 4883fb828814c..a838aac6e8d1a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -31,6 +31,6 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[U] = f(context, split.index, firstParent[T].iterator(split, context)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 7fb94840df99c..2ab967f4bb313 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -238,7 +238,7 @@ private[spark] object NewHadoopRDD { override def getPartitions: Array[Partition] = firstParent[T].partitions - override def compute(split: Partition, context: TaskContext) = { + override def compute(split: Partition, context: TaskContext): Iterator[U] = { val partition = split.asInstanceOf[NewHadoopPartition] val inputSplit = partition.serializableHadoopSplit.value f(inputSplit, firstParent[T].iterator(split, context)) diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 6fdfdb734d1b8..d71bb63000904 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -56,8 +56,8 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * order of the keys). */ // TODO: this currently doesn't work on P other than Tuple2! - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) - : RDD[(K, V)] = + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.length) + : RDD[(K, V)] = self.withScope { val part = new RangePartitioner(numPartitions, self, ascending) new ShuffledRDD[K, V, V](self, part) @@ -71,7 +71,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * This is more efficient than calling `repartition` and then sorting within each partition * because it can push the sorting down into the shuffle machinery. */ - def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = { + def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = self.withScope { new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering) } @@ -81,7 +81,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * performed efficiently by only scanning the partitions that might contain matching elements. * Otherwise, a standard `filter` is applied to all partitions. */ - def filterByRange(lower: K, upper: K): RDD[P] = { + def filterByRange(lower: K, upper: K): RDD[P] = self.withScope { def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper) 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 6b4f097ea9ae5..a6d5d2c94e17f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -29,7 +29,7 @@ import scala.util.DynamicVariable import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} @@ -75,7 +75,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) mergeCombiners: (C, C) => C, partitioner: Partitioner, mapSideCombine: Boolean = true, - serializer: Serializer = null): RDD[(K, C)] = { + serializer: Serializer = null): RDD[(K, C)] = self.withScope { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 if (keyClass.isArray) { if (mapSideCombine) { @@ -108,7 +108,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - numPartitions: Int): RDD[(K, C)] = { + numPartitions: Int): RDD[(K, C)] = self.withScope { combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) } @@ -122,7 +122,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * instead of creating a new U. */ def aggregateByKey[U: ClassTag](zeroValue: U, partitioner: Partitioner)(seqOp: (U, V) => U, - combOp: (U, U) => U): RDD[(K, U)] = { + combOp: (U, U) => U): RDD[(K, U)] = self.withScope { // Serialize the zero value to a byte array so that we can get a new clone of it on each key val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) @@ -131,7 +131,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) - combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) + // We will clean the combiner closure later in `combineByKey` + val cleanedSeqOp = self.context.clean(seqOp) + combineByKey[U]((v: V) => cleanedSeqOp(createZero(), v), cleanedSeqOp, combOp, partitioner) } /** @@ -144,7 +146,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * instead of creating a new U. */ def aggregateByKey[U: ClassTag](zeroValue: U, numPartitions: Int)(seqOp: (U, V) => U, - combOp: (U, U) => U): RDD[(K, U)] = { + combOp: (U, U) => U): RDD[(K, U)] = self.withScope { aggregateByKey(zeroValue, new HashPartitioner(numPartitions))(seqOp, combOp) } @@ -158,7 +160,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * instead of creating a new U. */ def aggregateByKey[U: ClassTag](zeroValue: U)(seqOp: (U, V) => U, - combOp: (U, U) => U): RDD[(K, U)] = { + combOp: (U, U) => U): RDD[(K, U)] = self.withScope { aggregateByKey(zeroValue, defaultPartitioner(self))(seqOp, combOp) } @@ -167,7 +169,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * may be added to the result an arbitrary number of times, and must not change the result * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { + def foldByKey( + zeroValue: V, + partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = self.withScope { // Serialize the zero value to a byte array so that we can get a new clone of it on each key val zeroBuffer = SparkEnv.get.serializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) @@ -177,7 +181,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) lazy val cachedSerializer = SparkEnv.get.serializer.newInstance() val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) + val cleanedFunc = self.context.clean(func) + combineByKey[V]((v: V) => cleanedFunc(createZero(), v), cleanedFunc, cleanedFunc, partitioner) } /** @@ -185,7 +190,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * may be added to the result an arbitrary number of times, and must not change the result * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = self.withScope { foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) } @@ -194,7 +199,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * may be added to the result an arbitrary number of times, and must not change the result * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { + def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = self.withScope { foldByKey(zeroValue, defaultPartitioner(self))(func) } @@ -213,7 +218,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def sampleByKey(withReplacement: Boolean, fractions: Map[K, Double], - seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + seed: Long = Utils.random.nextLong): RDD[(K, V)] = self.withScope { require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") @@ -242,9 +247,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * @return RDD containing the sampled subset */ @Experimental - def sampleByKeyExact(withReplacement: Boolean, + def sampleByKeyExact( + withReplacement: Boolean, fractions: Map[K, Double], - seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + seed: Long = Utils.random.nextLong): RDD[(K, V)] = self.withScope { require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") @@ -261,7 +267,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = { + def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = self.withScope { combineByKey[V]((v: V) => v, func, func, partitioner) } @@ -270,7 +276,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { + def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = self.withScope { reduceByKey(new HashPartitioner(numPartitions), func) } @@ -280,7 +286,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ * parallelism level. */ - def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { + def reduceByKey(func: (V, V) => V): RDD[(K, V)] = self.withScope { reduceByKey(defaultPartitioner(self), func) } @@ -289,7 +295,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * immediately to the master as a Map. This will also perform the merging locally on each mapper * before sending results to a reducer, similarly to a "combiner" in MapReduce. */ - def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { + def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = self.withScope { if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") @@ -317,7 +323,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** Alias for reduceByKeyLocally */ @deprecated("Use reduceByKeyLocally", "1.0.0") - def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func) + def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = self.withScope { + reduceByKeyLocally(func) + } /** * Count the number of elements for each key, collecting the results to a local Map. @@ -327,7 +335,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * To handle very large results, consider using rdd.mapValues(_ => 1L).reduceByKey(_ + _), which * returns an RDD[T, Long] instead of a map. */ - def countByKey(): Map[K, Long] = self.mapValues(_ => 1L).reduceByKey(_ + _).collect().toMap + def countByKey(): Map[K, Long] = self.withScope { + self.mapValues(_ => 1L).reduceByKey(_ + _).collect().toMap + } /** * :: Experimental :: @@ -336,7 +346,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) - : PartialResult[Map[K, BoundedDouble]] = { + : PartialResult[Map[K, BoundedDouble]] = self.withScope { self.map(_._1).countByValueApprox(timeout, confidence) } @@ -360,7 +370,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * @param partitioner Partitioner to use for the resulting RDD. */ @Experimental - def countApproxDistinctByKey(p: Int, sp: Int, partitioner: Partitioner): RDD[(K, Long)] = { + def countApproxDistinctByKey( + p: Int, + sp: Int, + partitioner: Partitioner): RDD[(K, Long)] = self.withScope { require(p >= 4, s"p ($p) must be >= 4") require(sp <= 32, s"sp ($sp) must be <= 32") require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") @@ -392,7 +405,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * It must be greater than 0.000017. * @param partitioner partitioner of the resulting RDD */ - def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { + def countApproxDistinctByKey( + relativeSD: Double, + partitioner: Partitioner): RDD[(K, Long)] = self.withScope { require(relativeSD > 0.000017, s"accuracy ($relativeSD) must be greater than 0.000017") val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt assert(p <= 32) @@ -410,7 +425,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * It must be greater than 0.000017. * @param numPartitions number of partitions of the resulting RDD */ - def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = { + def countApproxDistinctByKey( + relativeSD: Double, + numPartitions: Int): RDD[(K, Long)] = self.withScope { countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions)) } @@ -424,7 +441,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. */ - def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = { + def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = self.withScope { countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) } @@ -441,7 +458,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = self.withScope { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -465,14 +482,14 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. */ - def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = self.withScope { groupByKey(new HashPartitioner(numPartitions)) } /** * Return a copy of the RDD partitioned using the specified partitioner. */ - def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { + def partitionBy(partitioner: Partitioner): RDD[(K, V)] = self.withScope { if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -488,7 +505,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = self.withScope { this.cogroup(other, partitioner).flatMapValues( pair => for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w) ) @@ -500,7 +517,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W]( + other: RDD[(K, W)], + partitioner: Partitioner): RDD[(K, (V, Option[W]))] = self.withScope { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._2.isEmpty) { pair._1.iterator.map(v => (v, None)) @@ -517,7 +536,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * partition the output RDD. */ def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) - : RDD[(K, (Option[V], W))] = { + : RDD[(K, (Option[V], W))] = self.withScope { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._1.isEmpty) { pair._2.iterator.map(w => (None, w)) @@ -536,7 +555,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * in `this` have key k. Uses the given Partitioner to partition the output RDD. */ def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) - : RDD[(K, (Option[V], Option[W]))] = { + : RDD[(K, (Option[V], Option[W]))] = self.withScope { this.cogroup(other, partitioner).flatMapValues { case (vs, Seq()) => vs.iterator.map(v => (Some(v), None)) case (Seq(), ws) => ws.iterator.map(w => (None, Some(w))) @@ -549,7 +568,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * existing partitioner/parallelism level. */ def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) - : RDD[(K, C)] = { + : RDD[(K, C)] = self.withScope { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -563,7 +582,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ - def groupByKey(): RDD[(K, Iterable[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = self.withScope { groupByKey(defaultPartitioner(self)) } @@ -572,7 +591,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = self.withScope { join(other, defaultPartitioner(self, other)) } @@ -581,7 +600,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = self.withScope { join(other, new HashPartitioner(numPartitions)) } @@ -591,7 +610,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = self.withScope { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -601,7 +620,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W]( + other: RDD[(K, W)], + numPartitions: Int): RDD[(K, (V, Option[W]))] = self.withScope { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -611,7 +632,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = self.withScope { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -621,7 +642,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W]( + other: RDD[(K, W)], + numPartitions: Int): RDD[(K, (Option[V], W))] = self.withScope { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -634,7 +657,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ * parallelism level. */ - def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = { + def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = self.withScope { fullOuterJoin(other, defaultPartitioner(self, other)) } @@ -646,7 +669,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. */ - def fullOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = { + def fullOuterJoin[W]( + other: RDD[(K, W)], + numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = self.withScope { fullOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -656,7 +681,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Warning: this doesn't return a multimap (so if you have multiple values to the same key, only * one value per key is preserved in the map returned) */ - def collectAsMap(): Map[K, V] = { + def collectAsMap(): Map[K, V] = self.withScope { val data = self.collect() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) @@ -668,7 +693,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: V => U): RDD[(K, U)] = { + def mapValues[U](f: V => U): RDD[(K, U)] = self.withScope { val cleanF = self.context.clean(f) new MapPartitionsRDD[(K, U), (K, V)](self, (context, pid, iter) => iter.map { case (k, v) => (k, cleanF(v)) }, @@ -679,7 +704,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = self.withScope { val cleanF = self.context.clean(f) new MapPartitionsRDD[(K, U), (K, V)](self, (context, pid, iter) => iter.flatMap { case (k, v) => @@ -697,7 +722,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) other2: RDD[(K, W2)], other3: RDD[(K, W3)], partitioner: Partitioner) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -715,7 +740,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) - : RDD[(K, (Iterable[V], Iterable[W]))] = { + : RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -730,7 +755,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -748,7 +773,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * for that key in `this`, `other1`, `other2` and `other3`. */ def cogroup[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)]) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3)) } @@ -756,7 +781,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { cogroup(other, defaultPartitioner(self, other)) } @@ -765,7 +790,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -773,7 +798,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { + def cogroup[W]( + other: RDD[(K, W)], + numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { cogroup(other, new HashPartitioner(numPartitions)) } @@ -782,7 +809,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { cogroup(other1, other2, new HashPartitioner(numPartitions)) } @@ -795,24 +822,24 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) other2: RDD[(K, W2)], other3: RDD[(K, W3)], numPartitions: Int) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { cogroup(other1, other2, other3, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } /** Alias for cogroup. */ def groupWith[W1, W2, W3](other1: RDD[(K, W1)], other2: RDD[(K, W2)], other3: RDD[(K, W3)]) - : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { cogroup(other1, other2, other3, defaultPartitioner(self, other1, other2, other3)) } @@ -822,22 +849,27 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = - subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) + def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = self.withScope { + subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length))) + } /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = + def subtractByKey[W: ClassTag]( + other: RDD[(K, W)], + numPartitions: Int): RDD[(K, V)] = self.withScope { subtractByKey(other, new HashPartitioner(numPartitions)) + } /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = + def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = self.withScope { new SubtractedRDD[K, V, W](self, other, p) + } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. */ - def lookup(key: K): Seq[V] = { + def lookup(key: K): Seq[V] = self.withScope { self.partitioner match { case Some(p) => val index = p.getPartition(key) @@ -859,7 +891,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class * supporting the key and value types K and V in this RDD. */ - def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { + def saveAsHadoopFile[F <: OutputFormat[K, V]]( + path: String)(implicit fm: ClassTag[F]): Unit = self.withScope { saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -869,7 +902,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * supplied codec. */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( - path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { + path: String, + codec: Class[_ <: CompressionCodec])(implicit fm: ClassTag[F]): Unit = self.withScope { val runtimeClass = fm.runtimeClass saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec) } @@ -878,7 +912,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ - def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]]( + path: String)(implicit fm: ClassTag[F]): Unit = self.withScope { saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -891,8 +926,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: NewOutputFormat[_, _]], - conf: Configuration = self.context.hadoopConfiguration) - { + conf: Configuration = self.context.hadoopConfiguration): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf val job = new NewAPIHadoopJob(hadoopConf) @@ -912,7 +946,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], - codec: Class[_ <: CompressionCodec]) { + codec: Class[_ <: CompressionCodec]): Unit = self.withScope { saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, new JobConf(self.context.hadoopConfiguration), Some(codec)) } @@ -927,7 +961,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration), - codec: Option[Class[_ <: CompressionCodec]] = None) { + codec: Option[Class[_ <: CompressionCodec]] = None): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf hadoopConf.setOutputKeyClass(keyClass) @@ -960,7 +994,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * output paths required (e.g. a table name to write to) in the same way as it would be * configured for a Hadoop MapReduce job. */ - def saveAsNewAPIHadoopDataset(conf: Configuration) { + def saveAsNewAPIHadoopDataset(conf: Configuration): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf val job = new NewAPIHadoopJob(hadoopConf) @@ -995,7 +1029,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] require(writer != null, "Unable to obtain RecordWriter") var recordsWritten = 0L - try { + Utils.tryWithSafeFinally { while (iter.hasNext) { val pair = iter.next() writer.write(pair._1, pair._2) @@ -1004,7 +1038,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) recordsWritten += 1 } - } finally { + } { writer.close(hadoopContext) } committer.commitTask(hadoopContext) @@ -1027,7 +1061,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop * MapReduce job. */ - def saveAsHadoopDataset(conf: JobConf) { + def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf val wrappedConf = new SerializableWritable(hadoopConf) @@ -1068,7 +1102,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() var recordsWritten = 0L - try { + + Utils.tryWithSafeFinally { while (iter.hasNext) { val record = iter.next() writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) @@ -1077,7 +1112,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) recordsWritten += 1 } - } finally { + } { writer.close() } writer.commit() diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index f12d0cffaba34..e2394e28f8d26 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -98,7 +98,7 @@ private[spark] class ParallelCollectionRDD[T: ClassTag]( slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray } - override def compute(s: Partition, context: TaskContext) = { + override def compute(s: Partition, context: TaskContext): Iterator[T] = { new InterruptibleIterator(context, s.asInstanceOf[ParallelCollectionPartition[T]].iterator) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index f781a8d776f2a..a00f4c1cdff91 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -40,7 +40,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF .filter(s => partitionFilterFunc(s.index)).zipWithIndex .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } - override def getParents(partitionId: Int) = { + override def getParents(partitionId: Int): List[Int] = { List(partitions(partitionId).asInstanceOf[PartitionPruningRDDPartition].parentSplit.index) } } @@ -59,8 +59,10 @@ class PartitionPruningRDD[T: ClassTag]( @transient partitionFilterFunc: Int => Boolean) extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) { - override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator( - split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context) + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + firstParent[T].iterator( + split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context) + } override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions @@ -74,7 +76,7 @@ object PartitionPruningRDD { * Create a PartitionPruningRDD. This function can be used to create the PartitionPruningRDD * when its type T is not known at compile time. */ - def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = { + def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean): PartitionPruningRDD[T] = { new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 92b0641d0fb6e..7598ff617b399 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -60,6 +60,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( var rdds: Seq[RDD[T]] ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { require(rdds.length > 0) + require(rdds.forall(_.partitioner.isDefined)) require(rdds.flatMap(_.partitioner).toSet.size == 1, "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner)) 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 ed79032893d33..dc60d48927624 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -149,10 +149,10 @@ private[spark] class PipedRDD[T: ClassTag]( }.start() // Return an iterator that read lines from the process's stdout - val lines = Source.fromInputStream(proc.getInputStream).getLines + val lines = Source.fromInputStream(proc.getInputStream).getLines() new Iterator[String] { - def next() = lines.next() - def hasNext = { + def next(): String = lines.next() + def hasNext: Boolean = { if (lines.hasNext) { true } else { @@ -162,7 +162,7 @@ private[spark] class PipedRDD[T: ClassTag]( } // cleanup task working directory if used - if (workInTaskDirectory == true) { + if (workInTaskDirectory) { scala.util.control.Exception.ignoring(classOf[IOException]) { Utils.deleteRecursively(new File(taskDirectory)) } 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 a4c74ed03e330..b3b60578c92e8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -25,7 +25,7 @@ import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus -import org.apache.hadoop.io.{Writable, BytesWritable, NullWritable, Text} +import org.apache.hadoop.io.{BytesWritable, NullWritable, Text} import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.TextOutputFormat @@ -186,7 +186,7 @@ abstract class RDD[T: ClassTag]( } /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ - def getStorageLevel = storageLevel + def getStorageLevel: StorageLevel = storageLevel // Our dependencies and partitions will be gotten by calling subclass's methods below, and will // be overwritten when we're checkpointed @@ -277,12 +277,20 @@ abstract class RDD[T: ClassTag]( if (isCheckpointed) firstParent[T].iterator(split, context) else compute(split, context) } + /** + * Execute a block of code in a scope such that all new RDDs created in this body will + * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}. + * + * Note: Return statements are NOT allowed in the given body. + */ + private[spark] def withScope[U](body: => U): U = RDDOperationScope.withScope[U](sc)(body) + // Transformations (return a new RDD) /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[U: ClassTag](f: T => U): RDD[U] = { + def map[U: ClassTag](f: T => U): RDD[U] = withScope { val cleanF = sc.clean(f) new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.map(cleanF)) } @@ -291,7 +299,7 @@ abstract class RDD[T: ClassTag]( * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = { + def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = withScope { val cleanF = sc.clean(f) new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.flatMap(cleanF)) } @@ -299,7 +307,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: T => Boolean): RDD[T] = { + def filter(f: T => Boolean): RDD[T] = withScope { val cleanF = sc.clean(f) new MapPartitionsRDD[T, T]( this, @@ -310,13 +318,16 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = + def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope { map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) + } /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(): RDD[T] = distinct(partitions.size) + def distinct(): RDD[T] = withScope { + distinct(partitions.length) + } /** * Return a new RDD that has exactly numPartitions partitions. @@ -327,7 +338,7 @@ abstract class RDD[T: ClassTag]( * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, * which can avoid performing a shuffle. */ - def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = { + def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope { coalesce(numPartitions, shuffle = true) } @@ -352,7 +363,7 @@ abstract class RDD[T: ClassTag]( * data distributed using a hash partitioner. */ def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) - : RDD[T] = { + : RDD[T] = withScope { if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ val distributePartition = (index: Int, items: Iterator[T]) => { @@ -377,16 +388,17 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. - * + * * @param withReplacement can elements be sampled multiple times (replaced when sampled out) * @param fraction expected size of the sample as a fraction of this RDD's size * without replacement: probability that each element is chosen; fraction must be [0, 1] * with replacement: expected number of times each element is chosen; fraction must be >= 0 * @param seed seed for the random number generator */ - def sample(withReplacement: Boolean, + def sample( + withReplacement: Boolean, fraction: Double, - seed: Long = Utils.random.nextLong): RDD[T] = { + seed: Long = Utils.random.nextLong): RDD[T] = withScope { require(fraction >= 0.0, "Negative fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) @@ -403,15 +415,32 @@ abstract class RDD[T: ClassTag]( * * @return split RDDs in an array */ - def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = { + def randomSplit( + weights: Array[Double], + seed: Long = Utils.random.nextLong): Array[RDD[T]] = withScope { val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new PartitionwiseSampledRDD[T, T]( - this, new BernoulliCellSampler[T](x(0), x(1)), true, seed) + randomSampleWithRange(x(0), x(1), seed) }.toArray } + /** + * Internal method exposed for Random Splits in DataFrames. Samples an RDD given a probability + * range. + * @param lb lower bound to use for the Bernoulli sampler + * @param ub upper bound to use for the Bernoulli sampler + * @param seed the seed for the Random number generator + * @return A random sub-sample of the RDD without replacement. + */ + private[spark] def randomSampleWithRange(lb: Double, ub: Double, seed: Long): RDD[T] = { + this.mapPartitionsWithIndex { case (index, partition) => + val sampler = new BernoulliCellSampler[T](lb, ub) + sampler.setSeed(seed + index) + sampler.sample(partition) + } + } + /** * Return a fixed-size sampled subset of this RDD in an array * @@ -420,7 +449,9 @@ abstract class RDD[T: ClassTag]( * @param seed seed for the random number generator * @return sample of specified size in an array */ - def takeSample(withReplacement: Boolean, + // TODO: rewrite this without return statements so we can wrap it in a scope + def takeSample( + withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = { val numStDev = 10.0 @@ -468,7 +499,7 @@ abstract class RDD[T: ClassTag]( * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). */ - def union(other: RDD[T]): RDD[T] = { + def union(other: RDD[T]): RDD[T] = withScope { if (partitioner.isDefined && other.partitioner == partitioner) { new PartitionerAwareUnionRDD(sc, Array(this, other)) } else { @@ -480,7 +511,9 @@ abstract class RDD[T: ClassTag]( * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). */ - def ++(other: RDD[T]): RDD[T] = this.union(other) + def ++(other: RDD[T]): RDD[T] = withScope { + this.union(other) + } /** * Return this RDD sorted by the given key function. @@ -488,11 +521,12 @@ abstract class RDD[T: ClassTag]( def sortBy[K]( f: (T) => K, ascending: Boolean = true, - numPartitions: Int = this.partitions.size) - (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = + numPartitions: Int = this.partitions.length) + (implicit ord: Ordering[K], ctag: ClassTag[K]): RDD[T] = withScope { this.keyBy[K](f) .sortByKey(ascending, numPartitions) .values + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -500,7 +534,7 @@ abstract class RDD[T: ClassTag]( * * Note that this method performs a shuffle internally. */ - def intersection(other: RDD[T]): RDD[T] = { + def intersection(other: RDD[T]): RDD[T] = withScope { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys @@ -514,8 +548,9 @@ abstract class RDD[T: ClassTag]( * * @param partitioner Partitioner to use for the resulting RDD */ - def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null) - : RDD[T] = { + def intersection( + other: RDD[T], + partitioner: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = withScope { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys @@ -529,16 +564,14 @@ abstract class RDD[T: ClassTag]( * * @param numPartitions How many partitions to use in the resulting RDD */ - def intersection(other: RDD[T], numPartitions: Int): RDD[T] = { - this.map(v => (v, null)).cogroup(other.map(v => (v, null)), new HashPartitioner(numPartitions)) - .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } - .keys + def intersection(other: RDD[T], numPartitions: Int): RDD[T] = withScope { + intersection(other, new HashPartitioner(numPartitions)) } /** * Return an RDD created by coalescing all elements within each partition into an array. */ - def glom(): RDD[Array[T]] = { + def glom(): RDD[Array[T]] = withScope { new MapPartitionsRDD[Array[T], T](this, (context, pid, iter) => Iterator(iter.toArray)) } @@ -546,7 +579,9 @@ abstract class RDD[T: ClassTag]( * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of * elements (a, b) where a is in `this` and b is in `other`. */ - def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) + def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope { + new CartesianRDD(sc, this, other) + } /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements @@ -557,8 +592,9 @@ abstract class RDD[T: ClassTag]( * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ - def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope { groupBy[K](f, defaultPartitioner(this)) + } /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements @@ -569,8 +605,11 @@ abstract class RDD[T: ClassTag]( * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ - def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = + def groupBy[K]( + f: T => K, + numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope { groupBy(f, new HashPartitioner(numPartitions)) + } /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements @@ -582,7 +621,7 @@ abstract class RDD[T: ClassTag]( * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) - : RDD[(K, Iterable[T])] = { + : RDD[(K, Iterable[T])] = withScope { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -590,13 +629,16 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String): RDD[String] = new PipedRDD(this, command) + def pipe(command: String): RDD[String] = withScope { + new PipedRDD(this, command) + } /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String, env: Map[String, String]): RDD[String] = + def pipe(command: String, env: Map[String, String]): RDD[String] = withScope { new PipedRDD(this, command, env) + } /** * Return an RDD created by piping elements to a forked external process. @@ -604,7 +646,7 @@ abstract class RDD[T: ClassTag]( * * @param command command to run in forked process. * @param env environment variables to set. - * @param printPipeContext Before piping elements, this function is called as an oppotunity + * @param printPipeContext Before piping elements, this function is called as an opportunity * to pipe context data. Print line function (like out.println) will be * passed as printPipeContext's parameter. * @param printRDDElement Use this function to customize how to pipe elements. This function @@ -622,7 +664,7 @@ 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] = { + separateWorkingDir: Boolean = false): 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, @@ -636,9 +678,13 @@ abstract class RDD[T: ClassTag]( * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitions[U: ClassTag]( - f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter) - new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) + f: Iterator[T] => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = withScope { + val cleanedF = sc.clean(f) + new MapPartitionsRDD( + this, + (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(iter), + preservesPartitioning) } /** @@ -649,9 +695,13 @@ abstract class RDD[T: ClassTag]( * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitionsWithIndex[U: ClassTag]( - f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter) - new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = withScope { + val cleanedF = sc.clean(f) + new MapPartitionsRDD( + this, + (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter), + preservesPartitioning) } /** @@ -666,7 +716,7 @@ abstract class RDD[T: ClassTag]( @deprecated("use TaskContext.get", "1.2.0") def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = { + preservesPartitioning: Boolean = false): RDD[U] = withScope { val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter) new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } @@ -677,7 +727,8 @@ abstract class RDD[T: ClassTag]( */ @deprecated("use mapPartitionsWithIndex", "0.7.0") def mapPartitionsWithSplit[U: ClassTag]( - f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = withScope { mapPartitionsWithIndex(f, preservesPartitioning) } @@ -689,7 +740,7 @@ abstract class RDD[T: ClassTag]( @deprecated("use mapPartitionsWithIndex", "1.0.0") def mapWith[A, U: ClassTag] (constructA: Int => A, preservesPartitioning: Boolean = false) - (f: (T, A) => U): RDD[U] = { + (f: (T, A) => U): RDD[U] = withScope { mapPartitionsWithIndex((index, iter) => { val a = constructA(index) iter.map(t => f(t, a)) @@ -704,7 +755,7 @@ abstract class RDD[T: ClassTag]( @deprecated("use mapPartitionsWithIndex and flatMap", "1.0.0") def flatMapWith[A, U: ClassTag] (constructA: Int => A, preservesPartitioning: Boolean = false) - (f: (T, A) => Seq[U]): RDD[U] = { + (f: (T, A) => Seq[U]): RDD[U] = withScope { mapPartitionsWithIndex((index, iter) => { val a = constructA(index) iter.flatMap(t => f(t, a)) @@ -717,11 +768,11 @@ abstract class RDD[T: ClassTag]( * partition with the index of that partition. */ @deprecated("use mapPartitionsWithIndex and foreach", "1.0.0") - def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit) { + def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit): Unit = withScope { mapPartitionsWithIndex { (index, iter) => val a = constructA(index) iter.map(t => {f(t, a); t}) - }.foreach(_ => {}) + } } /** @@ -730,7 +781,7 @@ abstract class RDD[T: ClassTag]( * partition with the index of that partition. */ @deprecated("use mapPartitionsWithIndex and filter", "1.0.0") - def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = { + def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = withScope { mapPartitionsWithIndex((index, iter) => { val a = constructA(index) iter.filter(t => p(t, a)) @@ -743,16 +794,16 @@ abstract class RDD[T: ClassTag]( * partitions* and the *same number of elements in each partition* (e.g. one was made through * a map on the other). */ - def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { + def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = withScope { zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => new Iterator[(T, U)] { - def hasNext = (thisIter.hasNext, otherIter.hasNext) match { + def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match { case (true, true) => true case (false, false) => false case _ => throw new SparkException("Can only zip RDDs with " + "same number of elements in each partition") } - def next = (thisIter.next, otherIter.next) + def next(): (T, U) = (thisIter.next(), otherIter.next()) } } } @@ -765,33 +816,39 @@ abstract class RDD[T: ClassTag]( */ def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B], preservesPartitioning: Boolean) - (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) + } def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B]) - (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = withScope { + zipPartitions(rdd2, preservesPartitioning = false)(f) + } def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean) - (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = + (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = withScope { new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning) + } def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C]) - (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false) + (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = withScope { + zipPartitions(rdd2, rdd3, preservesPartitioning = false)(f) + } def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean) - (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = + (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = withScope { new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning) + } def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D]) - (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, false) + (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = withScope { + zipPartitions(rdd2, rdd3, rdd4, preservesPartitioning = false)(f) + } // Actions (launch a job to return a value to the user program) @@ -799,7 +856,7 @@ abstract class RDD[T: ClassTag]( /** * Applies a function f to all elements of this RDD. */ - def foreach(f: T => Unit) { + def foreach(f: T => Unit): Unit = withScope { val cleanF = sc.clean(f) sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) } @@ -807,7 +864,7 @@ abstract class RDD[T: ClassTag]( /** * Applies a function f to each partition of this RDD. */ - def foreachPartition(f: Iterator[T] => Unit) { + def foreachPartition(f: Iterator[T] => Unit): Unit = withScope { val cleanF = sc.clean(f) sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } @@ -815,7 +872,7 @@ abstract class RDD[T: ClassTag]( /** * Return an array that contains all of the elements in this RDD. */ - def collect(): Array[T] = { + def collect(): Array[T] = withScope { val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray) Array.concat(results: _*) } @@ -825,7 +882,7 @@ abstract class RDD[T: ClassTag]( * * The iterator will consume as much memory as the largest partition in this RDD. */ - def toLocalIterator: Iterator[T] = { + def toLocalIterator: Iterator[T] = withScope { def collectPartition(p: Int): Array[T] = { sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head } @@ -836,12 +893,14 @@ abstract class RDD[T: ClassTag]( * Return an array that contains all of the elements in this RDD. */ @deprecated("use collect", "1.0.0") - def toArray(): Array[T] = collect() + def toArray(): Array[T] = withScope { + collect() + } /** * Return an RDD that contains all matching values by applying `f`. */ - def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = { + def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = withScope { filter(f.isDefinedAt).map(f) } @@ -851,25 +910,29 @@ abstract class RDD[T: ClassTag]( * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ - def subtract(other: RDD[T]): RDD[T] = - subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) + def subtract(other: RDD[T]): RDD[T] = withScope { + subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.length))) + } /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], numPartitions: Int): RDD[T] = + def subtract(other: RDD[T], numPartitions: Int): RDD[T] = withScope { subtract(other, new HashPartitioner(numPartitions)) + } /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = { + def subtract( + other: RDD[T], + p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = withScope { if (partitioner == Some(p)) { // Our partitioner knows how to handle T (which, since we have a partitioner, is // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples val p2 = new Partitioner() { - override def numPartitions = p.numPartitions - override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) + override def numPartitions: Int = p.numPartitions + override def getPartition(k: Any): Int = p.getPartition(k.asInstanceOf[(Any, _)]._1) } // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies // anyway, and when calling .keys, will not have a partitioner set, even though @@ -885,7 +948,7 @@ abstract class RDD[T: ClassTag]( * Reduces the elements of this RDD using the specified commutative and * associative binary operator. */ - def reduce(f: (T, T) => T): T = { + def reduce(f: (T, T) => T): T = withScope { val cleanF = sc.clean(f) val reducePartition: Iterator[T] => Option[T] = iter => { if (iter.hasNext) { @@ -914,7 +977,7 @@ abstract class RDD[T: ClassTag]( * @param depth suggested depth of the tree (default: 2) * @see [[org.apache.spark.rdd.RDD#reduce]] */ - def treeReduce(f: (T, T) => T, depth: Int = 2): T = { + def treeReduce(f: (T, T) => T, depth: Int = 2): T = withScope { require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") val cleanF = context.clean(f) val reducePartition: Iterator[T] => Option[T] = iter => { @@ -946,7 +1009,7 @@ abstract class RDD[T: ClassTag]( * modify t1 and return it as its result value to avoid object allocation; however, it should not * modify t2. */ - def fold(zeroValue: T)(op: (T, T) => T): T = { + def fold(zeroValue: T)(op: (T, T) => T): T = withScope { // Clone the zero value since we will also be serializing it as part of tasks var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) val cleanOp = sc.clean(op) @@ -964,7 +1027,7 @@ abstract class RDD[T: ClassTag]( * allowed to modify and return their first argument instead of creating a new U to avoid memory * allocation. */ - def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { + def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope { // Clone the zero value since we will also be serializing it as part of tasks var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) @@ -984,26 +1047,29 @@ abstract class RDD[T: ClassTag]( def treeAggregate[U: ClassTag](zeroValue: U)( seqOp: (U, T) => U, combOp: (U, U) => U, - depth: Int = 2): U = { + depth: Int = 2): U = withScope { require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") - if (partitions.size == 0) { - return Utils.clone(zeroValue, context.env.closureSerializer.newInstance()) - } - val cleanSeqOp = context.clean(seqOp) - val cleanCombOp = context.clean(combOp) - val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) - var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it))) - var numPartitions = partiallyAggregated.partitions.size - val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) - // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation. - while (numPartitions > scale + numPartitions / scale) { - numPartitions /= scale - val curNumPartitions = numPartitions - partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) => - iter.map((i % curNumPartitions, _)) - }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values + if (partitions.length == 0) { + Utils.clone(zeroValue, context.env.closureSerializer.newInstance()) + } else { + val cleanSeqOp = context.clean(seqOp) + val cleanCombOp = context.clean(combOp) + val aggregatePartition = + (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) + var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it))) + var numPartitions = partiallyAggregated.partitions.length + val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) + // If creating an extra level doesn't help reduce + // the wall-clock time, we stop tree aggregation. + while (numPartitions > scale + numPartitions / scale) { + numPartitions /= scale + val curNumPartitions = numPartitions + partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { + (i, iter) => iter.map((i % curNumPartitions, _)) + }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values + } + partiallyAggregated.reduce(cleanCombOp) } - partiallyAggregated.reduce(cleanCombOp) } /** @@ -1017,7 +1083,9 @@ abstract class RDD[T: ClassTag]( * within a timeout, even if not all tasks have finished. */ @Experimental - def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + def countApprox( + timeout: Long, + confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L while (iter.hasNext) { @@ -1026,7 +1094,7 @@ abstract class RDD[T: ClassTag]( } result } - val evaluator = new CountEvaluator(partitions.size, confidence) + val evaluator = new CountEvaluator(partitions.length, confidence) sc.runApproximateJob(this, countElements, evaluator, timeout) } @@ -1038,7 +1106,7 @@ abstract class RDD[T: ClassTag]( * To handle very large results, consider using rdd.map(x => (x, 1L)).reduceByKey(_ + _), which * returns an RDD[T, Long] instead of a map. */ - def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { + def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = withScope { map(value => (value, null)).countByKey() } @@ -1049,8 +1117,7 @@ abstract class RDD[T: ClassTag]( @Experimental def countByValueApprox(timeout: Long, confidence: Double = 0.95) (implicit ord: Ordering[T] = null) - : PartialResult[Map[T, BoundedDouble]] = - { + : PartialResult[Map[T, BoundedDouble]] = withScope { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } @@ -1061,7 +1128,7 @@ abstract class RDD[T: ClassTag]( } map } - val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence) + val evaluator = new GroupedCountEvaluator[T](partitions.length, confidence) sc.runApproximateJob(this, countPartition, evaluator, timeout) } @@ -1083,7 +1150,7 @@ abstract class RDD[T: ClassTag]( * If `sp` equals 0, the sparse representation is skipped. */ @Experimental - def countApproxDistinct(p: Int, sp: Int): Long = { + def countApproxDistinct(p: Int, sp: Int): Long = withScope { require(p >= 4, s"p ($p) must be at least 4") require(sp <= 32, s"sp ($sp) cannot be greater than 32") require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") @@ -1109,7 +1176,7 @@ abstract class RDD[T: ClassTag]( * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. */ - def countApproxDistinct(relativeSD: Double = 0.05): Long = { + def countApproxDistinct(relativeSD: Double = 0.05): Long = withScope { val p = math.ceil(2.0 * math.log(1.054 / relativeSD) / math.log(2)).toInt countApproxDistinct(p, 0) } @@ -1127,7 +1194,9 @@ abstract class RDD[T: ClassTag]( * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ - def zipWithIndex(): RDD[(T, Long)] = new ZippedWithIndexRDD(this) + def zipWithIndex(): RDD[(T, Long)] = withScope { + new ZippedWithIndexRDD(this) + } /** * Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, @@ -1139,8 +1208,8 @@ abstract class RDD[T: ClassTag]( * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ - def zipWithUniqueId(): RDD[(T, Long)] = { - val n = this.partitions.size.toLong + def zipWithUniqueId(): RDD[(T, Long)] = withScope { + val n = this.partitions.length.toLong this.mapPartitionsWithIndex { case (k, iter) => iter.zipWithIndex.map { case (item, i) => (item, i * n + k) @@ -1156,48 +1225,50 @@ abstract class RDD[T: ClassTag]( * @note due to complications in the internal implementation, this method will raise * an exception if called on an RDD of `Nothing` or `Null`. */ - def take(num: Int): Array[T] = { + def take(num: Int): Array[T] = withScope { if (num == 0) { - return new Array[T](0) - } - - val buf = new ArrayBuffer[T] - val totalParts = this.partitions.length - var partsScanned = 0 - while (buf.size < num && partsScanned < totalParts) { - // The number of partitions to try in this iteration. It is ok for this number to be - // greater than totalParts because we actually cap it at totalParts in runJob. - var numPartsToTry = 1 - if (partsScanned > 0) { - // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, - // interpolate the number of partitions we need to try, but overestimate it by 50%. - // We also cap the estimation in the end. - if (buf.size == 0) { - numPartsToTry = partsScanned * 4 - } else { - // the left side of max is >=1 whenever partsScanned >= 2 - numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1) - numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) + new Array[T](0) + } else { + val buf = new ArrayBuffer[T] + val totalParts = this.partitions.length + var partsScanned = 0 + while (buf.size < num && partsScanned < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. + var numPartsToTry = 1 + if (partsScanned > 0) { + // If we didn't find any rows after the previous iteration, quadruple and retry. + // Otherwise, interpolate the number of partitions we need to try, but overestimate + // it by 50%. We also cap the estimation in the end. + if (buf.size == 0) { + numPartsToTry = partsScanned * 4 + } else { + // the left side of max is >=1 whenever partsScanned >= 2 + numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1) + numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) + } } - } - val left = num - buf.size - val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) - val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true) + val left = num - buf.size + val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true) - res.foreach(buf ++= _.take(num - buf.size)) - partsScanned += numPartsToTry - } + res.foreach(buf ++= _.take(num - buf.size)) + partsScanned += numPartsToTry + } - buf.toArray + buf.toArray + } } /** * Return the first element in this RDD. */ - def first(): T = take(1) match { - case Array(t) => t - case _ => throw new UnsupportedOperationException("empty collection") + def first(): T = withScope { + take(1) match { + case Array(t) => t + case _ => throw new UnsupportedOperationException("empty collection") + } } /** @@ -1215,7 +1286,9 @@ abstract class RDD[T: ClassTag]( * @param ord the implicit ordering for T * @return an array of top elements */ - def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse) + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = withScope { + takeOrdered(num)(ord.reverse) + } /** * Returns the first k (smallest) elements from this RDD as defined by the specified @@ -1233,7 +1306,7 @@ abstract class RDD[T: ClassTag]( * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = withScope { if (num == 0) { Array.empty } else { @@ -1243,7 +1316,7 @@ abstract class RDD[T: ClassTag]( queue ++= util.collection.Utils.takeOrdered(items, num)(ord) Iterator.single(queue) } - if (mapRDDs.partitions.size == 0) { + if (mapRDDs.partitions.length == 0) { Array.empty } else { mapRDDs.reduce { (queue1, queue2) => @@ -1258,13 +1331,17 @@ abstract class RDD[T: ClassTag]( * Returns the max of this RDD as defined by the implicit Ordering[T]. * @return the maximum element of the RDD * */ - def max()(implicit ord: Ordering[T]): T = this.reduce(ord.max) + def max()(implicit ord: Ordering[T]): T = withScope { + this.reduce(ord.max) + } /** * Returns the min of this RDD as defined by the implicit Ordering[T]. * @return the minimum element of the RDD * */ - def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min) + def min()(implicit ord: Ordering[T]): T = withScope { + this.reduce(ord.min) + } /** * @note due to complications in the internal implementation, this method will raise an @@ -1274,12 +1351,14 @@ abstract class RDD[T: ClassTag]( * @return true if and only if the RDD contains no elements at all. Note that an RDD * may be empty even when it has at least 1 partition. */ - def isEmpty(): Boolean = partitions.length == 0 || take(1).length == 0 + def isEmpty(): Boolean = withScope { + partitions.length == 0 || take(1).length == 0 + } /** * Save this RDD as a text file, using string representations of elements. */ - def saveAsTextFile(path: String) { + def saveAsTextFile(path: String): Unit = withScope { // https://issues.apache.org/jira/browse/SPARK-2075 // // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit @@ -1306,7 +1385,7 @@ abstract class RDD[T: ClassTag]( /** * Save this RDD as a compressed text file, using string representations of elements. */ - def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = withScope { // https://issues.apache.org/jira/browse/SPARK-2075 val nullWritableClassTag = implicitly[ClassTag[NullWritable]] val textClassTag = implicitly[ClassTag[Text]] @@ -1324,7 +1403,7 @@ abstract class RDD[T: ClassTag]( /** * Save this RDD as a SequenceFile of serialized objects. */ - def saveAsObjectFile(path: String) { + def saveAsObjectFile(path: String): Unit = withScope { this.mapPartitions(iter => iter.grouped(10).map(_.toArray)) .map(x => (NullWritable.get(), new BytesWritable(Utils.serialize(x)))) .saveAsSequenceFile(path) @@ -1333,12 +1412,13 @@ abstract class RDD[T: ClassTag]( /** * Creates tuples of the elements in this RDD by applying `f`. */ - def keyBy[K](f: T => K): RDD[(K, T)] = { - map(x => (f(x), x)) + def keyBy[K](f: T => K): RDD[(K, T)] = withScope { + val cleanedF = sc.clean(f) + map(x => (cleanedF(x), x)) } /** A private method for tests, to look at the contents of each partition */ - private[spark] def collectPartitions(): Array[Array[T]] = { + private[spark] def collectPartitions(): Array[Array[T]] = withScope { sc.runJob(this, (iter: Iterator[T]) => iter.toArray) } @@ -1377,6 +1457,17 @@ abstract class RDD[T: ClassTag]( /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ @transient private[spark] val creationSite = sc.getCallSite() + /** + * The scope associated with the operation that created this RDD. + * + * This is more flexible than the call site and can be defined hierarchically. For more + * detail, see the documentation of {{RDDOperationScope}}. This scope is not defined if the + * user instantiates this RDD himself without using any Spark operations. + */ + @transient private[spark] val scope: Option[RDDOperationScope] = { + Option(sc.getLocalProperty(SparkContext.RDD_SCOPE_KEY)).map(RDDOperationScope.fromJson) + } + private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] @@ -1394,7 +1485,7 @@ abstract class RDD[T: ClassTag]( } /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ - def context = sc + def context: SparkContext = sc /** * Private API for changing an RDD's ClassTag. @@ -1455,14 +1546,14 @@ abstract class RDD[T: ClassTag]( /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString: String = { // Get a debug description of an rdd without its children - def debugSelf (rdd: RDD[_]): Seq[String] = { + def debugSelf(rdd: RDD[_]): Seq[String] = { import Utils.bytesToString val persistence = if (storageLevel != StorageLevel.NONE) storageLevel.description else "" val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info => - " CachedPartitions: %d; MemorySize: %s; TachyonSize: %s; DiskSize: %s".format( + " CachedPartitions: %d; MemorySize: %s; ExternalBlockStoreSize: %s; DiskSize: %s".format( info.numCachedPartitions, bytesToString(info.memSize), - bytesToString(info.tachyonSize), bytesToString(info.diskSize))) + bytesToString(info.externalBlockStoreSize), bytesToString(info.diskSize))) s"$rdd [$persistence]" +: storageInfo } @@ -1489,7 +1580,7 @@ abstract class RDD[T: ClassTag]( } // The first RDD in the dependency stack has no parents, so no need for a +- def firstDebugString(rdd: RDD[_]): Seq[String] = { - val partitionStr = "(" + rdd.partitions.size + ")" + val partitionStr = "(" + rdd.partitions.length + ")" val leftOffset = (partitionStr.length - 1) / 2 val nextPrefix = (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset)) @@ -1499,7 +1590,7 @@ abstract class RDD[T: ClassTag]( } ++ debugChildren(rdd, nextPrefix) } def shuffleDebugString(rdd: RDD[_], prefix: String = "", isLastChild: Boolean): Seq[String] = { - val partitionStr = "(" + rdd.partitions.size + ")" + val partitionStr = "(" + rdd.partitions.length + ")" val leftOffset = (partitionStr.length - 1) / 2 val thisPrefix = prefix.replaceAll("\\|\\s+$", "") val nextPrefix = ( @@ -1512,10 +1603,11 @@ abstract class RDD[T: ClassTag]( case (desc: String, _) => s"$nextPrefix$desc" } ++ debugChildren(rdd, nextPrefix) } - def debugString(rdd: RDD[_], - prefix: String = "", - isShuffle: Boolean = true, - isLastChild: Boolean = false): Seq[String] = { + def debugString( + rdd: RDD[_], + prefix: String = "", + isShuffle: Boolean = true, + isLastChild: Boolean = false): Seq[String] = { if (isShuffle) { shuffleDebugString(rdd, prefix, isLastChild) } else { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index f67e5f1857979..1722c27e55003 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, Partition, SerializableWritable, SparkException} +import org.apache.spark._ import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** @@ -83,7 +83,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } // Create the output path for the checkpoint - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val path = RDDCheckpointData.rddCheckpointDataPath(rdd.context, rdd.id).get val fs = path.getFileSystem(rdd.context.hadoopConfiguration) if (!fs.mkdirs(path)) { throw new SparkException("Failed to create checkpoint path " + path) @@ -92,12 +92,17 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) - if (newRDD.partitions.size != rdd.partitions.size) { + if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { + rdd.context.cleaner.foreach { cleaner => + cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id) + } + } + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) + if (newRDD.partitions.length != rdd.partitions.length) { throw new SparkException( - "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.size + ") has different " + - "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") + "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.length + ") has different " + + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.length + ")") } // Change the dependencies and partitions of the RDD @@ -130,5 +135,17 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -// Used for synchronization -private[spark] object RDDCheckpointData +private[spark] object RDDCheckpointData { + def rddCheckpointDataPath(sc: SparkContext, rddId: Int): Option[Path] = { + sc.checkpointDir.map { dir => new Path(dir, "rdd-" + rddId) } + } + + def clearRDDCheckpointData(sc: SparkContext, rddId: Int): Unit = { + rddCheckpointDataPath(sc, rddId).foreach { path => + val fs = path.getFileSystem(sc.hadoopConfiguration) + if (fs.exists(path)) { + fs.delete(path, true) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala new file mode 100644 index 0000000000000..537b56b49f866 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -0,0 +1,137 @@ +/* + * 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.rdd + +import java.util.concurrent.atomic.AtomicInteger + +import com.fasterxml.jackson.annotation.{JsonIgnore, JsonInclude, JsonPropertyOrder} +import com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +import org.apache.spark.SparkContext + +/** + * A general, named code block representing an operation that instantiates RDDs. + * + * All RDDs instantiated in the corresponding code block will store a pointer to this object. + * Examples include, but will not be limited to, existing RDD operations, such as textFile, + * reduceByKey, and treeAggregate. + * + * An operation scope may be nested in other scopes. For instance, a SQL query may enclose + * scopes associated with the public RDD APIs it uses under the hood. + * + * There is no particular relationship between an operation scope and a stage or a job. + * A scope may live inside one stage (e.g. map) or span across multiple jobs (e.g. take). + */ +@JsonInclude(Include.NON_NULL) +@JsonPropertyOrder(Array("id", "name", "parent")) +private[spark] class RDDOperationScope( + val name: String, + val parent: Option[RDDOperationScope] = None) { + + val id: Int = RDDOperationScope.nextScopeId() + + def toJson: String = { + RDDOperationScope.jsonMapper.writeValueAsString(this) + } + + /** + * Return a list of scopes that this scope is a part of, including this scope itself. + * The result is ordered from the outermost scope (eldest ancestor) to this scope. + */ + @JsonIgnore + def getAllScopes: Seq[RDDOperationScope] = { + parent.map(_.getAllScopes).getOrElse(Seq.empty) ++ Seq(this) + } + + override def equals(other: Any): Boolean = { + other match { + case s: RDDOperationScope => + id == s.id && name == s.name && parent == s.parent + case _ => false + } + } + + override def toString: String = toJson +} + +/** + * A collection of utility methods to construct a hierarchical representation of RDD scopes. + * An RDD scope tracks the series of operations that created a given RDD. + */ +private[spark] object RDDOperationScope { + private val jsonMapper = new ObjectMapper().registerModule(DefaultScalaModule) + private val scopeCounter = new AtomicInteger(0) + + def fromJson(s: String): RDDOperationScope = { + jsonMapper.readValue(s, classOf[RDDOperationScope]) + } + + /** Return a globally unique operation scope ID. */ + def nextScopeId(): Int = scopeCounter.getAndIncrement + + /** + * Execute the given body such that all RDDs created in this body will have the same scope. + * The name of the scope will be the name of the method that immediately encloses this one. + * + * Note: Return statements are NOT allowed in body. + */ + private[spark] def withScope[T]( + sc: SparkContext, + allowNesting: Boolean = false)(body: => T): T = { + val callerMethodName = Thread.currentThread.getStackTrace()(3).getMethodName + withScope[T](sc, callerMethodName, allowNesting)(body) + } + + /** + * Execute the given body such that all RDDs created in this body will have the same scope. + * + * If nesting is allowed, this concatenates the previous scope with the new one in a way that + * signifies the hierarchy. Otherwise, if nesting is not allowed, then any children calls to + * this method executed in the body will have no effect. + * + * Note: Return statements are NOT allowed in body. + */ + private[spark] def withScope[T]( + sc: SparkContext, + name: String, + allowNesting: Boolean = false)(body: => T): T = { + // Save the old scope to restore it later + val scopeKey = SparkContext.RDD_SCOPE_KEY + val noOverrideKey = SparkContext.RDD_SCOPE_NO_OVERRIDE_KEY + val oldScopeJson = sc.getLocalProperty(scopeKey) + val oldScope = Option(oldScopeJson).map(RDDOperationScope.fromJson) + val oldNoOverride = sc.getLocalProperty(noOverrideKey) + try { + // Set the scope only if the higher level caller allows us to do so + if (sc.getLocalProperty(noOverrideKey) == null) { + sc.setLocalProperty(scopeKey, new RDDOperationScope(name, oldScope).toJson) + } + // Optionally disallow the child body to override our scope + if (!allowNesting) { + sc.setLocalProperty(noOverrideKey, "true") + } + body + } finally { + // Remember to restore any state that was modified before exiting + sc.setLocalProperty(scopeKey, oldScopeJson) + sc.setLocalProperty(noOverrideKey, oldNoOverride) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 059f8963691f0..3dfcf67f0eb66 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -85,7 +85,9 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported * file system. */ - def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { + def saveAsSequenceFile( + path: String, + codec: Option[Class[_ <: CompressionCodec]] = None): Unit = self.withScope { def anyToWritable[U <% Writable](u: U): Writable = u // TODO We cannot force the return type of `anyToWritable` be same as keyWritableClass and diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index d9fe6847254fa..2dc47f95937cb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,14 +17,12 @@ package org.apache.spark.rdd -import scala.reflect.ClassTag - import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { - override val index = idx + override val index: Int = idx override def hashCode(): Int = idx } diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index ed24ea22a661c..633aeba3bbae6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -76,14 +76,14 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def getPartitions: Array[Partition] = { val array = new Array[Partition](part.numPartitions) - for (i <- 0 until array.size) { + for (i <- 0 until array.length) { // Each CoGroupPartition will depend on rdd1 and rdd2 array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) => dependencies(j) match { case s: ShuffleDependency[_, _, _] => - new ShuffleCoGroupSplitDep(s.shuffleHandle) + None case _ => - new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i)) + Some(new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))) } }.toArray) } @@ -105,20 +105,26 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( seq } } - def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => - rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) + def integrate(depNum: Int, op: Product2[K, V] => Unit) = { + dependencies(depNum) match { + case oneToOneDependency: OneToOneDependency[_] => + val dependencyPartition = partition.narrowDeps(depNum).get.split + oneToOneDependency.rdd.iterator(dependencyPartition, context) + .asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - case ShuffleCoGroupSplitDep(handle) => - val iter = SparkEnv.get.shuffleManager - .getReader(handle, partition.index, partition.index + 1, context) - .read() - iter.foreach(op) + case shuffleDependency: ShuffleDependency[_, _, _] => + val iter = SparkEnv.get.shuffleManager + .getReader( + shuffleDependency.shuffleHandle, partition.index, partition.index + 1, context) + .read() + iter.foreach(op) + } } + // the first dep is rdd1; add all values to the map - integrate(partition.deps(0), t => getSeq(t._1) += t._2) + integrate(0, t => getSeq(t._1) += t._2) // the second dep is rdd2; remove all of its keys - integrate(partition.deps(1), t => map.remove(t._1)) + integrate(1, t => map.remove(t._1)) map.iterator.map { t => t._2.iterator.map { (t._1, _) } }.flatten } 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 aece683ff3199..3986645350a82 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -44,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag]( var parentPartition: Partition = rdd.partitions(parentRddPartitionIndex) - def preferredLocations() = rdd.preferredLocations(parentPartition) + def preferredLocations(): Seq[String] = rdd.preferredLocations(parentPartition) override val index: Int = idx @@ -63,7 +63,7 @@ class UnionRDD[T: ClassTag]( extends RDD[T](sc, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.size).sum) + val array = new Array[Partition](rdds.map(_.partitions.length).sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) @@ -76,8 +76,8 @@ class UnionRDD[T: ClassTag]( val deps = new ArrayBuffer[Dependency[_]] var pos = 0 for (rdd <- rdds) { - deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size) - pos += rdd.partitions.size + deps += new RangeDependency(rdd, 0, pos, rdd.partitions.length) + pos += rdd.partitions.length } deps } diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 95b2dd954e9f4..a96b6c3d23454 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -32,7 +32,7 @@ private[spark] class ZippedPartitionsPartition( override val index: Int = idx var partitionValues = rdds.map(rdd => rdd.partitions(idx)) - def partitions = partitionValues + def partitions: Seq[Partition] = partitionValues @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { @@ -52,8 +52,8 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( if (preservesPartitioning) firstParent[Any].partitioner else None override def getPartitions: Array[Partition] = { - val numParts = rdds.head.partitions.size - if (!rdds.forall(rdd => rdd.partitions.size == numParts)) { + val numParts = rdds.head.partitions.length + if (!rdds.forall(rdd => rdd.partitions.length == numParts)) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } Array.tabulate[Partition](numParts) { i => diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index 8c43a559409f2..523aaf2b860b5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -41,7 +41,7 @@ class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, L /** The start index of each partition. */ @transient private val startIndices: Array[Long] = { - val n = prev.partitions.size + val n = prev.partitions.length if (n == 0) { Array[Long]() } else if (n == 1) { diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala new file mode 100644 index 0000000000000..3e5b64265e919 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala @@ -0,0 +1,41 @@ +/* + * 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.rpc + +/** + * A callback that [[RpcEndpoint]] can use it to send back a message or failure. It's thread-safe + * and can be called in any thread. + */ +private[spark] trait RpcCallContext { + + /** + * Reply a message to the sender. If the sender is [[RpcEndpoint]], its [[RpcEndpoint.receive]] + * will be called. + */ + def reply(response: Any): Unit + + /** + * Report a failure to the sender. + */ + def sendFailure(e: Throwable): Unit + + /** + * The sender of this message. + */ + def sender: RpcEndpointRef +} diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala new file mode 100644 index 0000000000000..d2b2baef1d8c4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala @@ -0,0 +1,148 @@ +/* + * 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.rpc + +import org.apache.spark.SparkException + +/** + * A factory class to create the [[RpcEnv]]. It must have an empty constructor so that it can be + * created using Reflection. + */ +private[spark] trait RpcEnvFactory { + + def create(config: RpcEnvConfig): RpcEnv +} + +/** + * A trait that requires RpcEnv thread-safely sending messages to it. + * + * Thread-safety means processing of one message happens before processing of the next message by + * the same [[ThreadSafeRpcEndpoint]]. In the other words, changes to internal fields of a + * [[ThreadSafeRpcEndpoint]] are visible when processing the next message, and fields in the + * [[ThreadSafeRpcEndpoint]] need not be volatile or equivalent. + * + * However, there is no guarantee that the same thread will be executing the same + * [[ThreadSafeRpcEndpoint]] for different messages. + */ +private[spark] trait ThreadSafeRpcEndpoint extends RpcEndpoint + + +/** + * An end point for the RPC that defines what functions to trigger given a message. + * + * It is guaranteed that `onStart`, `receive` and `onStop` will be called in sequence. + * + * The lift-cycle will be: + * + * constructor onStart receive* onStop + * + * Note: `receive` can be called concurrently. If you want `receive` is thread-safe, please use + * [[ThreadSafeRpcEndpoint]] + * + * If any error is thrown from one of [[RpcEndpoint]] methods except `onError`, `onError` will be + * invoked with the cause. If `onError` throws an error, [[RpcEnv]] will ignore it. + */ +private[spark] trait RpcEndpoint { + + /** + * The [[RpcEnv]] that this [[RpcEndpoint]] is registered to. + */ + val rpcEnv: RpcEnv + + /** + * The [[RpcEndpointRef]] of this [[RpcEndpoint]]. `self` will become valid when `onStart` is + * called. And `self` will become `null` when `onStop` is called. + * + * Note: Because before `onStart`, [[RpcEndpoint]] has not yet been registered and there is not + * valid [[RpcEndpointRef]] for it. So don't call `self` before `onStart` is called. + */ + final def self: RpcEndpointRef = { + require(rpcEnv != null, "rpcEnv has not been initialized") + rpcEnv.endpointRef(this) + } + + /** + * Process messages from [[RpcEndpointRef.send]] or [[RpcCallContext.reply)]]. If receiving a + * unmatched message, [[SparkException]] will be thrown and sent to `onError`. + */ + def receive: PartialFunction[Any, Unit] = { + case _ => throw new SparkException(self + " does not implement 'receive'") + } + + /** + * Process messages from [[RpcEndpointRef.ask]]. If receiving a unmatched message, + * [[SparkException]] will be thrown and sent to `onError`. + */ + def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case _ => context.sendFailure(new SparkException(self + " won't reply anything")) + } + + /** + * Invoked when any exception is thrown during handling messages. + */ + def onError(cause: Throwable): Unit = { + // By default, throw e and let RpcEnv handle it + throw cause + } + + /** + * Invoked before [[RpcEndpoint]] starts to handle any message. + */ + def onStart(): Unit = { + // By default, do nothing. + } + + /** + * Invoked when [[RpcEndpoint]] is stopping. + */ + def onStop(): Unit = { + // By default, do nothing. + } + + /** + * Invoked when `remoteAddress` is connected to the current node. + */ + def onConnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when `remoteAddress` is lost. + */ + def onDisconnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when some network error happens in the connection between the current node and + * `remoteAddress`. + */ + def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * A convenient method to stop [[RpcEndpoint]]. + */ + final def stop(): Unit = { + val _self = self + if (_self != null) { + rpcEnv.stop(_self) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala new file mode 100644 index 0000000000000..69181edb9ad44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -0,0 +1,119 @@ +/* + * 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.rpc + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.FiniteDuration +import scala.reflect.ClassTag + +import org.apache.spark.util.RpcUtils +import org.apache.spark.{SparkException, Logging, SparkConf} + +/** + * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe. + */ +private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) + extends Serializable with Logging { + + private[this] val maxRetries = RpcUtils.numRetries(conf) + private[this] val retryWaitMs = RpcUtils.retryWaitMs(conf) + private[this] val defaultAskTimeout = RpcUtils.askTimeout(conf) + + /** + * return the address for the [[RpcEndpointRef]] + */ + def address: RpcAddress + + def name: String + + /** + * Sends a one-way asynchronous message. Fire-and-forget semantics. + */ + def send(message: Any): Unit + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to + * receive the reply within the specified timeout. + * + * This method only sends the message once and never retries. + */ + def ask[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to + * receive the reply within a default timeout. + * + * This method only sends the message once and never retries. + */ + def ask[T: ClassTag](message: Any): Future[T] = ask(message, defaultAskTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint]] and get its result within a default + * timeout, or throw a SparkException if this fails even after the default number of retries. + * The default `timeout` will be used in every trial of calling `sendWithReply`. Because this + * method retries, the message handling in the receiver side should be idempotent. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in an message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askWithRetry[T: ClassTag](message: Any): T = askWithRetry(message, defaultAskTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a + * specified timeout, throw a SparkException if this fails even after the specified number of + * retries. `timeout` will be used in every trial of calling `sendWithReply`. Because this method + * retries, the message handling in the receiver side should be idempotent. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in an message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @param timeout the timeout duration + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askWithRetry[T: ClassTag](message: Any, timeout: FiniteDuration): T = { + // TODO: Consider removing multiple attempts + var attempts = 0 + var lastException: Exception = null + while (attempts < maxRetries) { + attempts += 1 + try { + val future = ask[T](message, timeout) + val result = Await.result(future, timeout) + if (result == null) { + throw new SparkException("Actor returned null") + } + return result + } catch { + case ie: InterruptedException => throw ie + case e: Exception => + lastException = e + logWarning(s"Error sending message [message = $message] in $attempts attempts", e) + } + Thread.sleep(retryWaitMs) + } + + throw new SparkException( + s"Error sending message [message = $message]", lastException) + } +} diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala new file mode 100644 index 0000000000000..12b6b28d4d7ec --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -0,0 +1,184 @@ +/* + * 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.rpc + +import java.net.URI + +import scala.concurrent.{Await, Future} +import scala.language.postfixOps + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.util.{RpcUtils, Utils} + + +/** + * A RpcEnv implementation must have a [[RpcEnvFactory]] implementation with an empty constructor + * so that it can be created via Reflection. + */ +private[spark] object RpcEnv { + + private def getRpcEnvFactory(conf: SparkConf): RpcEnvFactory = { + // Add more RpcEnv implementations here + val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory") + val rpcEnvName = conf.get("spark.rpc", "akka") + val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName) + Class.forName(rpcEnvFactoryClassName, true, Utils.getContextOrSparkClassLoader). + newInstance().asInstanceOf[RpcEnvFactory] + } + + def create( + name: String, + host: String, + port: Int, + conf: SparkConf, + securityManager: SecurityManager): RpcEnv = { + // Using Reflection to create the RpcEnv to avoid to depend on Akka directly + val config = RpcEnvConfig(conf, name, host, port, securityManager) + getRpcEnvFactory(conf).create(config) + } + +} + + +/** + * An RPC environment. [[RpcEndpoint]]s need to register itself with a name to [[RpcEnv]] to + * receives messages. Then [[RpcEnv]] will process messages sent from [[RpcEndpointRef]] or remote + * nodes, and deliver them to corresponding [[RpcEndpoint]]s. For uncaught exceptions caught by + * [[RpcEnv]], [[RpcEnv]] will use [[RpcCallContext.sendFailure]] to send exceptions back to the + * sender, or logging them if no such sender or `NotSerializableException`. + * + * [[RpcEnv]] also provides some methods to retrieve [[RpcEndpointRef]]s given name or uri. + */ +private[spark] abstract class RpcEnv(conf: SparkConf) { + + private[spark] val defaultLookupTimeout = RpcUtils.lookupTimeout(conf) + + /** + * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement + * [[RpcEndpoint.self]]. Return `null` if the corresponding [[RpcEndpointRef]] does not exist. + */ + private[rpc] def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Return the address that [[RpcEnv]] is listening to. + */ + def address: RpcAddress + + /** + * Register a [[RpcEndpoint]] with a name and return its [[RpcEndpointRef]]. [[RpcEnv]] does not + * guarantee thread-safety. + */ + def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri` asynchronously. + */ + def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri`. This is a blocking action. + */ + def setupEndpointRefByURI(uri: String): RpcEndpointRef = { + Await.result(asyncSetupEndpointRefByURI(uri), defaultLookupTimeout) + } + + /** + * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName` + * asynchronously. + */ + def asyncSetupEndpointRef( + systemName: String, address: RpcAddress, endpointName: String): Future[RpcEndpointRef] = { + asyncSetupEndpointRefByURI(uriOf(systemName, address, endpointName)) + } + + /** + * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName`. + * This is a blocking action. + */ + def setupEndpointRef( + systemName: String, address: RpcAddress, endpointName: String): RpcEndpointRef = { + setupEndpointRefByURI(uriOf(systemName, address, endpointName)) + } + + /** + * Stop [[RpcEndpoint]] specified by `endpoint`. + */ + def stop(endpoint: RpcEndpointRef): Unit + + /** + * Shutdown this [[RpcEnv]] asynchronously. If need to make sure [[RpcEnv]] exits successfully, + * call [[awaitTermination()]] straight after [[shutdown()]]. + */ + def shutdown(): Unit + + /** + * Wait until [[RpcEnv]] exits. + * + * TODO do we need a timeout parameter? + */ + def awaitTermination(): Unit + + /** + * Create a URI used to create a [[RpcEndpointRef]]. Use this one to create the URI instead of + * creating it manually because different [[RpcEnv]] may have different formats. + */ + def uriOf(systemName: String, address: RpcAddress, endpointName: String): String +} + + +private[spark] case class RpcEnvConfig( + conf: SparkConf, + name: String, + host: String, + port: Int, + securityManager: SecurityManager) + + +/** + * Represents a host and port. + */ +private[spark] case class RpcAddress(host: String, port: Int) { + // TODO do we need to add the type of RpcEnv in the address? + + val hostPort: String = host + ":" + port + + override val toString: String = hostPort +} + + +private[spark] object RpcAddress { + + /** + * Return the [[RpcAddress]] represented by `uri`. + */ + def fromURI(uri: URI): RpcAddress = { + RpcAddress(uri.getHost, uri.getPort) + } + + /** + * Return the [[RpcAddress]] represented by `uri`. + */ + def fromURIString(uri: String): RpcAddress = { + fromURI(new java.net.URI(uri)) + } + + def fromSparkURL(sparkUrl: String): RpcAddress = { + val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) + RpcAddress(host, port) + } +} diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala new file mode 100644 index 0000000000000..ba0d468f111ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala @@ -0,0 +1,325 @@ +/* + * 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.rpc.akka + +import java.util.concurrent.ConcurrentHashMap + +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address} +import akka.event.Logging.Error +import akka.pattern.{ask => akkaAsk} +import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent} +import org.apache.spark.{SparkException, Logging, SparkConf} +import org.apache.spark.rpc._ +import org.apache.spark.util.{ActorLogReceive, AkkaUtils} + +/** + * A RpcEnv implementation based on Akka. + * + * TODO Once we remove all usages of Akka in other place, we can move this file to a new project and + * remove Akka from the dependencies. + * + * @param actorSystem + * @param conf + * @param boundPort + */ +private[spark] class AkkaRpcEnv private[akka] ( + val actorSystem: ActorSystem, conf: SparkConf, boundPort: Int) + extends RpcEnv(conf) with Logging { + + private val defaultAddress: RpcAddress = { + val address = actorSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress + // In some test case, ActorSystem doesn't bind to any address. + // So just use some default value since they are only some unit tests + RpcAddress(address.host.getOrElse("localhost"), address.port.getOrElse(boundPort)) + } + + override val address: RpcAddress = defaultAddress + + /** + * A lookup table to search a [[RpcEndpointRef]] for a [[RpcEndpoint]]. We need it to make + * [[RpcEndpoint.self]] work. + */ + private val endpointToRef = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]() + + /** + * Need this map to remove `RpcEndpoint` from `endpointToRef` via a `RpcEndpointRef` + */ + private val refToEndpoint = new ConcurrentHashMap[RpcEndpointRef, RpcEndpoint]() + + private def registerEndpoint(endpoint: RpcEndpoint, endpointRef: RpcEndpointRef): Unit = { + endpointToRef.put(endpoint, endpointRef) + refToEndpoint.put(endpointRef, endpoint) + } + + private def unregisterEndpoint(endpointRef: RpcEndpointRef): Unit = { + val endpoint = refToEndpoint.remove(endpointRef) + if (endpoint != null) { + endpointToRef.remove(endpoint) + } + } + + /** + * Retrieve the [[RpcEndpointRef]] of `endpoint`. + */ + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = endpointToRef.get(endpoint) + + override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { + @volatile var endpointRef: AkkaRpcEndpointRef = null + // Use lazy because the Actor needs to use `endpointRef`. + // So `actorRef` should be created after assigning `endpointRef`. + lazy val actorRef = actorSystem.actorOf(Props(new Actor with ActorLogReceive with Logging { + + assert(endpointRef != null) + + override def preStart(): Unit = { + // Listen for remote client network events + context.system.eventStream.subscribe(self, classOf[AssociationEvent]) + safelyCall(endpoint) { + endpoint.onStart() + } + } + + override def receiveWithLogging: Receive = { + case AssociatedEvent(_, remoteAddress, _) => + safelyCall(endpoint) { + endpoint.onConnected(akkaAddressToRpcAddress(remoteAddress)) + } + + case DisassociatedEvent(_, remoteAddress, _) => + safelyCall(endpoint) { + endpoint.onDisconnected(akkaAddressToRpcAddress(remoteAddress)) + } + + case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) => + safelyCall(endpoint) { + endpoint.onNetworkError(cause, akkaAddressToRpcAddress(remoteAddress)) + } + + case e: AssociationEvent => + // TODO ignore? + + case m: AkkaMessage => + logDebug(s"Received RPC message: $m") + safelyCall(endpoint) { + processMessage(endpoint, m, sender) + } + + case AkkaFailure(e) => + safelyCall(endpoint) { + throw e + } + + case message: Any => { + logWarning(s"Unknown message: $message") + } + + } + + override def postStop(): Unit = { + unregisterEndpoint(endpoint.self) + safelyCall(endpoint) { + endpoint.onStop() + } + } + + }), name = name) + endpointRef = new AkkaRpcEndpointRef(defaultAddress, actorRef, conf, initInConstructor = false) + registerEndpoint(endpoint, endpointRef) + // Now actorRef can be created safely + endpointRef.init() + endpointRef + } + + private def processMessage(endpoint: RpcEndpoint, m: AkkaMessage, _sender: ActorRef): Unit = { + val message = m.message + val needReply = m.needReply + val pf: PartialFunction[Any, Unit] = + if (needReply) { + endpoint.receiveAndReply(new RpcCallContext { + override def sendFailure(e: Throwable): Unit = { + _sender ! AkkaFailure(e) + } + + override def reply(response: Any): Unit = { + _sender ! AkkaMessage(response, false) + } + + // Some RpcEndpoints need to know the sender's address + override val sender: RpcEndpointRef = + new AkkaRpcEndpointRef(defaultAddress, _sender, conf) + }) + } else { + endpoint.receive + } + try { + pf.applyOrElse[Any, Unit](message, { message => + throw new SparkException(s"Unmatched message $message from ${_sender}") + }) + } catch { + case NonFatal(e) => + if (needReply) { + // If the sender asks a reply, we should send the error back to the sender + _sender ! AkkaFailure(e) + } else { + throw e + } + } + } + + /** + * Run `action` safely to avoid to crash the thread. If any non-fatal exception happens, it will + * call `endpoint.onError`. If `endpoint.onError` throws any non-fatal exception, just log it. + */ + private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { + try { + action + } catch { + case NonFatal(e) => { + try { + endpoint.onError(e) + } catch { + case NonFatal(e) => logError(s"Ignore error: ${e.getMessage}", e) + } + } + } + } + + private def akkaAddressToRpcAddress(address: Address): RpcAddress = { + RpcAddress(address.host.getOrElse(defaultAddress.host), + address.port.getOrElse(defaultAddress.port)) + } + + override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { + import actorSystem.dispatcher + actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout). + map(new AkkaRpcEndpointRef(defaultAddress, _, conf)) + } + + override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String = { + AkkaUtils.address( + AkkaUtils.protocol(actorSystem), systemName, address.host, address.port, endpointName) + } + + override def shutdown(): Unit = { + actorSystem.shutdown() + } + + override def stop(endpoint: RpcEndpointRef): Unit = { + require(endpoint.isInstanceOf[AkkaRpcEndpointRef]) + actorSystem.stop(endpoint.asInstanceOf[AkkaRpcEndpointRef].actorRef) + } + + override def awaitTermination(): Unit = { + actorSystem.awaitTermination() + } + + override def toString: String = s"${getClass.getSimpleName}($actorSystem)" +} + +private[spark] class AkkaRpcEnvFactory extends RpcEnvFactory { + + def create(config: RpcEnvConfig): RpcEnv = { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + config.name, config.host, config.port, config.conf, config.securityManager) + actorSystem.actorOf(Props(classOf[ErrorMonitor]), "ErrorMonitor") + new AkkaRpcEnv(actorSystem, config.conf, boundPort) + } +} + +/** + * Monitor errors reported by Akka and log them. + */ +private[akka] class ErrorMonitor extends Actor with ActorLogReceive with Logging { + + override def preStart(): Unit = { + context.system.eventStream.subscribe(self, classOf[Error]) + } + + override def receiveWithLogging: Actor.Receive = { + case Error(cause: Throwable, _, _, message: String) => logError(message, cause) + } +} + +private[akka] class AkkaRpcEndpointRef( + @transient defaultAddress: RpcAddress, + @transient _actorRef: => ActorRef, + @transient conf: SparkConf, + @transient initInConstructor: Boolean = true) + extends RpcEndpointRef(conf) with Logging { + + lazy val actorRef = _actorRef + + override lazy val address: RpcAddress = { + val akkaAddress = actorRef.path.address + RpcAddress(akkaAddress.host.getOrElse(defaultAddress.host), + akkaAddress.port.getOrElse(defaultAddress.port)) + } + + override lazy val name: String = actorRef.path.name + + private[akka] def init(): Unit = { + // Initialize the lazy vals + actorRef + address + name + } + + if (initInConstructor) { + init() + } + + override def send(message: Any): Unit = { + actorRef ! AkkaMessage(message, false) + } + + override def ask[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] = { + import scala.concurrent.ExecutionContext.Implicits.global + actorRef.ask(AkkaMessage(message, true))(timeout).flatMap { + case msg @ AkkaMessage(message, reply) => + if (reply) { + logError(s"Receive $msg but the sender cannot reply") + Future.failed(new SparkException(s"Receive $msg but the sender cannot reply")) + } else { + Future.successful(message) + } + case AkkaFailure(e) => + Future.failed(e) + }.mapTo[T] + } + + override def toString: String = s"${getClass.getSimpleName}($actorRef)" + +} + +/** + * A wrapper to `message` so that the receiver knows if the sender expects a reply. + * @param message + * @param needReply if the sender expects a reply message + */ +private[akka] case class AkkaMessage(message: Any, needReply: Boolean) + +/** + * A reply with the failure error from the receiver to the sender + */ +private[akka] case class AkkaFailure(e: Throwable) diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index fa83372bb4d11..e0edd7d4ae968 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -39,8 +39,11 @@ class AccumulableInfo ( } object AccumulableInfo { - def apply(id: Long, name: String, update: Option[String], value: String) = + def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = { new AccumulableInfo(id, name, update, value) + } - def apply(id: Long, name: String, value: String) = new AccumulableInfo(id, name, None, value) + def apply(id: Long, name: String, value: String): AccumulableInfo = { + new AccumulableInfo(id, name, None, value) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index b755d8fb15757..50a69379412d2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.CallSite */ private[spark] class ActiveJob( val jobId: Int, - val finalStage: Stage, + val finalStage: ResultStage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], val callSite: CallSite, 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 6d39a5e3fa64c..9f218c64cac2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -26,6 +26,7 @@ package org.apache.spark.scheduler private[spark] class ApplicationEventListener extends SparkListener { var appName: Option[String] = None var appId: Option[String] = None + var appAttemptId: Option[String] = None var sparkUser: Option[String] = None var startTime: Option[Long] = None var endTime: Option[Long] = None @@ -35,6 +36,7 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) appId = applicationStart.appId + appAttemptId = applicationStart.appAttemptId startTime = Some(applicationStart.time) sparkUser = Some(applicationStart.sparkUser) } 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 8feac6cb6b7a1..5d812918a13d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -19,17 +19,16 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} -import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.existentials import scala.language.postfixOps import scala.util.control.NonFatal -import akka.pattern.ask -import akka.util.Timeout +import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast @@ -37,6 +36,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -53,6 +53,10 @@ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task * a small number of times before cancelling the whole stage. * + * Here's a checklist to use when making or reviewing changes to this class: + * + * - When adding a new data structure, update `DAGSchedulerSuite.assertDataStructuresEmpty` to + * include the new structure. This will help to catch memory leaks. */ private[spark] class DAGScheduler( @@ -83,7 +87,7 @@ class DAGScheduler( private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] + private[scheduler] val shuffleToMapStage = new HashMap[Int, ShuffleMapStage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] // Stages we need to run whose parents aren't done @@ -114,6 +118,8 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] + private [scheduler] val outputCommitCoordinator = env.outputCommitCoordinator + // A closure serializer that we reuse. // This is only safe because DAGScheduler runs in a single thread. private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() @@ -126,13 +132,11 @@ class DAGScheduler( private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) private val messageScheduler = - Executors.newScheduledThreadPool(1, Utils.namedThreadFactory("dag-scheduler-message")) + ThreadUtils.newDaemonSingleThreadScheduledExecutor("dag-scheduler-message") private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) - private val outputCommitCoordinator = env.outputCommitCoordinator - // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessLoop.post(BeginEvent(task, taskInfo)) @@ -150,7 +154,7 @@ class DAGScheduler( result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { + taskMetrics: TaskMetrics): Unit = { eventProcessLoop.post( CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) } @@ -165,26 +169,23 @@ class DAGScheduler( taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) blockManagerId: BlockManagerId): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) - implicit val timeout = Timeout(600 seconds) - - Await.result( - blockManagerMaster.driverActor ? BlockManagerHeartbeat(blockManagerId), - timeout.duration).asInstanceOf[Boolean] + blockManagerMaster.driverEndpoint.askWithRetry[Boolean]( + BlockManagerHeartbeat(blockManagerId), 600 seconds) } // Called by TaskScheduler when an executor fails. - def executorLost(execId: String) { + def executorLost(execId: String): Unit = { eventProcessLoop.post(ExecutorLost(execId)) } // Called by TaskScheduler when a host is added - def executorAdded(execId: String, host: String) { + def executorAdded(execId: String, host: String): Unit = { eventProcessLoop.post(ExecutorAdded(execId, host)) } // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or // cancellation of the job itself. - def taskSetFailed(taskSet: TaskSet, reason: String) { + def taskSetFailed(taskSet: TaskSet, reason: String): Unit = { eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } @@ -210,40 +211,65 @@ class DAGScheduler( * The jobId value passed in will be used if the stage doesn't already exist with * a lower jobId (jobId always increases across jobs.) */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): Stage = { + private def getShuffleMapStage( + shuffleDep: ShuffleDependency[_, _, _], + jobId: Int): ShuffleMapStage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => // We are going to register ancestor shuffle dependencies registerShuffleDependencies(shuffleDep, jobId) // Then register current shuffleDep - val stage = - newOrUsedStage( - shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, - shuffleDep.rdd.creationSite) + val stage = newOrUsedShuffleStage(shuffleDep, jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage - + stage } } /** - * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation - * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided - * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage - * directly. + * Helper function to eliminate some code re-use when creating new stages. */ - private def newStage( + private def getParentStagesAndId(rdd: RDD[_], jobId: Int): (List[Stage], Int) = { + val parentStages = getParentStages(rdd, jobId) + val id = nextStageId.getAndIncrement() + (parentStages, id) + } + + /** + * Create a ShuffleMapStage as part of the (re)-creation of a shuffle map stage in + * newOrUsedShuffleStage. The stage will be associated with the provided jobId. + * Production of shuffle map stages should always use newOrUsedShuffleStage, not + * newShuffleMapStage directly. + */ + private def newShuffleMapStage( rdd: RDD[_], numTasks: Int, - shuffleDep: Option[ShuffleDependency[_, _, _]], + shuffleDep: ShuffleDependency[_, _, _], jobId: Int, - callSite: CallSite) - : Stage = - { - val parentStages = getParentStages(rdd, jobId) - val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, numTasks, shuffleDep, parentStages, jobId, callSite) + callSite: CallSite): ShuffleMapStage = { + val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, jobId) + val stage: ShuffleMapStage = new ShuffleMapStage(id, rdd, numTasks, parentStages, + jobId, callSite, shuffleDep) + + stageIdToStage(id) = stage + updateJobIdStageIdMaps(jobId, stage) + stage + } + + /** + * Create a ResultStage -- either directly for use as a result stage, or as part of the + * (re)-creation of a shuffle map stage in newOrUsedShuffleStage. The stage will be associated + * with the provided jobId. + */ + private def newResultStage( + rdd: RDD[_], + numTasks: Int, + jobId: Int, + callSite: CallSite): ResultStage = { + val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, jobId) + val stage: ResultStage = new ResultStage(id, rdd, numTasks, parentStages, jobId, callSite) + stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage @@ -255,20 +281,17 @@ class DAGScheduler( * present in the MapOutputTracker, then the number and location of available outputs are * recovered from the MapOutputTracker */ - private def newOrUsedStage( - rdd: RDD[_], - numTasks: Int, + private def newOrUsedShuffleStage( shuffleDep: ShuffleDependency[_, _, _], - jobId: Int, - callSite: CallSite) - : Stage = - { - val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) + jobId: Int): ShuffleMapStage = { + val rdd = shuffleDep.rdd + val numTasks = rdd.partitions.size + val stage = newShuffleMapStage(rdd, numTasks, shuffleDep, jobId, rdd.creationSite) if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) for (i <- 0 until locs.size) { - stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing + stage.outputLocs(i) = Option(locs(i)).toList // locs(i) will be null if missing } stage.numAvailableOutputs = locs.count(_ != null) } else { @@ -306,26 +329,23 @@ class DAGScheduler( } } waitingForVisit.push(rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } parents.toList } - // Find ancestor missing shuffle dependencies and register into shuffleToMapStage - private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) = { + /** Find ancestor missing shuffle dependencies and register into shuffleToMapStage */ + private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) { val parentsWithNoMapStage = getAncestorShuffleDependencies(shuffleDep.rdd) - while (!parentsWithNoMapStage.isEmpty) { + while (parentsWithNoMapStage.nonEmpty) { val currentShufDep = parentsWithNoMapStage.pop() - val stage = - newOrUsedStage( - currentShufDep.rdd, currentShufDep.rdd.partitions.size, currentShufDep, jobId, - currentShufDep.rdd.creationSite) + val stage = newOrUsedShuffleStage(currentShufDep, jobId) shuffleToMapStage(currentShufDep.shuffleId) = stage } } - // Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet + /** Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet */ private def getAncestorShuffleDependencies(rdd: RDD[_]): Stack[ShuffleDependency[_, _, _]] = { val parents = new Stack[ShuffleDependency[_, _, _]] val visited = new HashSet[RDD[_]] @@ -351,7 +371,7 @@ class DAGScheduler( } waitingForVisit.push(rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } parents @@ -382,7 +402,7 @@ class DAGScheduler( } } waitingForVisit.push(stage.rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } missing.toList @@ -392,7 +412,7 @@ class DAGScheduler( * Registers the given jobId among the jobs that need the given stage and * all of that stage's ancestors. */ - private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { + private def updateJobIdStageIdMaps(jobId: Int, stage: Stage): Unit = { def updateJobIdStageIdMapsList(stages: List[Stage]) { if (stages.nonEmpty) { val s = stages.head @@ -412,7 +432,7 @@ class DAGScheduler( * * @param job The job whose state to cleanup. */ - private def cleanupStateForJobAndIndependentStages(job: ActiveJob) { + private def cleanupStateForJobAndIndependentStages(job: ActiveJob): Unit = { val registeredStages = jobIdToStageIds.get(job.jobId) if (registeredStages.isEmpty || registeredStages.get.isEmpty) { logError("No stages registered for job " + job.jobId) @@ -474,8 +494,7 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null): JobWaiter[U] = - { + properties: Properties): JobWaiter[U] = { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length partitions.find(p => p >= maxPartitions || p < 0).foreach { p => @@ -493,7 +512,8 @@ class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) eventProcessLoop.post(JobSubmitted( - jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)) + jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, + SerializationUtils.clone(properties))) waiter } @@ -504,15 +524,13 @@ class DAGScheduler( callSite: CallSite, allowLocal: Boolean, resultHandler: (Int, U) => Unit, - properties: Properties = null) - { + properties: Properties): Unit = { val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { - case JobSucceeded => { + case JobSucceeded => logInfo("Job %d finished: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) - } case JobFailed(exception: Exception) => logInfo("Job %d failed: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) @@ -526,27 +544,26 @@ class DAGScheduler( evaluator: ApproximateEvaluator[U, R], callSite: CallSite, timeout: Long, - properties: Properties = null) - : PartialResult[R] = - { + properties: Properties): PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray val jobId = nextJobId.getAndIncrement() eventProcessLoop.post(JobSubmitted( - jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)) + jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, + SerializationUtils.clone(properties))) listener.awaitResult() // Will throw an exception if the job fails } /** * Cancel a job that is running or waiting in the queue. */ - def cancelJob(jobId: Int) { + def cancelJob(jobId: Int): Unit = { logInfo("Asked to cancel job " + jobId) eventProcessLoop.post(JobCancelled(jobId)) } - def cancelJobGroup(groupId: String) { + def cancelJobGroup(groupId: String): Unit = { logInfo("Asked to cancel job group " + groupId) eventProcessLoop.post(JobGroupCancelled(groupId)) } @@ -554,7 +571,7 @@ class DAGScheduler( /** * Cancel all jobs that are running or waiting in the queue. */ - def cancelAllJobs() { + def cancelAllJobs(): Unit = { eventProcessLoop.post(AllJobsCancelled) } @@ -631,15 +648,32 @@ class DAGScheduler( try { val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) - val taskContext = new TaskContextImpl(job.finalStage.id, job.partitions(0), taskAttemptId = 0, - attemptNumber = 0, runningLocally = true) - TaskContextHelper.setTaskContext(taskContext) + val taskMemoryManager = new TaskMemoryManager(env.executorMemoryManager) + val taskContext = + new TaskContextImpl( + job.finalStage.id, + job.partitions(0), + taskAttemptId = 0, + attemptNumber = 0, + taskMemoryManager = taskMemoryManager, + runningLocally = true) + TaskContext.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() - TaskContextHelper.unset() + TaskContext.unset() + // Note: this memory freeing logic is duplicated in Executor.run(); when changing this, + // make sure to update both copies. + val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() + if (freedMemory > 0) { + if (sc.getConf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { + throw new SparkException(s"Managed memory leak detected; size = $freedMemory bytes") + } else { + logError(s"Managed memory leak detected; size = $freedMemory bytes") + } + } } } catch { case e: Exception => @@ -674,8 +708,11 @@ class DAGScheduler( private[scheduler] def handleJobGroupCancelled(groupId: String) { // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter(activeJob => - groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val activeInGroup = activeJobs.filter { activeJob => + Option(activeJob.properties).exists { + _.getProperty(SparkContext.SPARK_JOB_GROUP_ID) == groupId + } + } val jobIds = activeInGroup.map(_.jobId) jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) submitWaitingStages() @@ -702,9 +739,10 @@ class DAGScheduler( // cancelling the stages because if the DAG scheduler is stopped, the entire application // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" - runningStages.foreach { stage => - stage.latestInfo.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + // The `toArray` here is necessary so that we don't iterate over `runningStages` while + // mutating it. + runningStages.toArray.foreach { stage => + markStageAsFinished(stage, Some(stageFailedMessage)) } listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error))) } @@ -722,13 +760,12 @@ class DAGScheduler( allowLocal: Boolean, callSite: CallSite, listener: JobListener, - properties: Properties = null) - { - var finalStage: Stage = null + properties: Properties) { + var finalStage: ResultStage = null try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = newStage(finalRDD, partitions.size, None, jobId, callSite) + finalStage = newResultStage(finalRDD, partitions.size, jobId, callSite) } catch { case e: Exception => logWarning("Creating new stage failed due to exception - job: " + jobId, e) @@ -773,7 +810,7 @@ class DAGScheduler( if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) - if (missing == Nil) { + if (missing.isEmpty) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) } else { @@ -794,22 +831,19 @@ class DAGScheduler( // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() + // First figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = { - if (stage.isShuffleMap) { - (0 until stage.numPartitions).filter(id => stage.outputLocs(id) == Nil) - } else { - val job = stage.resultOfJob.get - (0 until job.numPartitions).filter(id => !job.finished(id)) + stage match { + case stage: ShuffleMapStage => + (0 until stage.numPartitions).filter(id => stage.outputLocs(id).isEmpty) + case stage: ResultStage => + val job = stage.resultOfJob.get + (0 until job.numPartitions).filter(id => !job.finished(id)) } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } + val properties = jobIdToActiveJob.get(stage.jobId).map(_.properties).orNull runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are @@ -830,18 +864,21 @@ class DAGScheduler( try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). - val taskBinaryBytes: Array[Byte] = - if (stage.isShuffleMap) { - closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() - } else { - closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() - } + val taskBinaryBytes: Array[Byte] = stage match { + case stage: ShuffleMapStage => + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef).array() + case stage: ResultStage => + closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func): AnyRef).array() + } + taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage + + // Abort execution return case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") @@ -849,20 +886,22 @@ class DAGScheduler( return } - val tasks: Seq[Task[_]] = if (stage.isShuffleMap) { - partitionsToCompute.map { id => - val locs = getPreferredLocs(stage.rdd, id) - val part = stage.rdd.partitions(id) - new ShuffleMapTask(stage.id, taskBinary, part, locs) - } - } else { - val job = stage.resultOfJob.get - partitionsToCompute.map { id => - val p: Int = job.partitions(id) - val part = stage.rdd.partitions(p) - val locs = getPreferredLocs(stage.rdd, p) - new ResultTask(stage.id, taskBinary, part, locs, id) - } + val tasks: Seq[Task[_]] = stage match { + case stage: ShuffleMapStage => + partitionsToCompute.map { id => + val locs = getPreferredLocs(stage.rdd, id) + val part = stage.rdd.partitions(id) + new ShuffleMapTask(stage.id, taskBinary, part, locs) + } + + case stage: ResultStage => + val job = stage.resultOfJob.get + partitionsToCompute.map { id => + val p: Int = job.partitions(id) + val part = stage.rdd.partitions(p) + val locs = getPreferredLocs(stage.rdd, p) + new ResultTask(stage.id, taskBinary, part, locs, id) + } } if (tasks.size > 0) { @@ -873,13 +912,20 @@ class DAGScheduler( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } else { - // Because we posted SparkListenerStageSubmitted earlier, we should post - // SparkListenerStageCompleted here in case there are no tasks to run. - outputCommitCoordinator.stageEnd(stage.id) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) - logDebug("Stage " + stage + " is actually done; %b %d %d".format( - stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) - runningStages -= stage + // Because we posted SparkListenerStageSubmitted earlier, we should mark + // the stage as completed here in case there are no tasks to run + markStageAsFinished(stage, None) + + val debugString = stage match { + case stage: ShuffleMapStage => + s"Stage ${stage} is actually done; " + + s"(available: ${stage.isAvailable}," + + s"available outputs: ${stage.numAvailableOutputs}," + + s"partitions: ${stage.numPartitions})" + case stage : ResultStage => + s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" + } + logDebug(debugString) } } @@ -945,22 +991,6 @@ class DAGScheduler( } val stage = stageIdToStage(task.stageId) - - def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { - val serviceTime = stage.latestInfo.submissionTime match { - case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) - case _ => "Unknown" - } - if (errorMessage.isEmpty) { - logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stage.latestInfo.completionTime = Some(clock.getTimeMillis()) - } else { - stage.latestInfo.stageFailed(errorMessage.get) - logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) - } - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) - runningStages -= stage - } event.reason match { case Success => listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, @@ -968,7 +998,10 @@ class DAGScheduler( stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => - stage.resultOfJob match { + // Cast to ResultStage here because it's part of the ResultTask + // TODO Refactor this out to a function that accepts a ResultStage + val resultStage = stage.asInstanceOf[ResultStage] + resultStage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { updateAccumulators(event) @@ -976,7 +1009,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - markStageAsFinished(stage) + markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) listenerBus.post( SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) @@ -988,7 +1021,7 @@ class DAGScheduler( job.listener.taskSucceeded(rt.outputId, event.result) } catch { case e: Exception => - // TODO: Perhaps we want to mark the stage as failed? + // TODO: Perhaps we want to mark the resultStage as failed? job.listener.jobFailed(new SparkDriverExecutionException(e)) } } @@ -997,6 +1030,7 @@ class DAGScheduler( } case smt: ShuffleMapTask => + val shuffleStage = stage.asInstanceOf[ShuffleMapStage] updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId @@ -1004,50 +1038,54 @@ class DAGScheduler( if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { - stage.addOutputLoc(smt.partitionId, status) + shuffleStage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(stage) && stage.pendingTasks.isEmpty) { - markStageAsFinished(stage) + if (runningStages.contains(shuffleStage) && shuffleStage.pendingTasks.isEmpty) { + markStageAsFinished(shuffleStage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) logInfo("waiting: " + waitingStages) logInfo("failed: " + failedStages) - if (stage.shuffleDep.isDefined) { - // We supply true to increment the epoch number here in case this is a - // recomputation of the map outputs. In that case, some nodes may have cached - // locations with holes (from when we detected the error) and will need the - // epoch incremented to refetch them. - // TODO: Only increment the epoch number if this is not the first time - // we registered these map outputs. - mapOutputTracker.registerMapOutputs( - stage.shuffleDep.get.shuffleId, - stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeEpoch = true) - } + + // We supply true to increment the epoch number here in case this is a + // recomputation of the map outputs. In that case, some nodes may have cached + // locations with holes (from when we detected the error) and will need the + // epoch incremented to refetch them. + // TODO: Only increment the epoch number if this is not the first time + // we registered these map outputs. + mapOutputTracker.registerMapOutputs( + shuffleStage.shuffleDep.shuffleId, + shuffleStage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, + changeEpoch = true) + clearCacheLocs() - if (stage.outputLocs.exists(_ == Nil)) { - // Some tasks had failed; let's resubmit this stage + if (shuffleStage.outputLocs.contains(Nil)) { + // Some tasks had failed; let's resubmit this shuffleStage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " (" + stage.name + + logInfo("Resubmitting " + shuffleStage + " (" + shuffleStage.name + ") because some of its tasks had failed: " + - stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) - submitStage(stage) + shuffleStage.outputLocs.zipWithIndex.filter(_._1.isEmpty) + .map(_._2).mkString(", ")) + submitStage(shuffleStage) } else { val newlyRunnable = new ArrayBuffer[Stage] - for (stage <- waitingStages) { - logInfo("Missing parents for " + stage + ": " + getMissingParentStages(stage)) + for (shuffleStage <- waitingStages) { + logInfo("Missing parents for " + shuffleStage + ": " + + getMissingParentStages(shuffleStage)) } - for (stage <- waitingStages if getMissingParentStages(stage) == Nil) { - newlyRunnable += stage + for (shuffleStage <- waitingStages if getMissingParentStages(shuffleStage).isEmpty) + { + newlyRunnable += shuffleStage } waitingStages --= newlyRunnable runningStages ++= newlyRunnable for { - stage <- newlyRunnable.sortBy(_.id) - jobId <- activeJobForStage(stage) + shuffleStage <- newlyRunnable.sortBy(_.id) + jobId <- activeJobForStage(shuffleStage) } { - logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable") - submitMissingTasks(stage, jobId) + logInfo("Submitting " + shuffleStage + " (" + + shuffleStage.rdd + "), which is now runnable") + submitMissingTasks(shuffleStage, jobId) } } } @@ -1068,7 +1106,6 @@ class DAGScheduler( logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + s"due to a fetch failure from $mapStage (${mapStage.name})") markStageAsFinished(failedStage, Some(failureMessage)) - runningStages -= failedStage } if (disallowStageRetryForTest) { @@ -1184,6 +1221,26 @@ class DAGScheduler( submitWaitingStages() } + /** + * Marks a stage as finished and removes it from the list of running stages. + */ + private def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None): Unit = { + val serviceTime = stage.latestInfo.submissionTime match { + case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0) + case _ => "Unknown" + } + if (errorMessage.isEmpty) { + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) + stage.latestInfo.completionTime = Some(clock.getTimeMillis()) + } else { + stage.latestInfo.stageFailed(errorMessage.get) + logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + } + outputCommitCoordinator.stageEnd(stage.id) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + runningStages -= stage + } + /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. @@ -1204,9 +1261,7 @@ class DAGScheduler( } } - /** - * Fails a job and all stages that are only used by that job, and cleans up relevant state. - */ + /** Fails a job and all stages that are only used by that job, and cleans up relevant state. */ private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { val error = new SparkException(failureReason) var ableToCancelStages = true @@ -1235,8 +1290,7 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - stage.latestInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) + markStageAsFinished(stage, Some(failureReason)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) @@ -1254,9 +1308,7 @@ class DAGScheduler( } } - /** - * Return true if one of stage's ancestors is target. - */ + /** Return true if one of stage's ancestors is target. */ private def stageDependsOn(stage: Stage, target: Stage): Boolean = { if (stage == target) { return true @@ -1282,7 +1334,7 @@ class DAGScheduler( } } waitingForVisit.push(stage.rdd) - while (!waitingForVisit.isEmpty) { + while (waitingForVisit.nonEmpty) { visit(waitingForVisit.pop()) } visitedRdds.contains(target.rdd) @@ -1312,9 +1364,7 @@ class DAGScheduler( private def getPreferredLocsInternal( rdd: RDD[_], partition: Int, - visited: HashSet[(RDD[_],Int)]) - : Seq[TaskLocation] = - { + visited: HashSet[(RDD[_],Int)]): Seq[TaskLocation] = { // If the partition has already been visited, no need to re-visit. // This avoids exponential path exploration. SPARK-695 if (!visited.add((rdd,partition))) { @@ -1323,12 +1373,12 @@ class DAGScheduler( } // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) - if (!cached.isEmpty) { + if (cached.nonEmpty) { return cached } // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList - if (!rddPrefs.isEmpty) { + if (rddPrefs.nonEmpty) { return rddPrefs.map(TaskLocation(_)) } // If the RDD has narrow dependencies, pick the first partition of the first narrow dep @@ -1349,6 +1399,7 @@ class DAGScheduler( def stop() { logInfo("Stopping DAGScheduler") + messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() } @@ -1412,7 +1463,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler dagScheduler.sc.stop() } - override def onStop() { + override def onStop(): Unit = { // Cancel any active jobs in postStop hook dagScheduler.cleanUpAfterSchedulerStop() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 34fa6d27c3a45..529a5b2bf1a0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -47,21 +47,23 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ private[spark] class EventLoggingListener( appId: String, - logBaseDir: String, + appAttemptId : Option[String], + logBaseDir: URI, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appId: String, logBaseDir: String, sparkConf: SparkConf) = - this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: String, appAttemptId : Option[String], logBaseDir: URI, sparkConf: SparkConf) = + this(appId, appAttemptId, logBaseDir, sparkConf, + SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val fileSystem = Utils.getHadoopFileSystem(new URI(logBaseDir), hadoopConf) + private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) private val compressionCodec = if (shouldCompress) { Some(CompressionCodec.createCodec(sparkConf)) @@ -89,7 +91,7 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, compressionCodecName) + private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) /** * Creates the log file in the configured log directory. @@ -149,47 +151,60 @@ private[spark] class EventLoggingListener( } // Events that do not trigger a flush - override def onStageSubmitted(event: SparkListenerStageSubmitted) = - logEvent(event) - override def onTaskStart(event: SparkListenerTaskStart) = - logEvent(event) - override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = - logEvent(event) - override def onTaskEnd(event: SparkListenerTaskEnd) = - logEvent(event) - override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = - logEvent(event) + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event) + + override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) + + override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = logEvent(event) + + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = logEvent(event) + + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = logEvent(event) // Events that trigger a flush - override def onStageCompleted(event: SparkListenerStageCompleted) = + override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { logEvent(event, flushLogger = true) - override def onJobStart(event: SparkListenerJobStart) = - logEvent(event, flushLogger = true) - override def onJobEnd(event: SparkListenerJobEnd) = - logEvent(event, flushLogger = true) - override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) = + } + + override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true) + + override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true) + + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = { logEvent(event, flushLogger = true) - override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved) = + } + + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { logEvent(event, flushLogger = true) - override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = + } + + override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { logEvent(event, flushLogger = true) - override def onApplicationStart(event: SparkListenerApplicationStart) = + } + + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { logEvent(event, flushLogger = true) - override def onApplicationEnd(event: SparkListenerApplicationEnd) = + } + + override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { logEvent(event, flushLogger = true) - override def onExecutorAdded(event: SparkListenerExecutorAdded) = + } + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) - override def onExecutorRemoved(event: SparkListenerExecutorRemoved) = + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) + } // No-op because logging every update would be overkill - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } /** * Stop logging events. The event log file will be renamed so that it loses the * ".inprogress" suffix. */ - def stop() = { + def stop(): Unit = { writer.foreach(_.close()) val target = new Path(logPath) @@ -239,20 +254,32 @@ private[spark] object EventLoggingListener extends Logging { * we won't know which codec to use to decompress the metadata needed to open the file in * the first place. * + * The log file name will identify the compression codec used for the contents, if any. + * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. + * * @param logBaseDir Directory where the log file will be written. * @param appId A unique app ID. + * @param appAttemptId A unique attempt id of appId. May be the empty string. * @param compressionCodecName Name to identify the codec used to compress the contents * of the log, or None if compression is not enabled. * @return A path which consists of file-system-safe characters. */ def getLogPath( - logBaseDir: String, + logBaseDir: URI, appId: String, + appAttemptId: Option[String], compressionCodecName: Option[String] = None): String = { - val sanitizedAppId = appId.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase - // e.g. app_123, app_123.lzf - val logName = sanitizedAppId + compressionCodecName.map { "." + _ }.getOrElse("") - Utils.resolveURI(logBaseDir).toString.stripSuffix("/") + "/" + logName + val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) + val codec = compressionCodecName.map("." + _).getOrElse("") + if (appAttemptId.isDefined) { + base + "_" + sanitize(appAttemptId.get) + codec + } else { + base + codec + } + } + + private def sanitize(str: String): String = { + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 8aa528ac573d0..e55b76c36cc5f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -57,7 +57,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener private val stageIdToJobId = new HashMap[Int, Int] private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } createLogDir() diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 29879b374b801..382b09422a4a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -34,7 +34,7 @@ private[spark] class JobWaiter[T]( @volatile private var _jobFinished = totalTasks == 0 - def jobFinished = _jobFinished + def jobFinished: Boolean = _jobFinished // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero // partition RDDs), we set the jobResult directly to JobSucceeded. diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 759df023a6dcf..0b1d47cff3746 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -19,10 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable -import akka.actor.{ActorRef, Actor} - import org.apache.spark._ -import org.apache.spark.util.{AkkaUtils, ActorLogReceive} +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, RpcEndpoint} private sealed trait OutputCommitCoordinationMessage extends Serializable @@ -34,8 +32,8 @@ private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttem * policy. * * OutputCommitCoordinator is instantiated in both the drivers and executors. On executors, it is - * configured with a reference to the driver's OutputCommitCoordinatorActor, so requests to commit - * output will be forwarded to the driver's OutputCommitCoordinator. + * configured with a reference to the driver's OutputCommitCoordinatorEndpoint, so requests to + * commit output will be forwarded to the driver's OutputCommitCoordinator. * * This class was introduced in SPARK-4879; see that JIRA issue (and the associated pull requests) * for an extensive design discussion. @@ -43,10 +41,7 @@ private case class AskPermissionToCommitOutput(stage: Int, task: Long, taskAttem private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { // Initialized by SparkEnv - var coordinatorActor: Option[ActorRef] = None - private val timeout = AkkaUtils.askTimeout(conf) - private val maxAttempts = AkkaUtils.numRetries(conf) - private val retryInterval = AkkaUtils.retryWaitMs(conf) + var coordinatorRef: Option[RpcEndpointRef] = None private type StageId = Int private type PartitionId = Long @@ -64,6 +59,13 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { private val authorizedCommittersByStage: CommittersByStageMap = mutable.Map() private type CommittersByStageMap = mutable.Map[StageId, mutable.Map[PartitionId, TaskAttemptId]] + /** + * Returns whether the OutputCommitCoordinator's internal data structures are all empty. + */ + def isEmpty: Boolean = { + authorizedCommittersByStage.isEmpty + } + /** * Called by tasks to ask whether they can commit their output to HDFS. * @@ -81,9 +83,9 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { partition: PartitionId, attempt: TaskAttemptId): Boolean = { val msg = AskPermissionToCommitOutput(stage, partition, attempt) - coordinatorActor match { - case Some(actor) => - AkkaUtils.askWithReply[Boolean](msg, actor, maxAttempts, retryInterval, timeout) + coordinatorRef match { + case Some(endpointRef) => + endpointRef.askWithRetry[Boolean](msg) case None => logError( "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?") @@ -118,15 +120,17 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { logInfo( s"Task was denied committing, stage: $stage, partition: $partition, attempt: $attempt") case otherReason => - logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" + - s" clearing lock") - authorizedCommitters.remove(partition) + if (authorizedCommitters.get(partition).exists(_ == attempt)) { + logDebug(s"Authorized committer $attempt (stage=$stage, partition=$partition) failed;" + + s" clearing lock") + authorizedCommitters.remove(partition) + } } } def stop(): Unit = synchronized { - coordinatorActor.foreach(_ ! StopCoordinator) - coordinatorActor = None + coordinatorRef.foreach(_ send StopCoordinator) + coordinatorRef = None authorizedCommittersByStage.clear() } @@ -157,16 +161,20 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging { private[spark] object OutputCommitCoordinator { // This actor is used only for RPC - class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator) - extends Actor with ActorLogReceive with Logging { + private[spark] class OutputCommitCoordinatorEndpoint( + override val rpcEnv: RpcEnv, outputCommitCoordinator: OutputCommitCoordinator) + extends RpcEndpoint with Logging { - override def receiveWithLogging = { - case AskPermissionToCommitOutput(stage, partition, taskAttempt) => - sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt) + override def receive: PartialFunction[Any, Unit] = { case StopCoordinator => logInfo("OutputCommitCoordinator stopped!") - context.stop(self) - sender ! true + stop() + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case AskPermissionToCommitOutput(stage, partition, taskAttempt) => + context.reply( + outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt)) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala new file mode 100644 index 0000000000000..c0f3d5a13d623 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -0,0 +1,40 @@ +/* + * 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.scheduler + +import org.apache.spark.rdd.RDD +import org.apache.spark.util.CallSite + +/** + * The ResultStage represents the final stage in a job. + */ +private[spark] class ResultStage( + id: Int, + rdd: RDD[_], + numTasks: Int, + parents: List[Stage], + jobId: Int, + callSite: CallSite) + extends Stage(id, rdd, numTasks, parents, jobId, callSite) { + + // The active job for this result stage. Will be empty if the job has already finished + // (e.g., because the job was cancelled). + var resultOfJob: Option[ActiveJob] = None + + override def toString: String = "ResultStage " + id +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 4a9ff918afe25..c9a124113961f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -53,9 +53,11 @@ private[spark] class ResultTask[T, U]( override def runTask(context: TaskContext): U = { // Deserialize the RDD and the func using the broadcast variables. + val deserializeStartTime = System.currentTimeMillis() val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime metrics = Some(context.taskMetrics) func(context, rdd.iterator(partition, context)) @@ -64,5 +66,5 @@ private[spark] class ResultTask[T, U]( // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs - override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" + override def toString: String = "ResultTask(" + stageId + ", " + partitionId + ")" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 992c477493d8e..646820520ea1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -41,4 +41,12 @@ private[spark] trait SchedulerBackend { */ def applicationId(): String = appId + /** + * Get the attempt ID for this run, if the cluster manager supports multiple + * attempts. Applications run in client mode will not have attempt IDs. + * + * @return The application attempt id, if available. + */ + def applicationAttemptId(): Option[String] = None + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala new file mode 100644 index 0000000000000..d02210743484c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -0,0 +1,84 @@ +/* + * 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.scheduler + +import org.apache.spark.ShuffleDependency +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.CallSite + +/** + * The ShuffleMapStage represents the intermediate stages in a job. + */ +private[spark] class ShuffleMapStage( + id: Int, + rdd: RDD[_], + numTasks: Int, + parents: List[Stage], + jobId: Int, + callSite: CallSite, + val shuffleDep: ShuffleDependency[_, _, _]) + extends Stage(id, rdd, numTasks, parents, jobId, callSite) { + + override def toString: String = "ShuffleMapStage " + id + + var numAvailableOutputs: Long = 0 + + def isAvailable: Boolean = numAvailableOutputs == numPartitions + + val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) + + def addOutputLoc(partition: Int, status: MapStatus): Unit = { + val prevList = outputLocs(partition) + outputLocs(partition) = status :: prevList + if (prevList == Nil) { + numAvailableOutputs += 1 + } + } + + def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { + val prevList = outputLocs(partition) + val newList = prevList.filterNot(_.location == bmAddress) + outputLocs(partition) = newList + if (prevList != Nil && newList == Nil) { + numAvailableOutputs -= 1 + } + } + + /** + * Removes all shuffle outputs associated with this executor. Note that this will also remove + * outputs which are served by an external shuffle server (if one exists), as they are still + * registered with this execId. + */ + def removeOutputsOnExecutor(execId: String): Unit = { + var becameUnavailable = false + for (partition <- 0 until numPartitions) { + val prevList = outputLocs(partition) + val newList = prevList.filterNot(_.location.executorId == execId) + outputLocs(partition) = newList + if (prevList != Nil && newList == Nil) { + becameUnavailable = true + numAvailableOutputs -= 1 + } + } + if (becameUnavailable) { + logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( + this, execId, numAvailableOutputs, numPartitions, isAvailable)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 79709089c0da4..bd3dd23dfe1ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -33,7 +33,7 @@ import org.apache.spark.shuffle.ShuffleWriter * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param taskBinary broadcast version of of the RDD and the ShuffleDependency. Once deserialized, + * @param taskBinary broadcast version of the RDD and the ShuffleDependency. Once deserialized, * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling @@ -47,7 +47,7 @@ private[spark] class ShuffleMapTask( /** A constructor used only in test suites. This does not require passing in an RDD. */ def this(partitionId: Int) { - this(0, null, new Partition { override def index = 0 }, null) + this(0, null, new Partition { override def index: Int = 0 }, null) } @transient private val preferredLocs: Seq[TaskLocation] = { @@ -56,9 +56,11 @@ private[spark] class ShuffleMapTask( override def runTask(context: TaskContext): MapStatus = { // Deserialize the RDD using the broadcast variable. + val deserializeStartTime = System.currentTimeMillis() val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null @@ -83,5 +85,5 @@ private[spark] class ShuffleMapTask( override def preferredLocations: Seq[TaskLocation] = preferredLocs - override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partitionId) + override def toString: String = "ShuffleMapTask(%d, %d)".format(stageId, partitionId) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 52720d48ca67f..169d4fd3a94f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -110,8 +110,8 @@ case class SparkListenerExecutorMetricsUpdate( extends SparkListenerEvent @DeveloperApi -case class SparkListenerApplicationStart(appName: String, appId: Option[String], time: Long, - sparkUser: String) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, appId: Option[String], + time: Long, sparkUser: String, appAttemptId: Option[String]) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent @@ -300,7 +300,7 @@ private[spark] object StatsReportListener extends Logging { } def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { - def f(d: Double) = format.format(d) + def f(d: Double): String = format.format(d) showDistribution(heading, dOpt, f _) } @@ -346,7 +346,7 @@ private[spark] object StatsReportListener extends Logging { /** * Reformat a time interval in milliseconds to a prettier format for output */ - def millisToString(ms: Long) = { + def millisToString(ms: Long): String = { val (size, units) = if (ms > hours) { (ms.toDouble / hours, "hours") diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index cc13f57a49b89..5d0ddb8377c33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -21,7 +21,6 @@ import scala.collection.mutable.HashSet import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.CallSite /** @@ -47,29 +46,23 @@ import org.apache.spark.util.CallSite * be updated for each attempt. * */ -private[spark] class Stage( +private[spark] abstract class Stage( val id: Int, val rdd: RDD[_], val numTasks: Int, - val shuffleDep: Option[ShuffleDependency[_, _, _]], // Output shuffle if stage is a map stage val parents: List[Stage], val jobId: Int, val callSite: CallSite) extends Logging { - val isShuffleMap = shuffleDep.isDefined val numPartitions = rdd.partitions.size - val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) - var numAvailableOutputs = 0 /** Set of jobs that this stage belongs to. */ val jobIds = new HashSet[Int] - /** For stages that are the final (consists of only ResultTasks), link to the ActiveJob. */ - var resultOfJob: Option[ActiveJob] = None var pendingTasks = new HashSet[Task[_]] - private var nextAttemptId = 0 + private var nextAttemptId: Int = 0 val name = callSite.shortForm val details = callSite.longForm @@ -77,53 +70,6 @@ private[spark] class Stage( /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ var latestInfo: StageInfo = StageInfo.fromStage(this) - def isAvailable: Boolean = { - if (!isShuffleMap) { - true - } else { - numAvailableOutputs == numPartitions - } - } - - def addOutputLoc(partition: Int, status: MapStatus) { - val prevList = outputLocs(partition) - outputLocs(partition) = status :: prevList - if (prevList == Nil) { - numAvailableOutputs += 1 - } - } - - def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location == bmAddress) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - numAvailableOutputs -= 1 - } - } - - /** - * Removes all shuffle outputs associated with this executor. Note that this will also remove - * outputs which are served by an external shuffle server (if one exists), as they are still - * registered with this execId. - */ - def removeOutputsOnExecutor(execId: String) { - var becameUnavailable = false - for (partition <- 0 until numPartitions) { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location.executorId == execId) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - becameUnavailable = true - numAvailableOutputs -= 1 - } - } - if (becameUnavailable) { - logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( - this, execId, numAvailableOutputs, numPartitions, isAvailable)) - } - } - /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { val id = nextAttemptId @@ -133,11 +79,8 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - override def toString = "Stage " + id - - override def hashCode(): Int = id - - override def equals(other: Any): Boolean = other match { + override final def hashCode(): Int = id + override final def equals(other: Any): Boolean = other match { case stage: Stage => stage != null && stage.id == id case _ => false } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c6dc3369ba5cc..e439d2a7e1229 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -33,6 +33,7 @@ class StageInfo( val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo], + val parentIds: Seq[Int], val details: String) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None @@ -47,6 +48,18 @@ class StageInfo( failureReason = Some(reason) completionTime = Some(System.currentTimeMillis) } + + private[spark] def getStatusString: String = { + if (completionTime.isDefined) { + if (failureReason.isDefined) { + "failed" + } else { + "succeeded" + } + } else { + "running" + } + } } private[spark] object StageInfo { @@ -66,6 +79,7 @@ private[spark] object StageInfo { stage.name, numTasks.getOrElse(stage.numTasks), rddInfos, + stage.parents.map(_.id), stage.details) } } 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 4d9f940813b8e..586d1e06204c1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,9 +22,10 @@ import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import org.apache.spark.{TaskContextHelper, TaskContextImpl, TaskContext} +import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.unsafe.memory.TaskMemoryManager import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils @@ -52,9 +53,14 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex * @return the result of the task */ final def run(taskAttemptId: Long, attemptNumber: Int): T = { - context = new TaskContextImpl(stageId = stageId, partitionId = partitionId, - taskAttemptId = taskAttemptId, attemptNumber = attemptNumber, runningLocally = false) - TaskContextHelper.setTaskContext(context) + context = new TaskContextImpl( + stageId = stageId, + partitionId = partitionId, + taskAttemptId = taskAttemptId, + attemptNumber = attemptNumber, + taskMemoryManager = taskMemoryManager, + runningLocally = false) + TaskContext.setTaskContext(context) context.taskMetrics.setHostname(Utils.localHostName()) taskThread = Thread.currentThread() if (_killed) { @@ -64,10 +70,16 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex runTask(context) } finally { context.markTaskCompleted() - TaskContextHelper.unset() + TaskContext.unset() } } + private var taskMemoryManager: TaskMemoryManager = _ + + def setTaskMemoryManager(taskMemoryManager: TaskMemoryManager): Unit = { + this.taskMemoryManager = taskMemoryManager + } + def runTask(context: TaskContext): T def preferredLocations: Seq[TaskLocation] = Nil @@ -87,11 +99,18 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // initialized when kill() is invoked. @volatile @transient private var _killed = false + protected var _executorDeserializeTime: Long = 0 + /** * Whether the task has been killed. */ def killed: Boolean = _killed + /** + * Returns the amount of time spent deserializing the RDD and function to be run. + */ + def executorDeserializeTime: Long = _executorDeserializeTime + /** * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark * code and user code to properly handle the flag. This function should be idempotent so it can diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6fa1f2c880f7a..132a9ced77700 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -81,9 +81,11 @@ class TaskInfo( def status: String = { if (running) { - "RUNNING" - } else if (gettingResult) { - "GET RESULT" + if (gettingResult) { + "GET RESULT" + } else { + "RUNNING" + } } else if (failed) { "FAILED" } else if (successful) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 10c685f29d3ac..da07ce2c6ea49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -29,23 +29,22 @@ private[spark] sealed trait TaskLocation { /** * A location that includes both a host and an executor id on that host. */ -private [spark] case class ExecutorCacheTaskLocation(override val host: String, - val executorId: String) extends TaskLocation { -} +private [spark] +case class ExecutorCacheTaskLocation(override val host: String, executorId: String) + extends TaskLocation /** * A location on a host. */ private [spark] case class HostTaskLocation(override val host: String) extends TaskLocation { - override def toString = host + override def toString: String = host } /** * A location on a host that is cached by HDFS. */ -private [spark] case class HDFSCacheTaskLocation(override val host: String) - extends TaskLocation { - override def toString = TaskLocation.inMemoryLocationTag + host +private [spark] case class HDFSCacheTaskLocation(override val host: String) extends TaskLocation { + override def toString: String = TaskLocation.inMemoryLocationTag + host } private[spark] object TaskLocation { @@ -54,14 +53,16 @@ private[spark] object TaskLocation { // confusion. See RFC 952 and RFC 1123 for information about the format of hostnames. val inMemoryLocationTag = "hdfs_cache_" - def apply(host: String, executorId: String) = new ExecutorCacheTaskLocation(host, executorId) + def apply(host: String, executorId: String): TaskLocation = { + new ExecutorCacheTaskLocation(host, executorId) + } /** * Create a TaskLocation from a string returned by getPreferredLocations. * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the * location is cached. */ - def apply(str: String) = { + def apply(str: String): TaskLocation = { val hstr = str.stripPrefix(inMemoryLocationTag) if (hstr.equals(str)) { new HostTaskLocation(str) 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 3938580aeea59..391827c1d2156 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. @@ -35,7 +35,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul extends Logging { private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) - private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( + private val getTaskResultExecutor = ThreadUtils.newDaemonFixedThreadPool( THREADS, "task-result-getter") protected val serializer = new ThreadLocal[SerializerInstance] { 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 ed3418676e077..f25f3ed0d9037 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -73,9 +73,17 @@ private[spark] trait TaskScheduler { * @return An application ID */ def applicationId(): String = appId - + /** * Process a lost executor */ def executorLost(executorId: String, reason: ExecutorLossReason): Unit + + /** + * Get an application's attempt ID associated with the job. + * + * @return An application's Attempt ID + */ + def applicationAttemptId(): Option[String] + } 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 f33fd4450b2a6..b4b8a630694bb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,10 +62,10 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) + val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) @@ -142,11 +142,10 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") - import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, + SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } - } + }(sc.env.actorSystem.dispatcher) } } @@ -173,7 +172,7 @@ private[spark] class TaskSchedulerImpl( this.cancel() } } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + }, STARVATION_TIMEOUT_MS, STARVATION_TIMEOUT_MS) } hasReceivedTask = true } @@ -373,17 +372,17 @@ private[spark] class TaskSchedulerImpl( } def handleSuccessfulTask( - taskSetManager: TaskSetManager, - tid: Long, - taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager: TaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]): Unit = synchronized { taskSetManager.handleSuccessfulTask(tid, taskResult) } def handleFailedTask( - taskSetManager: TaskSetManager, - tid: Long, - taskState: TaskState, - reason: TaskEndReason) = synchronized { + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: TaskEndReason): Unit = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { // Need to revive offers again now that the task set manager state has been updated to @@ -394,7 +393,7 @@ private[spark] class TaskSchedulerImpl( def error(message: String) { synchronized { - if (activeTaskSets.size > 0) { + if (activeTaskSets.nonEmpty) { // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { @@ -407,8 +406,7 @@ private[spark] class TaskSchedulerImpl( // No task sets are active but we still got an error. Just exit since this // must mean the error is during registration. // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) + throw new SparkException(s"Exiting due to error from cluster scheduler: $message") } } } @@ -423,7 +421,7 @@ private[spark] class TaskSchedulerImpl( starvationTimer.cancel() } - override def defaultParallelism() = backend.defaultParallelism() + override def defaultParallelism(): Int = backend.defaultParallelism() // Check for speculatable tasks in all our active jobs. def checkSpeculatableTasks() { @@ -515,6 +513,8 @@ private[spark] class TaskSchedulerImpl( override def applicationId(): String = backend.applicationId() + override def applicationAttemptId(): Option[String] = backend.applicationAttemptId() + } 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 529237f0d35dc..7dc325283d961 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.nio.ByteBuffer import java.util.Arrays +import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -29,6 +30,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -97,7 +99,8 @@ private[spark] class TaskSetManager( var calculatedTasks = 0 val runningTasksSet = new HashSet[Long] - override def runningTasks = runningTasksSet.size + + override def runningTasks: Int = runningTasksSet.size // True once no more tasks should be launched for this task set manager. TaskSetManagers enter // the zombie state once at least one attempt of each task has completed successfully, or if the @@ -168,9 +171,9 @@ private[spark] class TaskSetManager( var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level - override def schedulableQueue = null + override def schedulableQueue: ConcurrentLinkedQueue[Schedulable] = null - override def schedulingMode = SchedulingMode.NONE + override def schedulingMode: SchedulingMode = SchedulingMode.NONE var emittedTaskSizeWarning = false @@ -585,7 +588,7 @@ private[spark] class TaskSetManager( /** * Marks the task as getting result and notifies the DAG Scheduler */ - def handleTaskGettingResult(tid: Long) = { + def handleTaskGettingResult(tid: Long): Unit = { val info = taskInfos(tid) info.markGettingResult() sched.dagScheduler.taskGettingResult(info) @@ -612,7 +615,7 @@ private[spark] class TaskSetManager( /** * Marks the task as successful and notifies the DAGScheduler that a task has ended. */ - def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = { + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = { val info = taskInfos(tid) val index = info.index info.markSuccessful() @@ -845,15 +848,18 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000") - level match { - case TaskLocality.PROCESS_LOCAL => - conf.get("spark.locality.wait.process", defaultWait).toLong - case TaskLocality.NODE_LOCAL => - conf.get("spark.locality.wait.node", defaultWait).toLong - case TaskLocality.RACK_LOCAL => - conf.get("spark.locality.wait.rack", defaultWait).toLong - case _ => 0L + val defaultWait = conf.get("spark.locality.wait", "3s") + val localityWaitKey = level match { + case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" + case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" + case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + case _ => null + } + + if (localityWaitKey != null) { + conf.getTimeAsMs(localityWaitKey, defaultWait) + } else { + 0L } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9bf74f4be198d..70364cea62a80 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{SerializableBuffer, Utils} private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable @@ -41,6 +42,7 @@ private[spark] object CoarseGrainedClusterMessages { // Executors to driver case class RegisterExecutor( executorId: String, + executorRef: RpcEndpointRef, hostPort: String, cores: Int, logUrls: Map[String, String]) @@ -70,6 +72,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage + case class SetupDriver(driver: RpcEndpointRef) extends CoarseGrainedClusterMessage + // Exchanged between the driver and the AM in Yarn client mode case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String) extends CoarseGrainedClusterMessage @@ -77,7 +81,7 @@ private[spark] object CoarseGrainedClusterMessages { // Messages exchanged between the driver and the cluster manager for executor allocation // In Yarn mode, these are exchanged between the driver and the AM - case object RegisterClusterManager extends CoarseGrainedClusterMessage + case class RegisterClusterManager(am: RpcEndpointRef) extends CoarseGrainedClusterMessage // Request executors by specifying the new total number of executors desired // This includes executors already pending or running 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 87ebf31139ce9..f107148f3b8c6 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 @@ -17,20 +17,16 @@ package org.apache.spark.scheduler.cluster +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.Await -import scala.concurrent.duration._ - -import akka.actor._ -import akka.pattern.ask -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.spark.rpc._ import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.util.{ThreadUtils, SerializableBuffer, AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -41,7 +37,7 @@ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Ut * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -49,7 +45,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Total number of executors that are currently registered var totalRegisteredExecutors = new AtomicInteger(0) val conf = scheduler.sc.conf - private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. @@ -57,8 +52,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val maxRegisteredWaitingTimeMs = + conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -71,48 +66,28 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] - class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { + class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) + extends ThreadSafeRpcEndpoint with Logging { + override protected def log = CoarseGrainedSchedulerBackend.this.log - private val addressToExecutorId = new HashMap[Address, String] - override def preStart() { - // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + private val addressToExecutorId = new HashMap[RpcAddress, String] - // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) - import context.dispatcher - context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) - } - - def receiveWithLogging = { - case RegisterExecutor(executorId, hostPort, cores, logUrls) => - Utils.checkHostPort(hostPort, "Host port expected " + hostPort) - if (executorDataMap.contains(executorId)) { - sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) - } else { - logInfo("Registered executor: " + sender + " with ID " + executorId) - sender ! RegisteredExecutor + private val reviveThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") - addressToExecutorId(sender.path.address) = executorId - totalCoreCount.addAndGet(cores) - totalRegisteredExecutors.addAndGet(1) - val (host, _) = Utils.parseHostPort(hostPort) - val data = new ExecutorData(sender, sender.path.address, host, cores, cores, logUrls) - // This must be synchronized because variables mutated - // in this block are read when requesting executors - CoarseGrainedSchedulerBackend.this.synchronized { - executorDataMap.put(executorId, data) - if (numPendingExecutors > 0) { - numPendingExecutors -= 1 - logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") - } - } - listenerBus.post( - SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) - makeOffers() + override def onStart() { + // Periodically revive offers to allow delay scheduling to work + val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") + + reviveThread.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ReviveOffers)) } + }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) + } + override def receive: PartialFunction[Any, Unit] = { case StatusUpdate(executorId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { @@ -133,33 +108,58 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste case KillTask(taskId, executorId, interruptThread) => executorDataMap.get(executorId) match { case Some(executorInfo) => - executorInfo.executorActor ! KillTask(taskId, executorId, interruptThread) + executorInfo.executorEndpoint.send(KillTask(taskId, executorId, interruptThread)) case None => // Ignoring the task kill since the executor is not registered. logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") } + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterExecutor(executorId, executorRef, hostPort, cores, logUrls) => + Utils.checkHostPort(hostPort, "Host port expected " + hostPort) + if (executorDataMap.contains(executorId)) { + context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + } else { + logInfo("Registered executor: " + executorRef + " with ID " + executorId) + context.reply(RegisteredExecutor) + addressToExecutorId(executorRef.address) = executorId + totalCoreCount.addAndGet(cores) + totalRegisteredExecutors.addAndGet(1) + val (host, _) = Utils.parseHostPort(hostPort) + val data = new ExecutorData(executorRef, executorRef.address, host, cores, cores, logUrls) + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap.put(executorId, data) + if (numPendingExecutors > 0) { + numPendingExecutors -= 1 + logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") + } + } + listenerBus.post( + SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) + makeOffers() + } + case StopDriver => - sender ! true - context.stop(self) + context.reply(true) + stop() case StopExecutors => logInfo("Asking each executor to shut down") for ((_, executorData) <- executorDataMap) { - executorData.executorActor ! StopExecutor + executorData.executorEndpoint.send(StopExecutor) } - sender ! true + context.reply(true) case RemoveExecutor(executorId, reason) => removeExecutor(executorId, reason) - sender ! true - - case DisassociatedEvent(_, address, _) => - addressToExecutorId.get(address).foreach(removeExecutor(_, - "remote Akka client disassociated")) + context.reply(true) case RetrieveSparkProps => - sender ! sparkProperties + context.reply(sparkProperties) } // Make fake resource offers on all executors @@ -169,6 +169,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste }.toSeq)) } + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, + "remote Rpc client disassociated")) + } + // Make fake resource offers on just one executor def makeOffers(executorId: String) { val executorData = executorDataMap(executorId) @@ -199,7 +204,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK - executorData.executorActor ! LaunchTask(new SerializableBuffer(serializedTask)) + executorData.executorEndpoint.send(LaunchTask(new SerializableBuffer(serializedTask))) } } } @@ -223,9 +228,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste case None => logError(s"Asked to remove non-existent executor $executorId") } } + + override def onStop() { + reviveThread.shutdownNow() + } } - var driverActor: ActorRef = null + var driverEndpoint: RpcEndpointRef = null val taskIdsOnSlave = new HashMap[String, HashSet[String]] override def start() { @@ -235,17 +244,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste properties += ((key, value)) } } + // TODO (prashant) send conf instead of properties - driverActor = actorSystem.actorOf( - Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) + driverEndpoint = rpcEnv.setupEndpoint( + CoarseGrainedSchedulerBackend.ENDPOINT_NAME, new DriverEndpoint(rpcEnv, properties)) } def stopExecutors() { try { - if (driverActor != null) { + if (driverEndpoint != null) { logInfo("Shutting down all executors") - val future = driverActor.ask(StopExecutors)(timeout) - Await.ready(future, timeout) + driverEndpoint.askWithRetry[Boolean](StopExecutors) } } catch { case e: Exception => @@ -256,22 +265,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste override def stop() { stopExecutors() try { - if (driverActor != null) { - val future = driverActor.ask(StopDriver)(timeout) - Await.ready(future, timeout) + if (driverEndpoint != null) { + driverEndpoint.askWithRetry[Boolean](StopDriver) } } catch { case e: Exception => - throw new SparkException("Error stopping standalone scheduler's driver actor", e) + throw new SparkException("Error stopping standalone scheduler's driver endpoint", e) } } override def reviveOffers() { - driverActor ! ReviveOffers + driverEndpoint.send(ReviveOffers) } override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - driverActor ! KillTask(taskId, executorId, interruptThread) + driverEndpoint.send(KillTask(taskId, executorId, interruptThread)) } override def defaultParallelism(): Int = { @@ -281,11 +289,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Called by subclasses when notified of a lost worker def removeExecutor(executorId: String, reason: String) { try { - val future = driverActor.ask(RemoveExecutor(executorId, reason))(timeout) - Await.ready(future, timeout) + driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason)) } catch { case e: Exception => - throw new SparkException("Error notifying standalone scheduler's driver actor", e) + throw new SparkException("Error notifying standalone scheduler's driver endpoint", e) } } @@ -297,9 +304,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") return true } false @@ -391,5 +398,5 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } private[spark] object CoarseGrainedSchedulerBackend { - val ACTOR_NAME = "CoarseGrainedScheduler" + val ENDPOINT_NAME = "CoarseGrainedScheduler" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 5e571efe76720..26e72c0bff38d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,20 +17,20 @@ package org.apache.spark.scheduler.cluster -import akka.actor.{Address, ActorRef} +import org.apache.spark.rpc.{RpcEndpointRef, RpcAddress} /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. * - * @param executorActor The ActorRef representing this executor + * @param executorEndpoint The ActorRef representing this executor * @param executorAddress The network address of this executor * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor */ private[cluster] class ExecutorData( - val executorActor: ActorRef, - val executorAddress: Address, + val executorEndpoint: RpcEndpointRef, + val executorAddress: RpcAddress, override val executorHost: String, var freeCores: Int, override val totalCores: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 06786a59524e7..0324c9dab910b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -19,16 +19,16 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.spark.rpc.RpcAddress import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.AkkaUtils private[spark] class SimrSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, driverFilePath: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with Logging { val tmpPath = new Path(driverFilePath + "_tmp") @@ -39,12 +39,9 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), - SparkEnv.driverActorSystemName, - sc.conf.get("spark.driver.host"), - sc.conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, + RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) val fs = FileSystem.get(conf) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ffd4825705755..ccf1dc5af6120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,17 +19,18 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.Semaphore +import org.apache.spark.rpc.RpcAddress import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String]) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with AppClientListener with Logging { @@ -48,12 +49,9 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), - SparkEnv.driverActorSystemName, - conf.get("spark.driver.host"), - conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, + RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val args = Seq( "--driver-url", driverUrl, "--executor-id", "{{EXECUTOR_ID}}", @@ -84,12 +82,11 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, sc.eventLogDir, sc.eventLogCodec) - + val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, + command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() - waitForRegistration() } @@ -121,9 +118,12 @@ private[spark] class SparkDeploySchedulerBackend( notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) - scheduler.error(reason) - // Ensure the application terminates, as we can no longer run jobs. - sc.stop() + try { + scheduler.error(reason) + } finally { + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index f14aaeea0a25c..2a3a5d925d06f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -19,14 +19,12 @@ package org.apache.spark.scheduler.cluster import scala.concurrent.{Future, ExecutionContext} -import akka.actor.{Actor, ActorRef, Props} -import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} - -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{ThreadUtils, RpcUtils} import scala.util.control.NonFatal @@ -37,7 +35,7 @@ import scala.util.control.NonFatal private[spark] abstract class YarnSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 @@ -45,28 +43,24 @@ private[spark] abstract class YarnSchedulerBackend( protected var totalExpectedExecutors = 0 - private val yarnSchedulerActor: ActorRef = - actorSystem.actorOf( - Props(new YarnSchedulerActor), - name = YarnSchedulerBackend.ACTOR_NAME) + private val yarnSchedulerEndpoint = rpcEnv.setupEndpoint( + YarnSchedulerBackend.ENDPOINT_NAME, new YarnSchedulerEndpoint(rpcEnv)) - private implicit val askTimeout = AkkaUtils.askTimeout(sc.conf) + private implicit val askTimeout = RpcUtils.askTimeout(sc.conf) /** * Request executors from the ApplicationMaster by specifying the total number desired. * This includes executors already pending or running. */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - AkkaUtils.askWithReply[Boolean]( - RequestExecutors(requestedTotal), yarnSchedulerActor, askTimeout) + yarnSchedulerEndpoint.askWithRetry[Boolean](RequestExecutors(requestedTotal)) } /** * Request that the ApplicationMaster kill the specified executors. */ override def doKillExecutors(executorIds: Seq[String]): Boolean = { - AkkaUtils.askWithReply[Boolean]( - KillExecutors(executorIds), yarnSchedulerActor, askTimeout) + yarnSchedulerEndpoint.askWithRetry[Boolean](KillExecutors(executorIds)) } override def sufficientResourcesRegistered(): Boolean = { @@ -96,64 +90,71 @@ private[spark] abstract class YarnSchedulerBackend( } /** - * An actor that communicates with the ApplicationMaster. + * An [[RpcEndpoint]] that communicates with the ApplicationMaster. */ - private class YarnSchedulerActor extends Actor { - private var amActor: Option[ActorRef] = None + private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) + extends ThreadSafeRpcEndpoint with Logging { + private var amEndpoint: Option[RpcEndpointRef] = None - implicit val askAmActorExecutor = ExecutionContext.fromExecutor( - Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-executor")) + private val askAmThreadPool = + ThreadUtils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") + implicit val askAmExecutor = ExecutionContext.fromExecutor(askAmThreadPool) - override def preStart(): Unit = { - // Listen for disassociation events - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } + override def receive: PartialFunction[Any, Unit] = { + case RegisterClusterManager(am) => + logInfo(s"ApplicationMaster registered as $am") + amEndpoint = Some(am) + + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) - override def receive = { - case RegisterClusterManager => - logInfo(s"ApplicationMaster registered as $sender") - amActor = Some(sender) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case r: RequestExecutors => - amActor match { - case Some(actor) => - val driverActor = sender + amEndpoint match { + case Some(am) => Future { - driverActor ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + context.reply(am.askWithRetry[Boolean](r)) } onFailure { - case NonFatal(e) => logError(s"Sending $r to AM was unsuccessful", e) + case NonFatal(e) => + logError(s"Sending $r to AM was unsuccessful", e) + context.sendFailure(e) } case None => logWarning("Attempted to request executors before the AM has registered!") - sender ! false + context.reply(false) } case k: KillExecutors => - amActor match { - case Some(actor) => - val driverActor = sender + amEndpoint match { + case Some(am) => Future { - driverActor ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + context.reply(am.askWithRetry[Boolean](k)) } onFailure { - case NonFatal(e) => logError(s"Sending $k to AM was unsuccessful", e) + case NonFatal(e) => + logError(s"Sending $k to AM was unsuccessful", e) + context.sendFailure(e) } case None => logWarning("Attempted to kill executors before the AM has registered!") - sender ! false + context.reply(false) } - case AddWebUIFilter(filterName, filterParams, proxyBase) => - addWebUIFilter(filterName, filterParams, proxyBase) - sender ! true + } - case d: DisassociatedEvent => - if (amActor.isDefined && sender == amActor.get) { - logWarning(s"ApplicationMaster has disassociated: $d") - } + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + if (amEndpoint.exists(_.address == remoteAddress)) { + logWarning(s"ApplicationMaster has disassociated: $remoteAddress") + } + } + + override def onStop(): Unit ={ + askAmThreadPool.shutdownNow() } } } private[spark] object YarnSchedulerBackend { - val ACTOR_NAME = "YarnScheduler" + val ENDPOINT_NAME = "YarnScheduler" } 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 e13de0f46ef89..dc59545b43314 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 @@ -18,20 +18,17 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{List => JList} -import java.util.Collections +import java.util.{Collections, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.mesos.{Scheduler => MScheduler} -import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} - -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException, TaskState} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -47,19 +44,12 @@ private[spark] class CoarseMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with MScheduler - with Logging { + with MesosSchedulerUtils { val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Driver for talking to Mesos - var driver: SchedulerDriver = null - // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt @@ -87,26 +77,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def start() { super.start() - - synchronized { - new Thread("CoarseMesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() - - waitForRegister() - } + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + startScheduler(master, CoarseMesosSchedulerBackend.this, fwInfo) } def createCommand(offer: Offer, numCores: Int): CommandInfo = { @@ -148,10 +120,12 @@ private[spark] class CoarseMesosSchedulerBackend( SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) + + val uri = conf.getOption("spark.executor.uri") + .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - val uri = conf.get("spark.executor.uri", null) - if (uri == null) { + if (uri.isEmpty) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" @@ -164,7 +138,7 @@ private[spark] class CoarseMesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = uri.split('/').last.split('.').head + val basename = uri.get.split('/').last.split('.').head command.setValue( s"cd $basename*; $prefixEnv " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + @@ -173,7 +147,7 @@ private[spark] class CoarseMesosSchedulerBackend( s" --hostname ${offer.getHostname}" + s" --cores $numCores" + s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } command.build() } @@ -183,18 +157,7 @@ private[spark] class CoarseMesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } - } + markRegistered() } override def disconnected(d: SchedulerDriver) {} @@ -207,7 +170,7 @@ private[spark] class CoarseMesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { synchronized { - val filters = Filters.newBuilder().setRefuseSeconds(-1).build() + val filters = Filters.newBuilder().setRefuseSeconds(5).build() for (offer <- offers) { val slaveId = offer.getSlaveId.toString @@ -233,9 +196,14 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", MemoryUtils.calculateTotalMemory(sc))) - .build() + + sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => + MesosSchedulerBackendUtil + .setupContainerBuilderDockerInfo(image, sc.conf, task.getContainerBuilder()) + } + d.launchTasks( - Collections.singleton(offer.getId), Collections.singletonList(task), filters) + Collections.singleton(offer.getId), Collections.singletonList(task.build()), filters) } else { // Filter it out d.launchTasks( @@ -245,14 +213,6 @@ private[spark] class CoarseMesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - private def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Build a Mesos resource protobuf object */ private def createResource(resourceName: String, quantity: Double): Protos.Resource = { Resource.newBuilder() @@ -284,7 +244,8 @@ private[spark] class CoarseMesosSchedulerBackend( "is Spark installed on it?") } } - driver.reviveOffers() // In case we'd rejected everything before but have now lost a node + // In case we'd rejected everything before but have now lost a node + mesosDriver.reviveOffers() } } } @@ -296,8 +257,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def stop() { super.stop() - if (driver != null) { - driver.stop() + if (mesosDriver != null) { + mesosDriver.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala index aa3ec0f8cfb9c..8df4f3b554c41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -24,7 +24,7 @@ private[spark] object MemoryUtils { val OVERHEAD_FRACTION = 0.10 val OVERHEAD_MINIMUM = 384 - def calculateTotalMemory(sc: SparkContext) = { + def calculateTotalMemory(sc: SparkContext): Int = { sc.conf.getInt("spark.mesos.executor.memoryOverhead", math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala new file mode 100644 index 0000000000000..3efc536f1456c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -0,0 +1,134 @@ +/* + * 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.scheduler.cluster.mesos + +import scala.collection.JavaConversions._ + +import org.apache.curator.framework.CuratorFramework +import org.apache.zookeeper.CreateMode +import org.apache.zookeeper.KeeperException.NoNodeException + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.util.Utils + +/** + * Persistence engine factory that is responsible for creating new persistence engines + * to store Mesos cluster mode state. + */ +private[spark] abstract class MesosClusterPersistenceEngineFactory(conf: SparkConf) { + def createEngine(path: String): MesosClusterPersistenceEngine +} + +/** + * Mesos cluster persistence engine is responsible for persisting Mesos cluster mode + * specific state, so that on failover all the state can be recovered and the scheduler + * can resume managing the drivers. + */ +private[spark] trait MesosClusterPersistenceEngine { + def persist(name: String, obj: Object): Unit + def expunge(name: String): Unit + def fetch[T](name: String): Option[T] + def fetchAll[T](): Iterable[T] +} + +/** + * Zookeeper backed persistence engine factory. + * All Zk engines created from this factory shares the same Zookeeper client, so + * all of them reuses the same connection pool. + */ +private[spark] class ZookeeperMesosClusterPersistenceEngineFactory(conf: SparkConf) + extends MesosClusterPersistenceEngineFactory(conf) { + + lazy val zk = SparkCuratorUtil.newClient(conf, "spark.mesos.deploy.zookeeper.url") + + def createEngine(path: String): MesosClusterPersistenceEngine = { + new ZookeeperMesosClusterPersistenceEngine(path, zk, conf) + } +} + +/** + * Black hole persistence engine factory that creates black hole + * persistence engines, which stores nothing. + */ +private[spark] class BlackHoleMesosClusterPersistenceEngineFactory + extends MesosClusterPersistenceEngineFactory(null) { + def createEngine(path: String): MesosClusterPersistenceEngine = { + new BlackHoleMesosClusterPersistenceEngine + } +} + +/** + * Black hole persistence engine that stores nothing. + */ +private[spark] class BlackHoleMesosClusterPersistenceEngine extends MesosClusterPersistenceEngine { + override def persist(name: String, obj: Object): Unit = {} + override def fetch[T](name: String): Option[T] = None + override def expunge(name: String): Unit = {} + override def fetchAll[T](): Iterable[T] = Iterable.empty[T] +} + +/** + * Zookeeper based Mesos cluster persistence engine, that stores cluster mode state + * into Zookeeper. Each engine object is operating under one folder in Zookeeper, but + * reuses a shared Zookeeper client. + */ +private[spark] class ZookeeperMesosClusterPersistenceEngine( + baseDir: String, + zk: CuratorFramework, + conf: SparkConf) + extends MesosClusterPersistenceEngine with Logging { + private val WORKING_DIR = + conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir + + SparkCuratorUtil.mkdir(zk, WORKING_DIR) + + def path(name: String): String = { + WORKING_DIR + "/" + name + } + + override def expunge(name: String): Unit = { + zk.delete().forPath(path(name)) + } + + override def persist(name: String, obj: Object): Unit = { + val serialized = Utils.serialize(obj) + val zkPath = path(name) + zk.create().withMode(CreateMode.PERSISTENT).forPath(zkPath, serialized) + } + + override def fetch[T](name: String): Option[T] = { + val zkPath = path(name) + + try { + val fileData = zk.getData().forPath(zkPath) + Some(Utils.deserialize[T](fileData)) + } catch { + case e: NoNodeException => None + case e: Exception => { + logWarning("Exception while reading persisted file, deleting", e) + zk.delete().forPath(zkPath) + None + } + } + } + + override def fetchAll[T](): Iterable[T] = { + zk.getChildren.forPath(WORKING_DIR).map(fetch[T]).flatten + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala new file mode 100644 index 0000000000000..06f0e2881c344 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -0,0 +1,635 @@ +/* + * 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.scheduler.cluster.mesos + +import java.io.File +import java.util.concurrent.locks.ReentrantLock +import java.util.{Collections, Date, List => JList} + +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.mesos.Protos.Environment.Variable +import org.apache.mesos.Protos.TaskStatus.Reason +import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.{Scheduler, SchedulerDriver} +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.Utils +import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} + + +/** + * Tracks the current state of a Mesos Task that runs a Spark driver. + * @param driverDescription Submitted driver description from + * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] + * @param taskId Mesos TaskID generated for the task + * @param slaveId Slave ID that the task is assigned to + * @param mesosTaskStatus The last known task status update. + * @param startDate The date the task was launched + */ +private[spark] class MesosClusterSubmissionState( + val driverDescription: MesosDriverDescription, + val taskId: TaskID, + val slaveId: SlaveID, + var mesosTaskStatus: Option[TaskStatus], + var startDate: Date, + var finishDate: Option[Date]) + extends Serializable { + + def copy(): MesosClusterSubmissionState = { + new MesosClusterSubmissionState( + driverDescription, taskId, slaveId, mesosTaskStatus, startDate, finishDate) + } +} + +/** + * Tracks the retry state of a driver, which includes the next time it should be scheduled + * and necessary information to do exponential backoff. + * This class is not thread-safe, and we expect the caller to handle synchronizing state. + * @param lastFailureStatus Last Task status when it failed. + * @param retries Number of times it has been retried. + * @param nextRetry Time at which it should be retried next + * @param waitTime The amount of time driver is scheduled to wait until next retry. + */ +private[spark] class MesosClusterRetryState( + val lastFailureStatus: TaskStatus, + val retries: Int, + val nextRetry: Date, + val waitTime: Int) extends Serializable { + def copy(): MesosClusterRetryState = + new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime) +} + +/** + * The full state of the cluster scheduler, currently being used for displaying + * information on the UI. + * @param frameworkId Mesos Framework id for the cluster scheduler. + * @param masterUrl The Mesos master url + * @param queuedDrivers All drivers queued to be launched + * @param launchedDrivers All launched or running drivers + * @param finishedDrivers All terminated drivers + * @param pendingRetryDrivers All drivers pending to be retried + */ +private[spark] class MesosClusterSchedulerState( + val frameworkId: String, + val masterUrl: Option[String], + val queuedDrivers: Iterable[MesosDriverDescription], + val launchedDrivers: Iterable[MesosClusterSubmissionState], + val finishedDrivers: Iterable[MesosClusterSubmissionState], + val pendingRetryDrivers: Iterable[MesosDriverDescription]) + +/** + * The full state of a Mesos driver, that is being used to display driver information on the UI. + */ +private[spark] class MesosDriverState( + val state: String, + val description: MesosDriverDescription, + val submissionState: Option[MesosClusterSubmissionState] = None) + +/** + * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode + * as Mesos tasks in a Mesos cluster. + * All drivers are launched asynchronously by the framework, which will eventually be launched + * by one of the slaves in the cluster. The results of the driver will be stored in slave's task + * sandbox which is accessible by visiting the Mesos UI. + * This scheduler supports recovery by persisting all its state and performs task reconciliation + * on recover, which gets all the latest state for all the drivers from Mesos master. + */ +private[spark] class MesosClusterScheduler( + engineFactory: MesosClusterPersistenceEngineFactory, + conf: SparkConf) + extends Scheduler with MesosSchedulerUtils { + var frameworkUrl: String = _ + private val metricsSystem = + MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf)) + private val master = conf.get("spark.master") + private val appName = conf.get("spark.app.name") + private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) + private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) + private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute + private val schedulerState = engineFactory.createEngine("scheduler") + private val stateLock = new ReentrantLock() + private val finishedDrivers = + new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers) + private var frameworkId: String = null + // Holds all the launched drivers and current launch state, keyed by driver id. + private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]() + // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation. + // All drivers that are loaded after failover are added here, as we need get the latest + // state of the tasks from Mesos. + private val pendingRecover = new mutable.HashMap[String, SlaveID]() + // Stores all the submitted drivers that hasn't been launched. + private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() + // All supervised drivers that are waiting to retry after termination. + private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() + private val queuedDriversState = engineFactory.createEngine("driverQueue") + private val launchedDriversState = engineFactory.createEngine("launchedDrivers") + private val pendingRetryDriversState = engineFactory.createEngine("retryList") + // Flag to mark if the scheduler is ready to be called, which is until the scheduler + // is registered with Mesos master. + @volatile protected var ready = false + private var masterInfo: Option[MasterInfo] = None + + def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { + val c = new CreateSubmissionResponse + if (!ready) { + c.success = false + c.message = "Scheduler is not ready to take requests" + return c + } + + stateLock.synchronized { + if (isQueueFull()) { + c.success = false + c.message = "Already reached maximum submission size" + return c + } + c.submissionId = desc.submissionId + queuedDriversState.persist(desc.submissionId, desc) + queuedDrivers += desc + c.success = true + } + c + } + + def killDriver(submissionId: String): KillSubmissionResponse = { + val k = new KillSubmissionResponse + if (!ready) { + k.success = false + k.message = "Scheduler is not ready to take requests" + return k + } + k.submissionId = submissionId + stateLock.synchronized { + // We look for the requested driver in the following places: + // 1. Check if submission is running or launched. + // 2. Check if it's still queued. + // 3. Check if it's in the retry list. + // 4. Check if it has already completed. + if (launchedDrivers.contains(submissionId)) { + val task = launchedDrivers(submissionId) + mesosDriver.killTask(task.taskId) + k.success = true + k.message = "Killing running driver" + } else if (removeFromQueuedDrivers(submissionId)) { + k.success = true + k.message = "Removed driver while it's still pending" + } else if (removeFromPendingRetryDrivers(submissionId)) { + k.success = true + k.message = "Removed driver while it's being retried" + } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { + k.success = false + k.message = "Driver already terminated" + } else { + k.success = false + k.message = "Cannot find driver" + } + } + k + } + + def getDriverStatus(submissionId: String): SubmissionStatusResponse = { + val s = new SubmissionStatusResponse + if (!ready) { + s.success = false + s.message = "Scheduler is not ready to take requests" + return s + } + s.submissionId = submissionId + stateLock.synchronized { + if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { + s.success = true + s.driverState = "QUEUED" + } else if (launchedDrivers.contains(submissionId)) { + s.success = true + s.driverState = "RUNNING" + launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString) + } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { + s.success = true + s.driverState = "FINISHED" + finishedDrivers + .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus + .foreach(state => s.message = state.toString) + } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { + val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + .get.retryState.get.lastFailureStatus + s.success = true + s.driverState = "RETRYING" + s.message = status.toString + } else { + s.success = false + s.driverState = "NOT_FOUND" + } + } + s + } + + /** + * Gets the driver state to be displayed on the Web UI. + */ + def getDriverState(submissionId: String): Option[MesosDriverState] = { + stateLock.synchronized { + queuedDrivers.find(_.submissionId.equals(submissionId)) + .map(d => new MesosDriverState("QUEUED", d)) + .orElse(launchedDrivers.get(submissionId) + .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d)))) + .orElse(finishedDrivers.find(_.driverDescription.submissionId.equals(submissionId)) + .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d)))) + .orElse(pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + .map(d => new MesosDriverState("RETRYING", d))) + } + } + + private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity + + /** + * Recover scheduler state that is persisted. + * We still need to do task reconciliation to be up to date of the latest task states + * as it might have changed while the scheduler is failing over. + */ + private def recoverState(): Unit = { + stateLock.synchronized { + launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state => + launchedDrivers(state.taskId.getValue) = state + pendingRecover(state.taskId.getValue) = state.slaveId + } + queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d) + // There is potential timing issue where a queued driver might have been launched + // but the scheduler shuts down before the queued driver was able to be removed + // from the queue. We try to mitigate this issue by walking through all queued drivers + // and remove if they're already launched. + queuedDrivers + .filter(d => launchedDrivers.contains(d.submissionId)) + .foreach(d => removeFromQueuedDrivers(d.submissionId)) + pendingRetryDriversState.fetchAll[MesosDriverDescription]() + .foreach(s => pendingRetryDrivers += s) + // TODO: Consider storing finished drivers so we can show them on the UI after + // failover. For now we clear the history on each recovery. + finishedDrivers.clear() + } + } + + /** + * Starts the cluster scheduler and wait until the scheduler is registered. + * This also marks the scheduler to be ready for requests. + */ + def start(): Unit = { + // TODO: Implement leader election to make sure only one framework running in the cluster. + val fwId = schedulerState.fetch[String]("frameworkId") + val builder = FrameworkInfo.newBuilder() + .setUser(Utils.getCurrentUserName()) + .setName(appName) + .setWebuiUrl(frameworkUrl) + .setCheckpoint(true) + .setFailoverTimeout(Integer.MAX_VALUE) // Setting to max so tasks keep running on crash + fwId.foreach { id => + builder.setId(FrameworkID.newBuilder().setValue(id).build()) + frameworkId = id + } + recoverState() + metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) + metricsSystem.start() + startScheduler(master, MesosClusterScheduler.this, builder.build()) + ready = true + } + + def stop(): Unit = { + ready = false + metricsSystem.report() + metricsSystem.stop() + mesosDriver.stop(true) + } + + override def registered( + driver: SchedulerDriver, + newFrameworkId: FrameworkID, + masterInfo: MasterInfo): Unit = { + logInfo("Registered as framework ID " + newFrameworkId.getValue) + if (newFrameworkId.getValue != frameworkId) { + frameworkId = newFrameworkId.getValue + schedulerState.persist("frameworkId", frameworkId) + } + markRegistered() + + stateLock.synchronized { + this.masterInfo = Some(masterInfo) + if (!pendingRecover.isEmpty) { + // Start task reconciliation if we need to recover. + val statuses = pendingRecover.collect { + case (taskId, slaveId) => + val newStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId).build()) + .setSlaveId(slaveId) + .setState(MesosTaskState.TASK_STAGING) + .build() + launchedDrivers.get(taskId).map(_.mesosTaskStatus.getOrElse(newStatus)) + .getOrElse(newStatus) + } + // TODO: Page the status updates to avoid trying to reconcile + // a large amount of tasks at once. + driver.reconcileTasks(statuses) + } + } + } + + private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = { + val appJar = CommandInfo.URI.newBuilder() + .setValue(desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() + val builder = CommandInfo.newBuilder().addUris(appJar) + val entries = + (conf.getOption("spark.executor.extraLibraryPath").toList ++ + desc.command.libraryPathEntries) + val prefixEnv = if (!entries.isEmpty) { + Utils.libraryPathEnvPrefix(entries) + } else { + "" + } + val envBuilder = Environment.newBuilder() + desc.command.environment.foreach { case (k, v) => + envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v).build()) + } + // Pass all spark properties to executor. + val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v" }.mkString(" ") + envBuilder.addVariables( + Variable.newBuilder().setName("SPARK_EXECUTOR_OPTS").setValue(executorOpts)) + val cmdOptions = generateCmdOption(desc) + val executorUri = desc.schedulerProperties.get("spark.executor.uri") + .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) + val appArguments = desc.command.arguments.mkString(" ") + val cmd = if (executorUri.isDefined) { + builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) + val folderBasename = executorUri.get.split('/').last.split('.').head + val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" + val cmdJar = s"../${desc.jarUrl.split("/").last}" + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" + } else { + val executorSparkHome = desc.schedulerProperties.get("spark.mesos.executor.home") + .orElse(conf.getOption("spark.home")) + .orElse(Option(System.getenv("SPARK_HOME"))) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } + val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath + val cmdJar = desc.jarUrl.split("/").last + s"$cmdExecutable ${cmdOptions.mkString(" ")} $cmdJar $appArguments" + } + builder.setValue(cmd) + builder.setEnvironment(envBuilder.build()) + builder.build() + } + + private def generateCmdOption(desc: MesosDriverDescription): Seq[String] = { + var options = Seq( + "--name", desc.schedulerProperties("spark.app.name"), + "--class", desc.command.mainClass, + "--master", s"mesos://${conf.get("spark.master")}", + "--driver-cores", desc.cores.toString, + "--driver-memory", s"${desc.mem}M") + desc.schedulerProperties.get("spark.executor.memory").map { v => + options ++= Seq("--executor-memory", v) + } + desc.schedulerProperties.get("spark.cores.max").map { v => + options ++= Seq("--total-executor-cores", v) + } + options + } + + private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) { + override def toString(): String = { + s"Offer id: ${offer.getId.getValue}, cpu: $cpu, mem: $mem" + } + } + + /** + * This method takes all the possible candidates and attempt to schedule them with Mesos offers. + * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled + * logic on each task. + */ + private def scheduleTasks( + candidates: Seq[MesosDriverDescription], + afterLaunchCallback: (String) => Boolean, + currentOffers: List[ResourceOffer], + tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = { + for (submission <- candidates) { + val driverCpu = submission.cores + val driverMem = submission.mem + logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") + val offerOption = currentOffers.find { o => + o.cpu >= driverCpu && o.mem >= driverMem + } + if (offerOption.isEmpty) { + logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " + + s"cpu: $driverCpu, mem: $driverMem") + } else { + val offer = offerOption.get + offer.cpu -= driverCpu + offer.mem -= driverMem + val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() + val cpuResource = Resource.newBuilder() + .setName("cpus").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverCpu)).build() + val memResource = Resource.newBuilder() + .setName("mem").setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(driverMem)).build() + val commandInfo = buildDriverCommand(submission) + val appName = submission.schedulerProperties("spark.app.name") + val taskInfo = TaskInfo.newBuilder() + .setTaskId(taskId) + .setName(s"Driver for $appName") + .setSlaveId(offer.offer.getSlaveId) + .setCommand(commandInfo) + .addResources(cpuResource) + .addResources(memResource) + .build() + val queuedTasks = tasks.getOrElseUpdate(offer.offer.getId, new ArrayBuffer[TaskInfo]) + queuedTasks += taskInfo + logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + + submission.submissionId) + val newState = new MesosClusterSubmissionState(submission, taskId, offer.offer.getSlaveId, + None, new Date(), None) + launchedDrivers(submission.submissionId) = newState + launchedDriversState.persist(submission.submissionId, newState) + afterLaunchCallback(submission.submissionId) + } + } + } + + override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { + val currentOffers = offers.map { o => + new ResourceOffer( + o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) + }.toList + logTrace(s"Received offers from Mesos: \n${currentOffers.mkString("\n")}") + val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() + val currentTime = new Date() + + stateLock.synchronized { + // We first schedule all the supervised drivers that are ready to retry. + // This list will be empty if none of the drivers are marked as supervise. + val driversToRetry = pendingRetryDrivers.filter { d => + d.retryState.get.nextRetry.before(currentTime) + } + scheduleTasks( + driversToRetry, + removeFromPendingRetryDrivers, + currentOffers, + tasks) + // Then we walk through the queued drivers and try to schedule them. + scheduleTasks( + queuedDrivers, + removeFromQueuedDrivers, + currentOffers, + tasks) + } + tasks.foreach { case (offerId, tasks) => + driver.launchTasks(Collections.singleton(offerId), tasks) + } + offers + .filter(o => !tasks.keySet.contains(o.getId)) + .foreach(o => driver.declineOffer(o.getId)) + } + + def getSchedulerState(): MesosClusterSchedulerState = { + def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { + val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) + buffer.copyToBuffer(newBuffer) + newBuffer + } + stateLock.synchronized { + new MesosClusterSchedulerState( + frameworkId, + masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), + copyBuffer(queuedDrivers), + launchedDrivers.values.map(_.copy()).toList, + finishedDrivers.map(_.copy()).toList, + copyBuffer(pendingRetryDrivers)) + } + } + + override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} + override def disconnected(driver: SchedulerDriver): Unit = {} + override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { + logInfo(s"Framework re-registered with master ${masterInfo.getId}") + } + override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} + override def error(driver: SchedulerDriver, error: String): Unit = { + logError("Error received: " + error) + } + + /** + * Check if the task state is a recoverable state that we can relaunch the task. + * Task state like TASK_ERROR are not relaunchable state since it wasn't able + * to be validated by Mesos. + */ + private def shouldRelaunch(state: MesosTaskState): Boolean = { + state == MesosTaskState.TASK_FAILED || + state == MesosTaskState.TASK_KILLED || + state == MesosTaskState.TASK_LOST + } + + override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = { + val taskId = status.getTaskId.getValue + stateLock.synchronized { + if (launchedDrivers.contains(taskId)) { + if (status.getReason == Reason.REASON_RECONCILIATION && + !pendingRecover.contains(taskId)) { + // Task has already received update and no longer requires reconciliation. + return + } + val state = launchedDrivers(taskId) + // Check if the driver is supervise enabled and can be relaunched. + if (state.driverDescription.supervise && shouldRelaunch(status.getState)) { + removeFromLaunchedDrivers(taskId) + state.finishDate = Some(new Date()) + val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState + val (retries, waitTimeSec) = retryState + .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) } + .getOrElse{ (1, 1) } + val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) + + val newDriverDescription = state.driverDescription.copy( + retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec))) + pendingRetryDrivers += newDriverDescription + pendingRetryDriversState.persist(taskId, newDriverDescription) + } else if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { + removeFromLaunchedDrivers(taskId) + state.finishDate = Some(new Date()) + if (finishedDrivers.size >= retainedDrivers) { + val toRemove = math.max(retainedDrivers / 10, 1) + finishedDrivers.trimStart(toRemove) + } + finishedDrivers += state + } + state.mesosTaskStatus = Option(status) + } else { + logError(s"Unable to find driver $taskId in status update") + } + } + } + + override def frameworkMessage( + driver: SchedulerDriver, + executorId: ExecutorID, + slaveId: SlaveID, + message: Array[Byte]): Unit = {} + + override def executorLost( + driver: SchedulerDriver, + executorId: ExecutorID, + slaveId: SlaveID, + status: Int): Unit = {} + + private def removeFromQueuedDrivers(id: String): Boolean = { + val index = queuedDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + queuedDrivers.remove(index) + queuedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromLaunchedDrivers(id: String): Boolean = { + if (launchedDrivers.remove(id).isDefined) { + launchedDriversState.expunge(id) + true + } else { + false + } + } + + private def removeFromPendingRetryDrivers(id: String): Boolean = { + val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id)) + if (index != -1) { + pendingRetryDrivers.remove(index) + pendingRetryDriversState.expunge(id) + true + } else { + false + } + } + + def getQueuedDriversSize: Int = queuedDrivers.size + def getLaunchedDriversSize: Int = launchedDrivers.size + def getPendingRetryDriversSize: Int = pendingRetryDrivers.size +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala new file mode 100644 index 0000000000000..1fe94974c8e36 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSource.scala @@ -0,0 +1,40 @@ +/* + * 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.scheduler.cluster.mesos + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.source.Source + +private[mesos] class MesosClusterSchedulerSource(scheduler: MesosClusterScheduler) + extends Source { + override def sourceName: String = "mesos_cluster" + override def metricRegistry: MetricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name("waitingDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.getQueuedDriversSize + }) + + metricRegistry.register(MetricRegistry.name("launchedDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.getLaunchedDriversSize + }) + + metricRegistry.register(MetricRegistry.name("retryDrivers"), new Gauge[Int] { + override def getValue: Int = scheduler.getPendingRetryDriversSize + }) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 06bb527522141..db0a080b3b0c0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -18,23 +18,19 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{ArrayList => JArrayList, List => JList} -import java.util.Collections +import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} +import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Scheduler => MScheduler} -import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, - ExecutorInfo => MesosExecutorInfo, _} - +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} -import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkException, TaskState} /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a @@ -47,14 +43,7 @@ private[spark] class MesosSchedulerBackend( master: String) extends SchedulerBackend with MScheduler - with Logging { - - // Lock used to wait for scheduler to be registered - var isRegistered = false - val registeredLock = new Object() - - // Driver for talking to Mesos - var driver: SchedulerDriver = null + with MesosSchedulerUtils { // Which slave IDs we have executors on val slaveIdsWithExecutors = new HashSet[String] @@ -68,29 +57,14 @@ private[spark] class MesosSchedulerBackend( // The listener bus to publish executor added/removed events. val listenerBus = sc.listenerBus + private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) + @volatile var appId: String = _ override def start() { - synchronized { - classLoader = Thread.currentThread.getContextClassLoader - - new Thread("MesosSchedulerBackend driver") { - setDaemon(true) - override def run() { - val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() - driver = new MesosSchedulerDriver(scheduler, fwInfo, master) - try { - val ret = driver.run() - logInfo("driver.run() returned with code " + ret) - } catch { - case e: Exception => logError("driver.run() failed", e) - } - } - }.start() - - waitForRegister() - } + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() + classLoader = Thread.currentThread.getContextClassLoader + startScheduler(master, MesosSchedulerBackend.this, fwInfo) } def createExecutorInfo(execId: String): MesosExecutorInfo = { @@ -123,23 +97,25 @@ private[spark] class MesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.get("spark.executor.uri", null) + val uri = sc.conf.getOption("spark.executor.uri") + .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val executorBackendName = classOf[MesosExecutorBackend].getName - if (uri == null) { + if (uri.isEmpty) { val executorPath = new File(executorSparkHome, "/bin/spark-class").getCanonicalPath command.setValue(s"$prefixEnv $executorPath $executorBackendName") } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = uri.split('/').last.split('.').head + val basename = uri.get.split('/').last.split('.').head command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) } val cpus = Resource.newBuilder() .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder() - .setValue(scheduler.CPUS_PER_TASK).build()) + .setValue(mesosExecutorCores).build()) .build() val memory = Resource.newBuilder() .setName("mem") @@ -148,13 +124,19 @@ private[spark] class MesosSchedulerBackend( Value.Scalar.newBuilder() .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) .build() - MesosExecutorInfo.newBuilder() + val executorInfo = MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) .addResources(cpus) .addResources(memory) - .build() + + sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => + MesosSchedulerBackendUtil + .setupContainerBuilderDockerInfo(image, sc.conf, executorInfo.getContainerBuilder()) + } + + executorInfo.build() } /** @@ -179,18 +161,7 @@ private[spark] class MesosSchedulerBackend( inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() - } - } - } - - def waitForRegister() { - registeredLock.synchronized { - while (!isRegistered) { - registeredLock.wait() - } + markRegistered() } } @@ -220,10 +191,9 @@ private[spark] class MesosSchedulerBackend( val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task - cpus >= 2 * scheduler.CPUS_PER_TASK) || + cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)) || (slaveIdsWithExecutors.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) } @@ -232,10 +202,9 @@ private[spark] class MesosSchedulerBackend( val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { - // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? - getResource(o.getResourcesList, "cpus").toInt - - scheduler.CPUS_PER_TASK + // If the Mesos executor has not been started on this slave yet, set aside a few + // cores for the Mesos executor by offering fewer cores to the Spark executor + (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt } new WorkerOffer( o.getSlaveId.getValue, @@ -287,14 +256,6 @@ private[spark] class MesosSchedulerBackend( } } - /** Helper function to pull out a resource from a Mesos Resources protobuf */ - def getResource(res: JList[Resource], name: String): Double = { - for (r <- res if r.getName == name) { - return r.getScalar.getValue - } - 0 - } - /** Turn a Spark TaskDescription into a Mesos task */ def createMesosTask(task: TaskDescription, slaveId: String): MesosTaskInfo = { val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() @@ -339,13 +300,13 @@ private[spark] class MesosSchedulerBackend( } override def stop() { - if (driver != null) { - driver.stop() + if (mesosDriver != null) { + mesosDriver.stop() } } override def reviveOffers() { - driver.reviveOffers() + mesosDriver.reviveOffers() } override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} @@ -380,14 +341,14 @@ private[spark] class MesosSchedulerBackend( } override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { - driver.killTask( + mesosDriver.killTask( TaskID.newBuilder() .setValue(taskId.toString).build() ) } // TODO: query Mesos for number of cores - override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8) override def applicationId(): String = Option(appId).getOrElse { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala new file mode 100644 index 0000000000000..928c5cfed417a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -0,0 +1,142 @@ +/* + * 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.scheduler.cluster.mesos + +import org.apache.mesos.Protos.{ContainerInfo, Volume} +import org.apache.mesos.Protos.ContainerInfo.DockerInfo + +import org.apache.spark.{Logging, SparkConf} + +/** + * A collection of utility functions which can be used by both the + * MesosSchedulerBackend and the CoarseMesosSchedulerBackend. + */ +private[mesos] object MesosSchedulerBackendUtil extends Logging { + /** + * Parse a comma-delimited list of volume specs, each of which + * takes the form [host-dir:]container-dir[:rw|:ro]. + */ + def parseVolumesSpec(volumes: String): List[Volume] = { + volumes.split(",").map(_.split(":")).flatMap { spec => + val vol: Volume.Builder = Volume + .newBuilder() + .setMode(Volume.Mode.RW) + spec match { + case Array(container_path) => + Some(vol.setContainerPath(container_path)) + case Array(container_path, "rw") => + Some(vol.setContainerPath(container_path)) + case Array(container_path, "ro") => + Some(vol.setContainerPath(container_path) + .setMode(Volume.Mode.RO)) + case Array(host_path, container_path) => + Some(vol.setContainerPath(container_path) + .setHostPath(host_path)) + case Array(host_path, container_path, "rw") => + Some(vol.setContainerPath(container_path) + .setHostPath(host_path)) + case Array(host_path, container_path, "ro") => + Some(vol.setContainerPath(container_path) + .setHostPath(host_path) + .setMode(Volume.Mode.RO)) + case spec => { + logWarning(s"Unable to parse volume specs: $volumes. " + + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"") + None + } + } + } + .map { _.build() } + .toList + } + + /** + * Parse a comma-delimited list of port mapping specs, each of which + * takes the form host_port:container_port[:udp|:tcp] + * + * Note: + * the docker form is [ip:]host_port:container_port, but the DockerInfo + * message has no field for 'ip', and instead has a 'protocol' field. + * Docker itself only appears to support TCP, so this alternative form + * anticipates the expansion of the docker form to allow for a protocol + * and leaves open the chance for mesos to begin to accept an 'ip' field + */ + def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = { + portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] => + val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping + .newBuilder() + .setProtocol("tcp") + spec match { + case Array(host_port, container_port) => + Some(portmap.setHostPort(host_port.toInt) + .setContainerPort(container_port.toInt)) + case Array(host_port, container_port, protocol) => + Some(portmap.setHostPort(host_port.toInt) + .setContainerPort(container_port.toInt) + .setProtocol(protocol)) + case spec => { + logWarning(s"Unable to parse port mapping specs: $portmaps. " + + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"") + None + } + } + } + .map { _.build() } + .toList + } + + /** + * Construct a DockerInfo structure and insert it into a ContainerInfo + */ + def addDockerInfo( + container: ContainerInfo.Builder, + image: String, + volumes: Option[List[Volume]] = None, + network: Option[ContainerInfo.DockerInfo.Network] = None, + portmaps: Option[List[ContainerInfo.DockerInfo.PortMapping]] = None):Unit = { + + val docker = ContainerInfo.DockerInfo.newBuilder().setImage(image) + + network.foreach(docker.setNetwork) + portmaps.foreach(_.foreach(docker.addPortMappings)) + container.setType(ContainerInfo.Type.DOCKER) + container.setDocker(docker.build()) + volumes.foreach(_.foreach(container.addVolumes)) + } + + /** + * Setup a docker containerizer + */ + def setupContainerBuilderDockerInfo( + imageName: String, + conf: SparkConf, + builder: ContainerInfo.Builder): Unit = { + val volumes = conf + .getOption("spark.mesos.executor.docker.volumes") + .map(parseVolumesSpec) + val portmaps = conf + .getOption("spark.mesos.executor.docker.portmaps") + .map(parsePortMappingsSpec) + addDockerInfo( + builder, + imageName, + volumes = volumes, + portmaps = portmaps) + logDebug("setupContainerDockerInfo: using docker image: " + imageName) + } +} 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 new file mode 100644 index 0000000000000..d11228f3d016a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -0,0 +1,95 @@ +/* + * 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.scheduler.cluster.mesos + +import java.util.List +import java.util.concurrent.CountDownLatch + +import scala.collection.JavaConversions._ + +import org.apache.mesos.Protos.{FrameworkInfo, Resource, Status} +import org.apache.mesos.{MesosSchedulerDriver, Scheduler} +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * Shared trait for implementing a Mesos Scheduler. This holds common state and helper + * methods and Mesos scheduler will use. + */ +private[mesos] trait MesosSchedulerUtils extends Logging { + // Lock used to wait for scheduler to be registered + private final val registerLatch = new CountDownLatch(1) + + // Driver for talking to Mesos + protected var mesosDriver: MesosSchedulerDriver = null + + /** + * Starts the MesosSchedulerDriver with the provided information. This method returns + * only after the scheduler has registered with Mesos. + * @param masterUrl Mesos master connection URL + * @param scheduler Scheduler object + * @param fwInfo FrameworkInfo to pass to the Mesos master + */ + def startScheduler(masterUrl: String, scheduler: Scheduler, fwInfo: FrameworkInfo): Unit = { + synchronized { + if (mesosDriver != null) { + registerLatch.await() + return + } + + new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { + setDaemon(true) + + override def run() { + mesosDriver = new MesosSchedulerDriver(scheduler, fwInfo, masterUrl) + try { + val ret = mesosDriver.run() + logInfo("driver.run() returned with code " + ret) + if (ret.equals(Status.DRIVER_ABORTED)) { + System.exit(1) + } + } catch { + case e: Exception => { + logError("driver.run() failed", e) + System.exit(1) + } + } + } + }.start() + + registerLatch.await() + } + } + + /** + * Signal that the scheduler has registered with Mesos. + */ + protected def markRegistered(): Unit = { + registerLatch.countDown() + } + + /** + * Get the amount of resources for the specified type from the resource list + */ + protected def getResource(res: List[Resource], name: String): Double = { + for (r <- res if r.getName == name) { + return r.getScalar.getValue + } + 0.0 + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index d95426d918e19..e64d06c4d3cfc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -18,17 +18,14 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit -import scala.concurrent.duration._ -import scala.language.postfixOps - -import akka.actor.{Actor, ActorRef, Props} - -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.util.ActorLogReceive +import org.apache.spark.util.{ThreadUtils, Utils} private case class ReviveOffers() @@ -39,17 +36,19 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private case class StopExecutor() /** - * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on - * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend + * Calls to LocalBackend are all serialized through LocalEndpoint. Using an RpcEndpoint makes the + * calls on LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend * and the TaskSchedulerImpl. */ -private[spark] class LocalActor( +private[spark] class LocalEndpoint( + override val rpcEnv: RpcEnv, scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, private val totalCores: Int) - extends Actor with ActorLogReceive with Logging { + extends ThreadSafeRpcEndpoint with Logging { - import context.dispatcher // to use Akka's scheduler.scheduleOnce() + private val reviveThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("local-revive-thread") private var freeCores = totalCores @@ -59,7 +58,7 @@ private[spark] class LocalActor( private val executor = new Executor( localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) - override def receiveWithLogging = { + override def receive: PartialFunction[Any, Unit] = { case ReviveOffers => reviveOffers() @@ -72,11 +71,15 @@ private[spark] class LocalActor( case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case StopExecutor => executor.stop() + context.reply(true) } + def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) val tasks = scheduler.resourceOffers(offers).flatten @@ -87,9 +90,17 @@ private[spark] class LocalActor( } if (tasks.isEmpty && scheduler.activeTaskSets.nonEmpty) { // Try to reviveOffer after 1 second, because scheduler may wait for locality timeout - context.system.scheduler.scheduleOnce(1000 millis, self, ReviveOffers) + reviveThread.schedule(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ReviveOffers)) + } + }, 1000, TimeUnit.MILLISECONDS) } } + + override def onStop(): Unit = { + reviveThread.shutdownNow() + } } /** @@ -97,35 +108,37 @@ private[spark] class LocalActor( * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ -private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) - extends SchedulerBackend with ExecutorBackend { +private[spark] class LocalBackend( + conf: SparkConf, + scheduler: TaskSchedulerImpl, + val totalCores: Int) + extends SchedulerBackend with ExecutorBackend with Logging { private val appId = "local-" + System.currentTimeMillis - var localActor: ActorRef = null + var localEndpoint: RpcEndpointRef = null override def start() { - localActor = SparkEnv.get.actorSystem.actorOf( - Props(new LocalActor(scheduler, this, totalCores)), - "LocalBackendActor") + localEndpoint = SparkEnv.get.rpcEnv.setupEndpoint( + "LocalBackendEndpoint", new LocalEndpoint(SparkEnv.get.rpcEnv, scheduler, this, totalCores)) } override def stop() { - localActor ! StopExecutor + localEndpoint.ask(StopExecutor) } override def reviveOffers() { - localActor ! ReviveOffers + localEndpoint.send(ReviveOffers) } - override def defaultParallelism() = + override def defaultParallelism(): Int = scheduler.conf.getInt("spark.default.parallelism", totalCores) override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - localActor ! KillTask(taskId, interruptThread) + localEndpoint.send(KillTask(taskId, interruptThread)) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { - localActor ! StatusUpdate(taskId, state, serializedData) + localEndpoint.send(StatusUpdate(taskId, state, serializedData)) } override def applicationId(): String = appId diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 1baa0e009f3ae..dfbde7c8a1b0d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -59,9 +59,10 @@ private[spark] class JavaSerializationStream( } private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) -extends DeserializationStream { + extends DeserializationStream { + private val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass) = + override def resolveClass(desc: ObjectStreamClass): Class[_] = Class.forName(desc.getName, false, loader) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index dc7aa99738c17..b7bc087855b9f 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -49,10 +49,21 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSize = - (conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) * 1024 * 1024).toInt + private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") + + if (bufferSizeKb >= 2048) { + throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + + s"2048 mb, got: + $bufferSizeKb mb.") + } + private val bufferSize = (bufferSizeKb * 1024).toInt + + val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt + if (maxBufferSizeMb >= 2048) { + throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + + s"2048 mb, got: + $maxBufferSizeMb mb.") + } + private val maxBufferSize = maxBufferSizeMb * 1024 * 1024 - private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) private val userRegistrator = conf.getOption("spark.kryo.registrator") @@ -60,7 +71,7 @@ class KryoSerializer(conf: SparkConf) .split(',') .filter(!_.isEmpty) - def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator @@ -163,7 +174,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ } catch { case e: KryoException if e.getMessage.startsWith("Buffer overflow") => throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + - "increase spark.kryoserializer.buffer.max.mb value.") + "increase spark.kryoserializer.buffer.max value.") } ByteBuffer.wrap(output.toBytes) } @@ -189,6 +200,16 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ override def deserializeStream(s: InputStream): DeserializationStream = { new KryoDeserializationStream(kryo, s) } + + /** + * Returns true if auto-reset is on. The only reason this would be false is if the user-supplied + * registrator explicitly turns auto-reset off. + */ + def getAutoReset(): Boolean = { + val field = classOf[Kryo].getDeclaredField("autoReset") + field.setAccessible(true) + field.get(kryo).asInstanceOf[Boolean] + } } /** diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index cecb992579655..5abfa467c0ec8 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -23,6 +23,7 @@ import java.security.AccessController import scala.annotation.tailrec import scala.collection.mutable +import scala.util.control.NonFatal import org.apache.spark.Logging @@ -35,8 +36,15 @@ private[serializer] object SerializationDebugger extends Logging { */ def improveException(obj: Any, e: NotSerializableException): NotSerializableException = { if (enableDebugging && reflect != null) { - new NotSerializableException( - e.getMessage + "\nSerialization stack:\n" + find(obj).map("\t- " + _).mkString("\n")) + try { + new NotSerializableException( + e.getMessage + "\nSerialization stack:\n" + find(obj).map("\t- " + _).mkString("\n")) + } catch { + case NonFatal(t) => + // Fall back to old exception + logWarning("Exception in serialization debugger", t) + e + } } else { e } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index ca6e971d227fb..c381672a4f588 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -101,7 +101,12 @@ abstract class SerializerInstance { */ @DeveloperApi abstract class SerializationStream { + /** The most general-purpose method to write an object. */ def writeObject[T: ClassTag](t: T): SerializationStream + /** Writes the object representing the key of a key-value pair. */ + def writeKey[T: ClassTag](key: T): SerializationStream = writeObject(key) + /** Writes the object representing the value of a key-value pair. */ + def writeValue[T: ClassTag](value: T): SerializationStream = writeObject(value) def flush(): Unit def close(): Unit @@ -120,7 +125,12 @@ abstract class SerializationStream { */ @DeveloperApi abstract class DeserializationStream { + /** The most general-purpose method to read an object. */ def readObject[T: ClassTag](): T + /** Reads the object representing the key of a key-value pair. */ + def readKey[T: ClassTag](): T = readObject[T]() + /** Reads the object representing the value of a key-value pair. */ + def readValue[T: ClassTag](): T = readObject[T]() def close(): Unit /** @@ -141,4 +151,25 @@ abstract class DeserializationStream { DeserializationStream.this.close() } } + + /** + * Read the elements of this stream through an iterator over key-value pairs. This can only be + * called once, as reading each element will consume data from the input source. + */ + def asKeyValueIterator: Iterator[(Any, Any)] = new NextIterator[(Any, Any)] { + override protected def getNext() = { + try { + (readKey[Any](), readValue[Any]()) + } catch { + case eof: EOFException => { + finished = true + null + } + } + } + + override protected def close() { + DeserializationStream.this.close() + } + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 7de2f9cbb2866..e9b4e2b955dc8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -67,7 +67,7 @@ private[spark] trait ShuffleWriterGroup { // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getHashBasedShuffleBlockData(). private[spark] class FileShuffleBlockManager(conf: SparkConf) - extends ShuffleBlockManager with Logging { + extends ShuffleBlockResolver with Logging { private val transportConf = SparkTransportConf.fromSparkConf(conf) @@ -78,7 +78,8 @@ class FileShuffleBlockManager(conf: SparkConf) private val consolidateShuffleFiles = conf.getBoolean("spark.shuffle.consolidateFiles", false) - private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided + private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused @@ -106,17 +107,19 @@ class FileShuffleBlockManager(conf: SparkConf) * when the writers are closed successfully */ def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer, - writeMetrics: ShuffleWriteMetrics) = { + writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleId) private var fileGroup: ShuffleFileGroup = null + val openStartTime = System.nanoTime + val serializerInstance = serializer.newInstance() val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) - blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize, + blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializerInstance, bufferSize, writeMetrics) } } else { @@ -132,9 +135,13 @@ class FileShuffleBlockManager(conf: SparkConf) logWarning(s"Failed to remove existing shuffle file $blockFile") } } - blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize, writeMetrics) + blockManager.getDiskWriter(blockId, blockFile, serializerInstance, bufferSize, + writeMetrics) } } + // Creating the file to write to and creating a disk writer both involve interacting with + // the disk, so should be included in the shuffle write time. + writeMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) override def releaseWriters(success: Boolean) { if (consolidateShuffleFiles) { @@ -171,11 +178,6 @@ class FileShuffleBlockManager(conf: SparkConf) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockData(blockId) - Some(segment.nioByteBuffer()) - } - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. @@ -268,7 +270,7 @@ object FileShuffleBlockManager { new PrimitiveVector[Long]() } - def apply(bucketId: Int) = files(bucketId) + def apply(bucketId: Int): File = files(bucketId) def recordMapOutput(mapId: Int, offsets: Array[Long], lengths: Array[Long]) { assert(offsets.length == lengths.length) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index b292587d37028..a1741e2875c16 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -26,6 +26,9 @@ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ +import org.apache.spark.util.Utils + +import IndexShuffleBlockManager.NOOP_REDUCE_ID /** * Create and maintain the shuffle blocks' mapping between logic block and physical file location. @@ -39,25 +42,18 @@ import org.apache.spark.storage._ // Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). private[spark] -class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { +class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockResolver { private lazy val blockManager = SparkEnv.get.blockManager private val transportConf = SparkTransportConf.fromSparkConf(conf) - /** - * Mapping to a single shuffleBlockId with reduce ID 0. - * */ - def consolidateId(shuffleId: Int, mapId: Int): ShuffleBlockId = { - ShuffleBlockId(shuffleId, mapId, 0) - } - def getDataFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, 0)) + blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } private def getIndexFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, 0)) + blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } /** @@ -80,27 +76,22 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { * end of the output file. This will be used by getBlockLocation to figure out where each block * begins and ends. * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]) = { + def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) - try { + Utils.tryWithSafeFinally { // We take in lengths of each block, need to convert it to offsets. var offset = 0L out.writeLong(offset) - for (length <- lengths) { offset += length out.writeLong(offset) } - } finally { + } { out.close() } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - Some(getBlockData(blockId).nioByteBuffer()) - } - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index @@ -121,5 +112,13 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { } } - override def stop() = {} + override def stop(): Unit = {} +} + +private[spark] object IndexShuffleBlockManager { + // No-op reduce ID used in interactions with disk store and BlockObjectWriter. + // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort + // shuffle outputs for several reduces are glommed into a single file. + // TODO: Avoid this entirely by having the DiskBlockObjectWriter not require a BlockId. + val NOOP_REDUCE_ID = 0 } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala similarity index 68% rename from core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala rename to core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index b521f0c7fc77e..4342b0d598b16 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -22,15 +22,19 @@ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId private[spark] -trait ShuffleBlockManager { +/** + * Implementers of this trait understand how to retrieve block data for a logical shuffle block + * identifier (i.e. map, reduce, and shuffle). Implementations may use files or file segments to + * encapsulate shuffle data. This is used by the BlockStore to abstract over different shuffle + * implementations when shuffle data is retrieved. + */ +trait ShuffleBlockResolver { type ShuffleId = Int /** - * Get shuffle block data managed by the local ShuffleBlockManager. - * @return Some(ByteBuffer) if block found, otherwise None. + * Retrieve the data for the specified block. If the data for that block is not available, + * throws an unspecified exception. */ - def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] - def getBlockData(blockId: ShuffleBlockId): ManagedBuffer def stop(): Unit diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala index 13c7115f88afa..e04c97fe61894 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleHandle.scala @@ -17,9 +17,12 @@ package org.apache.spark.shuffle +import org.apache.spark.annotation.DeveloperApi + /** * An opaque handle to a shuffle, used by a ShuffleManager to pass information about it to tasks. * * @param shuffleId ID of the shuffle */ -private[spark] abstract class ShuffleHandle(val shuffleId: Int) extends Serializable {} +@DeveloperApi +abstract class ShuffleHandle(val shuffleId: Int) extends Serializable {} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index a44a8e1249256..978366d1a1d1b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -55,7 +55,10 @@ private[spark] trait ShuffleManager { */ def unregisterShuffle(shuffleId: Int): Boolean - def shuffleBlockManager: ShuffleBlockManager + /** + * Return a resolver capable of retrieving shuffle block data based on block coordinates. + */ + def shuffleBlockResolver: ShuffleBlockResolver /** Shut down this ShuffleManager. */ def stop(): Unit diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index b934480cfb9be..f6e6fe5defe09 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -23,7 +23,7 @@ import org.apache.spark.scheduler.MapStatus * Obtained inside a map task to write out records to the shuffle system. */ private[spark] trait ShuffleWriter[K, V] { - /** Write a bunch of records to this task's output */ + /** Write a sequence of records to this task's output */ def write(records: Iterator[_ <: Product2[K, V]]): Unit /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 7a2c5ae32d98b..80374adc44296 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -79,7 +79,8 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { blockManager, blocksByAddress, serializer, - SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) + // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility + SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 62e0629b34400..2a7df8dd5bd83 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -53,20 +53,20 @@ private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) : ShuffleWriter[K, V] = { new HashShuffleWriter( - shuffleBlockManager, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + shuffleBlockResolver, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - shuffleBlockManager.removeShuffle(shuffleId) + shuffleBlockResolver.removeShuffle(shuffleId) } - override def shuffleBlockManager: FileShuffleBlockManager = { + override def shuffleBlockResolver: FileShuffleBlockManager = { fileShuffleBlockManager } /** Shut down this ShuffleManager. */ override def stop(): Unit = { - shuffleBlockManager.stop() + shuffleBlockResolver.stop() } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 755f17d6aa15a..cd27c9e07a3cd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -63,7 +63,7 @@ private[spark] class HashShuffleWriter[K, V]( for (elem <- iter) { val bucketId = dep.partitioner.getPartition(elem._1) - shuffle.writers(bucketId).write(elem) + shuffle.writers(bucketId).write(elem._1, elem._2) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index bda30a56d808e..0497036192154 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -58,7 +58,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager val baseShuffleHandle = handle.asInstanceOf[BaseShuffleHandle[K, V, _]] shuffleMapNumber.putIfAbsent(baseShuffleHandle.shuffleId, baseShuffleHandle.numMaps) new SortShuffleWriter( - shuffleBlockManager, baseShuffleHandle, mapId, context) + shuffleBlockResolver, baseShuffleHandle, mapId, context) } /** Remove a shuffle's metadata from the ShuffleManager. */ @@ -66,18 +66,19 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager if (shuffleMapNumber.containsKey(shuffleId)) { val numMaps = shuffleMapNumber.remove(shuffleId) (0 until numMaps).map{ mapId => - shuffleBlockManager.removeDataByMap(shuffleId, mapId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId) } } true } - override def shuffleBlockManager: IndexShuffleBlockManager = { + override def shuffleBlockResolver: IndexShuffleBlockManager = { indexShuffleBlockManager } /** Shut down this ShuffleManager. */ override def stop(): Unit = { - shuffleBlockManager.stop() + shuffleBlockResolver.stop() } } + diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index fa2e617762f55..a066435df6fb0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -58,13 +58,15 @@ private[spark] class SortShuffleWriter[K, V, C]( // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't // care whether the keys get sorted in each partition; that will be done on the reduce side // if the operation being run is sortByKey. - sorter = new ExternalSorter[K, V, V]( - None, Some(dep.partitioner), None, dep.serializer) + sorter = new ExternalSorter[K, V, V](None, Some(dep.partitioner), None, dep.serializer) sorter.insertAll(records) } + // Don't bother including the time to open the merged output file in the shuffle write time, + // because it just opens a single file, so is typically too fast to measure accurately + // (see SPARK-3570). val outputFile = shuffleBlockManager.getDataFile(dep.shuffleId, mapId) - val blockId = shuffleBlockManager.consolidateId(dep.shuffleId, mapId) + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) @@ -97,3 +99,4 @@ private[spark] class SortShuffleWriter[K, V, C]( } } } + diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala new file mode 100644 index 0000000000000..5783df5d8220c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala @@ -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. + */ +package org.apache.spark.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.jobs.UIData.JobUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllJobsResource(ui: SparkUI) { + + @GET + def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = { + val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = + AllJobsResource.getStatusToJobs(ui) + val adjStatuses: JList[JobExecutionStatus] = { + if (statuses.isEmpty) { + Arrays.asList(JobExecutionStatus.values(): _*) + } else { + statuses + } + } + val jobInfos = for { + (status, jobs) <- statusToJobs + job <- jobs if adjStatuses.contains(status) + } yield { + AllJobsResource.convertJobData(job, ui.jobProgressListener, false) + } + jobInfos.sortBy{- _.jobId} + } + +} + +private[v1] object AllJobsResource { + + def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = { + val statusToJobs = ui.jobProgressListener.synchronized { + Seq( + JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq, + JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq, + JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq + ) + } + statusToJobs + } + + def convertJobData( + job: JobUIData, + listener: JobProgressListener, + includeStageDetails: Boolean): JobData = { + listener.synchronized { + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap { _.description } + new JobData( + jobId = job.jobId, + name = lastStageName, + description = lastStageDescription, + submissionTime = job.submissionTime.map{new Date(_)}, + completionTime = job.completionTime.map{new Date(_)}, + stageIds = job.stageIds, + jobGroup = job.jobGroup, + status = job.status, + numTasks = job.numTasks, + numActiveTasks = job.numActiveTasks, + numCompletedTasks = job.numCompletedTasks, + numSkippedTasks = job.numCompletedTasks, + numFailedTasks = job.numFailedTasks, + numActiveStages = job.numActiveStages, + numCompletedStages = job.completedStageIndices.size, + numSkippedStages = job.numSkippedStages, + numFailedStages = job.numFailedStages + ) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala new file mode 100644 index 0000000000000..645ede26a0879 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -0,0 +1,104 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils} +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.storage.StorageListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllRDDResource(ui: SparkUI) { + + @GET + def rddList(): Seq[RDDStorageInfo] = { + val storageStatusList = ui.storageListener.storageStatusList + val rddInfos = ui.storageListener.rddInfoList + rddInfos.map{rddInfo => + AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList, + includeDetails = false) + } + } + +} + +private[spark] object AllRDDResource { + + def getRDDStorageInfo( + rddId: Int, + listener: StorageListener, + includeDetails: Boolean): Option[RDDStorageInfo] = { + val storageStatusList = listener.storageStatusList + listener.rddInfoList.find { _.id == rddId }.map { rddInfo => + getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails) + } + } + + def getRDDStorageInfo( + rddId: Int, + rddInfo: RDDInfo, + storageStatusList: Seq[StorageStatus], + includeDetails: Boolean): RDDStorageInfo = { + val workers = storageStatusList.map { (rddId, _) } + val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) + val blocks = storageStatusList + .flatMap { _.rddBlocksById(rddId) } + .sortWith { _._1.name < _._1.name } + .map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + } + + val dataDistribution = if (includeDetails) { + Some(storageStatusList.map { status => + new RDDDataDistribution( + address = status.blockManagerId.hostPort, + memoryUsed = status.memUsedByRdd(rddId), + memoryRemaining = status.memRemaining, + diskUsed = status.diskUsedByRdd(rddId) + ) } ) + } else { + None + } + val partitions = if (includeDetails) { + Some(blocks.map { case (id, block, locations) => + new RDDPartitionInfo( + blockName = id.name, + storageLevel = block.storageLevel.description, + memoryUsed = block.memSize, + diskUsed = block.diskSize, + executors = locations + ) + } ) + } else { + None + } + + new RDDStorageInfo( + id = rddId, + name = rddInfo.name, + numPartitions = rddInfo.numPartitions, + numCachedPartitions = rddInfo.numCachedPartitions, + storageLevel = rddInfo.storageLevel.description, + memoryUsed = rddInfo.memSize, + diskUsed = rddInfo.diskSize, + dataDistribution = dataDistribution, + partitions = partitions + ) + } +} 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 new file mode 100644 index 0000000000000..50608588f09ae --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -0,0 +1,309 @@ +/* + * 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.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +import javax.ws.rs.{GET, PathParam, 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.util.Distribution + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllStagesResource(ui: SparkUI) { + + @GET + def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = { + val listener = ui.jobProgressListener + val stageAndStatus = AllStagesResource.stagesAndStatus(ui) + val adjStatuses = { + if (statuses.isEmpty()) { + Arrays.asList(StageStatus.values(): _*) + } else { + statuses + } + } + for { + (status, stageList) <- stageAndStatus + stageInfo: StageInfo <- stageList if adjStatuses.contains(status) + stageUiData: StageUIData <- listener.synchronized { + listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId)) + } + } yield { + AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) + } + } +} + +private[v1] object AllStagesResource { + def stageUiToStageData( + status: StageStatus, + stageInfo: StageInfo, + stageUiData: StageUIData, + includeDetails: Boolean): StageData = { + + val taskData = if (includeDetails) { + Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } ) + } else { + None + } + val executorSummary = if (includeDetails) { + Some(stageUiData.executorSummary.map { case (k, summary) => + k -> new ExecutorStageSummary( + taskTime = summary.taskTime, + failedTasks = summary.failedTasks, + succeededTasks = summary.succeededTasks, + inputBytes = summary.inputBytes, + outputBytes = summary.outputBytes, + shuffleRead = summary.shuffleRead, + shuffleWrite = summary.shuffleWrite, + memoryBytesSpilled = summary.memoryBytesSpilled, + diskBytesSpilled = summary.diskBytesSpilled + ) + }) + } else { + None + } + + val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq + + new StageData( + status = status, + stageId = stageInfo.stageId, + attemptId = stageInfo.attemptId, + numActiveTasks = stageUiData.numActiveTasks, + numCompleteTasks = stageUiData.numCompleteTasks, + numFailedTasks = stageUiData.numFailedTasks, + executorRunTime = stageUiData.executorRunTime, + inputBytes = stageUiData.inputBytes, + inputRecords = stageUiData.inputRecords, + outputBytes = stageUiData.outputBytes, + outputRecords = stageUiData.outputRecords, + shuffleReadBytes = stageUiData.shuffleReadTotalBytes, + shuffleReadRecords = stageUiData.shuffleReadRecords, + shuffleWriteBytes = stageUiData.shuffleWriteBytes, + shuffleWriteRecords = stageUiData.shuffleWriteRecords, + memoryBytesSpilled = stageUiData.memoryBytesSpilled, + diskBytesSpilled = stageUiData.diskBytesSpilled, + schedulingPool = stageUiData.schedulingPool, + name = stageInfo.name, + details = stageInfo.details, + accumulatorUpdates = accumulableInfo, + tasks = taskData, + executorSummary = executorSummary + ) + } + + def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = { + val listener = ui.jobProgressListener + listener.synchronized { + Seq( + StageStatus.ACTIVE -> listener.activeStages.values.toSeq, + StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq, + StageStatus.FAILED -> listener.failedStages.reverse.toSeq, + StageStatus.PENDING -> listener.pendingStages.values.toSeq + ) + } + } + + def convertTaskData(uiData: TaskUIData): TaskData = { + new TaskData( + taskId = uiData.taskInfo.taskId, + index = uiData.taskInfo.index, + attempt = uiData.taskInfo.attempt, + launchTime = new Date(uiData.taskInfo.launchTime), + executorId = uiData.taskInfo.executorId, + host = uiData.taskInfo.host, + taskLocality = uiData.taskInfo.taskLocality.toString(), + speculative = uiData.taskInfo.speculative, + accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, + errorMessage = uiData.errorMessage, + taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics } + ) + } + + def taskMetricDistributions( + allTaskData: Iterable[TaskUIData], + quantiles: Array[Double]): TaskMetricDistributions = { + + val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq + + def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = + Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) + + // We need to do a lot of similar munging to nested metrics here. For each one, + // we want (a) extract the values for nested metrics (b) make a distribution for each metric + // (c) shove the distribution into the right field in our return type and (d) only return + // a result if the option is defined for any of the tasks. MetricHelper is a little util + // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just + // implement one "build" method, which just builds the quantiles for each field. + + val inputMetrics: Option[InputMetricDistributions] = + new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = { + raw.inputMetrics + } + + def build: InputMetricDistributions = new InputMetricDistributions( + bytesRead = submetricQuantiles(_.bytesRead), + recordsRead = submetricQuantiles(_.recordsRead) + ) + }.metricOption + + val outputMetrics: Option[OutputMetricDistributions] = + new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { + def getSubmetrics(raw:InternalTaskMetrics): Option[InternalOutputMetrics] = { + raw.outputMetrics + } + def build: OutputMetricDistributions = new OutputMetricDistributions( + bytesWritten = submetricQuantiles(_.bytesWritten), + recordsWritten = submetricQuantiles(_.recordsWritten) + ) + }.metricOption + + val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] = + new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = { + raw.shuffleReadMetrics + } + def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( + readBytes = submetricQuantiles(_.totalBytesRead), + readRecords = submetricQuantiles(_.recordsRead), + remoteBytesRead = submetricQuantiles(_.remoteBytesRead), + remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), + localBlocksFetched = submetricQuantiles(_.localBlocksFetched), + totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), + fetchWaitTime = submetricQuantiles(_.fetchWaitTime) + ) + }.metricOption + + val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] = + new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, + quantiles) { + def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = { + raw.shuffleWriteMetrics + } + def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( + writeBytes = submetricQuantiles(_.shuffleBytesWritten), + writeRecords = submetricQuantiles(_.shuffleRecordsWritten), + writeTime = submetricQuantiles(_.shuffleWriteTime) + ) + }.metricOption + + new TaskMetricDistributions( + quantiles = quantiles, + executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), + executorRunTime = metricQuantiles(_.executorRunTime), + resultSize = metricQuantiles(_.resultSize), + jvmGcTime = metricQuantiles(_.jvmGCTime), + resultSerializationTime = metricQuantiles(_.resultSerializationTime), + memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled), + diskBytesSpilled = metricQuantiles(_.diskBytesSpilled), + inputMetrics = inputMetrics, + outputMetrics = outputMetrics, + shuffleReadMetrics = shuffleReadMetrics, + shuffleWriteMetrics = shuffleWriteMetrics + ) + } + + def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { + new AccumulableInfo(acc.id, acc.name, acc.update, acc.value) + } + + def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { + new TaskMetrics( + executorDeserializeTime = internal.executorDeserializeTime, + executorRunTime = internal.executorRunTime, + resultSize = internal.resultSize, + jvmGcTime = internal.jvmGCTime, + resultSerializationTime = internal.resultSerializationTime, + memoryBytesSpilled = internal.memoryBytesSpilled, + diskBytesSpilled = internal.diskBytesSpilled, + inputMetrics = internal.inputMetrics.map { convertInputMetrics }, + outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics }, + shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics }, + shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics } + ) + } + + def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = { + new InputMetrics( + bytesRead = internal.bytesRead, + recordsRead = internal.recordsRead + ) + } + + def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = { + new OutputMetrics( + bytesWritten = internal.bytesWritten, + recordsWritten = internal.recordsWritten + ) + } + + def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = { + new ShuffleReadMetrics( + remoteBlocksFetched = internal.remoteBlocksFetched, + localBlocksFetched = internal.localBlocksFetched, + fetchWaitTime = internal.fetchWaitTime, + remoteBytesRead = internal.remoteBytesRead, + totalBlocksFetched = internal.totalBlocksFetched, + recordsRead = internal.recordsRead + ) + } + + def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { + new ShuffleWriteMetrics( + bytesWritten = internal.shuffleBytesWritten, + writeTime = internal.shuffleWriteTime, + recordsWritten = internal.shuffleRecordsWritten + ) + } +} + +/** + * Helper for getting distributions from nested metric types. Many of the metrics we want are + * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle + * the options (returning None if the metrics are all empty), and extract the quantiles for each + * metric. After creating an instance, call metricOption to get the result type. + */ +private[v1] abstract class MetricHelper[I,O]( + rawMetrics: Seq[InternalTaskMetrics], + quantiles: Array[Double]) { + + def getSubmetrics(raw: InternalTaskMetrics): Option[I] + + def build: O + + val data: Seq[I] = rawMetrics.flatMap(getSubmetrics) + + /** applies the given function to all input metrics, and returns the quantiles */ + def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { + Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) + } + + def metricOption: Option[O] = { + if (data.isEmpty) { + None + } else { + Some(build) + } + } +} 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 new file mode 100644 index 0000000000000..17b521f3e1d41 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -0,0 +1,94 @@ +/* + * 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.status.api.v1 + +import java.util.{Arrays, Date, List => JList} +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) { + + @GET + def appList( + @QueryParam("status") status: JList[ApplicationStatus], + @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, + @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam) + : Iterator[ApplicationInfo] = { + val allApps = uiRoot.getApplicationInfoList + val adjStatus = { + if (status.isEmpty) { + Arrays.asList(ApplicationStatus.values(): _*) + } else { + status + } + } + val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED) + val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING) + allApps.filter { app => + val anyRunning = app.attempts.exists(!_.completed) + // if any attempt is still running, we consider the app to also still be running + val statusOk = (!anyRunning && includeCompleted) || + (anyRunning && includeRunning) + // keep the app if *any* attempts fall in the right time window + val dateOk = app.attempts.exists { attempt => + attempt.startTime.getTime >= minDate.timestamp && + attempt.startTime.getTime <= maxDate.timestamp + } + statusOk && dateOk + } + } +} + +private[spark] object ApplicationsListResource { + def appHistoryInfoToPublicAppInfo(app: ApplicationHistoryInfo): ApplicationInfo = { + new ApplicationInfo( + id = app.id, + name = app.name, + attempts = app.attempts.map { internalAttemptInfo => + new ApplicationAttemptInfo( + attemptId = internalAttemptInfo.attemptId, + startTime = new Date(internalAttemptInfo.startTime), + endTime = new Date(internalAttemptInfo.endTime), + sparkUser = internalAttemptInfo.sparkUser, + completed = internalAttemptInfo.completed + ) + } + ) + } + + def convertApplicationInfo( + internal: InternalApplicationInfo, + completed: Boolean): ApplicationInfo = { + // standalone application info always has just one attempt + new ApplicationInfo( + id = internal.id, + name = internal.desc.name, + attempts = Seq(new ApplicationAttemptInfo( + attemptId = None, + startTime = new Date(internal.startTime), + endTime = new Date(internal.endTime), + sparkUser = internal.desc.user, + completed = completed + )) + ) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala new file mode 100644 index 0000000000000..8ad4656b4dada --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -0,0 +1,36 @@ +/* +* 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.status.api.v1 + +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.exec.ExecutorsPage + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class ExecutorListResource(ui: SparkUI) { + + @GET + def executorList(): Seq[ExecutorSummary] = { + val listener = ui.executorsListener + val storageStatusList = listener.storageStatusList + (0 until storageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala new file mode 100644 index 0000000000000..202a5191ad57d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -0,0 +1,93 @@ +/* + * 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.status.api.v1 + +import java.io.OutputStream +import java.lang.annotation.Annotation +import java.lang.reflect.Type +import java.text.SimpleDateFormat +import java.util.{Calendar, SimpleTimeZone} +import javax.ws.rs.Produces +import javax.ws.rs.core.{MediaType, MultivaluedMap} +import javax.ws.rs.ext.{MessageBodyWriter, Provider} + +import com.fasterxml.jackson.annotation.JsonInclude +import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature} + +/** + * This class converts the POJO metric responses into json, using jackson. + * + * This doesn't follow the standard jersey-jackson plugin options, because we want to stick + * with an old version of jersey (since we have it from yarn anyway) and don't want to pull in lots + * of dependencies from a new plugin. + * + * Note that jersey automatically discovers this class based on its package and its annotations. + */ +@Provider +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ + + val mapper = new ObjectMapper() { + override def writeValueAsString(t: Any): String = { + super.writeValueAsString(t) + } + } + mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule) + mapper.enable(SerializationFeature.INDENT_OUTPUT) + mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL) + mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat) + + override def isWriteable( + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType): Boolean = { + true + } + + override def writeTo( + t: Object, + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType, + multivaluedMap: MultivaluedMap[String, AnyRef], + outputStream: OutputStream): Unit = { + t match { + case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) + case _ => mapper.writeValue(outputStream, t) + } + } + + override def getSize( + t: Object, + aClass: Class[_], + `type`: Type, + annotations: Array[Annotation], + mediaType: MediaType): Long = { + -1L + } +} + +private[spark] object JacksonMessageWriter { + def makeISODateFormat: SimpleDateFormat = { + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) + iso8601.setCalendar(cal) + iso8601 + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala new file mode 100644 index 0000000000000..c3ec45f54681b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala @@ -0,0 +1,255 @@ +/* + * 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.status.api.v1 + +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.apache.spark.SecurityManager +import org.apache.spark.ui.SparkUI + +/** + * Main entry point for serving spark application metrics as json, using JAX-RS. + * + * Each resource should have endpoints that return **public** classes defined in api.scala. Mima + * binary compatibility checks ensure that we don't inadvertently make changes that break the api. + * The returned objects are automatically converted to json by jackson with JacksonMessageWriter. + * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden + * files". Any changes and additions should be reflected there as well -- see the notes in + * HistoryServerSuite. + */ +@Path("/v1") +private[v1] class JsonRootResource extends UIRootFromServletContext { + + @Path("applications") + def getApplicationList(): ApplicationListResource = { + new ApplicationListResource(uiRoot) + } + + @Path("applications/{appId}") + def getApplication(): OneApplicationResource = { + new OneApplicationResource(uiRoot) + } + + @Path("applications/{appId}/{attemptId}/jobs") + def getJobs( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllJobsResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllJobsResource(ui) + } + } + + @Path("applications/{appId}/jobs") + def getJobs(@PathParam("appId") appId: String): AllJobsResource = { + uiRoot.withSparkUI(appId, None) { ui => + new AllJobsResource(ui) + } + } + + @Path("applications/{appId}/jobs/{jobId: \\d+}") + def getJob(@PathParam("appId") appId: String): OneJobResource = { + uiRoot.withSparkUI(appId, None) { ui => + new OneJobResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}") + def getJob( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneJobResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneJobResource(ui) + } + } + + @Path("applications/{appId}/executors") + def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = { + uiRoot.withSparkUI(appId, None) { ui => + new ExecutorListResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/executors") + def getExecutors( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): ExecutorListResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new ExecutorListResource(ui) + } + } + + + @Path("applications/{appId}/stages") + def getStages(@PathParam("appId") appId: String): AllStagesResource= { + uiRoot.withSparkUI(appId, None) { ui => + new AllStagesResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/stages") + def getStages( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllStagesResource= { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllStagesResource(ui) + } + } + + @Path("applications/{appId}/stages/{stageId: \\d+}") + def getStage(@PathParam("appId") appId: String): OneStageResource= { + uiRoot.withSparkUI(appId, None) { ui => + new OneStageResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}") + def getStage( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneStageResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneStageResource(ui) + } + } + + @Path("applications/{appId}/storage/rdd") + def getRdds(@PathParam("appId") appId: String): AllRDDResource = { + uiRoot.withSparkUI(appId, None) { ui => + new AllRDDResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/storage/rdd") + def getRdds( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllRDDResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new AllRDDResource(ui) + } + } + + @Path("applications/{appId}/storage/rdd/{rddId: \\d+}") + def getRdd(@PathParam("appId") appId: String): OneRDDResource = { + uiRoot.withSparkUI(appId, None) { ui => + new OneRDDResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}") + def getRdd( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): OneRDDResource = { + uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + new OneRDDResource(ui) + } + } + +} + +private[spark] object JsonRootResource { + + def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = { + val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) + jerseyContext.setContextPath("/json") + 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) + UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) + jerseyContext.addServlet(holder, "/*") + jerseyContext + } +} + +/** + * 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 + * interface needed for them all to expose application info as json. + */ +private[spark] trait UIRoot { + def getSparkUI(appKey: String): Option[SparkUI] + def getApplicationInfoList: Iterator[ApplicationInfo] + + /** + * Get the spark UI with the given appID, and apply a function + * to it. If there is no such app, throw an appropriate exception + */ + def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { + val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) + getSparkUI(appKey) match { + case Some(ui) => + f(ui) + case None => throw new NotFoundException("no such app: " + appId) + } + } + def securityManager: SecurityManager +} + +private[v1] object UIRootFromServletContext { + + private val attribute = getClass.getCanonicalName + + def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = { + contextHandler.setAttribute(attribute, uiRoot) + } + + def getUiRoot(context: ServletContext): UIRoot = { + context.getAttribute(attribute).asInstanceOf[UIRoot] + } +} + +private[v1] trait UIRootFromServletContext { + @Context + var servletContext: ServletContext = _ + + def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext) +} + +private[v1] class NotFoundException(msg: String) extends WebApplicationException( + new NoSuchElementException(msg), + Response + .status(Response.Status.NOT_FOUND) + .entity(ErrorWrapper(msg)) + .build() +) + +private[v1] class BadParameterException(msg: String) extends WebApplicationException( + new IllegalArgumentException(msg), + Response + .status(Response.Status.BAD_REQUEST) + .entity(ErrorWrapper(msg)) + .build() +) { + def this(param: String, exp: String, actual: String) = { + this(raw"""Bad value for parameter "$param". Expected a $exp, got "$actual"""") + } +} + +/** + * Signal to JacksonMessageWriter to not convert the message into json (which would result in an + * extra set of quotes). + */ +private[v1] case class ErrorWrapper(s: String) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala new file mode 100644 index 0000000000000..b5ef72649e295 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -0,0 +1,31 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs.core.MediaType +import javax.ws.rs.{Produces, PathParam, GET} + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneApplicationResource(uiRoot: UIRoot) { + + @GET + def getApp(@PathParam("appId") appId: String): ApplicationInfo = { + val apps = uiRoot.getApplicationInfoList.find { _.id == appId } + apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala new file mode 100644 index 0000000000000..6d8a60d480aed --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala @@ -0,0 +1,41 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.UIData.JobUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneJobResource(ui: SparkUI) { + + @GET + def oneJob(@PathParam("jobId") jobId: Int): JobData = { + val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = + AllJobsResource.getStatusToJobs(ui) + val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId} + jobOpt.map { job => + AllJobsResource.convertJobData(job, ui.jobProgressListener, false) + }.getOrElse { + throw new NotFoundException("unknown job: " + jobId) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala new file mode 100644 index 0000000000000..07b224fac4786 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala @@ -0,0 +1,34 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneRDDResource(ui: SparkUI) { + + @GET + def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo = { + AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse( + throw new NotFoundException(s"no rdd found w/ id $rddId") + ) + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala new file mode 100644 index 0000000000000..fd24aea63a8a1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala @@ -0,0 +1,150 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.SparkException +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.api.v1.StageStatus._ +import org.apache.spark.status.api.v1.TaskSorting._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.jobs.UIData.StageUIData + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneStageResource(ui: SparkUI) { + + @GET + @Path("") + def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { + withStage(stageId){ stageAttempts => + stageAttempts.map { stage => + AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, + includeDetails = true) + } + } + } + + @GET + @Path("/{stageAttemptId: \\d+}") + def oneAttemptData( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { + withStageAttempt(stageId, stageAttemptId) { stage => + AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, + includeDetails = true) + } + } + + @GET + @Path("/{stageAttemptId: \\d+}/taskSummary") + def taskSummary( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String) + : TaskMetricDistributions = { + withStageAttempt(stageId, stageAttemptId) { stage => + val quantiles = quantileString.split(",").map { s => + try { + s.toDouble + } catch { + case nfe: NumberFormatException => + throw new BadParameterException("quantiles", "double", s) + } + } + AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles) + } + } + + @GET + @Path("/{stageAttemptId: \\d+}/taskList") + def taskList( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @DefaultValue("0") @QueryParam("offset") offset: Int, + @DefaultValue("20") @QueryParam("length") length: Int, + @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { + withStageAttempt(stageId, stageAttemptId) { stage => + val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq + .sorted(OneStageResource.ordering(sortBy)) + tasks.slice(offset, offset + length) + } + } + + private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData) + + private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = { + val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId) + if (stageAttempts.isEmpty) { + throw new NotFoundException("unknown stage: " + stageId) + } else { + f(stageAttempts) + } + } + + private def findStageStatusUIData( + listener: JobProgressListener, + stageId: Int): Seq[StageStatusInfoUi] = { + listener.synchronized { + def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = { + infos.filter { _.stageId == stageId }.map { info => + val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId), + // this is an internal error -- we should always have uiData + throw new SparkException( + s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}") + ) + StageStatusInfoUi(status, info, ui) + } + } + getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++ + getStatusInfoUi(COMPLETE, listener.completedStages) ++ + getStatusInfoUi(FAILED, listener.failedStages) ++ + getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq) + } + } + + private def withStageAttempt[T]( + stageId: Int, + stageAttemptId: Int) + (f: StageStatusInfoUi => T): T = { + withStage(stageId) { attempts => + val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId } + oneAttempt match { + case Some(stage) => + f(stage) + case None => + val stageAttempts = attempts.map { _.info.attemptId } + throw new NotFoundException(s"unknown attempt for stage $stageId. " + + s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}") + } + } + } +} + +object OneStageResource { + def ordering(taskSorting: TaskSorting): Ordering[TaskData] = { + val extractor: (TaskData => Long) = td => + taskSorting match { + case ID => td.taskId + case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) + case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L) + } + Ordering.by(extractor) + } +} 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 new file mode 100644 index 0000000000000..95fbd96ade5ab --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.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.status.api.v1 + +import javax.ws.rs.WebApplicationException +import javax.ws.rs.core.Response + +import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} + +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( + Response + .status(Response.Status.FORBIDDEN) + .entity(raw"""user "$user"is not authorized""") + .build() + ) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala new file mode 100644 index 0000000000000..cee29786c3019 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala @@ -0,0 +1,55 @@ +/* + * 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.status.api.v1 + +import java.text.SimpleDateFormat +import java.util.TimeZone +import javax.ws.rs.WebApplicationException +import javax.ws.rs.core.Response +import javax.ws.rs.core.Response.Status + +import scala.util.Try + +private[v1] class SimpleDateParam(val originalValue: String) { + val timestamp: Long = { + SimpleDateParam.formats.collectFirst { + case fmt if Try(fmt.parse(originalValue)).isSuccess => + fmt.parse(originalValue).getTime() + }.getOrElse( + throw new WebApplicationException( + Response + .status(Status.BAD_REQUEST) + .entity("Couldn't parse date: " + originalValue) + .build() + ) + ) + } +} + +private[v1] object SimpleDateParam { + + val formats: Seq[SimpleDateFormat] = { + + val gmtDay = new SimpleDateFormat("yyyy-MM-dd") + gmtDay.setTimeZone(TimeZone.getTimeZone("GMT")) + + Seq( + new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz"), + gmtDay + ) + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala new file mode 100644 index 0000000000000..ef3c8570d8186 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -0,0 +1,228 @@ +/* + * 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.status.api.v1 + +import java.util.Date + +import scala.collection.Map + +import org.apache.spark.JobExecutionStatus + +class ApplicationInfo private[spark]( + val id: String, + val name: String, + val attempts: Seq[ApplicationAttemptInfo]) + +class ApplicationAttemptInfo private[spark]( + val attemptId: Option[String], + val startTime: Date, + val endTime: Date, + val sparkUser: String, + val completed: Boolean = false) + +class ExecutorStageSummary private[spark]( + val taskTime : Long, + val failedTasks : Int, + val succeededTasks : Int, + val inputBytes : Long, + val outputBytes : Long, + val shuffleRead : Long, + val shuffleWrite : Long, + val memoryBytesSpilled : Long, + val diskBytesSpilled : Long) + +class ExecutorSummary private[spark]( + val id: String, + val hostPort: String, + val rddBlocks: Int, + val memoryUsed: Long, + val diskUsed: Long, + val activeTasks: Int, + val failedTasks: Int, + val completedTasks: Int, + val totalTasks: Int, + val totalDuration: Long, + val totalInputBytes: Long, + val totalShuffleRead: Long, + val totalShuffleWrite: Long, + val maxMemory: Long, + val executorLogs: Map[String, String]) + +class JobData private[spark]( + val jobId: Int, + val name: String, + val description: Option[String], + val submissionTime: Option[Date], + val completionTime: Option[Date], + val stageIds: Seq[Int], + val jobGroup: Option[String], + val status: JobExecutionStatus, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numSkippedTasks: Int, + val numFailedTasks: Int, + val numActiveStages: Int, + val numCompletedStages: Int, + val numSkippedStages: Int, + val numFailedStages: Int) + +// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage +// page ... does anybody pay attention to it? +class RDDStorageInfo private[spark]( + val id: Int, + val name: String, + val numPartitions: Int, + val numCachedPartitions: Int, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val dataDistribution: Option[Seq[RDDDataDistribution]], + val partitions: Option[Seq[RDDPartitionInfo]]) + +class RDDDataDistribution private[spark]( + val address: String, + val memoryUsed: Long, + val memoryRemaining: Long, + val diskUsed: Long) + +class RDDPartitionInfo private[spark]( + val blockName: String, + val storageLevel: String, + val memoryUsed: Long, + val diskUsed: Long, + val executors: Seq[String]) + +class StageData private[spark]( + val status: StageStatus, + val stageId: Int, + val attemptId: Int, + val numActiveTasks: Int , + val numCompleteTasks: Int, + val numFailedTasks: Int, + + val executorRunTime: Long, + + val inputBytes: Long, + val inputRecords: Long, + val outputBytes: Long, + val outputRecords: Long, + val shuffleReadBytes: Long, + val shuffleReadRecords: Long, + val shuffleWriteBytes: Long, + val shuffleWriteRecords: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + + val name: String, + val details: String, + val schedulingPool: String, + + val accumulatorUpdates: Seq[AccumulableInfo], + val tasks: Option[Map[Long, TaskData]], + val executorSummary:Option[Map[String,ExecutorStageSummary]]) + +class TaskData private[spark]( + val taskId: Long, + val index: Int, + val attempt: Int, + val launchTime: Date, + val executorId: String, + val host: String, + val taskLocality: String, + val speculative: Boolean, + val accumulatorUpdates: Seq[AccumulableInfo], + val errorMessage: Option[String] = None, + val taskMetrics: Option[TaskMetrics] = None) + +class TaskMetrics private[spark]( + val executorDeserializeTime: Long, + val executorRunTime: Long, + val resultSize: Long, + val jvmGcTime: Long, + val resultSerializationTime: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + val inputMetrics: Option[InputMetrics], + val outputMetrics: Option[OutputMetrics], + val shuffleReadMetrics: Option[ShuffleReadMetrics], + val shuffleWriteMetrics: Option[ShuffleWriteMetrics]) + +class InputMetrics private[spark]( + val bytesRead: Long, + val recordsRead: Long) + +class OutputMetrics private[spark]( + val bytesWritten: Long, + val recordsWritten: Long) + +class ShuffleReadMetrics private[spark]( + val remoteBlocksFetched: Int, + val localBlocksFetched: Int, + val fetchWaitTime: Long, + val remoteBytesRead: Long, + val totalBlocksFetched: Int, + val recordsRead: Long) + +class ShuffleWriteMetrics private[spark]( + val bytesWritten: Long, + val writeTime: Long, + val recordsWritten: Long) + +class TaskMetricDistributions private[spark]( + val quantiles: IndexedSeq[Double], + + val executorDeserializeTime: IndexedSeq[Double], + val executorRunTime: IndexedSeq[Double], + val resultSize: IndexedSeq[Double], + val jvmGcTime: IndexedSeq[Double], + val resultSerializationTime: IndexedSeq[Double], + val memoryBytesSpilled: IndexedSeq[Double], + val diskBytesSpilled: IndexedSeq[Double], + + val inputMetrics: Option[InputMetricDistributions], + val outputMetrics: Option[OutputMetricDistributions], + val shuffleReadMetrics: Option[ShuffleReadMetricDistributions], + val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions]) + +class InputMetricDistributions private[spark]( + val bytesRead: IndexedSeq[Double], + val recordsRead: IndexedSeq[Double]) + +class OutputMetricDistributions private[spark]( + val bytesWritten: IndexedSeq[Double], + val recordsWritten: IndexedSeq[Double]) + +class ShuffleReadMetricDistributions private[spark]( + val readBytes: IndexedSeq[Double], + val readRecords: IndexedSeq[Double], + val remoteBlocksFetched: IndexedSeq[Double], + val localBlocksFetched: IndexedSeq[Double], + val fetchWaitTime: IndexedSeq[Double], + val remoteBytesRead: IndexedSeq[Double], + val totalBlocksFetched: IndexedSeq[Double]) + +class ShuffleWriteMetricDistributions private[spark]( + val writeBytes: IndexedSeq[Double], + val writeRecords: IndexedSeq[Double], + val writeTime: IndexedSeq[Double]) + +class AccumulableInfo private[spark]( + val id: Long, + val name: String, + val update: Option[String], + val value: String) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 1f012941c85ab..c186fd360fef6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -35,13 +35,13 @@ sealed abstract class BlockId { def name: String // convenience methods - def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None - def isRDD = isInstanceOf[RDDBlockId] - def isShuffle = isInstanceOf[ShuffleBlockId] - def isBroadcast = isInstanceOf[BroadcastBlockId] + def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None + def isRDD: Boolean = isInstanceOf[RDDBlockId] + def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] + def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] - override def toString = name - override def hashCode = name.hashCode + override def toString: String = name + override def hashCode: Int = name.hashCode override def equals(other: Any): Boolean = other match { case o: BlockId => getClass == o.getClass && name.equals(o.name) case _ => false @@ -50,54 +50,54 @@ sealed abstract class BlockId { @DeveloperApi case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { - def name = "rdd_" + rddId + "_" + splitIndex + override def name: String = "rdd_" + rddId + "_" + splitIndex } // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getBlockData(). @DeveloperApi case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } @DeveloperApi case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" } @DeveloperApi case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { - def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) + override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } @DeveloperApi case class TaskResultBlockId(taskId: Long) extends BlockId { - def name = "taskresult_" + taskId + override def name: String = "taskresult_" + taskId } @DeveloperApi case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { - def name = "input-" + streamId + "-" + uniqueId + override def name: String = "input-" + streamId + "-" + uniqueId } /** Id associated with temporary local data managed as blocks. Not serializable. */ private[spark] case class TempLocalBlockId(id: UUID) extends BlockId { - def name = "temp_local_" + id + override def name: String = "temp_local_" + id } /** Id associated with temporary shuffle data managed as blocks. Not serializable. */ private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId { - def name = "temp_shuffle_" + id + override def name: String = "temp_shuffle_" + id } // Intended only for testing purposes private[spark] case class TestBlockId(id: String) extends BlockId { - def name = "test_" + id + override def name: String = "test_" + id } @DeveloperApi @@ -112,7 +112,7 @@ object BlockId { val TEST = "test_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ - def apply(id: String) = id match { + def apply(id: String): BlockId = id match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => 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 80d66e59132da..a46fecd2274ef 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -26,18 +26,18 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ -import org.apache.spark.executor._ +import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo -import org.apache.spark.serializer.Serializer +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.serializer.{SerializerInstance, Serializer} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util._ @@ -50,11 +50,8 @@ private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( val data: Iterator[Any], - readMethod: DataReadMethod.Value, - bytes: Long) { - val inputMetrics = new InputMetrics(readMethod) - inputMetrics.incBytesRead(bytes) -} + val readMethod: DataReadMethod.Value, + val bytes: Long) /** * Manager running on every node (driver and executors) which provides interfaces for putting and @@ -64,7 +61,7 @@ private[spark] class BlockResult( */ private[spark] class BlockManager( executorId: String, - actorSystem: ActorSystem, + rpcEnv: RpcEnv, val master: BlockManagerMaster, defaultSerializer: Serializer, maxMemory: Long, @@ -81,19 +78,11 @@ private[spark] class BlockManager( private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] // Actual storage of where blocks are kept - private var tachyonInitialized = false + private var externalBlockStoreInitialized = false private[spark] val memoryStore = new MemoryStore(this, maxMemory) private[spark] val diskStore = new DiskStore(this, diskBlockManager) - private[spark] lazy val tachyonStore: TachyonStore = { - val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") - val appFolderName = conf.get("spark.tachyonStore.folderName") - val tachyonStorePath = s"$storeDir/$appFolderName/${this.executorId}" - val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") - val tachyonBlockManager = - new TachyonBlockManager(this, tachyonStorePath, tachyonMaster) - tachyonInitialized = true - new TachyonStore(this, tachyonBlockManager) - } + private[spark] lazy val externalBlockStore: ExternalBlockStore = + new ExternalBlockStore(this, executorId) private[spark] val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) @@ -122,7 +111,8 @@ private[spark] class BlockManager( // standard BlockTransferService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { val transConf = SparkTransportConf.fromSparkConf(conf, numUsableCores) - new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled()) + new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled(), + securityManager.isSaslEncryptionEnabled()) } else { blockTransferService } @@ -136,9 +126,9 @@ private[spark] class BlockManager( // Whether to compress shuffle output temporarily spilled to disk private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) - private val slaveActor = actorSystem.actorOf( - Props(new BlockManagerSlaveActor(this, mapOutputTracker)), - name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) + private val slaveEndpoint = rpcEnv.setupEndpoint( + "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, + new BlockManagerSlaveEndpoint(rpcEnv, this, mapOutputTracker)) // Pending re-registration action being executed asynchronously or null if none is pending. // Accesses should synchronize on asyncReregisterLock. @@ -167,7 +157,7 @@ private[spark] class BlockManager( */ def this( execId: String, - actorSystem: ActorSystem, + rpcEnv: RpcEnv, master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, @@ -176,7 +166,7 @@ private[spark] class BlockManager( blockTransferService: BlockTransferService, securityManager: SecurityManager, numUsableCores: Int) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), + this(execId, rpcEnv, master, serializer, BlockManager.getMaxMemory(conf), conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager, numUsableCores) } @@ -186,7 +176,7 @@ private[spark] class BlockManager( * where it is only learned after registration with the TaskScheduler). * * This method initializes the BlockTransferService and ShuffleClient, registers with the - * BlockManagerMaster, starts the BlockManagerWorker actor, and registers with a local shuffle + * BlockManagerMaster, starts the BlockManagerWorker endpoint, and registers with a local shuffle * service if configured. */ def initialize(appId: String): Unit = { @@ -202,7 +192,7 @@ private[spark] class BlockManager( blockManagerId } - master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) // Register Executors' configuration with the local shuffle service, if one should exist. if (externalShuffleServiceEnabled && !blockManagerId.isDriver) { @@ -265,7 +255,7 @@ private[spark] class BlockManager( def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") - master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) reportAllBlocks() } @@ -301,7 +291,7 @@ private[spark] class BlockManager( */ override def getBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { - shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) .asInstanceOf[Option[ByteBuffer]] @@ -323,13 +313,13 @@ private[spark] class BlockManager( /** * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. + * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { blockInfo.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L - // Assume that block is not in Tachyon + // Assume that block is not in external block store BlockStatus(info.level, memSize, diskSize, 0L) } } @@ -379,10 +369,10 @@ private[spark] class BlockManager( if (info.tellMaster) { val storageLevel = status.storageLevel val inMemSize = Math.max(status.memSize, droppedMemorySize) - val inTachyonSize = status.tachyonSize + val inExternalBlockStoreSize = status.externalBlockStoreSize val onDiskSize = status.diskSize master.updateBlockInfo( - blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) + blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inExternalBlockStoreSize) } else { true } @@ -400,15 +390,17 @@ private[spark] class BlockManager( BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) - val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) + val inExternalBlockStore = level.useOffHeap && externalBlockStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false - val replication = if (inMem || inTachyon || onDisk) level.replication else 1 - val storageLevel = StorageLevel(onDisk, inMem, inTachyon, deserialized, replication) + val replication = if (inMem || inExternalBlockStore || onDisk) level.replication else 1 + val storageLevel = + StorageLevel(onDisk, inMem, inExternalBlockStore, deserialized, replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L - val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else 0L + val externalBlockStoreSize = + if (inExternalBlockStore) externalBlockStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - BlockStatus(storageLevel, memSize, diskSize, tachyonSize) + BlockStatus(storageLevel, memSize, diskSize, externalBlockStoreSize) } } } @@ -439,14 +431,10 @@ private[spark] class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { - val shuffleBlockManager = shuffleManager.shuffleBlockManager - shuffleBlockManager.getBytes(blockId.asInstanceOf[ShuffleBlockId]) match { - case Some(bytes) => - Some(bytes) - case None => - throw new BlockException( - blockId, s"Block $blockId not found on disk, though it should be") - } + val shuffleBlockManager = shuffleManager.shuffleBlockResolver + // TODO: This should gracefully handle case where local block is not available. Currently + // downstream code will throw an exception. + Option(shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) } else { doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } @@ -492,11 +480,11 @@ private[spark] class BlockManager( } } - // Look for the block in Tachyon + // Look for the block in external block store if (level.useOffHeap) { - logDebug(s"Getting block $blockId from tachyon") - if (tachyonStore.contains(blockId)) { - tachyonStore.getBytes(blockId) match { + logDebug(s"Getting block $blockId from ExternalBlockStore") + if (externalBlockStore.contains(blockId)) { + externalBlockStore.getBytes(blockId) match { case Some(bytes) => if (!asBlockResult) { return Some(bytes) @@ -505,7 +493,7 @@ private[spark] class BlockManager( dataDeserialize(blockId, bytes), DataReadMethod.Memory, info.size)) } case None => - logDebug(s"Block $blockId not found in tachyon") + logDebug(s"Block $blockId not found in externalBlockStore") } } } @@ -535,9 +523,14 @@ private[spark] class BlockManager( /* We'll store the bytes in memory if the block's storage level includes * "memory serialized", or if it should be cached as objects in memory * but we only requested its serialized bytes. */ - val copyForMemory = ByteBuffer.allocate(bytes.limit) - copyForMemory.put(bytes) - memoryStore.putBytes(blockId, copyForMemory, level) + memoryStore.putBytes(blockId, bytes.limit, () => { + // https://issues.apache.org/jira/browse/SPARK-6076 + // If the file size is bigger than the free memory, OOM will happen. So if we cannot + // put it into MemoryStore, copyForMemory should not be created. That's why this + // action is put into a `() => ByteBuffer` and created lazily. + val copyForMemory = ByteBuffer.allocate(bytes.limit) + copyForMemory.put(bytes) + }) bytes.rewind() } if (!asBlockResult) { @@ -645,13 +638,13 @@ private[spark] class BlockManager( def getDiskWriter( blockId: BlockId, file: File, - serializer: Serializer, + serializerInstance: SerializerInstance, bufferSize: Int, writeMetrics: ShuffleWriteMetrics): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites, - writeMetrics) + new DiskBlockObjectWriter(blockId, file, serializerInstance, bufferSize, compressStream, + syncWrites, writeMetrics) } /** @@ -768,8 +761,8 @@ private[spark] class BlockManager( // We will drop it to disk later if the memory store can't hold it. (true, memoryStore) } else if (putLevel.useOffHeap) { - // Use tachyon for off-heap storage - (false, tachyonStore) + // Use external block store + (false, externalBlockStore) } else if (putLevel.useDisk) { // Don't get back the bytes from put unless we replicate them (putLevel.replication > 1, diskStore) @@ -804,7 +797,7 @@ private[spark] class BlockManager( val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (putBlockStatus.storageLevel != StorageLevel.NONE) { - // Now that the block is in either the memory, tachyon, or disk store, + // Now that the block is in either the memory, externalBlockStore, or disk store, // let other threads read it, and tell the master about it. marked = true putBlockInfo.markReady(size) @@ -991,15 +984,23 @@ private[spark] class BlockManager( putIterator(blockId, Iterator(value), level, tellMaster) } + def dropFromMemory( + blockId: BlockId, + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { + dropFromMemory(blockId, () => data) + } + /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. * + * If `data` is not put on disk, it won't be created. + * * Return the block status if the given block has been updated, else None. */ def dropFromMemory( blockId: BlockId, - data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { + data: () => Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull @@ -1023,7 +1024,7 @@ private[spark] class BlockManager( // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { logInfo(s"Writing block $blockId to disk") - data match { + data() match { case Left(elements) => diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => @@ -1093,10 +1094,11 @@ private[spark] class BlockManager( // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) - val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false - if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { + val removedFromExternalBlockStore = + if (externalBlockStoreInitialized) externalBlockStore.remove(blockId) else false + if (!removedFromMemory && !removedFromDisk && !removedFromExternalBlockStore) { logWarning(s"Block $blockId could not be removed as it was not found in either " + - "the disk, memory, or tachyon store") + "the disk, memory, or external block store") } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { @@ -1130,7 +1132,7 @@ private[spark] class BlockManager( val level = info.level if (level.useMemory) { memoryStore.remove(id) } if (level.useDisk) { diskStore.remove(id) } - if (level.useOffHeap) { tachyonStore.remove(id) } + if (level.useOffHeap) { externalBlockStore.remove(id) } iterator.remove() logInfo(s"Dropped block $id") } @@ -1206,12 +1208,12 @@ private[spark] class BlockManager( shuffleClient.close() } diskBlockManager.stop() - actorSystem.stop(slaveActor) + rpcEnv.stop(slaveEndpoint) blockInfo.clear() memoryStore.clear() diskStore.clear() - if (tachyonInitialized) { - tachyonStore.clear() + if (externalBlockStoreInitialized) { + externalBlockStore.clear() } metadataCleaner.cancel() broadcastCleaner.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b177a59c721df..69ac37511e730 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -60,7 +60,10 @@ class BlockManagerId private ( def port: Int = port_ - def isDriver: Boolean = { executorId == SparkContext.DRIVER_IDENTIFIER } + def isDriver: Boolean = { + executorId == SparkContext.DRIVER_IDENTIFIER || + executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER + } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) @@ -77,11 +80,11 @@ class BlockManagerId private ( @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = s"BlockManagerId($executorId, $host, $port)" + override def toString: String = s"BlockManagerId($executorId, $host, $port)" override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port - override def equals(that: Any) = that match { + override def equals(that: Any): Boolean = that match { case id: BlockManagerId => executorId == id.executorId && port == id.port && host == id.host case _ => @@ -100,10 +103,10 @@ private[spark] object BlockManagerId { * @param port Port of the block manager. * @return A new [[org.apache.spark.storage.BlockManagerId]]. */ - def apply(execId: String, host: String, port: Int) = + def apply(execId: String, host: String, port: Int): BlockManagerId = getCachedBlockManagerId(new BlockManagerId(execId, host, port)) - def apply(in: ObjectInput) = { + def apply(in: ObjectInput): BlockManagerId = { val obj = new BlockManagerId() obj.readExternal(in) getCachedBlockManagerId(obj) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 654796f23c96e..a85e1c7632973 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -20,35 +20,31 @@ package org.apache.spark.storage import scala.concurrent.{Await, Future} import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor._ - +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.RpcUtils private[spark] class BlockManagerMaster( - var driverActor: ActorRef, + var driverEndpoint: RpcEndpointRef, conf: SparkConf, isDriver: Boolean) extends Logging { - private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) - private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) - - val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = AkkaUtils.askTimeout(conf) + val timeout = RpcUtils.askTimeout(conf) - /** Remove a dead executor from the driver actor. This is only called on the driver side. */ + /** Remove a dead executor from the driver endpoint. This is only called on the driver side. */ def removeExecutor(execId: String) { tell(RemoveExecutor(execId)) logInfo("Removed " + execId + " successfully in removeExecutor") } /** Register the BlockManager's id with the driver. */ - def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + def registerBlockManager( + blockManagerId: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef): Unit = { logInfo("Trying to register BlockManager") - tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) + tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) logInfo("Registered BlockManager") } @@ -58,38 +54,39 @@ class BlockManagerMaster( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long): Boolean = { - val res = askDriverWithReply[Boolean]( - UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) + externalBlockStoreSize: Long): Boolean = { + val res = driverEndpoint.askWithRetry[Boolean]( + UpdateBlockInfo(blockManagerId, blockId, storageLevel, + memSize, diskSize, externalBlockStoreSize)) logDebug(s"Updated info of block $blockId") res } /** Get locations of the blockId from the driver */ def getLocations(blockId: BlockId): Seq[BlockManagerId] = { - askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId)) + driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetLocations(blockId)) } /** Get locations of multiple blockIds from the driver */ def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { - askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) + driverEndpoint.askWithRetry[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } /** * Check if block manager master has a block. Note that this can be used to check for only * those blocks that are reported to block manager master. */ - def contains(blockId: BlockId) = { + def contains(blockId: BlockId): Boolean = { !getLocations(blockId).isEmpty } /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { - askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) + driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId)) } - def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { - askDriverWithReply[Option[(String, Int)]](GetActorSystemHostPortForExecutor(executorId)) + def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { + driverEndpoint.askWithRetry[Option[(String, Int)]](GetRpcHostPortForExecutor(executorId)) } /** @@ -97,12 +94,12 @@ class BlockManagerMaster( * blocks that the driver knows about. */ def removeBlock(blockId: BlockId) { - askDriverWithReply(RemoveBlock(blockId)) + driverEndpoint.askWithRetry[Boolean](RemoveBlock(blockId)) } /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) + val future = driverEndpoint.askWithRetry[Future[Seq[Int]]](RemoveRdd(rddId)) future.onFailure { case e: Exception => logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}") @@ -114,7 +111,7 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given shuffle. */ def removeShuffle(shuffleId: Int, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + val future = driverEndpoint.askWithRetry[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) future.onFailure { case e: Exception => logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}") @@ -126,7 +123,7 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given broadcast. */ def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { - val future = askDriverWithReply[Future[Seq[Int]]]( + val future = driverEndpoint.askWithRetry[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) future.onFailure { case e: Exception => @@ -145,11 +142,11 @@ class BlockManagerMaster( * amount of remaining memory. */ def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { - askDriverWithReply[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + driverEndpoint.askWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) } def getStorageStatus: Array[StorageStatus] = { - askDriverWithReply[Array[StorageStatus]](GetStorageStatus) + driverEndpoint.askWithRetry[Array[StorageStatus]](GetStorageStatus) } /** @@ -165,11 +162,12 @@ class BlockManagerMaster( askSlaves: Boolean = true): Map[BlockManagerId, BlockStatus] = { val msg = GetBlockStatus(blockId, askSlaves) /* - * To avoid potential deadlocks, the use of Futures is necessary, because the master actor + * To avoid potential deadlocks, the use of Futures is necessary, because the master endpoint * should not block on waiting for a block manager, which can in turn be waiting for the - * master actor for a response to a prior message. + * master endpoint for a response to a prior message. */ - val response = askDriverWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + val response = driverEndpoint. + askWithRetry[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) val (blockManagerIds, futures) = response.unzip val result = Await.result(Future.sequence(futures), timeout) if (result == null) { @@ -193,33 +191,28 @@ class BlockManagerMaster( filter: BlockId => Boolean, askSlaves: Boolean): Seq[BlockId] = { val msg = GetMatchingBlockIds(filter, askSlaves) - val future = askDriverWithReply[Future[Seq[BlockId]]](msg) + val future = driverEndpoint.askWithRetry[Future[Seq[BlockId]]](msg) Await.result(future, timeout) } - /** Stop the driver actor, called only on the Spark driver node */ + /** Stop the driver endpoint, called only on the Spark driver node */ def stop() { - if (driverActor != null && isDriver) { + if (driverEndpoint != null && isDriver) { tell(StopBlockManagerMaster) - driverActor = null + driverEndpoint = null logInfo("BlockManagerMaster stopped") } } - /** Send a one-way message to the master actor, to which we expect it to reply with true. */ + /** Send a one-way message to the master endpoint, to which we expect it to reply with true. */ private def tell(message: Any) { - if (!askDriverWithReply[Boolean](message)) { - throw new SparkException("BlockManagerMasterActor returned false, expected true.") + if (!driverEndpoint.askWithRetry[Boolean](message)) { + throw new SparkException("BlockManagerMasterEndpoint returned false, expected true.") } } - /** - * Send a message to the driver actor and get its result within a default timeout, or - * throw a SparkException if this fails. - */ - private def askDriverWithReply[T](message: Any): T = { - AkkaUtils.askWithReply(message, driverActor, AKKA_RETRY_ATTEMPTS, AKKA_RETRY_INTERVAL_MS, - timeout) - } +} +private[spark] object BlockManagerMaster { + val DRIVER_ENDPOINT_NAME = "BlockManagerMaster" } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala similarity index 79% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 787b0f96bec32..3afb4c3c02e2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -21,25 +21,26 @@ import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.collection.JavaConversions._ -import scala.concurrent.Future -import scala.concurrent.duration._ +import scala.concurrent.{ExecutionContext, Future} -import akka.actor.{Actor, ActorRef} -import akka.pattern.ask - -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, ThreadSafeRpcEndpoint} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} +import org.apache.spark.util.{ThreadUtils, Utils} /** - * BlockManagerMasterActor is an actor on the master node to track statuses of - * all slaves' block managers. + * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses + * of all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) - extends Actor with ActorLogReceive with Logging { +class BlockManagerMasterEndpoint( + override val rpcEnv: RpcEnv, + val isLocal: Boolean, + conf: SparkConf, + listenerBus: LiveListenerBus) + extends ThreadSafeRpcEndpoint with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] @@ -50,68 +51,67 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - private val akkaTimeout = AkkaUtils.askTimeout(conf) + private val askThreadPool = ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool") + private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool) - override def receiveWithLogging = { - case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => - register(blockManagerId, maxMemSize, slaveActor) - sender ! true + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint) => + register(blockManagerId, maxMemSize, slaveEndpoint) + context.reply(true) case UpdateBlockInfo( - blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => - sender ! updateBlockInfo( - blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) + blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize) => + context.reply(updateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize)) case GetLocations(blockId) => - sender ! getLocations(blockId) + context.reply(getLocations(blockId)) case GetLocationsMultipleBlockIds(blockIds) => - sender ! getLocationsMultipleBlockIds(blockIds) + context.reply(getLocationsMultipleBlockIds(blockIds)) case GetPeers(blockManagerId) => - sender ! getPeers(blockManagerId) + context.reply(getPeers(blockManagerId)) - case GetActorSystemHostPortForExecutor(executorId) => - sender ! getActorSystemHostPortForExecutor(executorId) + case GetRpcHostPortForExecutor(executorId) => + context.reply(getRpcHostPortForExecutor(executorId)) case GetMemoryStatus => - sender ! memoryStatus + context.reply(memoryStatus) case GetStorageStatus => - sender ! storageStatus + context.reply(storageStatus) case GetBlockStatus(blockId, askSlaves) => - sender ! blockStatus(blockId, askSlaves) + context.reply(blockStatus(blockId, askSlaves)) case GetMatchingBlockIds(filter, askSlaves) => - sender ! getMatchingBlockIds(filter, askSlaves) + context.reply(getMatchingBlockIds(filter, askSlaves)) case RemoveRdd(rddId) => - sender ! removeRdd(rddId) + context.reply(removeRdd(rddId)) case RemoveShuffle(shuffleId) => - sender ! removeShuffle(shuffleId) + context.reply(removeShuffle(shuffleId)) case RemoveBroadcast(broadcastId, removeFromDriver) => - sender ! removeBroadcast(broadcastId, removeFromDriver) + context.reply(removeBroadcast(broadcastId, removeFromDriver)) case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) - sender ! true + context.reply(true) case RemoveExecutor(execId) => removeExecutor(execId) - sender ! true + context.reply(true) case StopBlockManagerMaster => - sender ! true - context.stop(self) + context.reply(true) + stop() case BlockManagerHeartbeat(blockManagerId) => - sender ! heartbeatReceived(blockManagerId) + context.reply(heartbeatReceived(blockManagerId)) - case other => - logWarning("Got unknown message: " + other) } private def removeRdd(rddId: Int): Future[Seq[Int]] = { @@ -129,22 +129,20 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. // The dispatcher is used as an implicit argument into the Future sequence construction. - import context.dispatcher val removeMsg = RemoveRdd(rddId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + bm.slaveEndpoint.ask[Int](removeMsg) }.toSeq ) } private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { - // Nothing to do in the BlockManagerMasterActor data structures - import context.dispatcher + // Nothing to do in the BlockManagerMasterEndpoint data structures val removeMsg = RemoveShuffle(shuffleId) Future.sequence( blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] + bm.slaveEndpoint.ask[Boolean](removeMsg) }.toSeq ) } @@ -155,14 +153,13 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus * from the executors, but not from the driver. */ private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { - import context.dispatcher val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) val requiredBlockManagers = blockManagerInfo.values.filter { info => removeFromDriver || !info.blockManagerId.isDriver } Future.sequence( requiredBlockManagers.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + bm.slaveEndpoint.ask[Int](removeMsg) }.toSeq ) } @@ -217,7 +214,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // Remove the block from the slave's BlockManager. // Doesn't actually wait for a confirmation and the message might get lost. // If message loss becomes frequent, we should add retry logic here. - blockManager.get.slaveActor.ask(RemoveBlock(blockId))(akkaTimeout) + blockManager.get.slaveEndpoint.ask[Boolean](RemoveBlock(blockId)) } } } @@ -247,17 +244,16 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private def blockStatus( blockId: BlockId, askSlaves: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { - import context.dispatcher val getBlockStatus = GetBlockStatus(blockId) /* - * Rather than blocking on the block status query, master actor should simply return + * Rather than blocking on the block status query, master endpoint should simply return * Futures to avoid potential deadlocks. This can arise if there exists a block manager - * that is also waiting for this master actor's response to a previous message. + * that is also waiting for this master endpoint's response to a previous message. */ blockManagerInfo.values.map { info => val blockStatusFuture = if (askSlaves) { - info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo[Option[BlockStatus]] + info.slaveEndpoint.ask[Option[BlockStatus]](getBlockStatus) } else { Future { info.getStatus(blockId) } } @@ -276,13 +272,12 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private def getMatchingBlockIds( filter: BlockId => Boolean, askSlaves: Boolean): Future[Seq[BlockId]] = { - import context.dispatcher val getMatchingBlockIds = GetMatchingBlockIds(filter) Future.sequence( blockManagerInfo.values.map { info => val future = if (askSlaves) { - info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo[Seq[BlockId]] + info.slaveEndpoint.ask[Seq[BlockId]](getMatchingBlockIds) } else { Future { info.blocks.keys.filter(filter).toSeq } } @@ -291,7 +286,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus ).map(_.flatten.toSeq) } - private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + private def register(id: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef) { val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -308,7 +303,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockManagerIdByExecutor(id.executorId) = id blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveActor) + id, System.currentTimeMillis(), maxMemSize, slaveEndpoint) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) } @@ -319,7 +314,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long): Boolean = { + externalBlockStoreSize: Long): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { @@ -337,7 +332,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } blockManagerInfo(blockManagerId).updateBlockInfo( - blockId, storageLevel, memSize, diskSize, tachyonSize) + blockId, storageLevel, memSize, diskSize, externalBlockStoreSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { @@ -379,19 +374,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } /** - * Returns the hostname and port of an executor's actor system, based on the Akka address of its - * BlockManagerSlaveActor. + * Returns the hostname and port of an executor, based on the [[RpcEnv]] address of its + * [[BlockManagerSlaveEndpoint]]. */ - private def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { + private def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { for ( blockManagerId <- blockManagerIdByExecutor.get(executorId); - info <- blockManagerInfo.get(blockManagerId); - host <- info.slaveActor.path.address.host; - port <- info.slaveActor.path.address.port + info <- blockManagerInfo.get(blockManagerId) ) yield { - (host, port) + (info.slaveEndpoint.address.host, info.slaveEndpoint.address.port) } } + + override def onStop(): Unit = { + askThreadPool.shutdownNow() + } } @DeveloperApi @@ -399,8 +396,8 @@ case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) { - def isCached: Boolean = memSize + diskSize + tachyonSize > 0 + externalBlockStoreSize: Long) { + def isCached: Boolean = memSize + diskSize + externalBlockStoreSize > 0 } @DeveloperApi @@ -412,7 +409,7 @@ private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, val maxMem: Long, - val slaveActor: ActorRef) + val slaveEndpoint: RpcEndpointRef) extends Logging { private var _lastSeenMs: Long = timeMs @@ -421,7 +418,7 @@ private[spark] class BlockManagerInfo( // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) + def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId)) def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() @@ -432,7 +429,7 @@ private[spark] class BlockManagerInfo( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) { + externalBlockStoreSize: Long) { updateLastSeenMs() @@ -448,9 +445,9 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory, on-disk or on-Tachyon. + /* isValid means it is either stored in-memory, on-disk or on-externalBlockStore. * The memSize here indicates the data size in or dropped from memory, - * tachyonSize here indicates the data size in or dropped from Tachyon, + * externalBlockStoreSize here indicates the data size in or dropped from externalBlockStore, * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ @@ -467,9 +464,9 @@ private[spark] class BlockManagerInfo( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } if (storageLevel.useOffHeap) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, tachyonSize)) - logInfo("Added %s on tachyon on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(tachyonSize))) + _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, externalBlockStoreSize)) + logInfo("Added %s on ExternalBlockStore on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(externalBlockStoreSize))) } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. @@ -485,8 +482,9 @@ private[spark] class BlockManagerInfo( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } if (blockStatus.storageLevel.useOffHeap) { - logInfo("Removed %s on %s on tachyon (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize))) + logInfo("Removed %s on %s on externalBlockStore (size: %s)".format( + blockId, blockManagerId.hostPort, + Utils.bytesToString(blockStatus.externalBlockStoreSize))) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 48247453edef0..1683576067fe8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -19,8 +19,7 @@ package org.apache.spark.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} -import akka.actor.ActorRef - +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils private[spark] object BlockManagerMessages { @@ -52,7 +51,7 @@ private[spark] object BlockManagerMessages { case class RegisterBlockManager( blockManagerId: BlockManagerId, maxMemSize: Long, - sender: ActorRef) + sender: RpcEndpointRef) extends ToBlockManagerMaster case class UpdateBlockInfo( @@ -61,7 +60,7 @@ private[spark] object BlockManagerMessages { var storageLevel: StorageLevel, var memSize: Long, var diskSize: Long, - var tachyonSize: Long) + var externalBlockStoreSize: Long) extends ToBlockManagerMaster with Externalizable { @@ -73,7 +72,7 @@ private[spark] object BlockManagerMessages { storageLevel.writeExternal(out) out.writeLong(memSize) out.writeLong(diskSize) - out.writeLong(tachyonSize) + out.writeLong(externalBlockStoreSize) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -82,7 +81,7 @@ private[spark] object BlockManagerMessages { storageLevel = StorageLevel(in) memSize = in.readLong() diskSize = in.readLong() - tachyonSize = in.readLong() + externalBlockStoreSize = in.readLong() } } @@ -92,7 +91,7 @@ private[spark] object BlockManagerMessages { case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class GetActorSystemHostPortForExecutor(executorId: String) extends ToBlockManagerMaster + case class GetRpcHostPortForExecutor(executorId: String) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index 8462871e798a5..543df4e1350dd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -17,41 +17,43 @@ package org.apache.spark.storage -import scala.concurrent.Future - -import akka.actor.{ActorRef, Actor} +import scala.concurrent.{ExecutionContext, Future} +import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} +import org.apache.spark.util.ThreadUtils import org.apache.spark.{Logging, MapOutputTracker, SparkEnv} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.ActorLogReceive /** - * An actor to take commands from the master to execute options. For example, + * An RpcEndpoint to take commands from the master to execute options. For example, * this is used to remove blocks from the slave's BlockManager. */ private[storage] -class BlockManagerSlaveActor( +class BlockManagerSlaveEndpoint( + override val rpcEnv: RpcEnv, blockManager: BlockManager, mapOutputTracker: MapOutputTracker) - extends Actor with ActorLogReceive with Logging { + extends RpcEndpoint with Logging { - import context.dispatcher + private val asyncThreadPool = + ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool") + private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool) // Operations that involve removing blocks may be slow and should be done asynchronously - override def receiveWithLogging = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RemoveBlock(blockId) => - doAsync[Boolean]("removing block " + blockId, sender) { + doAsync[Boolean]("removing block " + blockId, context) { blockManager.removeBlock(blockId) true } case RemoveRdd(rddId) => - doAsync[Int]("removing RDD " + rddId, sender) { + doAsync[Int]("removing RDD " + rddId, context) { blockManager.removeRdd(rddId) } case RemoveShuffle(shuffleId) => - doAsync[Boolean]("removing shuffle " + shuffleId, sender) { + doAsync[Boolean]("removing shuffle " + shuffleId, context) { if (mapOutputTracker != null) { mapOutputTracker.unregisterShuffle(shuffleId) } @@ -59,30 +61,34 @@ class BlockManagerSlaveActor( } case RemoveBroadcast(broadcastId, _) => - doAsync[Int]("removing broadcast " + broadcastId, sender) { + doAsync[Int]("removing broadcast " + broadcastId, context) { blockManager.removeBroadcast(broadcastId, tellMaster = true) } case GetBlockStatus(blockId, _) => - sender ! blockManager.getStatus(blockId) + context.reply(blockManager.getStatus(blockId)) case GetMatchingBlockIds(filter, _) => - sender ! blockManager.getMatchingBlockIds(filter) + context.reply(blockManager.getMatchingBlockIds(filter)) } - private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) { + private def doAsync[T](actionMessage: String, context: RpcCallContext)(body: => T) { val future = Future { logDebug(actionMessage) body } future.onSuccess { case response => logDebug("Done " + actionMessage + ", response is " + response) - responseActor ! response - logDebug("Sent response: " + response + " to " + responseActor) + context.reply(response) + logDebug("Sent response: " + response + " to " + context.sender) } future.onFailure { case t: Throwable => logError("Error in " + actionMessage, t) - responseActor ! null.asInstanceOf[T] + context.sendFailure(t) } } + + override def onStop(): Unit = { + asyncThreadPool.shutdownNow() + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 81164178b9e8e..499dd97c0656a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -21,8 +21,9 @@ import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel import org.apache.spark.Logging -import org.apache.spark.serializer.{SerializationStream, Serializer} +import org.apache.spark.serializer.{SerializerInstance, SerializationStream} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.util.Utils /** * An interface for writing JVM objects to some underlying storage. This interface allows @@ -32,7 +33,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics * This interface does not support concurrent writes. Also, once the writer has * been opened, it cannot be reopened again. */ -private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { +private[spark] abstract class BlockObjectWriter(val blockId: BlockId) extends OutputStream { def open(): BlockObjectWriter @@ -53,9 +54,14 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { def revertPartialWritesAndClose() /** - * Writes an object. + * Writes a key-value pair. */ - def write(value: Any) + def write(key: Any, value: Any) + + /** + * Notify the writer that a record worth of bytes has been written with writeBytes. + */ + def recordWritten() /** * Returns the file segment of committed data that this Writer has written. @@ -70,7 +76,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { private[spark] class DiskBlockObjectWriter( blockId: BlockId, file: File, - serializer: Serializer, + serializerInstance: SerializerInstance, bufferSize: Int, compressStream: OutputStream => OutputStream, syncWrites: Boolean, @@ -82,11 +88,13 @@ private[spark] class DiskBlockObjectWriter( { /** Intercepts write calls and tracks total time spent writing. Not thread safe. */ private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream { - def write(i: Int): Unit = callWithTiming(out.write(i)) - override def write(b: Array[Byte]) = callWithTiming(out.write(b)) - override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len)) - override def close() = out.close() - override def flush() = out.flush() + override def write(i: Int): Unit = callWithTiming(out.write(i)) + override def write(b: Array[Byte]): Unit = callWithTiming(out.write(b)) + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + callWithTiming(out.write(b, off, len)) + } + override def close(): Unit = out.close() + override def flush(): Unit = out.flush() } /** The file channel, used for repositioning / truncating the file. */ @@ -131,20 +139,24 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() bs = compressStream(new BufferedOutputStream(ts, bufferSize)) - objOut = serializer.newInstance().serializeStream(bs) + objOut = serializerInstance.serializeStream(bs) initialized = true this } override def close() { if (initialized) { - if (syncWrites) { - // Force outstanding writes to disk and track how long it takes - objOut.flush() - def sync = fos.getFD.sync() - callWithTiming(sync) + Utils.tryWithSafeFinally { + if (syncWrites) { + // Force outstanding writes to disk and track how long it takes + objOut.flush() + callWithTiming { + fos.getFD.sync() + } + } + } { + objOut.close() } - objOut.close() channel = null bs = null @@ -196,12 +208,32 @@ private[spark] class DiskBlockObjectWriter( } } - override def write(value: Any) { + override def write(key: Any, value: Any) { + if (!initialized) { + open() + } + + objOut.writeKey(key) + objOut.writeValue(value) + numRecordsWritten += 1 + writeMetrics.incShuffleRecordsWritten(1) + + if (numRecordsWritten % 32 == 0) { + updateBytesWritten() + } + } + + override def write(b: Int): Unit = throw new UnsupportedOperationException() + + override def write(kvBytes: Array[Byte], offs: Int, len: Int): Unit = { if (!initialized) { open() } - objOut.writeObject(value) + bs.write(kvBytes, offs, len) + } + + override def recordWritten(): Unit = { numRecordsWritten += 1 writeMetrics.incShuffleRecordsWritten(1) @@ -231,7 +263,7 @@ private[spark] class DiskBlockObjectWriter( } // For testing - private[spark] def flush() { + private[spark] override def flush() { objOut.flush() bs.flush() } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 12cd8ea3bdf1f..5764c16902c66 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -47,6 +47,8 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } + // The content of subDirs is immutable but the content of subDirs(i) is mutable. And the content + // of subDirs(i) is protected by the lock of subDirs(i) private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private val shutdownHook = addShutdownHook() @@ -61,20 +63,17 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon val subDirId = (hash / localDirs.length) % subDirsPerLocalDir // Create the subdirectory if it doesn't already exist - var subDir = subDirs(dirId)(subDirId) - if (subDir == null) { - subDir = subDirs(dirId).synchronized { - val old = subDirs(dirId)(subDirId) - if (old != null) { - old - } else { - val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) - if (!newDir.exists() && !newDir.mkdir()) { - throw new IOException(s"Failed to create local dir in $newDir.") - } - subDirs(dirId)(subDirId) = newDir - newDir + val subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val newDir = new File(localDirs(dirId), "%02x".format(subDirId)) + if (!newDir.exists() && !newDir.mkdir()) { + throw new IOException(s"Failed to create local dir in $newDir.") } + subDirs(dirId)(subDirId) = newDir + newDir } } @@ -91,7 +90,12 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** List all the files currently stored on disk by the disk manager. */ def getAllFiles(): Seq[File] = { // Get all the files inside the array of array of directories - subDirs.flatten.filter(_ != null).flatMap { dir => + subDirs.flatMap { dir => + dir.synchronized { + // Copy the content of dir because it may be modified in other threads + dir.clone() + } + }.filter(_ != null).flatMap { dir => val files = dir.listFiles() if (files != null) files else Seq.empty } @@ -134,24 +138,21 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } } - private def addShutdownHook(): Thread = { - val shutdownHook = new Thread("delete Spark local dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - DiskBlockManager.this.doStop() - } + private def addShutdownHook(): AnyRef = { + Utils.addShutdownHook { () => + logDebug("Shutdown hook called") + DiskBlockManager.this.doStop() } - Runtime.getRuntime.addShutdownHook(shutdownHook) - shutdownHook } /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { // Remove the shutdown hook. It causes memory leaks if we leave it around. try { - Runtime.getRuntime.removeShutdownHook(shutdownHook) + Utils.removeShutdownHook(shutdownHook) } catch { - case e: IllegalStateException => None + case e: Exception => + logError(s"Exception while removing shutdown hook.", e) } doStop() } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 61ef5ff168791..1f45956282166 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -31,8 +31,7 @@ import org.apache.spark.util.Utils private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { - val minMemoryMapBytes = blockManager.conf.getLong( - "spark.storage.memoryMapThreshold", 2 * 1024L * 1024L) + val minMemoryMapBytes = blockManager.conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") override def getSize(blockId: BlockId): Long = { diskManager.getFile(blockId.name).length @@ -46,10 +45,13 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val channel = new FileOutputStream(file).getChannel - while (bytes.remaining > 0) { - channel.write(bytes) + Utils.tryWithSafeFinally { + while (bytes.remaining > 0) { + channel.write(bytes) + } + } { + channel.close() } - channel.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) @@ -75,9 +77,9 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) try { - try { + Utils.tryWithSafeFinally { blockManager.dataSerializeStream(blockId, outputStream, values) - } finally { + } { // Close outputStream here because it should be closed before file is deleted. outputStream.close() } @@ -106,8 +108,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc private def getBytes(file: File, offset: Long, length: Long): Option[ByteBuffer] = { val channel = new RandomAccessFile(file, "r").getChannel - - try { + Utils.tryWithSafeFinally { // For small files, directly read rather than memory map if (length < minMemoryMapBytes) { val buf = ByteBuffer.allocate(length.toInt) @@ -123,7 +124,7 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc } else { Some(channel.map(MapMode.READ_ONLY, offset, length)) } - } finally { + } { channel.close() } } diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala new file mode 100644 index 0000000000000..8964762df6af3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala @@ -0,0 +1,102 @@ +/* + * 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.storage + +import java.nio.ByteBuffer + +/** + * An abstract class that the concrete external block manager has to inherit. + * The class has to have a no-argument constructor, and will be initialized by init, + * which is invoked by ExternalBlockStore. The main input parameter is blockId for all + * the methods, which is the unique identifier for Block in one Spark application. + * + * The underlying external block manager should avoid any name space conflicts among multiple + * Spark applications. For example, creating different directory for different applications + * by randomUUID + * + */ +private[spark] abstract class ExternalBlockManager { + + override def toString: String = {"External Block Store"} + + /** + * Initialize a concrete block manager implementation. Subclass should initialize its internal + * data structure, e.g, file system, in this function, which is invoked by ExternalBlockStore + * right after the class is constructed. The function should throw IOException on failure + * + * @throws java.io.IOException if there is any file system failure during the initialization. + */ + def init(blockManager: BlockManager, executorId: String): Unit + + /** + * Drop the block from underlying external block store, if it exists.. + * @return true on successfully removing the block + * false if the block could not be removed as it was not found + * + * @throws java.io.IOException if there is any file system failure in removing the block. + */ + def removeBlock(blockId: BlockId): Boolean + + /** + * Used by BlockManager to check the existence of the block in the underlying external + * block store. + * @return true if the block exists. + * false if the block does not exists. + * + * @throws java.io.IOException if there is any file system failure in checking + * the block existence. + */ + def blockExists(blockId: BlockId): Boolean + + /** + * Put the given block to the underlying external block store. Note that in normal case, + * putting a block should never fail unless something wrong happens to the underlying + * external block store, e.g., file system failure, etc. In this case, IOException + * should be thrown. + * + * @throws java.io.IOException if there is any file system failure in putting the block. + */ + def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit + + /** + * Retrieve the block bytes. + * @return Some(ByteBuffer) if the block bytes is successfully retrieved + * None if the block does not exist in the external block store. + * + * @throws java.io.IOException if there is any file system failure in getting the block. + */ + def getBytes(blockId: BlockId): Option[ByteBuffer] + + /** + * Get the size of the block saved in the underlying external block store, + * which is saved before by putBytes. + * @return size of the block + * 0 if the block does not exist + * + * @throws java.io.IOException if there is any file system failure in getting the block size. + */ + def getSize(blockId: BlockId): Long + + /** + * Clean up any information persisted in the underlying external block store, + * e.g., the directory, files, etc,which is invoked by the shutdown hook of ExternalBlockStore + * during system shutdown. + * + */ + def shutdown() +} diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala new file mode 100644 index 0000000000000..0bf770306ae9b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala @@ -0,0 +1,181 @@ +/* + * 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.storage + +import java.nio.ByteBuffer +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import scala.util.control.NonFatal + + +/** + * Stores BlockManager blocks on ExternalBlockStore. + * We capture any potential exception from underlying implementation + * and return with the expected failure value + */ +private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: String) + extends BlockStore(blockManager: BlockManager) with Logging { + + lazy val externalBlockManager: Option[ExternalBlockManager] = createBlkManager() + + logInfo("ExternalBlockStore started") + + override def getSize(blockId: BlockId): Long = { + try { + externalBlockManager.map(_.getSize(blockId)).getOrElse(0) + } catch { + case NonFatal(t) => + logError(s"error in getSize from $blockId", t) + 0L + } + } + + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putIntoExternalBlockStore(blockId, bytes, returnValues = true) + } + + override def putArray( + blockId: BlockId, + values: Array[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + putIterator(blockId, values.toIterator, level, returnValues) + } + + override def putIterator( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + logDebug(s"Attempting to write values for block $blockId") + val bytes = blockManager.dataSerialize(blockId, values) + putIntoExternalBlockStore(blockId, bytes, returnValues) + } + + private def putIntoExternalBlockStore( + blockId: BlockId, + bytes: ByteBuffer, + returnValues: Boolean): PutResult = { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val byteBuffer = bytes.duplicate() + byteBuffer.rewind() + logDebug(s"Attempting to put block $blockId into ExtBlk store") + // we should never hit here if externalBlockManager is None. Handle it anyway for safety. + try { + val startTime = System.currentTimeMillis + if (externalBlockManager.isDefined) { + externalBlockManager.get.putBytes(blockId, bytes) + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file in ExternalBlockStore in %d ms".format( + blockId, Utils.bytesToString(byteBuffer.limit), finishTime - startTime)) + + if (returnValues) { + PutResult(bytes.limit(), Right(bytes.duplicate())) + } else { + PutResult(bytes.limit(), null) + } + } else { + logError(s"error in putBytes $blockId") + PutResult(bytes.limit(), null, Seq((blockId, BlockStatus.empty))) + } + } catch { + case NonFatal(t) => + logError(s"error in putBytes $blockId", t) + PutResult(bytes.limit(), null, Seq((blockId, BlockStatus.empty))) + } + } + + // We assume the block is removed even if exception thrown + override def remove(blockId: BlockId): Boolean = { + try { + externalBlockManager.map(_.removeBlock(blockId)).getOrElse(true) + } catch { + case NonFatal(t) => + logError(s"error in removing $blockId", t) + true + } + } + + override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) + } + + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + try { + externalBlockManager.flatMap(_.getBytes(blockId)) + } catch { + case NonFatal(t) => + logError(s"error in getBytes from $blockId", t) + None + } + } + + override def contains(blockId: BlockId): Boolean = { + try { + val ret = externalBlockManager.map(_.blockExists(blockId)).getOrElse(false) + if (!ret) { + logInfo(s"remove block $blockId") + blockManager.removeBlock(blockId, true) + } + ret + } catch { + case NonFatal(t) => + logError(s"error in getBytes from $blockId", t) + false + } + } + + private def addShutdownHook() { + Runtime.getRuntime.addShutdownHook(new Thread("ExternalBlockStore shutdown hook") { + override def run(): Unit = Utils.logUncaughtExceptions { + logDebug("Shutdown hook called") + externalBlockManager.map(_.shutdown()) + } + }) + } + + // Create concrete block manager and fall back to Tachyon by default for backward compatibility. + private def createBlkManager(): Option[ExternalBlockManager] = { + val clsName = blockManager.conf.getOption(ExternalBlockStore.BLOCK_MANAGER_NAME) + .getOrElse(ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME) + + try { + val instance = Class.forName(clsName) + .newInstance() + .asInstanceOf[ExternalBlockManager] + instance.init(blockManager, executorId) + addShutdownHook(); + Some(instance) + } catch { + case NonFatal(t) => + logError("Cannot initialize external block store", t) + None + } + } +} + +private[spark] object ExternalBlockStore extends Logging { + val MAX_DIR_CREATION_ATTEMPTS = 10 + val SUB_DIRS_PER_DIR = "64" + val BASE_DIR = "spark.externalBlockStore.baseDir" + val FOLD_NAME = "spark.externalBlockStore.folderName" + val MASTER_URL = "spark.externalBlockStore.url" + val BLOCK_MANAGER_NAME = "spark.externalBlockStore.blockManager" + val DEFAULT_BLOCK_MANAGER_NAME = "org.apache.spark.storage.TachyonBlockManager" +} diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 132502b75f8cd..95e2d688d9b17 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -24,5 +24,7 @@ import java.io.File * based off an offset and a length. */ private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { - override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) + override def toString: String = { + "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 1be860aea63d0..ed609772e6979 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -98,6 +98,26 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } + /** + * Use `size` to test if there is enough space in MemoryStore. If so, create the ByteBuffer and + * put it into MemoryStore. Otherwise, the ByteBuffer won't be created. + * + * The caller should guarantee that `size` is correct. + */ + def putBytes(blockId: BlockId, size: Long, _bytes: () => ByteBuffer): PutResult = { + // Work on a duplicate - since the original input might be used elsewhere. + lazy val bytes = _bytes().duplicate().rewind().asInstanceOf[ByteBuffer] + val putAttempt = tryToPut(blockId, () => bytes, size, deserialized = false) + val data = + if (putAttempt.success) { + assert(bytes.limit == size) + Right(bytes.duplicate()) + } else { + null + } + PutResult(size, data, putAttempt.droppedBlocks) + } + override def putArray( blockId: BlockId, values: Array[Any], @@ -312,11 +332,22 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId.asRDDId.map(_.rddId) } + private def tryToPut( + blockId: BlockId, + value: Any, + size: Long, + deserialized: Boolean): ResultWithDroppedBlocks = { + tryToPut(blockId, () => value, size, deserialized) + } + /** * Try to put in a set of values, if we can free up enough space. The value should either be * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size * must also be passed by the caller. * + * `value` will be lazily created. If it cannot be put into MemoryStore or disk, `value` won't be + * created to avoid OOM since it may be a big ByteBuffer. + * * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for @@ -326,7 +357,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) */ private def tryToPut( blockId: BlockId, - value: Any, + value: () => Any, size: Long, deserialized: Boolean): ResultWithDroppedBlocks = { @@ -345,7 +376,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks ++= freeSpaceResult.droppedBlocks if (enoughFreeSpace) { - val entry = new MemoryEntry(value, size, deserialized) + val entry = new MemoryEntry(value(), size, deserialized) entries.synchronized { entries.put(blockId, entry) currentMemory += size @@ -357,12 +388,12 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. - val data = if (deserialized) { - Left(value.asInstanceOf[Array[Any]]) + lazy val data = if (deserialized) { + Left(value().asInstanceOf[Array[Any]]) } else { - Right(value.asInstanceOf[ByteBuffer].duplicate()) + Right(value().asInstanceOf[ByteBuffer].duplicate()) } - val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + val droppedBlockStatus = blockManager.dropFromMemory(blockId, () => data) droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } // Release the unroll memory used because we no longer need the underlying Array diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 120c327a7e580..96062626b5045 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.storage import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDDOperationScope, RDD} import org.apache.spark.util.Utils @DeveloperApi @@ -26,32 +26,36 @@ class RDDInfo( val id: Int, val name: String, val numPartitions: Int, - var storageLevel: StorageLevel) + var storageLevel: StorageLevel, + val parentIds: Seq[Int], + val scope: Option[RDDOperationScope] = None) extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - var tachyonSize = 0L + var externalBlockStoreSize = 0L - def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0 + def isCached: Boolean = + (memSize + diskSize + externalBlockStoreSize > 0) && numCachedPartitions > 0 - override def toString = { + override def toString: String = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + - "MemorySize: %s; TachyonSize: %s; DiskSize: %s").format( + "MemorySize: %s; ExternalBlockStoreSize: %s; DiskSize: %s").format( name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + bytesToString(memSize), bytesToString(externalBlockStoreSize), bytesToString(diskSize)) } - override def compare(that: RDDInfo) = { + override def compare(that: RDDInfo): Int = { this.id - that.id } } private[spark] object RDDInfo { def fromRdd(rdd: RDD[_]): RDDInfo = { - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) + val parentIds = rdd.dependencies.map(_.rdd.id) + new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel, parentIds, rdd.scope) } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 8f28ef49a8a6f..d0faab62c9e9e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,17 +17,15 @@ package org.apache.spark.storage -import java.io.{InputStream, IOException} import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} -import scala.util.{Failure, Success, Try} +import scala.util.{Failure, Try} import org.apache.spark.{Logging, TaskContext} -import org.apache.spark.network.BlockTransferService import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.serializer.Serializer +import org.apache.spark.serializer.{SerializerInstance, Serializer} import org.apache.spark.util.{CompletionIterator, Utils} /** @@ -106,6 +104,8 @@ final class ShuffleBlockFetcherIterator( private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + private[this] val serializerInstance: SerializerInstance = serializer.newInstance() + /** * Whether the iterator is still active. If isZombie is true, the callback interface will no * longer place fetched blocks into [[results]]. @@ -299,7 +299,7 @@ final class ShuffleBlockFetcherIterator( // the scheduler gets a FetchFailedException. Try(buf.createInputStream()).map { is0 => val is = blockManager.wrapForCompression(blockId, is0) - val iter = serializer.newInstance().deserializeStream(is).asIterator + val iter = serializerInstance.deserializeStream(is).asKeyValueIterator CompletionIterator[Any, Iterator[Any]](iter, { // Once the iterator is exhausted, release the buffer and set currentResult to null // so we don't release it again in cleanup. diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index e5e1cf5a69a19..703bce3e6b85b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -26,9 +26,9 @@ import org.apache.spark.util.Utils /** * :: DeveloperApi :: * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, - * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to - * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on - * multiple nodes. + * or ExternalBlockStore, whether to drop the RDD to disk if it falls out of memory or + * ExternalBlockStore, whether to keep the data in memory in a serialized format, and whether + * to replicate the RDD partitions on multiple nodes. * * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the @@ -50,11 +50,11 @@ class StorageLevel private( def this() = this(false, true, false, false) // For deserialization - def useDisk = _useDisk - def useMemory = _useMemory - def useOffHeap = _useOffHeap - def deserialized = _deserialized - def replication = _replication + def useDisk: Boolean = _useDisk + def useMemory: Boolean = _useMemory + def useOffHeap: Boolean = _useOffHeap + def deserialized: Boolean = _deserialized + def replication: Int = _replication assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") @@ -80,7 +80,7 @@ class StorageLevel private( false } - def isValid = (useMemory || useDisk || useOffHeap) && (replication > 0) + def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0) def toInt: Int = { var ret = 0 @@ -126,7 +126,7 @@ class StorageLevel private( var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") - result += (if (useOffHeap) "Tachyon " else "") + result += (if (useOffHeap) "ExternalBlockStore " else "") result += (if (deserialized) "Deserialized " else "Serialized ") result += s"${replication}x Replicated" result @@ -183,7 +183,7 @@ object StorageLevel { useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, - replication: Int) = { + replication: Int): StorageLevel = { getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) } @@ -197,7 +197,7 @@ object StorageLevel { useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, - replication: Int = 1) = { + replication: Int = 1): StorageLevel = { getCachedStorageLevel(new StorageLevel(useDisk, useMemory, false, deserialized, replication)) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index def49e80a3605..ec711480ebf30 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,20 +19,23 @@ package org.apache.spark.storage import scala.collection.mutable -import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ /** * :: DeveloperApi :: * A SparkListener that maintains executor storage status. + * + * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageStatusListener extends SparkListener { // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() - def storageStatusList = executorIdToStorageStatus.values.toSeq + def storageStatusList: Seq[StorageStatus] = synchronized { + executorIdToStorageStatus.values.toSeq + } /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { @@ -56,7 +59,7 @@ class StorageStatusListener extends SparkListener { } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { @@ -67,7 +70,7 @@ class StorageStatusListener extends SparkListener { } } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { updateStorageStatus(unpersistRDD.rddId) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 2bd6b749be261..c4ac30092f807 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -199,33 +199,34 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize - val changeInTachyon = newBlockStatus.tachyonSize - oldBlockStatus.tachyonSize + val changeInExternalBlockStore = + newBlockStatus.externalBlockStoreSize - oldBlockStatus.externalBlockStoreSize val level = newBlockStatus.storageLevel // Compute new info from old info - val (oldMem, oldDisk, oldTachyon) = blockId match { + val (oldMem, oldDisk, oldExternalBlockStore) = blockId match { case RDDBlockId(rddId, _) => _rddStorageInfo.get(rddId) - .map { case (mem, disk, tachyon, _) => (mem, disk, tachyon) } + .map { case (mem, disk, externalBlockStore, _) => (mem, disk, externalBlockStore) } .getOrElse((0L, 0L, 0L)) case _ => _nonRddStorageInfo } val newMem = math.max(oldMem + changeInMem, 0L) val newDisk = math.max(oldDisk + changeInDisk, 0L) - val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) + val newExternalBlockStore = math.max(oldExternalBlockStore + changeInExternalBlockStore, 0L) // Set the correct info blockId match { case RDDBlockId(rddId, _) => // If this RDD is no longer persisted, remove it - if (newMem + newDisk + newTachyon == 0) { + if (newMem + newDisk + newExternalBlockStore == 0) { _rddStorageInfo.remove(rddId) } else { - _rddStorageInfo(rddId) = (newMem, newDisk, newTachyon, level) + _rddStorageInfo(rddId) = (newMem, newDisk, newExternalBlockStore, level) } case _ => - _nonRddStorageInfo = (newMem, newDisk, newTachyon) + _nonRddStorageInfo = (newMem, newDisk, newExternalBlockStore) } } @@ -247,13 +248,13 @@ private[spark] object StorageUtils { val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum val memSize = statuses.map(_.memUsedByRdd(rddId)).sum val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum - val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum + val externalBlockStoreSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum rddInfo.storageLevel = storageLevel rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize rddInfo.diskSize = diskSize - rddInfo.tachyonSize = tachyonSize + rddInfo.externalBlockStoreSize = externalBlockStoreSize } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index af873034215a9..bdc6276e41915 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -17,13 +17,16 @@ package org.apache.spark.storage +import java.io.IOException +import java.nio.ByteBuffer import java.text.SimpleDateFormat import java.util.{Date, Random} -import tachyon.client.TachyonFS -import tachyon.client.TachyonFile +import com.google.common.io.ByteStreams +import tachyon.client.{ReadType, WriteType, TachyonFS, TachyonFile} +import tachyon.TachyonURI -import org.apache.spark.Logging +import org.apache.spark.{SparkException, SparkConf, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.util.Utils @@ -32,39 +35,101 @@ import org.apache.spark.util.Utils * Creates and maintains the logical mapping between logical blocks and tachyon fs locations. By * default, one block is mapped to one file with a name given by its BlockId. * - * @param rootDirs The directories to use for storing block files. Data will be hashed among these. */ -private[spark] class TachyonBlockManager( - blockManager: BlockManager, - rootDirs: String, - val master: String) - extends Logging { +private[spark] class TachyonBlockManager() extends ExternalBlockManager with Logging { - val client = if (master != null && master != "") TachyonFS.get(master) else null - - if (client == null) { - logError("Failed to connect to the Tachyon as the master address is not configured") - System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_INITIALIZE) - } - - private val MAX_DIR_CREATION_ATTEMPTS = 10 - private val subDirsPerTachyonDir = - blockManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt + var blockManager: BlockManager =_ + var rootDirs: String = _ + var master: String = _ + var client: tachyon.client.TachyonFS = _ + private var subDirsPerTachyonDir: Int = _ // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; // then, inside this directory, create multiple subdirectories that we will hash files into, // in order to avoid having really large inodes at the top level in Tachyon. - private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() - private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) + private var tachyonDirs: Array[TachyonFile] = _ + private var subDirs: Array[Array[tachyon.client.TachyonFile]] = _ + + + override def init(blockManager: BlockManager, executorId: String): Unit = { + this.blockManager = blockManager + val storeDir = blockManager.conf.get(ExternalBlockStore.BASE_DIR, "/tmp_spark_tachyon") + val appFolderName = blockManager.conf.get(ExternalBlockStore.FOLD_NAME) + + rootDirs = s"$storeDir/$appFolderName/$executorId" + master = blockManager.conf.get(ExternalBlockStore.MASTER_URL, "tachyon://localhost:19998") + client = if (master != null && master != "") TachyonFS.get(new TachyonURI(master)) else null + // original implementation call System.exit, we change it to run without extblkstore support + if (client == null) { + logError("Failed to connect to the Tachyon as the master address is not configured") + throw new IOException("Failed to connect to the Tachyon as the master " + + "address is not configured") + } + subDirsPerTachyonDir = blockManager.conf.get("spark.externalBlockStore.subDirectories", + ExternalBlockStore.SUB_DIRS_PER_DIR).toInt + + // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; + // then, inside this directory, create multiple subdirectories that we will hash files into, + // in order to avoid having really large inodes at the top level in Tachyon. + tachyonDirs = createTachyonDirs() + subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) + tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) + } + + override def toString: String = {"ExternalBlockStore-Tachyon"} + + override def removeBlock(blockId: BlockId): Boolean = { + val file = getFile(blockId) + if (fileExists(file)) { + removeFile(file) + } else { + false + } + } + + override def blockExists(blockId: BlockId): Boolean = { + val file = getFile(blockId) + fileExists(file) + } + + override def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit = { + val file = getFile(blockId) + val os = file.getOutStream(WriteType.TRY_CACHE) + os.write(bytes.array()) + os.close() + } - addShutdownHook() + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + val file = getFile(blockId) + if (file == null || file.getLocationHosts.size == 0) { + return None + } + val is = file.getInStream(ReadType.CACHE) + assert (is != null) + try { + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + ByteStreams.readFully(is, bs) + Some(ByteBuffer.wrap(bs)) + } catch { + case ioe: IOException => + logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) + None + } finally { + is.close() + } + } + + override def getSize(blockId: BlockId): Long = { + getFile(blockId.name).length + } def removeFile(file: TachyonFile): Boolean = { - client.delete(file.getPath(), false) + client.delete(new TachyonURI(file.getPath()), false) } def fileExists(file: TachyonFile): Boolean = { - client.exist(file.getPath()) + client.exist(new TachyonURI(file.getPath())) } def getFile(filename: String): TachyonFile = { @@ -81,7 +146,7 @@ private[spark] class TachyonBlockManager( if (old != null) { old } else { - val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) + val path = new TachyonURI(s"${tachyonDirs(dirId)}/${"%02x".format(subDirId)}") client.mkdir(path) val newDir = client.getFile(path) subDirs(dirId)(subDirId) = newDir @@ -89,7 +154,7 @@ private[spark] class TachyonBlockManager( } } } - val filePath = subDir + "/" + filename + val filePath = new TachyonURI(s"$subDir/$filename") if(!client.exist(filePath)) { client.createFile(filePath) } @@ -109,11 +174,11 @@ private[spark] class TachyonBlockManager( var tachyonDirId: String = null var tries = 0 val rand = new Random() - while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { + while (!foundLocalDir && tries < ExternalBlockStore.MAX_DIR_CREATION_ATTEMPTS) { tries += 1 try { tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId + val path = new TachyonURI(s"$rootDir/spark-tachyon-$tachyonDirId") if (!client.exist(path)) { foundLocalDir = client.mkdir(path) tachyonDir = client.getFile(path) @@ -124,32 +189,27 @@ private[spark] class TachyonBlockManager( } } if (!foundLocalDir) { - logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + - rootDir) - System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) + logError("Failed " + ExternalBlockStore.MAX_DIR_CREATION_ATTEMPTS + + " attempts to create tachyon dir in " + rootDir) + System.exit(ExecutorExitCode.EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR) } logInfo("Created tachyon directory at " + tachyonDir) tachyonDir } } - private def addShutdownHook() { - tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - tachyonDirs.foreach { tachyonDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { - Utils.deleteRecursively(tachyonDir, client) - } - } catch { - case e: Exception => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) - } + override def shutdown() { + logDebug("Shutdown hook called") + tachyonDirs.foreach { tachyonDir => + try { + if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { + Utils.deleteRecursively(tachyonDir, client) } - client.close() + } catch { + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } - }) + } + client.close() } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala deleted file mode 100644 index 233d1e2b7c616..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ /dev/null @@ -1,128 +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.storage - -import java.io.IOException -import java.nio.ByteBuffer - -import com.google.common.io.ByteStreams -import tachyon.client.{ReadType, WriteType} - -import org.apache.spark.Logging -import org.apache.spark.util.Utils - -/** - * Stores BlockManager blocks on Tachyon. - */ -private[spark] class TachyonStore( - blockManager: BlockManager, - tachyonManager: TachyonBlockManager) - extends BlockStore(blockManager: BlockManager) with Logging { - - logInfo("TachyonStore started") - - override def getSize(blockId: BlockId): Long = { - tachyonManager.getFile(blockId.name).length - } - - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { - putIntoTachyonStore(blockId, bytes, returnValues = true) - } - - override def putArray( - blockId: BlockId, - values: Array[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - putIterator(blockId, values.toIterator, level, returnValues) - } - - override def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - logDebug(s"Attempting to write values for block $blockId") - val bytes = blockManager.dataSerialize(blockId, values) - putIntoTachyonStore(blockId, bytes, returnValues) - } - - private def putIntoTachyonStore( - blockId: BlockId, - bytes: ByteBuffer, - returnValues: Boolean): PutResult = { - // So that we do not modify the input offsets ! - // duplicate does not copy buffer, so inexpensive - val byteBuffer = bytes.duplicate() - byteBuffer.rewind() - logDebug(s"Attempting to put block $blockId into Tachyon") - val startTime = System.currentTimeMillis - val file = tachyonManager.getFile(blockId) - val os = file.getOutStream(WriteType.TRY_CACHE) - os.write(byteBuffer.array()) - os.close() - val finishTime = System.currentTimeMillis - logDebug("Block %s stored as %s file in Tachyon in %d ms".format( - blockId, Utils.bytesToString(byteBuffer.limit), finishTime - startTime)) - - if (returnValues) { - PutResult(bytes.limit(), Right(bytes.duplicate())) - } else { - PutResult(bytes.limit(), null) - } - } - - override def remove(blockId: BlockId): Boolean = { - val file = tachyonManager.getFile(blockId) - if (tachyonManager.fileExists(file)) { - tachyonManager.removeFile(file) - } else { - false - } - } - - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { - getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) - } - - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val file = tachyonManager.getFile(blockId) - if (file == null || file.getLocationHosts.size == 0) { - return None - } - val is = file.getInStream(ReadType.CACHE) - assert (is != null) - try { - val size = file.length - val bs = new Array[Byte](size.asInstanceOf[Int]) - ByteStreams.readFully(is, bs) - Some(ByteBuffer.wrap(bs)) - } catch { - case ioe: IOException => - logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) - None - } finally { - is.close() - } - } - - override def contains(blockId: BlockId): Boolean = { - val file = tachyonManager.getFile(blockId) - tachyonManager.fileExists(file) - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 67f572e79314d..77c0bc8b5360a 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -65,7 +65,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { val stageIds = sc.statusTracker.getActiveStageIds() val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1) .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId()) - if (stages.size > 0) { + if (stages.length > 0) { show(now, stages.take(3)) // display at most 3 stages in same time } } @@ -81,7 +81,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { val total = s.numTasks() val header = s"[Stage ${s.stageId()}:" val tailer = s"(${s.numCompletedTasks()} + ${s.numActiveTasks()}) / $total]" - val w = width - header.size - tailer.size + val w = width - header.length - tailer.length val bar = if (w > 0) { val percent = w * s.numCompletedTasks() / total (0 until w).map { i => 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 95f254a9ef22a..dfd6fdb5e9993 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -114,10 +114,25 @@ private[spark] object JettyUtils extends Logging { srcPath: String, destPath: String, beforeRedirect: HttpServletRequest => Unit = x => (), - basePath: String = ""): ServletContextHandler = { + basePath: String = "", + httpMethods: Set[String] = Set("GET")): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { - override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse): Unit = { + if (httpMethods.contains("GET")) { + doRequest(request, response) + } else { + response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + override def doPost(request: HttpServletRequest, response: HttpServletResponse): Unit = { + if (httpMethods.contains("POST")) { + doRequest(request, response) + } else { + response.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) + } + } + private def doRequest(request: HttpServletRequest, response: HttpServletResponse): Unit = { beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 0c24ad2760e08..bfe4a180e8a6f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,6 +17,9 @@ package org.apache.spark.ui +import java.util.Date + +import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener @@ -25,6 +28,7 @@ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} +import org.apache.spark.ui.scope.RDDOperationGraphListener /** * Top level user interface for a Spark application. @@ -32,35 +36,43 @@ import org.apache.spark.ui.storage.{StorageListener, StorageTab} private[spark] class SparkUI private ( val sc: Option[SparkContext], val conf: SparkConf, - val securityManager: SecurityManager, + securityManager: SecurityManager, val environmentListener: EnvironmentListener, val storageStatusListener: StorageStatusListener, val executorsListener: ExecutorsListener, val jobProgressListener: JobProgressListener, val storageListener: StorageListener, + val operationGraphListener: RDDOperationGraphListener, var appName: String, - val basePath: String) + val basePath: String, + val startTime: Long) extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") - with Logging { + with Logging + with UIRoot { val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + + val stagesTab = new StagesTab(this) + /** Initialize all components of the server. */ def initialize() { attachTab(new JobsTab(this)) - val stagesTab = new StagesTab(this) attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) - attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) + attachHandler(JsonRootResource.getJsonServlet(this)) + // This should be POST only, but, the YARN AM proxy won't proxy POSTs + attachHandler(createRedirectHandler( + "/stages/stage/kill", "/stages", stagesTab.handleKillRequest, + httpMethods = Set("GET", "POST"))) } initialize() - def getAppName = appName + def getAppName: String = appName /** Set the app name for this UI. */ def setAppName(name: String) { @@ -79,6 +91,24 @@ private[spark] class SparkUI private ( private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" + + def getSparkUI(appId: String): Option[SparkUI] = { + if (appId == appName) Some(this) else None + } + + def getApplicationInfoList: Iterator[ApplicationInfo] = { + Iterator(new ApplicationInfo( + id = appName, + name = appName, + attempts = Seq(new ApplicationAttemptInfo( + attemptId = None, + startTime = new Date(startTime), + endTime = new Date(-1), + sparkUser = "", + completed = false + )) + )) + } } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) @@ -91,6 +121,9 @@ private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + val DEFAULT_POOL_NAME = "default" + val DEFAULT_RETAINED_STAGES = 1000 + val DEFAULT_RETAINED_JOBS = 1000 def getUIPort(conf: SparkConf): Int = { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) @@ -102,9 +135,10 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, jobProgressListener: JobProgressListener, securityManager: SecurityManager, - appName: String): SparkUI = { + appName: String, + startTime: Long): SparkUI = { create(Some(sc), conf, listenerBus, securityManager, appName, - jobProgressListener = Some(jobProgressListener)) + jobProgressListener = Some(jobProgressListener), startTime = startTime) } def createHistoryUI( @@ -112,8 +146,9 @@ private[spark] object SparkUI { listenerBus: SparkListenerBus, securityManager: SecurityManager, appName: String, - basePath: String): SparkUI = { - create(None, conf, listenerBus, securityManager, appName, basePath) + basePath: String, + startTime: Long): SparkUI = { + create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) } /** @@ -130,7 +165,8 @@ private[spark] object SparkUI { securityManager: SecurityManager, appName: String, basePath: String = "", - jobProgressListener: Option[JobProgressListener] = None): SparkUI = { + jobProgressListener: Option[JobProgressListener] = None, + startTime: Long): SparkUI = { val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { val listener = new JobProgressListener(conf) @@ -142,13 +178,16 @@ private[spark] object SparkUI { val storageStatusListener = new StorageStatusListener val executorsListener = new ExecutorsListener(storageStatusListener) val storageListener = new StorageListener(storageStatusListener) + val operationGraphListener = new RDDOperationGraphListener(conf) listenerBus.addListener(environmentListener) listenerBus.addListener(storageStatusListener) listenerBus.addListener(executorsListener) listenerBus.addListener(storageListener) + listenerBus.addListener(operationGraphListener) new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, - executorsListener, _jobProgressListener, storageListener, appName, basePath) + executorsListener, _jobProgressListener, storageListener, operationGraphListener, + appName, basePath, startTime) } } diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index cae6870c2ab20..24f3236456248 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -24,7 +24,9 @@ private[spark] object ToolTips { scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results.""" - val TASK_DESERIALIZATION_TIME = "Time spent deserializing the task closure on the executor." + val TASK_DESERIALIZATION_TIME = + """Time spent deserializing the task closure on the executor, including the time to read the + broadcasted task.""" val SHUFFLE_READ_BLOCKED_TIME = "Time that the task spent blocked waiting for shuffle data to be read from remote machines." 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 b5022fe853c49..2f3fb181e4026 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -23,6 +23,7 @@ import java.util.{Locale, Date} import scala.xml.{Node, Text} import org.apache.spark.Logging +import org.apache.spark.ui.scope.RDDOperationGraph /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { @@ -149,20 +150,34 @@ private[spark] object UIUtils extends Logging { } } - def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource + def prependBaseUri(basePath: String = "", resource: String = ""): String = { + uiRoot + basePath + resource + } - def commonHeaderNodes = { + def commonHeaderNodes: Seq[Node] = { + + + + + } + + def vizHeaderNodes: Seq[Node] = { + + + + } /** Returns a spark page with correctly formatted headers */ @@ -171,7 +186,8 @@ private[spark] object UIUtils extends Logging { content: => Seq[Node], activeTab: SparkUITab, refreshInterval: Option[Int] = None, - helpText: Option[String] = None): Seq[Node] = { + helpText: Option[String] = None, + showVisualization: Boolean = false): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -189,6 +205,7 @@ private[spark] object UIUtils extends Logging { {commonHeaderNodes} + {if (showVisualization) vizHeaderNodes else Seq.empty} {appName} - {title} @@ -313,4 +330,47 @@ private[spark] object UIUtils extends Logging {
    } + + /** Return a "DAG visualization" DOM element that expands into a visualization for a stage. */ + def showDagVizForStage(stageId: Int, graph: Option[RDDOperationGraph]): Seq[Node] = { + showDagViz(graph.toSeq, forJob = false) + } + + /** Return a "DAG visualization" DOM element that expands into a visualization for a job. */ + def showDagVizForJob(jobId: Int, graphs: Seq[RDDOperationGraph]): Seq[Node] = { + showDagViz(graphs, forJob = true) + } + + /** + * Return a "DAG visualization" DOM element that expands into a visualization on the UI. + * + * This populates metadata necessary for generating the visualization on the front-end in + * a format that is expected by spark-dag-viz.js. Any changes in the format here must be + * reflected there. + */ + private def showDagViz(graphs: Seq[RDDOperationGraph], forJob: Boolean): Seq[Node] = { +
    + + + DAG visualization + +
    +
    + { + graphs.map { g => + + } + } +
    +
    + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index fc1844600f1cb..5fbcd6bb8ad94 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui +import java.util.concurrent.Semaphore + import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} @@ -51,7 +53,7 @@ private[spark] object UIWorkloadGenerator { val nJobSet = args(2).toInt val sc = new SparkContext(conf) - def setProperties(s: String) = { + def setProperties(s: String): Unit = { if(schedulingMode == SchedulingMode.FAIR) { sc.setLocalProperty("spark.scheduler.pool", s) } @@ -59,7 +61,7 @@ private[spark] object UIWorkloadGenerator { } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) - def nextFloat() = new Random().nextFloat() + def nextFloat(): Float = new Random().nextFloat() val jobs = Seq[(String, () => Long)]( ("Count", baseData.count), @@ -88,6 +90,8 @@ private[spark] object UIWorkloadGenerator { ("Job with delays", baseData.map(x => Thread.sleep(100)).count) ) + val barrier = new Semaphore(-nJobSet * jobs.size + 1) + (1 to nJobSet).foreach { _ => for ((desc, job) <- jobs) { new Thread { @@ -99,12 +103,17 @@ private[spark] object UIWorkloadGenerator { } catch { case e: Exception => println("Job Failed: " + desc) + } finally { + barrier.release() } } }.start Thread.sleep(INTER_JOB_WAIT_MS) } } + + // Waiting for threads. + barrier.acquire() sc.stop() } } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ea548f23120d9..384f2ad26e281 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -37,7 +37,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. */ private[spark] abstract class WebUI( - securityManager: SecurityManager, + val securityManager: SecurityManager, port: Int, conf: SparkConf, basePath: String = "", @@ -48,7 +48,7 @@ private[spark] abstract class WebUI( protected val handlers = ArrayBuffer[ServletContextHandler]() protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None - protected val localHostName = Utils.localHostName() + protected val localHostName = Utils.localHostNameForURI() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) @@ -77,15 +77,9 @@ private[spark] abstract class WebUI( val pagePath = "/" + page.prefix val renderHandler = createServletHandler(pagePath, (request: HttpServletRequest) => page.render(request), securityManager, basePath) - val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) attachHandler(renderHandler) - attachHandler(renderJsonHandler) pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) .append(renderHandler) - pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) - .append(renderJsonHandler) - } /** Attach a handler to this UI. */ diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 956608d7c0cbe..b247e4cdc3bd4 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -22,11 +22,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.status.api.v1.ExecutorSummary import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.util.Utils -/** Summary information about an executor to display in the UI. */ -// Needs to be private[ui] because of a false positive MiMa failure. +// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive private[ui] case class ExecutorSummaryInfo( id: String, hostPort: String, @@ -44,6 +44,7 @@ private[ui] case class ExecutorSummaryInfo( maxMemory: Long, executorLogs: Map[String, String]) + private[ui] class ExecutorsPage( parent: ExecutorsTab, threadDumpEnabled: Boolean) @@ -55,7 +56,8 @@ private[ui] class ExecutorsPage( val maxMem = storageStatusList.map(_.maxMem).sum val memUsed = storageStatusList.map(_.memUsed).sum val diskUsed = storageStatusList.map(_.diskUsed).sum - val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) + val execInfo = for (statusId <- 0 until storageStatusList.size) yield + ExecutorsPage.getExecInfo(listener, statusId) val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty @@ -111,7 +113,7 @@ private[ui] class ExecutorsPage( } /** Render an HTML row representing an executor */ - private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = { + private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = { val maximumMemory = info.maxMemory val memoryUsed = info.memoryUsed val diskUsed = info.diskUsed @@ -170,8 +172,11 @@ private[ui] class ExecutorsPage( } +} + +private[spark] object ExecutorsPage { /** Represent an executor's info as a map given a storage status index */ - private def getExecInfo(statusId: Int): ExecutorSummaryInfo = { + def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort @@ -189,7 +194,7 @@ private[ui] class ExecutorsPage( val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) - new ExecutorSummaryInfo( + new ExecutorSummary( execId, hostPort, rddBlocks, diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 3afd7ef07d7c9..0a08b000e2d03 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -22,8 +22,9 @@ import scala.collection.mutable.HashMap import org.apache.spark.ExceptionFailure import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.storage.{StorageStatus, StorageStatusListener} import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.jobs.UIData.ExecutorUIData private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { val listener = parent.executorsListener @@ -54,20 +55,30 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() val executorToLogUrls = HashMap[String, Map[String, String]]() + val executorIdToData = HashMap[String, ExecutorUIData]() - def storageStatusList = storageStatusListener.storageStatusList + def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) = synchronized { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap + executorIdToData(eid) = ExecutorUIData(executorAdded.time) } - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onExecutorRemoved( + executorRemoved: SparkListenerExecutorRemoved): Unit = synchronized { + val eid = executorRemoved.executorId + val uiData = executorIdToData(eid) + uiData.finishTime = Some(executorRemoved.time) + uiData.finishReason = Some(executorRemoved.reason) + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId 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 bd923d78a86ce..09323d1d80ad6 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 @@ -17,17 +17,183 @@ package org.apache.spark.ui.jobs -import scala.xml.{Node, NodeSeq} +import scala.collection.mutable.{HashMap, ListBuffer} +import scala.xml.{Node, NodeSeq, Unparsed} +import java.util.Date import javax.servlet.http.HttpServletRequest -import org.apache.spark.ui.{WebUIPage, UIUtils} -import org.apache.spark.ui.jobs.UIData.JobUIData +import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData} +import org.apache.spark.JobExecutionStatus /** Page showing list of all ongoing and recently finished jobs */ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { - private val startTime: Option[Long] = parent.sc.map(_.startTime) - private val listener = parent.listener + private val JOBS_LEGEND = +
    + + Succeeded + + Failed + + Running +
    .toString.filter(_ != '\n') + + private val EXECUTORS_LEGEND = +
    + + Added + + Removed +
    .toString.filter(_ != '\n') + + private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { + val lastStageInfo = Option(job.stageIds) + .filter(_.nonEmpty) + .flatMap { ids => parent.jobProgresslistener.stageIdToInfo.get(ids.max)} + val lastStageData = lastStageInfo.flatMap { s => + parent.jobProgresslistener.stageIdToData.get((s.stageId, s.attemptId)) + } + val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val description = lastStageData.flatMap(_.description).getOrElse("") + (name, description) + } + + private def makeJobEvent(jobUIDatas: Seq[JobUIData]): Seq[String] = { + jobUIDatas.filter { jobUIData => + jobUIData.status != JobExecutionStatus.UNKNOWN && jobUIData.submissionTime.isDefined + }.map { jobUIData => + val jobId = jobUIData.jobId + val status = jobUIData.status + val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData) + val displayJobDescription = if (jobDescription.isEmpty) jobName else jobDescription + val submissionTime = jobUIData.submissionTime.get + val completionTimeOpt = jobUIData.completionTime + val completionTime = completionTimeOpt.getOrElse(System.currentTimeMillis()) + val classNameByStatus = status match { + case JobExecutionStatus.SUCCEEDED => "succeeded" + case JobExecutionStatus.FAILED => "failed" + case JobExecutionStatus.RUNNING => "running" + } + + val jobEventJsonAsStr = + s""" + |{ + | 'className': 'job application-timeline-object ${classNameByStatus}', + | 'group': 'jobs', + | 'start': new Date(${submissionTime}), + | 'end': new Date(${completionTime}), + | 'content': '
    Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + } else { + "" + } + }">' + + | '${displayJobDescription} (Job ${jobId})
    ' + |} + """.stripMargin + jobEventJsonAsStr + } + } + + private def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + val events = ListBuffer[String]() + executorUIDatas.foreach { + case (executorId, event) => + val addedEvent = + s""" + |{ + | 'className': 'executor added', + | 'group': 'executors', + | 'start': new Date(${event.startTime}), + | 'content': '
    Executor ${executorId} added
    ' + |} + """.stripMargin + events += addedEvent + + if (event.finishTime.isDefined) { + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${event.finishTime.get}), + | 'content': '
    Reason: ${event.finishReason.get}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${executorId} removed
    ' + |} + """.stripMargin + events += removedEvent + } + } + events.toSeq + } + + private def makeTimeline( + jobs: Seq[JobUIData], + executors: HashMap[String, ExecutorUIData], + startTime: Long): Seq[Node] = { + + val jobEventJsonAsStrSeq = makeJobEvent(jobs) + val executorEventJsonAsStrSeq = makeExecutorEvent(executors) + + val groupJsonArrayAsStr = + s""" + |[ + | { + | 'id': 'executors', + | 'content': '
    Executors
    ${EXECUTORS_LEGEND}', + | }, + | { + | 'id': 'jobs', + | 'content': '
    Jobs
    ${JOBS_LEGEND}', + | } + |] + """.stripMargin + + val eventArrayAsStr = + (jobEventJsonAsStrSeq ++ executorEventJsonAsStrSeq).mkString("[", ",", "]") + + + + Event timeline + ++ + ++ + + } private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) @@ -42,15 +208,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def makeRow(job: JobUIData): Seq[Node] = { - val lastStageInfo = Option(job.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => listener.stageIdToInfo.get(ids.max) } - val lastStageData = lastStageInfo.flatMap { s => - listener.stageIdToData.get((s.stageId, s.attemptId)) - } - - val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(job) val duration: Option[Long] = { job.submissionTime.map { start => val end = job.completionTime.getOrElse(System.currentTimeMillis()) @@ -61,7 +219,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") val detailUrl = "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) - + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} @@ -95,11 +253,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def render(request: HttpServletRequest): Seq[Node] = { + val listener = parent.jobProgresslistener listener.synchronized { + val startTime = listener.startTime val activeJobs = listener.activeJobs.values.toSeq val completedJobs = listener.completedJobs.reverse.toSeq val failedJobs = listener.failedJobs.reverse.toSeq - val now = System.currentTimeMillis val activeJobsTable = jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse) @@ -112,14 +271,20 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val shouldShowCompletedJobs = completedJobs.nonEmpty val shouldShowFailedJobs = failedJobs.nonEmpty + val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) { + s"${completedJobs.size}" + } else { + s"${listener.numCompletedJobs}, only showing ${completedJobs.size}" + } + val summary: NodeSeq =
      - {if (startTime.isDefined) { + {if (parent.sc.isDefined) { // Total duration is not meaningful unless the UI is live
    • Total Duration: - {UIUtils.formatDuration(now - startTime.get)} + {UIUtils.formatDuration(System.currentTimeMillis() - startTime)}
    • }}
    • @@ -136,9 +301,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } { if (shouldShowCompletedJobs) { -
    • +
    • Completed Jobs: - {completedJobs.size} + {completedJobNumStr}
    • } } @@ -146,7 +311,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { if (shouldShowFailedJobs) {
    • Failed Jobs: - {failedJobs.size} + {listener.numFailedJobs}
    • } } @@ -154,18 +319,23 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
    var content = summary + val executorListener = parent.executorListener + content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, + executorListener.executorIdToData, startTime) + if (shouldShowActiveJobs) { content ++=

    Active Jobs ({activeJobs.size})

    ++ activeJobsTable } if (shouldShowCompletedJobs) { - content ++=

    Completed Jobs ({completedJobs.size})

    ++ + content ++=

    Completed Jobs ({completedJobNumStr})

    ++ completedJobsTable } if (shouldShowFailedJobs) { content ++=

    Failed Jobs ({failedJobs.size})

    ++ failedJobsTable } + val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" + " Click on a job's title to see information about the stages of tasks associated with" + " the job." diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 527f960af2dfc..a37f739ab9c66 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -27,7 +27,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc - private val listener = parent.listener + private val listener = parent.progressListener private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { @@ -42,18 +42,18 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) val pendingStagesTable = new StageTableBase(pendingStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler) + parent.progressListener, isFairScheduler = parent.isFairScheduler) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -64,6 +64,12 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val shouldShowCompletedStages = completedStages.nonEmpty val shouldShowFailedStages = failedStages.nonEmpty + val completedStageNumStr = if (numCompletedStages == completedStages.size) { + s"$numCompletedStages" + } else { + s"$numCompletedStages, only showing ${completedStages.size}" + } + val summary: NodeSeq =
      @@ -98,9 +104,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { } { if (shouldShowCompletedStages) { -
    • +
    • Completed Stages: - {numCompletedStages} + {completedStageNumStr}
    • } } @@ -132,7 +138,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { pendingStagesTable.toNodeSeq } if (shouldShowCompletedStages) { - content ++=

      Completed Stages ({numCompletedStages})

      ++ + content ++=

      Completed Stages ({completedStageNumStr})

      ++ completedStagesTable.toNodeSeq } if (shouldShowFailedStages) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 1f8536d1b7195..d5cdbfac104f8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { - private val listener = parent.listener + private val listener = parent.progressListener def toNodeSeq: Seq[Node] = { listener.synchronized { 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 7541d3e9c72e7..7163217e1fed0 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 @@ -17,20 +17,168 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable -import scala.xml.{NodeSeq, Node} +import java.util.Date + +import scala.collection.mutable.{Buffer, HashMap, ListBuffer} +import scala.xml.{NodeSeq, Node, Unparsed} import javax.servlet.http.HttpServletRequest import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.jobs.UIData.ExecutorUIData /** Page showing statistics and stage list for a given job */ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { - private val listener = parent.listener + + private val STAGES_LEGEND = +
      + + Completed + + Failed + + Active +
      .toString.filter(_ != '\n') + + private val EXECUTORS_LEGEND = +
      + + Added + + Removed +
      .toString.filter(_ != '\n') + + private def makeStageEvent(stageInfos: Seq[StageInfo]): Seq[String] = { + stageInfos.map { stage => + val stageId = stage.stageId + val attemptId = stage.attemptId + val name = stage.name + val status = stage.getStatusString + val submissionTime = stage.submissionTime.get + val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) + + s""" + |{ + | 'className': 'stage job-timeline-object ${status}', + | 'group': 'stages', + | 'start': new Date(${submissionTime}), + | 'end': new Date(${completionTime}), + | 'content': '
      Completion Time: ${UIUtils.formatDate(new Date(completionTime))}""" + } else { + "" + } + }">' + + | '${name} (Stage ${stageId}.${attemptId})
      ', + |} + """.stripMargin + } + } + + def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + val events = ListBuffer[String]() + executorUIDatas.foreach { + case (executorId, event) => + val addedEvent = + s""" + |{ + | 'className': 'executor added', + | 'group': 'executors', + | 'start': new Date(${event.startTime}), + | 'content': '
      Executor ${executorId} added
      ' + |} + """.stripMargin + events += addedEvent + + if (event.finishTime.isDefined) { + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${event.finishTime.get}), + | 'content': '
      Reason: ${event.finishReason.get}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${executorId} removed
      ' + |} + """.stripMargin + events += removedEvent + } + } + events.toSeq + } + + private def makeTimeline( + stages: Seq[StageInfo], + executors: HashMap[String, ExecutorUIData], + appStartTime: Long): Seq[Node] = { + + val stageEventJsonAsStrSeq = makeStageEvent(stages) + val executorsJsonAsStrSeq = makeExecutorEvent(executors) + + val groupJsonArrayAsStr = + s""" + |[ + | { + | 'id': 'executors', + | 'content': '
      Executors
      ${EXECUTORS_LEGEND}', + | }, + | { + | 'id': 'stages', + | 'content': '
      Stages
      ${STAGES_LEGEND}', + | } + |] + """.stripMargin + + val eventArrayAsStr = + (stageEventJsonAsStrSeq ++ executorsJsonAsStrSeq).mkString("[", ",", "]") + + + + Event timeline + ++ + ++ + + } def render(request: HttpServletRequest): Seq[Node] = { + val listener = parent.jobProgresslistener + listener.synchronized { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") @@ -39,7 +187,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val jobDataOption = listener.jobIdToData.get(jobId) if (jobDataOption.isEmpty) { val content = -
      +

      No information to display for job {jobId}

      return UIUtils.headerSparkPage( @@ -51,14 +199,14 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { // This could be empty if the JobProgressListener hasn't received information about the // stage or if the stage information has been garbage collected listener.stageIdToInfo.getOrElse(stageId, - new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown")) } - val activeStages = mutable.Buffer[StageInfo]() - val completedStages = mutable.Buffer[StageInfo]() + val activeStages = Buffer[StageInfo]() + val completedStages = Buffer[StageInfo]() // If the job is completed, then any pending stages are displayed as "skipped": - val pendingOrSkippedStages = mutable.Buffer[StageInfo]() - val failedStages = mutable.Buffer[StageInfo]() + val pendingOrSkippedStages = Buffer[StageInfo]() + val failedStages = Buffer[StageInfo]() for (stage <- stages) { if (stage.submissionTime.isEmpty) { pendingOrSkippedStages += stage @@ -75,18 +223,18 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) val pendingOrSkippedStagesTable = new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, - parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler) + parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler) val shouldShowActiveStages = activeStages.nonEmpty val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty @@ -154,6 +302,16 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
      var content = summary + val appStartTime = listener.startTime + val executorListener = parent.executorListener + val operationGraphListener = parent.operationGraphListener + + content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, + executorListener.executorIdToData, appStartTime) + + content ++= UIUtils.showDagVizForJob( + jobId, operationGraphListener.getOperationGraphForJob(jobId)) + if (shouldShowActiveStages) { content ++=

      Active Stages ({activeStages.size})

      ++ activeStagesTable.toNodeSeq @@ -174,7 +332,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { content ++=

      Failed Stages ({failedStages.size})

      ++ failedStagesTable.toNodeSeq } - UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent) + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true) } } } 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 937d95a934b59..246e191d64776 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 @@ -25,6 +25,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData._ /** @@ -38,22 +39,25 @@ import org.apache.spark.ui.jobs.UIData._ @DeveloperApi class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { - import JobProgressListener._ - // Define a handful of type aliases so that data structures' types can serve as documentation. // These type aliases are public because they're used in the types of public fields: type JobId = Int + type JobGroupId = String type StageId = Int type StageAttemptId = Int type PoolName = String type ExecutorId = String + // Applicatin: + @volatile var startTime = -1L + // Jobs: val activeJobs = new HashMap[JobId, JobUIData] val completedJobs = ListBuffer[JobUIData]() val failedJobs = ListBuffer[JobUIData]() val jobIdToData = new HashMap[JobId, JobUIData] + val jobGroupToJobIds = new HashMap[JobGroupId, HashSet[JobId]] // Stages: val pendingStages = new HashMap[StageId, StageInfo] @@ -70,18 +74,21 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // JobProgressListener's retention limits. var numCompletedStages = 0 var numFailedStages = 0 + var numCompletedJobs = 0 + var numFailedJobs = 0 // Misc: val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]() - def blockManagerIds = executorIdToBlockManagerId.values.toSeq + + def blockManagerIds: Seq[BlockManagerId] = executorIdToBlockManagerId.values.toSeq var schedulingMode: Option[SchedulingMode] = None // To limit the total memory usage of JobProgressListener, we only track information for a fixed // number of non-active jobs and stages (there is no limit for active jobs and stages): - val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) - val retainedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS) + val retainedStages = conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) + val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) // We can test for memory leaks by ensuring that collections that track non-active jobs and // stages do not grow without bound and that collections for active jobs/stages eventually become @@ -119,7 +126,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { Map( "jobIdToData" -> jobIdToData.size, "stageIdToData" -> stageIdToData.size, - "stageIdToStageInfo" -> stageIdToInfo.size + "stageIdToStageInfo" -> stageIdToInfo.size, + "jobGroupToJobIds" -> jobGroupToJobIds.values.map(_.size).sum, + // Since jobGroupToJobIds is map of sets, check that we don't leak keys with empty values: + "jobGroupToJobIds keySet" -> jobGroupToJobIds.keys.size ) } @@ -140,13 +150,25 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { if (jobs.size > retainedJobs) { val toRemove = math.max(retainedJobs / 10, 1) jobs.take(toRemove).foreach { job => - jobIdToData.remove(job.jobId) + // Remove the job's UI data, if it exists + jobIdToData.remove(job.jobId).foreach { removedJob => + // A null jobGroupId is used for jobs that are run without a job group + val jobGroupId = removedJob.jobGroup.orNull + // Remove the job group -> job mapping entry, if it exists + jobGroupToJobIds.get(jobGroupId).foreach { jobsInGroup => + jobsInGroup.remove(job.jobId) + // If this was the last job in this job group, remove the map entry for the job group + if (jobsInGroup.isEmpty) { + jobGroupToJobIds.remove(jobGroupId) + } + } + } } jobs.trimStart(toRemove) } } - override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { val jobGroup = for ( props <- Option(jobStart.properties); group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) @@ -158,6 +180,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageIds = jobStart.stageIds, jobGroup = jobGroup, status = JobExecutionStatus.RUNNING) + // A null jobGroupId is used for jobs that are run without a job group + jobGroupToJobIds.getOrElseUpdate(jobGroup.orNull, new HashSet[JobId]).add(jobStart.jobId) jobStart.stageInfos.foreach(x => pendingStages(x.stageId) = x) // Compute (a potential underestimate of) the number of tasks that will be run by this job. // This may be an underestimate because the job start event references all of the result @@ -182,7 +206,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized { val jobData = activeJobs.remove(jobEnd.jobId).getOrElse { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) @@ -195,10 +219,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { completedJobs += jobData trimJobsIfNecessary(completedJobs) jobData.status = JobExecutionStatus.SUCCEEDED + numCompletedJobs += 1 case JobFailed(exception) => failedJobs += jobData trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED + numFailedJobs += 1 } for (stageId <- jobData.stageIds) { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => @@ -219,7 +245,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { val stage = stageCompleted.stageInfo stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { @@ -260,13 +286,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val stage = stageSubmitted.stageInfo activeStages(stage.stageId) = stage pendingStages.remove(stage.stageId) val poolName = Option(stageSubmitted.properties).map { - p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) - }.getOrElse(DEFAULT_POOL_NAME) + p => p.getProperty("spark.scheduler.pool", SparkUI.DEFAULT_POOL_NAME) + }.getOrElse(SparkUI.DEFAULT_POOL_NAME) stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) @@ -288,7 +314,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val taskInfo = taskStart.taskInfo if (taskInfo != null) { val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { @@ -312,7 +338,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // stageToTaskInfos already has the updated status. } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task // completion event is for. Let's just drop it here. This means we might have some speculation @@ -497,10 +523,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } -} - -private object JobProgressListener { - val DEFAULT_POOL_NAME = "default" - val DEFAULT_RETAINED_STAGES = 1000 - val DEFAULT_RETAINED_JOBS = 1000 + override def onApplicationStart(appStarted: SparkListenerApplicationStart) { + startTime = appStarted.time + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index b2bbfdee56946..77ca60b000a9b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -24,8 +24,12 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { val sc = parent.sc val killEnabled = parent.killEnabled - def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - val listener = parent.jobProgressListener + val jobProgresslistener = parent.jobProgressListener + val executorListener = parent.executorsListener + val operationGraphListener = parent.operationGraphListener + + def isFairScheduler: Boolean = + jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR) attachPage(new AllJobsPage(this)) attachPage(new JobPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index f47cdc935e539..f3e0b38523f32 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,13 +21,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.{Schedulable, StageInfo} +import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { private val sc = parent.sc - private val listener = parent.listener + private val listener = parent.progressListener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -40,7 +40,7 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { case None => Seq[StageInfo]() } val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) // For now, pool information is only accessible in live UIs diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index df1899e7a9b84..9ba2af54dacf4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.UIUtils /** Table showing list of pools */ private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { - private val listener = parent.listener + private val listener = parent.progressListener def toNodeSeq: Seq[Node] = { listener.synchronized { 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 110f8780a9a12..89d175b06b947 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 @@ -20,22 +20,24 @@ package org.apache.spark.ui.jobs import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.xml.{Node, Unparsed} +import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ +import org.apache.spark.ui.scope.RDDOperationGraph import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { - private val listener = parent.listener + private val progressListener = parent.progressListener + private val operationGraphListener = parent.operationGraphListener def render(request: HttpServletRequest): Seq[Node] = { - listener.synchronized { + progressListener.synchronized { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") @@ -44,23 +46,31 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val stageId = parameterId.toInt val stageAttemptId = parameterAttempt.toInt - val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId)) + val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) + + val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" + if (stageDataOption.isEmpty) { + val content = +
      +

      No information to display for Stage {stageId} (Attempt {stageAttemptId})

      +
      + return UIUtils.headerSparkPage(stageHeader, content, parent) - if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { + } + if (stageDataOption.get.taskData.isEmpty) { val content =

      Summary Metrics

      No tasks have started yet

      Tasks

      No tasks have started yet
      - return UIUtils.headerSparkPage( - s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent) + return UIUtils.headerSparkPage(stageHeader, content, parent) } val stageData = stageDataOption.get val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) - val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables + val accumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables val hasAccumulators = accumulables.size > 0 val summary = @@ -169,8 +179,12 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
    + val dagViz = UIUtils.showDagVizForStage( + stageId, operationGraphListener.getOperationGraphForStage(stageId)) + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo) = {acc.name}{acc.value} + def accumulableRow(acc: AccumulableInfo): Elem = + {acc.name}{acc.value} val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, accumulables.values.toSeq) @@ -268,11 +282,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(serializationTimes) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => - if (info.gettingResultTime > 0) { - (info.finishTime - info.gettingResultTime).toDouble - } else { - 0.0 - } + getGettingResultTime(info).toDouble } val gettingResultQuantiles = @@ -293,10 +303,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val schedulerDelayQuantiles = schedulerDelayTitle +: getFormattedTimeQuantiles(schedulerDelays) - def getFormattedSizeQuantiles(data: Seq[Double]) = + def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) - def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = { + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) + : Seq[Elem] = { val recordDist = getDistributionQuantiles(records).iterator getDistributionQuantiles(data).map(d => {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} @@ -436,13 +447,14 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val content = summary ++ showAdditionalMetrics ++ + dagViz ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ maybeAccumulableTable ++

    Tasks

    ++ taskTable - UIUtils.headerSparkPage("Details for Stage %d".format(stageId), content, parent) + UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } } @@ -462,7 +474,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) - val gettingResultTime = info.gettingResultTime + val gettingResultTime = getGettingResultTime(info) val maybeAccumulators = info.accumulables val accumulatorsReadable = maybeAccumulators.map{acc => s"${acc.name}: ${acc.update.get}"} @@ -625,6 +637,19 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {errorSummary}{details} } + private def getGettingResultTime(info: TaskInfo): Long = { + if (info.gettingResultTime > 0) { + if (info.finishTime > 0) { + info.finishTime - info.gettingResultTime + } else { + // The task is still fetching the result. + System.currentTimeMillis - info.gettingResultTime + } + } else { + 0L + } + } + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { val totalExecutionTime = if (info.gettingResult) { @@ -636,6 +661,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } val executorOverhead = (metrics.executorDeserializeTime + metrics.resultSerializationTime) - math.max(0, totalExecutionTime - metrics.executorRunTime - executorOverhead) + math.max( + 0, + totalExecutionTime - metrics.executorRunTime - executorOverhead - getGettingResultTime(info)) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5865850fa09b5..a33243d4252bf 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -73,20 +73,26 @@ private[ui] class StageTableBase( } private def makeDescription(s: StageInfo): Seq[Node] = { - // scalastyle:off + val basePathUri = UIUtils.prependBaseUri(basePath) + val killLink = if (killEnabled) { - val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(basePath), s.stageId) - val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" - .format(s.stageId) - - (kill) - + val confirm = + s"if (window.confirm('Are you sure you want to kill stage ${s.stageId} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" + // SPARK-6846 this should be POST-only but YARN AM won't proxy POST + /* + val killLinkUri = s"$basePathUri/stages/stage/kill/" +
    + + + (kill) +
    + */ + val killLinkUri = s"$basePathUri/stages/stage/kill/?id=${s.stageId}&terminate=true" + (kill) } - // scalastyle:on - val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) + val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -98,11 +104,9 @@ private[ui] class StageTableBase( @@ -175,7 +179,8 @@ private[ui] class StageTableBase( } /** Render an HTML row that represents a stage */ - private def renderStageRow(s: StageInfo): Seq[Node] = {stageRow(s)} + private def renderStageRow(s: StageInfo): Seq[Node] = + {stageRow(s)} } private[ui] class FailedStageTable( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 937261de00e3a..55169956d8304 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -26,19 +26,20 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc val killEnabled = parent.killEnabled - val listener = parent.jobProgressListener + val progressListener = parent.jobProgressListener + val operationGraphListener = parent.operationGraphListener attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) - def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + def isFairScheduler: Boolean = progressListener.schedulingMode.exists(_ == SchedulingMode.FAIR) - def handleKillRequest(request: HttpServletRequest) = { - if ((killEnabled) && (parent.securityManager.checkModifyPermissions(request.getRemoteUser))) { + def handleKillRequest(request: HttpServletRequest): Unit = { + if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt - if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + if (stageId >= 0 && killFlag && progressListener.activeStages.contains(stageId)) { sc.get.cancelStage(stageId) } // Do a quick pause here to give Spark time to kill the stage so it shows up as 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 dbf1ceeda1878..3d96113aa5fe9 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 @@ -24,7 +24,7 @@ import org.apache.spark.util.collection.OpenHashSet import scala.collection.mutable.HashMap -private[jobs] object UIData { +private[spark] object UIData { class ExecutorSummary { var taskTime : Long = 0 @@ -94,11 +94,11 @@ private[jobs] object UIData { var taskData = new HashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] - def hasInput = inputBytes > 0 - def hasOutput = outputBytes > 0 - def hasShuffleRead = shuffleReadTotalBytes > 0 - def hasShuffleWrite = shuffleWriteBytes > 0 - def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 + def hasInput: Boolean = inputBytes > 0 + def hasOutput: Boolean = outputBytes > 0 + def hasShuffleRead: Boolean = shuffleReadTotalBytes > 0 + def hasShuffleWrite: Boolean = shuffleWriteBytes > 0 + def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 && diskBytesSpilled > 0 } /** @@ -108,4 +108,9 @@ private[jobs] object UIData { var taskInfo: TaskInfo, var taskMetrics: Option[TaskMetrics] = None, var errorMessage: Option[String] = None) + + case class ExecutorUIData( + val startTime: Long, + var finishTime: Option[Long] = None, + var finishReason: Option[String] = None) } diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala new file mode 100644 index 0000000000000..a18c193540ce3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -0,0 +1,205 @@ +/* + * 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.ui.scope + +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +import org.apache.spark.Logging +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.storage.StorageLevel + +/** + * A representation of a generic cluster graph used for storing information on RDD operations. + * + * Each graph is defined with a set of edges and a root cluster, which may contain children + * nodes and children clusters. Additionally, a graph may also have edges that enter or exit + * the graph from nodes that belong to adjacent graphs. + */ +private[ui] case class RDDOperationGraph( + edges: Seq[RDDOperationEdge], + outgoingEdges: Seq[RDDOperationEdge], + incomingEdges: Seq[RDDOperationEdge], + rootCluster: RDDOperationCluster) + +/** A node in an RDDOperationGraph. This represents an RDD. */ +private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean) + +/** + * A directed edge connecting two nodes in an RDDOperationGraph. + * This represents an RDD dependency. + */ +private[ui] case class RDDOperationEdge(fromId: Int, toId: Int) + +/** + * A cluster that groups nodes together in an RDDOperationGraph. + * + * This represents any grouping of RDDs, including operation scopes (e.g. textFile, flatMap), + * stages, jobs, or any higher level construct. A cluster may be nested inside of other clusters. + */ +private[ui] class RDDOperationCluster(val id: String, val name: String) { + private val _childNodes = new ListBuffer[RDDOperationNode] + private val _childClusters = new ListBuffer[RDDOperationCluster] + + def childNodes: Seq[RDDOperationNode] = _childNodes.iterator.toSeq + def childClusters: Seq[RDDOperationCluster] = _childClusters.iterator.toSeq + def attachChildNode(childNode: RDDOperationNode): Unit = { _childNodes += childNode } + def attachChildCluster(childCluster: RDDOperationCluster): Unit = { + _childClusters += childCluster + } + + /** Return all the nodes container in this cluster, including ones nested in other clusters. */ + def getAllNodes: Seq[RDDOperationNode] = { + _childNodes ++ _childClusters.flatMap(_.childNodes) + } +} + +private[ui] object RDDOperationGraph extends Logging { + + /** + * Construct a RDDOperationGraph for a given stage. + * + * The root cluster represents the stage, and all children clusters represent RDD operations. + * Each node represents an RDD, and each edge represents a dependency between two RDDs pointing + * from the parent to the child. + * + * This does not currently merge common operation scopes across stages. This may be worth + * supporting in the future if we decide to group certain stages within the same job under + * a common scope (e.g. part of a SQL query). + */ + def makeOperationGraph(stage: StageInfo): RDDOperationGraph = { + val edges = new ListBuffer[RDDOperationEdge] + val nodes = new mutable.HashMap[Int, RDDOperationNode] + val clusters = new mutable.HashMap[String, RDDOperationCluster] // indexed by cluster ID + + // Root cluster is the stage cluster + val stageClusterId = s"stage_${stage.stageId}" + val stageClusterName = s"Stage ${stage.stageId}" + + { if (stage.attemptId == 0) "" else s" (attempt ${stage.attemptId})" } + val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName) + + // Find nodes, edges, and operation scopes that belong to this stage + stage.rddInfos.foreach { rdd => + edges ++= rdd.parentIds.map { parentId => RDDOperationEdge(parentId, rdd.id) } + + // TODO: differentiate between the intention to cache an RDD and whether it's actually cached + val node = nodes.getOrElseUpdate( + rdd.id, RDDOperationNode(rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE)) + + if (rdd.scope == null) { + // This RDD has no encompassing scope, so we put it directly in the root cluster + // This should happen only if an RDD is instantiated outside of a public RDD API + rootCluster.attachChildNode(node) + } else { + // Otherwise, this RDD belongs to an inner cluster, + // which may be nested inside of other clusters + val rddScopes = rdd.scope.map { scope => scope.getAllScopes }.getOrElse(Seq.empty) + val rddClusters = rddScopes.map { scope => + val clusterId = scope.name + "_" + scope.id + val clusterName = scope.name + clusters.getOrElseUpdate(clusterId, new RDDOperationCluster(clusterId, clusterName)) + } + // Build the cluster hierarchy for this RDD + rddClusters.sliding(2).foreach { pc => + if (pc.size == 2) { + val parentCluster = pc(0) + val childCluster = pc(1) + parentCluster.attachChildCluster(childCluster) + } + } + // Attach the outermost cluster to the root cluster, and the RDD to the innermost cluster + rddClusters.headOption.foreach { cluster => rootCluster.attachChildCluster(cluster) } + rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) } + } + } + + // Classify each edge as internal, outgoing or incoming + // This information is needed to reason about how stages relate to each other + val internalEdges = new ListBuffer[RDDOperationEdge] + val outgoingEdges = new ListBuffer[RDDOperationEdge] + val incomingEdges = new ListBuffer[RDDOperationEdge] + edges.foreach { case e: RDDOperationEdge => + val fromThisGraph = nodes.contains(e.fromId) + val toThisGraph = nodes.contains(e.toId) + (fromThisGraph, toThisGraph) match { + case (true, true) => internalEdges += e + case (true, false) => outgoingEdges += e + case (false, true) => incomingEdges += e + // should never happen + case _ => logWarning(s"Found an orphan edge in stage ${stage.stageId}: $e") + } + } + + RDDOperationGraph(internalEdges, outgoingEdges, incomingEdges, rootCluster) + } + + /** + * Generate the content of a dot file that describes the specified graph. + * + * Note that this only uses a minimal subset of features available to the DOT specification. + * Part of the styling must be done here because the rendering library must take certain + * attributes into account when arranging the graph elements. More style is added in the + * visualization later through post-processing in JavaScript. + * + * For the complete DOT specification, see http://www.graphviz.org/Documentation/dotguide.pdf. + */ + def makeDotFile(graph: RDDOperationGraph, forJob: Boolean): String = { + val dotFile = new StringBuilder + dotFile.append("digraph G {\n") + dotFile.append(makeDotSubgraph(graph.rootCluster, forJob, indent = " ")) + graph.edges.foreach { edge => + dotFile.append(s""" ${edge.fromId}->${edge.toId} [lineInterpolate="basis"];\n""") + } + dotFile.append("}") + val result = dotFile.toString() + logDebug(result) + result + } + + /** + * Return the dot representation of a node in an RDDOperationGraph. + * + * On the job page, is displayed as a small circle without labels. + * On the stage page, it is displayed as a box with an embedded label. + */ + private def makeDotNode(node: RDDOperationNode, forJob: Boolean): String = { + if (forJob) { + s"""${node.id} [label=" " shape="circle" padding="5" labelStyle="font-size: 0"]""" + } else { + s"""${node.id} [label="${node.name} (${node.id})"]""" + } + } + + /** Return the dot representation of a subgraph in an RDDOperationGraph. */ + private def makeDotSubgraph( + scope: RDDOperationCluster, + forJob: Boolean, + indent: String): String = { + val subgraph = new StringBuilder + subgraph.append(indent + s"subgraph cluster${scope.id} {\n") + subgraph.append(indent + s""" label="${scope.name}";\n""") + scope.childNodes.foreach { node => + subgraph.append(indent + s" ${makeDotNode(node, forJob)};\n") + } + scope.childClusters.foreach { cscope => + subgraph.append(makeDotSubgraph(cscope, forJob, indent + " ")) + } + subgraph.append(indent + "}\n") + subgraph.toString() + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala new file mode 100644 index 0000000000000..2884a49f31122 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.scope + +import scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler._ +import org.apache.spark.ui.SparkUI + +/** + * A SparkListener that constructs a DAG of RDD operations. + */ +private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListener { + private val jobIdToStageIds = new mutable.HashMap[Int, Seq[Int]] + private val stageIdToGraph = new mutable.HashMap[Int, RDDOperationGraph] + private val stageIds = new mutable.ArrayBuffer[Int] + + // How many jobs or stages to retain graph metadata for + private val retainedStages = + conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) + + /** Return the graph metadata for the given stage, or None if no such information exists. */ + def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = { + jobIdToStageIds.get(jobId) + .map { sids => sids.flatMap { sid => stageIdToGraph.get(sid) } } + .getOrElse { Seq.empty } + } + + /** Return the graph metadata for the given stage, or None if no such information exists. */ + def getOperationGraphForStage(stageId: Int): Option[RDDOperationGraph] = { + stageIdToGraph.get(stageId) + } + + /** On job start, construct a RDDOperationGraph for each stage in the job for display later. */ + override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { + val jobId = jobStart.jobId + val stageInfos = jobStart.stageInfos + + stageInfos.foreach { stageInfo => + stageIds += stageInfo.stageId + stageIdToGraph(stageInfo.stageId) = RDDOperationGraph.makeOperationGraph(stageInfo) + } + jobIdToStageIds(jobId) = stageInfos.map(_.stageId).sorted + + // Remove graph metadata for old stages + if (stageIds.size >= retainedStages) { + val toRemove = math.max(retainedStages / 10, 1) + stageIds.take(toRemove).foreach { id => stageIdToGraph.remove(id) } + stageIds.trimStart(toRemove) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 199f731b92bcc..05f94a7507f4f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,8 +21,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ @@ -32,28 +32,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val rddId = parameterId.toInt - val storageStatusList = listener.storageStatusList - val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { - // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) - } + val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,includeDetails = true) + .getOrElse { + // Rather than crashing, render an "RDD Not Found" page + return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) + } // Worker table - val workers = storageStatusList.map((rddId, _)) - val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers, - id = Some("rdd-storage-by-worker-table")) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, + rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table")) // Block table - val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) - val blocks = storageStatusList - .flatMap(_.rddBlocksById(rddId)) - .sortWith(_._1.name < _._1.name) - .map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) - } - val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks, + val blockTable = UIUtils.listingTable(blockHeader, blockRow, rddStorageInfo.partitions.get, id = Some("rdd-storage-by-block-table")) val content = @@ -62,23 +53,23 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
    • Storage Level: - {rddInfo.storageLevel.description} + {rddStorageInfo.storageLevel}
    • Cached Partitions: - {rddInfo.numCachedPartitions} + {rddStorageInfo.numCachedPartitions}
    • Total Partitions: - {rddInfo.numPartitions} + {rddStorageInfo.numPartitions}
    • Memory Size: - {Utils.bytesToString(rddInfo.memSize)} + {Utils.bytesToString(rddStorageInfo.memoryUsed)}
    • Disk Size: - {Utils.bytesToString(rddInfo.diskSize)} + {Utils.bytesToString(rddStorageInfo.diskUsed)}
    @@ -86,19 +77,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
    -

    Data Distribution on {workers.size} Executors

    +

    Data Distribution on {rddStorageInfo.dataDistribution.size} Executors

    {workerTable}
    -

    {blocks.size} Partitions

    +

    {rddStorageInfo.partitions.size} Partitions

    {blockTable}
    ; - UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent) + UIUtils.headerSparkPage("RDD Storage Info for " + rddStorageInfo.name, content, parent) } /** Header fields for the worker table */ @@ -116,34 +107,32 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { "Executors") /** Render an HTML row representing a worker */ - private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { - val (rddId, status) = worker + private def workerRow(worker: RDDDataDistribution): Seq[Node] = { - {status.blockManagerId.host + ":" + status.blockManagerId.port} + {worker.address} - {Utils.bytesToString(status.memUsedByRdd(rddId))} - ({Utils.bytesToString(status.memRemaining)} Remaining) + {Utils.bytesToString(worker.memoryUsed)} + ({Utils.bytesToString(worker.memoryRemaining)} Remaining) - {Utils.bytesToString(status.diskUsedByRdd(rddId))} + {Utils.bytesToString(worker.diskUsed)} } /** Render an HTML row representing a block */ - private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { - val (id, block, locations) = row + private def blockRow(row: RDDPartitionInfo): Seq[Node] = { - {id} + {row.blockName} - {block.storageLevel.description} + {row.storageLevel} - - {Utils.bytesToString(block.memSize)} + + {Utils.bytesToString(row.memoryUsed)} - - {Utils.bytesToString(block.diskSize)} + + {Utils.bytesToString(row.diskUsed)} - {locations.map(l => {l}
    )} + {row.executors.map(l => {l}
    )} } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 6ced6052d2b18..07db783c572cf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ @@ -42,7 +42,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Cached Partitions", "Fraction Cached", "Size in Memory", - "Size in Tachyon", + "Size in ExternalBlockStore", "Size on Disk") /** Render an HTML row representing an RDD */ @@ -59,7 +59,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} - {Utils.bytesToString(rdd.tachyonSize)} + {Utils.bytesToString(rdd.externalBlockStoreSize)} {Utils.bytesToString(rdd.diskSize)} // scalastyle:on diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index a81291d505583..0351749700962 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -35,15 +35,19 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag /** * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the BlockManagerUI. + * + * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing - def storageStatusList = storageStatusListener.storageStatusList + def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ - def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + def rddInfoList: Seq[RDDInfo] = synchronized { + _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + } /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { @@ -56,19 +60,19 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar * Assumes the storage status list is fully up-to-date. This implies the corresponding * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. */ - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val metrics = taskEnd.taskMetrics if (metrics != null && metrics.updatedBlocks.isDefined) { updateRDDInfo(metrics.updatedBlocks.get) } } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { // Remove all partitions that are no longer cached in current completed stage val completedRddIds = stageCompleted.stageInfo.rddInfos.map(r => r.id).toSet _rddInfoMap.retain { case (id, info) => @@ -76,7 +80,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar } } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { _rddInfoMap.remove(unpersistRDD.rddId) } } diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala index 332d0cbb2dc0c..81a7cbde01ce5 100644 --- a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala +++ b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala @@ -43,7 +43,13 @@ private[spark] trait ActorLogReceive { private val _receiveWithLogging = receiveWithLogging - override def isDefinedAt(o: Any): Boolean = _receiveWithLogging.isDefinedAt(o) + override def isDefinedAt(o: Any): Boolean = { + val handled = _receiveWithLogging.isDefinedAt(o) + if (!handled) { + log.debug(s"Received unexpected actor system event: $o") + } + handled + } override def apply(o: Any): Unit = { if (log.isDebugEnabled) { diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 48a6ede05e17b..de3316d083a22 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -19,8 +19,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await -import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.Try +import scala.concurrent.duration.FiniteDuration import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -66,7 +65,8 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) + val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", + conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -78,8 +78,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") + val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() @@ -102,14 +102,14 @@ private[spark] object AkkaUtils extends Logging { |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize @@ -125,16 +125,6 @@ private[spark] object AkkaUtils extends Logging { (actorSystem, boundPort) } - /** Returns the default Spark timeout to use for Akka ask operations. */ - def askTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getLong("spark.akka.askTimeout", 30), "seconds") - } - - /** Returns the default Spark timeout to use for Akka remote actor lookup. */ - def lookupTimeout(conf: SparkConf): FiniteDuration = { - Duration.create(conf.getLong("spark.akka.lookupTimeout", 30), "seconds") - } - private val AKKA_MAX_FRAME_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 /** Returns the configured max frame size for Akka messages in bytes. */ @@ -150,16 +140,6 @@ private[spark] object AkkaUtils extends Logging { /** Space reserved for extra data in an Akka message besides serialized task or task result. */ val reservedSizeBytes = 200 * 1024 - /** Returns the configured number of times to retry connecting */ - def numRetries(conf: SparkConf): Int = { - conf.getInt("spark.akka.num.retries", 3) - } - - /** Returns the configured number of milliseconds to wait on each retry */ - def retryWaitMs(conf: SparkConf): Int = { - conf.getInt("spark.akka.retry.wait", 3000) - } - /** * Send a message to the given actor and get its result within a default timeout, or * throw a SparkException if this fails. @@ -179,7 +159,7 @@ private[spark] object AkkaUtils extends Logging { message: Any, actor: ActorRef, maxAttempts: Int, - retryInterval: Int, + retryInterval: Long, timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { @@ -203,7 +183,9 @@ private[spark] object AkkaUtils extends Logging { lastException = e logWarning(s"Error sending message [message = $message] in $attempts attempts", e) } - Thread.sleep(retryInterval) + if (attempts < maxAttempts) { + Thread.sleep(retryInterval) + } } throw new SparkException( @@ -216,7 +198,7 @@ private[spark] object AkkaUtils extends Logging { val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) - val timeout = AkkaUtils.lookupTimeout(conf) + val timeout = RpcUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } @@ -230,7 +212,7 @@ private[spark] object AkkaUtils extends Logging { val executorActorSystemName = SparkEnv.executorActorSystemName Utils.checkHost(host, "Expected hostname") val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) - val timeout = AkkaUtils.lookupTimeout(conf) + val timeout = RpcUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index e3f52f6ff1e63..6fe32e469c732 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -19,17 +19,20 @@ package org.apache.spark.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} -import scala.collection.mutable.Map -import scala.collection.mutable.Set +import scala.collection.mutable.{Map, Set} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.{Logging, SparkEnv, SparkException} +/** + * A cleaner that renders closures serializable if they can be done so safely. + */ private[spark] object ClosureCleaner extends Logging { + // Get an ASM class reader for a given class from the JAR that loaded it - private def getClassReader(cls: Class[_]): ClassReader = { + private[util] def getClassReader(cls: Class[_]): ClassReader = { // Copy data over, before delegating to ClassReader - else we can run out of open file handles. val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" val resourceStream = cls.getResourceAsStream(className) @@ -55,10 +58,14 @@ private[spark] object ClosureCleaner extends Logging { private def getOuterClasses(obj: AnyRef): List[Class[_]] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { f.setAccessible(true) - if (isClosure(f.getType)) { - return f.getType :: getOuterClasses(f.get(obj)) - } else { - return f.getType :: Nil // Stop at the first $outer that is not a closure + val outer = f.get(obj) + // The outer pointer may be null if we have cleaned this closure before + if (outer != null) { + if (isClosure(f.getType)) { + return f.getType :: getOuterClasses(outer) + } else { + return f.getType :: Nil // Stop at the first $outer that is not a closure + } } } Nil @@ -68,16 +75,23 @@ private[spark] object ClosureCleaner extends Logging { private def getOuterObjects(obj: AnyRef): List[AnyRef] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { f.setAccessible(true) - if (isClosure(f.getType)) { - return f.get(obj) :: getOuterObjects(f.get(obj)) - } else { - return f.get(obj) :: Nil // Stop at the first $outer that is not a closure + val outer = f.get(obj) + // The outer pointer may be null if we have cleaned this closure before + if (outer != null) { + if (isClosure(f.getType)) { + return outer :: getOuterObjects(outer) + } else { + return outer :: Nil // Stop at the first $outer that is not a closure + } } } Nil } - private def getInnerClasses(obj: AnyRef): List[Class[_]] = { + /** + * Return a list of classes that represent closures enclosed in the given closure object. + */ + private def getInnerClosureClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) var stack = List[Class[_]](obj.getClass) while (!stack.isEmpty) { @@ -90,7 +104,7 @@ private[spark] object ClosureCleaner extends Logging { stack = cls :: stack } } - return (seen - obj.getClass).toList + (seen - obj.getClass).toList } private def createNullValue(cls: Class[_]): AnyRef = { @@ -101,21 +115,129 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean(func: AnyRef, checkSerializable: Boolean = true) { + /** + * Clean the given closure in place. + * + * More specifically, this renders the given closure serializable as long as it does not + * explicitly reference unserializable objects. + * + * @param closure the closure to clean + * @param checkSerializable whether to verify that the closure is serializable after cleaning + * @param cleanTransitively whether to clean enclosing closures transitively + */ + def clean( + closure: AnyRef, + checkSerializable: Boolean = true, + cleanTransitively: Boolean = true): Unit = { + clean(closure, checkSerializable, cleanTransitively, Map.empty) + } + + /** + * Helper method to clean the given closure in place. + * + * The mechanism is to traverse the hierarchy of enclosing closures and null out any + * references along the way that are not actually used by the starting closure, but are + * nevertheless included in the compiled anonymous classes. Note that it is unsafe to + * simply mutate the enclosing closures in place, as other code paths may depend on them. + * Instead, we clone each enclosing closure and set the parent pointers accordingly. + * + * By default, closures are cleaned transitively. This means we detect whether enclosing + * objects are actually referenced by the starting one, either directly or transitively, + * and, if not, sever these closures from the hierarchy. In other words, in addition to + * nulling out unused field references, we also null out any parent pointers that refer + * to enclosing objects not actually needed by the starting closure. We determine + * transitivity by tracing through the tree of all methods ultimately invoked by the + * inner closure and record all the fields referenced in the process. + * + * For instance, transitive cleaning is necessary in the following scenario: + * + * class SomethingNotSerializable { + * def someValue = 1 + * def scope(name: String)(body: => Unit) = body + * def someMethod(): Unit = scope("one") { + * def x = someValue + * def y = 2 + * scope("two") { println(y + 1) } + * } + * } + * + * In this example, scope "two" is not serializable because it references scope "one", which + * references SomethingNotSerializable. Note that, however, the body of scope "two" does not + * actually depend on SomethingNotSerializable. This means we can safely null out the parent + * pointer of a cloned scope "one" and set it the parent of scope "two", such that scope "two" + * no longer references SomethingNotSerializable transitively. + * + * @param func the starting closure to clean + * @param checkSerializable whether to verify that the closure is serializable after cleaning + * @param cleanTransitively whether to clean enclosing closures transitively + * @param accessedFields a map from a class to a set of its fields that are accessed by + * the starting closure + */ + private def clean( + func: AnyRef, + checkSerializable: Boolean, + cleanTransitively: Boolean, + accessedFields: Map[Class[_], Set[String]]): Unit = { + + if (!isClosure(func.getClass)) { + logWarning("Expected a closure; got " + func.getClass.getName) + return + } + + // TODO: clean all inner closures first. This requires us to find the inner objects. // TODO: cache outerClasses / innerClasses / accessedFields + + if (func == null) { + return + } + + logDebug(s"+++ Cleaning closure $func (${func.getClass.getName}}) +++") + + // A list of classes that represents closures enclosed in the given one + val innerClasses = getInnerClosureClasses(func) + + // A list of enclosing objects and their respective classes, from innermost to outermost + // An outer object at a given index is of type outer class at the same index val outerClasses = getOuterClasses(func) - val innerClasses = getInnerClasses(func) val outerObjects = getOuterObjects(func) - val accessedFields = Map[Class[_], Set[String]]() - + // For logging purposes only + val declaredFields = func.getClass.getDeclaredFields + val declaredMethods = func.getClass.getDeclaredMethods + + logDebug(" + declared fields: " + declaredFields.size) + declaredFields.foreach { f => logDebug(" " + f) } + logDebug(" + declared methods: " + declaredMethods.size) + declaredMethods.foreach { m => logDebug(" " + m) } + logDebug(" + inner classes: " + innerClasses.size) + innerClasses.foreach { c => logDebug(" " + c.getName) } + logDebug(" + outer classes: " + outerClasses.size) + outerClasses.foreach { c => logDebug(" " + c.getName) } + logDebug(" + outer objects: " + outerObjects.size) + outerObjects.foreach { o => logDebug(" " + o) } + + // Fail fast if we detect return statements in closures getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) - - for (cls <- outerClasses) - accessedFields(cls) = Set[String]() - for (cls <- func.getClass :: innerClasses) - getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0) - // logInfo("accessedFields: " + accessedFields) + + // If accessed fields is not populated yet, we assume that + // the closure we are trying to clean is the starting one + if (accessedFields.isEmpty) { + logDebug(s" + populating accessed fields because this is the starting closure") + // Initialize accessed fields with the outer classes first + // This step is needed to associate the fields to the correct classes later + for (cls <- outerClasses) { + accessedFields(cls) = Set[String]() + } + // Populate accessed fields by visiting all fields and methods accessed by this and + // all of its inner closures. If transitive cleaning is enabled, this may recursively + // visits methods that belong to other classes in search of transitively referenced fields. + for (cls <- func.getClass :: innerClasses) { + getClassReader(cls).accept(new FieldAccessFinder(accessedFields, cleanTransitively), 0) + } + } + + logDebug(s" + fields accessed by starting closure: " + accessedFields.size) + accessedFields.foreach { f => logDebug(" " + f) } val inInterpreter = { try { @@ -126,34 +248,68 @@ private[spark] object ClosureCleaner extends Logging { } } + // List of outer (class, object) pairs, ordered from outermost to innermost + // Note that all outer objects but the outermost one (first one in this list) must be closures var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse - var outer: AnyRef = null + var parent: AnyRef = null if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) { // The closure is ultimately nested inside a class; keep the object of that // class without cloning it since we don't want to clone the user's objects. - outer = outerPairs.head._2 + // Note that we still need to keep around the outermost object itself because + // we need it to clone its child closure later (see below). + logDebug(s" + outermost object is not a closure, so do not clone it: ${outerPairs.head}") + parent = outerPairs.head._2 // e.g. SparkContext outerPairs = outerPairs.tail + } else if (outerPairs.size > 0) { + logDebug(s" + outermost object is a closure, so we just keep it: ${outerPairs.head}") + } else { + logDebug(" + there are no enclosing objects!") } + // Clone the closure objects themselves, nulling out any fields that are not // used in the closure we're working on or any of its inner closures. for ((cls, obj) <- outerPairs) { - outer = instantiateClass(cls, outer, inInterpreter) + logDebug(s" + cloning the object $obj of class ${cls.getName}") + // We null out these unused references by cloning each object and then filling in all + // required fields from the original object. We need the parent here because the Java + // language specification requires the first constructor parameter of any closure to be + // its enclosing object. + val clone = instantiateClass(cls, parent, inInterpreter) for (fieldName <- accessedFields(cls)) { val field = cls.getDeclaredField(fieldName) field.setAccessible(true) val value = field.get(obj) - // logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); - field.set(outer, value) + field.set(clone, value) } + // If transitive cleaning is enabled, we recursively clean any enclosing closure using + // the already populated accessed fields map of the starting closure + if (cleanTransitively && isClosure(clone.getClass)) { + logDebug(s" + cleaning cloned closure $clone recursively (${cls.getName})") + // No need to check serializable here for the outer closures because we're + // only interested in the serializability of the starting closure + clean(clone, checkSerializable = false, cleanTransitively, accessedFields) + } + parent = clone } - if (outer != null) { - // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); + // Update the parent pointer ($outer) of this closure + if (parent != null) { val field = func.getClass.getDeclaredField("$outer") field.setAccessible(true) - field.set(func, outer) + // If the starting closure doesn't actually need our enclosing object, then just null it out + if (accessedFields.contains(func.getClass) && + !accessedFields(func.getClass).contains("$outer")) { + logDebug(s" + the starting closure doesn't actually need $parent, so we null it out") + field.set(func, null) + } else { + // Update this closure's parent pointer to point to our enclosing object, + // which could either be a cloned closure or the original user object + field.set(func, parent) + } } - + + logDebug(s" +++ closure $func (${func.getClass.getName}) is now cleaned +++") + if (checkSerializable) { ensureSerializable(func) } @@ -161,21 +317,25 @@ private[spark] object ClosureCleaner extends Logging { private def ensureSerializable(func: AnyRef) { try { - SparkEnv.get.closureSerializer.newInstance().serialize(func) + if (SparkEnv.get != null) { + SparkEnv.get.closureSerializer.newInstance().serialize(func) + } } catch { case ex: Exception => throw new SparkException("Task not serializable", ex) } } - private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { - // logInfo("Creating a " + cls + " with outer = " + outer) + private def instantiateClass( + cls: Class[_], + enclosingObject: AnyRef, + inInterpreter: Boolean): AnyRef = { if (!inInterpreter) { // This is a bona fide closure class, whose constructor has no effects // other than to set its fields, so use its constructor val cons = cls.getConstructors()(0) val params = cons.getParameterTypes.map(createNullValue).toArray - if (outer != null) { - params(0) = outer // First param is always outer object + if (enclosingObject != null) { + params(0) = enclosingObject // First param is always enclosing object } return cons.newInstance(params: _*).asInstanceOf[AnyRef] } else { @@ -184,26 +344,27 @@ private[spark] object ClosureCleaner extends Logging { val parentCtor = classOf[java.lang.Object].getDeclaredConstructor() val newCtor = rf.newConstructorForSerialization(cls, parentCtor) val obj = newCtor.newInstance().asInstanceOf[AnyRef] - if (outer != null) { - // logInfo("3: Setting $outer on " + cls + " to " + outer); + if (enclosingObject != null) { val field = cls.getDeclaredField("$outer") field.setAccessible(true) - field.set(obj, outer) + field.set(obj, enclosingObject) } obj } } } -private[spark] -class ReturnStatementFinder extends ClassVisitor(ASM4) { +private[spark] class ReturnStatementInClosureException + extends SparkException("Return statements aren't allowed in Spark closures") + +private class ReturnStatementFinder extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name.contains("apply")) { new MethodVisitor(ASM4) { override def visitTypeInsn(op: Int, tp: String) { if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) { - throw new SparkException("Return statements aren't allowed in Spark closures") + throw new ReturnStatementInClosureException } } } @@ -213,26 +374,65 @@ class ReturnStatementFinder extends ClassVisitor(ASM4) { } } -private[spark] -class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { - override def visitMethod(access: Int, name: String, desc: String, - sig: String, exceptions: Array[String]): MethodVisitor = { +/** Helper class to identify a method. */ +private case class MethodIdentifier[T](cls: Class[T], name: String, desc: String) + +/** + * Find the fields accessed by a given class. + * + * The resulting fields are stored in the mutable map passed in through the constructor. + * This map is assumed to have its keys already populated with the classes of interest. + * + * @param fields the mutable map that stores the fields to return + * @param findTransitively if true, find fields indirectly referenced through method calls + * @param specificMethod if not empty, visit only this specific method + * @param visitedMethods a set of visited methods to avoid cycles + */ +private[util] class FieldAccessFinder( + fields: Map[Class[_], Set[String]], + findTransitively: Boolean, + specificMethod: Option[MethodIdentifier[_]] = None, + visitedMethods: Set[MethodIdentifier[_]] = Set.empty) + extends ClassVisitor(ASM4) { + + override def visitMethod( + access: Int, + name: String, + desc: String, + sig: String, + exceptions: Array[String]): MethodVisitor = { + + // If we are told to visit only a certain method and this is not the one, ignore it + if (specificMethod.isDefined && + (specificMethod.get.name != name || specificMethod.get.desc != desc)) { + return null + } + new MethodVisitor(ASM4) { override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { if (op == GETFIELD) { - for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { - output(cl) += name + for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) { + fields(cl) += name } } } - override def visitMethodInsn(op: Int, owner: String, name: String, - desc: String) { - // Check for calls a getter method for a variable in an interpreter wrapper object. - // This means that the corresponding field will be accessed, so we should save it. - if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) { - for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { - output(cl) += name + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) { + // Check for calls a getter method for a variable in an interpreter wrapper object. + // This means that the corresponding field will be accessed, so we should save it. + if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) { + fields(cl) += name + } + // Optionally visit other methods to find fields that are transitively referenced + if (findTransitively) { + val m = MethodIdentifier(cl, name, desc) + if (!visitedMethods.contains(m)) { + // Keep track of visited methods to avoid potential infinite cycles + visitedMethods += m + ClosureCleaner.getClassReader(cl).accept( + new FieldAccessFinder(fields, findTransitively, Some(m), visitedMethods), 0) + } } } } @@ -240,9 +440,14 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor } } -private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { +private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null + // TODO: Recursively find inner closures that we indirectly reference, e.g. + // val closure1 = () = { () => 1 } + // val closure2 = () => { (1 to 5).map(closure1) } + // The second closure technically has two inner closures, but this finder only finds one + override def visit(version: Int, access: Int, name: String, sig: String, superName: String, interfaces: Array[String]) { myName = name diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 390310243ee0a..9044aaeef2d48 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -27,8 +27,8 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat // scalastyle:on private[this] var completed = false - def next() = sub.next() - def hasNext = { + def next(): A = sub.next() + def hasNext: Boolean = { val r = sub.hasNext if (!r && !completed) { completed = true @@ -37,13 +37,13 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat r } - def completion() + def completion(): Unit } private[spark] object CompletionIterator { - def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A,I] = { + def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A, I] = { new CompletionIterator[A,I](sub) { - def completion() = completionFunction + def completion(): Unit = completionFunction } } } diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index a465298c8c5ab..9aea8efa38c7a 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -57,7 +57,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va out.println } - def statCounter = StatCounter(data.slice(startIdx, endIdx)) + def statCounter: StatCounter = StatCounter(data.slice(startIdx, endIdx)) /** * print a summary of this distribution to the given PrintStream. diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index b0ed908b84424..e9b2b8d24b476 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -76,9 +76,21 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging { def stop(): Unit = { if (stopped.compareAndSet(false, true)) { eventThread.interrupt() - eventThread.join() - // Call onStop after the event thread exits to make sure onReceive happens before onStop - onStop() + var onStopCalled = false + try { + eventThread.join() + // Call onStop after the event thread exits to make sure onReceive happens before onStop + onStopCalled = true + onStop() + } catch { + case ie: InterruptedException => + Thread.currentThread().interrupt() + if (!onStopCalled) { + // ie is thrown from `eventThread.join()`. Otherwise, we should not call `onStop` since + // it's already called. + onStop() + } + } } else { // Keep quiet to allow calling `stop` multiple times. } 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 474f79fb756f6..3f162d1f6c3eb 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -28,10 +28,11 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ +import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.storage._ -import org.apache.spark._ /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- @@ -194,7 +195,8 @@ private[spark] object JsonProtocol { ("App Name" -> applicationStart.appName) ~ ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ - ("User" -> applicationStart.sparkUser) + ("User" -> applicationStart.sparkUser) ~ + ("App Attempt ID" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { @@ -227,6 +229,7 @@ private[spark] object JsonProtocol { def stageInfoToJson(stageInfo: StageInfo): JValue = { val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).toList) + val parentIds = JArray(stageInfo.parentIds.map(JInt(_)).toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) @@ -235,6 +238,7 @@ private[spark] object JsonProtocol { ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ + ("Parent IDs" -> parentIds) ~ ("Details" -> stageInfo.details) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ @@ -367,20 +371,23 @@ private[spark] object JsonProtocol { def rddInfoToJson(rddInfo: RDDInfo): JValue = { val storageLevel = storageLevelToJson(rddInfo.storageLevel) + val parentIds = JArray(rddInfo.parentIds.map(JInt(_)).toList) ("RDD ID" -> rddInfo.id) ~ ("Name" -> rddInfo.name) ~ + ("Scope" -> rddInfo.scope.map(_.toJson)) ~ + ("Parent IDs" -> parentIds) ~ ("Storage Level" -> storageLevel) ~ ("Number of Partitions" -> rddInfo.numPartitions) ~ ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ - ("Tachyon Size" -> rddInfo.tachyonSize) ~ + ("ExternalBlockStore Size" -> rddInfo.externalBlockStoreSize) ~ ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ - ("Use Tachyon" -> storageLevel.useOffHeap) ~ + ("Use ExternalBlockStore" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -389,7 +396,7 @@ private[spark] object JsonProtocol { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ ("Memory Size" -> blockStatus.memSize) ~ - ("Tachyon Size" -> blockStatus.tachyonSize) ~ + ("ExternalBlockStore Size" -> blockStatus.externalBlockStoreSize) ~ ("Disk Size" -> blockStatus.diskSize) } @@ -518,7 +525,7 @@ private[spark] object JsonProtocol { // The "Stage Infos" field was added in Spark 1.2.0 val stageInfos = Utils.jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) } @@ -562,7 +569,8 @@ private[spark] object JsonProtocol { val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - SparkListenerApplicationStart(appName, appId, time, sparkUser) + val appAttemptId = Utils.jsonOption(json \ "App Attempt ID").map(_.extract[String]) + SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { @@ -597,7 +605,10 @@ private[spark] object JsonProtocol { val attemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] - val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_)) + val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) + val parentIds = Utils.jsonOption(json \ "Parent IDs") + .map { l => l.extract[List[JValue]].map(_.extract[Int]) } + .getOrElse(Seq.empty) val details = (json \ "Details").extractOpt[String].getOrElse("") val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) @@ -607,7 +618,8 @@ private[spark] object JsonProtocol { case None => Seq[AccumulableInfo]() } - val stageInfo = new StageInfo(stageId, attemptId, stageName, numTasks, rddInfos, details) + val stageInfo = new StageInfo( + stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -783,17 +795,25 @@ private[spark] object JsonProtocol { def rddInfoFromJson(json: JValue): RDDInfo = { val rddId = (json \ "RDD ID").extract[Int] val name = (json \ "Name").extract[String] + val scope = Utils.jsonOption(json \ "Scope") + .map(_.extract[String]) + .map(RDDOperationScope.fromJson) + val parentIds = Utils.jsonOption(json \ "Parent IDs") + .map { l => l.extract[List[JValue]].map(_.extract[Int]) } + .getOrElse(Seq.empty) val storageLevel = storageLevelFromJson(json \ "Storage Level") val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] - val tachyonSize = (json \ "Tachyon Size").extract[Long] + // fallback to tachyon for backward compatibility + val externalBlockStoreSize = (json \ "ExternalBlockStore Size").toSome + .getOrElse(json \ "Tachyon Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) + val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel, parentIds, scope) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.tachyonSize = tachyonSize + rddInfo.externalBlockStoreSize = externalBlockStoreSize rddInfo.diskSize = diskSize rddInfo } @@ -801,18 +821,22 @@ private[spark] object JsonProtocol { def storageLevelFromJson(json: JValue): StorageLevel = { val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] - val useTachyon = (json \ "Use Tachyon").extract[Boolean] + // fallback to tachyon for backward compatability + val useExternalBlockStore = (json \ "Use ExternalBlockStore").toSome + .getOrElse(json \ "Use Tachyon").extract[Boolean] val deserialized = (json \ "Deserialized").extract[Boolean] val replication = (json \ "Replication").extract[Int] - StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication) + StorageLevel(useDisk, useMemory, useExternalBlockStore, deserialized, replication) } def blockStatusFromJson(json: JValue): BlockStatus = { val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - val tachyonSize = (json \ "Tachyon Size").extract[Long] - BlockStatus(storageLevel, memorySize, diskSize, tachyonSize) + // fallback to tachyon for backward compatability + val externalBlockStoreSize = (json \ "ExternalBlockStore Size").toSome + .getOrElse(json \ "Tachyon Size").extract[Long] + BlockStatus(storageLevel, memorySize, diskSize, externalBlockStoreSize) } def executorInfoFromJson(json: JValue): ExecutorInfo = { diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index d60b8b9a31a9b..a725767d08cc2 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -19,9 +19,12 @@ package org.apache.spark.util import java.util.concurrent.CopyOnWriteArrayList +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.Logging +import org.apache.spark.scheduler.SparkListener /** * An event bus which posts events to its listeners. @@ -64,4 +67,9 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { */ def onPostEvent(listener: L, event: E): Unit + private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { + val c = implicitly[ClassTag[T]].runtimeClass + listeners.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq + } + } diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index cf89c1782fd67..1718554061985 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -39,31 +39,27 @@ private[spark] class ManualClock(private var time: Long) extends Clock { /** * @param timeToSet new time (in milliseconds) that the clock should represent */ - def setTime(timeToSet: Long) = - synchronized { - time = timeToSet - notifyAll() - } + def setTime(timeToSet: Long): Unit = synchronized { + time = timeToSet + notifyAll() + } /** * @param timeToAdd time (in milliseconds) to add to the clock's time */ - def advance(timeToAdd: Long) = - synchronized { - time += timeToAdd - notifyAll() - } + def advance(timeToAdd: Long): Unit = synchronized { + time += timeToAdd + notifyAll() + } /** * @param targetTime block until the clock time is set or advanced to at least this time * @return current time reported by the clock when waiting finishes */ - def waitTillTime(targetTime: Long): Long = - synchronized { - while (time < targetTime) { - wait(100) - } - getTimeMillis() + def waitTillTime(targetTime: Long): Long = synchronized { + while (time < targetTime) { + wait(100) } - + getTimeMillis() + } } diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index ac40f19ed6799..2bbfc988a99a8 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -67,15 +67,16 @@ private[spark] object MetadataCleanerType extends Enumeration { type MetadataCleanerType = Value - def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = - "spark.cleaner.ttl." + which.toString + def systemProperty(which: MetadataCleanerType.MetadataCleanerType): String = { + "spark.cleaner.ttl." + which.toString + } } // TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { - def getDelaySeconds(conf: SparkConf) = { - conf.getInt("spark.cleaner.ttl", -1) + def getDelaySeconds(conf: SparkConf): Int = { + conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index 74fa77b68de0b..dad888548ed10 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -43,7 +43,7 @@ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef this } - override def toString = "(" + _1 + "," + _2 + ")" + override def toString: String = "(" + _1 + "," + _2 + ")" override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]] } diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 6d8d9e8da3678..73d126ff6254e 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -22,7 +22,7 @@ package org.apache.spark.util */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { - override def findClass(name: String) = { + override def findClass(name: String): Class[_] = { super.findClass(name) } diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala new file mode 100644 index 0000000000000..f16cc8e7e42c6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -0,0 +1,60 @@ +/* + * 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 scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.spark.{SparkEnv, SparkConf} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} + +object RpcUtils { + + /** + * Retrieve a [[RpcEndpointRef]] which is located in the driver via its name. + */ + def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { + val driverActorSystemName = SparkEnv.driverActorSystemName + val driverHost: String = conf.get("spark.driver.host", "localhost") + val driverPort: Int = conf.getInt("spark.driver.port", 7077) + Utils.checkHost(driverHost, "Expected hostname") + rpcEnv.setupEndpointRef(driverActorSystemName, RpcAddress(driverHost, driverPort), name) + } + + /** Returns the configured number of times to retry connecting */ + def numRetries(conf: SparkConf): Int = { + conf.getInt("spark.rpc.numRetries", 3) + } + + /** Returns the configured number of milliseconds to wait on each retry */ + def retryWaitMs(conf: SparkConf): Long = { + conf.getTimeAsMs("spark.rpc.retry.wait", "3s") + } + + /** Returns the default Spark timeout to use for RPC ask operations. */ + def askTimeout(conf: SparkConf): FiniteDuration = { + conf.getTimeAsSeconds("spark.rpc.askTimeout", + conf.get("spark.network.timeout", "120s")) seconds + } + + /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ + def lookupTimeout(conf: SparkConf): FiniteDuration = { + conf.getTimeAsSeconds("spark.rpc.lookupTimeout", + conf.get("spark.network.timeout", "120s")) seconds + } +} diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index 770ff9d5ad6ae..a06b6f84ef11b 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -27,7 +27,7 @@ import java.nio.channels.Channels */ private[spark] class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable { - def value = buffer + def value: ByteBuffer = buffer private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 26ffbf9350388..d91c3294ddb8b 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -47,6 +47,11 @@ private[spark] object SizeEstimator extends Logging { private val FLOAT_SIZE = 4 private val DOUBLE_SIZE = 8 + // Fields can be primitive types, sizes are: 1, 2, 4, 8. Or fields can be pointers. The size of + // a pointer is 4 or 8 depending on the JVM (32-bit or 64-bit) and UseCompressedOops flag. + // The sizes should be in descending order, as we will use that information for fields placement. + private val fieldSizes = List(8, 4, 2, 1) + // Alignment boundary for objects // TODO: Is this arch dependent ? private val ALIGN_SIZE = 8 @@ -171,7 +176,7 @@ private[spark] object SizeEstimator extends Logging { // general all ClassLoaders and Classes will be shared between objects anyway. } else { val classInfo = getClassInfo(cls) - state.size += classInfo.shellSize + state.size += alignSize(classInfo.shellSize) for (field <- classInfo.pointerFields) { state.enqueue(field.get(obj)) } @@ -179,7 +184,7 @@ private[spark] object SizeEstimator extends Logging { } // Estimate the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. - private val ARRAY_SIZE_FOR_SAMPLING = 200 + private val ARRAY_SIZE_FOR_SAMPLING = 400 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING private def visitArray(array: AnyRef, arrayClass: Class[_], state: SearchState) { @@ -204,26 +209,41 @@ private[spark] object SizeEstimator extends Logging { } } else { // Estimate the size of a large array by sampling elements without replacement. - var size = 0.0 + // To exclude the shared objects that the array elements may link, sample twice + // and use the min one to caculate array size. val rand = new Random(42) - val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) - var numElementsDrawn = 0 - while (numElementsDrawn < ARRAY_SAMPLE_SIZE) { - var index = 0 - do { - index = rand.nextInt(length) - } while (drawn.contains(index)) - drawn.add(index) - val elem = ScalaRunTime.array_apply(array, index).asInstanceOf[AnyRef] - size += SizeEstimator.estimate(elem, state.visited) - numElementsDrawn += 1 - } - state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong + val drawn = new OpenHashSet[Int](2 * ARRAY_SAMPLE_SIZE) + val s1 = sampleArray(array, state, rand, drawn, length) + val s2 = sampleArray(array, state, rand, drawn, length) + val size = math.min(s1, s2) + state.size += math.max(s1, s2) + + (size * ((length - ARRAY_SAMPLE_SIZE) / (ARRAY_SAMPLE_SIZE))).toLong + } + } + } + + private def sampleArray( + array: AnyRef, + state: SearchState, + rand: Random, + drawn: OpenHashSet[Int], + length: Int): Long = { + var size = 0L + for (i <- 0 until ARRAY_SAMPLE_SIZE) { + var index = 0 + do { + index = rand.nextInt(length) + } while (drawn.contains(index)) + drawn.add(index) + val obj = ScalaRunTime.array_apply(array, index).asInstanceOf[AnyRef] + if (obj != null) { + size += SizeEstimator.estimate(obj, state.visited).toLong } } + size } - private def primitiveSize(cls: Class[_]): Long = { + private def primitiveSize(cls: Class[_]): Int = { if (cls == classOf[Byte]) { BYTE_SIZE } else if (cls == classOf[Boolean]) { @@ -259,21 +279,50 @@ private[spark] object SizeEstimator extends Logging { val parent = getClassInfo(cls.getSuperclass) var shellSize = parent.shellSize var pointerFields = parent.pointerFields + val sizeCount = Array.fill(fieldSizes.max + 1)(0) + // iterate through the fields of this class and gather information. for (field <- cls.getDeclaredFields) { if (!Modifier.isStatic(field.getModifiers)) { val fieldClass = field.getType if (fieldClass.isPrimitive) { - shellSize += primitiveSize(fieldClass) + sizeCount(primitiveSize(fieldClass)) += 1 } else { field.setAccessible(true) // Enable future get()'s on this field - shellSize += pointerSize + sizeCount(pointerSize) += 1 pointerFields = field :: pointerFields } } } - shellSize = alignSize(shellSize) + // Based on the simulated field layout code in Aleksey Shipilev's report: + // http://cr.openjdk.java.net/~shade/papers/2013-shipilev-fieldlayout-latest.pdf + // The code is in Figure 9. + // The simplified idea of field layout consists of 4 parts (see more details in the report): + // + // 1. field alignment: HotSpot lays out the fields aligned by their size. + // 2. object alignment: HotSpot rounds instance size up to 8 bytes + // 3. consistent fields layouts throughout the hierarchy: This means we should layout + // superclass first. And we can use superclass's shellSize as a starting point to layout the + // other fields in this class. + // 4. class alignment: HotSpot rounds field blocks up to to HeapOopSize not 4 bytes, confirmed + // with Aleksey. see https://bugs.openjdk.java.net/browse/CODETOOLS-7901322 + // + // The real world field layout is much more complicated. There are three kinds of fields + // order in Java 8. And we don't consider the @contended annotation introduced by Java 8. + // see the HotSpot classloader code, layout_fields method for more details. + // hg.openjdk.java.net/jdk8/jdk8/hotspot/file/tip/src/share/vm/classfile/classFileParser.cpp + var alignedSize = shellSize + for (size <- fieldSizes if sizeCount(size) > 0) { + val count = sizeCount(size) + // If there are internal gaps, smaller field can fit in. + alignedSize = math.max(alignedSize, alignSizeUp(shellSize, size) + size * count) + shellSize += size * count + } + + // Should choose a larger size to be new shellSize and clearly alignedSize >= shellSize, and + // round up the instance filed blocks + shellSize = alignSizeUp(alignedSize, pointerSize) // Create and cache a new ClassInfo val newInfo = new ClassInfo(shellSize, pointerFields) @@ -281,8 +330,15 @@ private[spark] object SizeEstimator extends Logging { newInfo } - private def alignSize(size: Long): Long = { - val rem = size % ALIGN_SIZE - if (rem == 0) size else (size + ALIGN_SIZE - rem) - } + private def alignSize(size: Long): Long = alignSizeUp(size, ALIGN_SIZE) + + /** + * Compute aligned size. The alignSize must be 2^n, otherwise the result will be wrong. + * When alignSize = 2^n, alignSize - 1 = 2^n - 1. The binary representation of (alignSize - 1) + * will only have n trailing 1s(0b00...001..1). ~(alignSize - 1) will be 0b11..110..0. Hence, + * (size + alignSize - 1) & ~(alignSize - 1) will set the last n bits to zeros, which leads to + * multiple of alignSize. + */ + private def alignSizeUp(size: Long, alignSize: Int): Long = + (size + alignSize - 1) & ~(alignSize - 1) } diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index d80eed455c427..8586da1996cf3 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -141,8 +141,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { object StatCounter { /** Build a StatCounter from a list of values. */ - def apply(values: TraversableOnce[Double]) = new StatCounter(values) + def apply(values: TraversableOnce[Double]): StatCounter = new StatCounter(values) /** Build a StatCounter from a list of values passed as variable-length arguments. */ - def apply(values: Double*) = new StatCounter(values) + def apply(values: Double*): StatCounter = new StatCounter(values) } diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala new file mode 100644 index 0000000000000..098a4b79496b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.util + +import java.util.concurrent._ + +import com.google.common.util.concurrent.ThreadFactoryBuilder + +private[spark] object ThreadUtils { + + /** + * Create a thread factory that names threads with a prefix and also sets the threads to daemon. + */ + def namedThreadFactory(prefix: String): ThreadFactory = { + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(prefix + "-%d").build() + } + + /** + * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Wrapper over newSingleThreadExecutor. + */ + def newDaemonSingleThreadExecutor(threadName: String): ExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + Executors.newSingleThreadExecutor(threadFactory) + } + + /** + * Wrapper over newSingleThreadScheduledExecutor. + */ + def newDaemonSingleThreadScheduledExecutor(threadName: String): ScheduledExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + Executors.newSingleThreadScheduledExecutor(threadFactory) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala index f5be5856c2109..310c0c109416c 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -82,7 +82,7 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo this } - override def update(key: A, value: B) = this += ((key, value)) + override def update(key: A, value: B): Unit = this += ((key, value)) override def apply(key: A): B = internalMap.apply(key) @@ -92,14 +92,14 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo override def size: Int = internalMap.size - override def foreach[U](f: ((A, B)) => U) = nonNullReferenceMap.foreach(f) + override def foreach[U](f: ((A, B)) => U): Unit = nonNullReferenceMap.foreach(f) def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) def toMap: Map[A, B] = iterator.toMap /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ - def clearOldValues(threshTime: Long) = internalMap.clearOldValues(threshTime) + def clearOldValues(threshTime: Long): Unit = internalMap.clearOldValues(threshTime) /** Remove entries with values that are no longer strongly reachable. */ def clearNullValues() { 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 fa56bb09e4e5c..be4db02ab86d0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,8 +21,8 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.{PriorityQueue, Properties, Locale, Random, UUID} +import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ @@ -30,11 +30,11 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Failure, Success, Try} import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.{ByteStreams, Files} -import com.google.common.util.concurrent.ThreadFactoryBuilder +import com.google.common.net.InetAddresses import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -42,10 +42,13 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ + +import tachyon.TachyonURI import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -62,9 +65,21 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + val DEFAULT_SHUTDOWN_PRIORITY = 100 + + /** + * The shutdown priority of the SparkContext instance. This is lower than the default + * priority, so that by default hooks are run before the context is shut down. + */ + val SPARK_CONTEXT_SHUTDOWN_PRIORITY = 50 + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null + + private val shutdownHooks = new SparkShutdownHookManager() + shutdownHooks.install() + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -85,7 +100,7 @@ private[spark] object Utils extends Logging { def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { val bis = new ByteArrayInputStream(bytes) val ois = new ObjectInputStream(bis) { - override def resolveClass(desc: ObjectStreamClass) = + override def resolveClass(desc: ObjectStreamClass): Class[_] = Class.forName(desc.getName, false, loader) } ois.readObject.asInstanceOf[T] @@ -106,11 +121,10 @@ private[spark] object Utils extends Logging { /** Serialize via nested stream using specific serializer */ def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)( - f: SerializationStream => Unit) = { + f: SerializationStream => Unit): Unit = { val osWrapper = ser.serializeStream(new OutputStream { - def write(b: Int) = os.write(b) - - override def write(b: Array[Byte], off: Int, len: Int) = os.write(b, off, len) + override def write(b: Int): Unit = os.write(b) + override def write(b: Array[Byte], off: Int, len: Int): Unit = os.write(b, off, len) }) try { f(osWrapper) @@ -121,10 +135,9 @@ private[spark] object Utils extends Logging { /** Deserialize via nested stream using specific serializer */ def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)( - f: DeserializationStream => Unit) = { + f: DeserializationStream => Unit): Unit = { val isWrapper = ser.deserializeStream(new InputStream { - def read(): Int = is.read() - + override def read(): Int = is.read() override def read(b: Array[Byte], off: Int, len: Int): Int = is.read(b, off, len) }) try { @@ -137,7 +150,7 @@ private[spark] object Utils extends Logging { /** * Get the ClassLoader which loaded Spark. */ - def getSparkClassLoader = getClass.getClassLoader + def getSparkClassLoader: ClassLoader = getClass.getClassLoader /** * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that @@ -146,7 +159,7 @@ private[spark] object Utils extends Logging { * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently * active loader when setting up ClassLoader delegation chains. */ - def getContextOrSparkClassLoader = + def getContextOrSparkClassLoader: ClassLoader = Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) /** Determines whether the provided class is loadable in the current thread. */ @@ -155,12 +168,14 @@ private[spark] object Utils extends Logging { } /** Preferred alternative to Class.forName(className) */ - def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader) + def classForName(className: String): Class[_] = { + Class.forName(className, true, getContextOrSparkClassLoader) + } /** * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ - def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { + def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput): Unit = { if (bb.hasArray) { out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) } else { @@ -174,18 +189,16 @@ private[spark] object Utils extends Logging { private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Add a shutdown hook to delete the temp dirs when the JVM exits - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dirs") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - shutdownDeletePaths.foreach { dirPath => - try { - Utils.deleteRecursively(new File(dirPath)) - } catch { - case e: Exception => logError(s"Exception while deleting Spark temp dir: $dirPath", e) - } + addShutdownHook { () => + logDebug("Shutdown hook called") + shutdownDeletePaths.foreach { dirPath => + try { + Utils.deleteRecursively(new File(dirPath)) + } catch { + case e: Exception => logError(s"Exception while deleting Spark temp dir: $dirPath", e) } } - }) + } // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { @@ -313,7 +326,7 @@ private[spark] object Utils extends Logging { transferToEnabled: Boolean = false): Long = { var count = 0L - try { + tryWithSafeFinally { if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. @@ -353,7 +366,7 @@ private[spark] object Utils extends Logging { } } count - } finally { + } { if (closeStreams) { try { in.close() @@ -611,9 +624,10 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 - uc.setConnectTimeout(timeout) - uc.setReadTimeout(timeout) + val timeoutMs = + conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 + uc.setConnectTimeout(timeoutMs) + uc.setReadTimeout(timeoutMs) uc.connect() val in = uc.getInputStream() downloadFile(url, in, targetFile, fileOverwrite) @@ -789,13 +803,12 @@ private[spark] object Utils extends Logging { * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). * Note, this is typically not used from within core spark. */ - lazy val localIpAddress: String = findLocalIpAddress() - lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) + private lazy val localIpAddress: InetAddress = findLocalInetAddress() - private def findLocalIpAddress(): String = { + private def findLocalInetAddress(): InetAddress = { val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") if (defaultIpOverride != null) { - defaultIpOverride + InetAddress.getByName(defaultIpOverride) } else { val address = InetAddress.getLocalHost if (address.isLoopbackAddress) { @@ -806,15 +819,20 @@ private[spark] object Utils extends Logging { // It's more proper to pick ip address following system output order. val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.toList val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse + for (ni <- reOrderedNetworkIFs) { - for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && - !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { + val addresses = ni.getInetAddresses.toList + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress) + if (addresses.nonEmpty) { + val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) + // because of Inet6Address.toHostName may add interface at the end if it knows about it + val strippedAddress = InetAddress.getByAddress(addr.getAddress) // We've found an address that looks reasonable! logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + - " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + - " instead (on interface " + ni.getName + ")") + " a loopback address: " + address.getHostAddress + "; using " + + strippedAddress.getHostAddress + " instead (on interface " + ni.getName + ")") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") - return addr.getHostAddress + return strippedAddress } } logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + @@ -822,7 +840,7 @@ private[spark] object Utils extends Logging { " external IP address!") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") } - address.getHostAddress + address } } @@ -842,11 +860,14 @@ private[spark] object Utils extends Logging { * Get the local machine's hostname. */ def localHostName(): String = { - customHostname.getOrElse(localIpAddressHostname) + customHostname.getOrElse(localIpAddress.getHostAddress) } - def getAddressHostName(address: String): String = { - InetAddress.getByName(address).getHostName + /** + * Get the local machine's URI. + */ + def localHostNameForURI(): String = { + customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) } def checkHost(host: String, message: String = "") { @@ -883,34 +904,6 @@ private[spark] object Utils extends Logging { hostPortParseResults.get(hostPort) } - private val daemonThreadFactoryBuilder: ThreadFactoryBuilder = - new ThreadFactoryBuilder().setDaemon(true) - - /** - * Create a thread factory that names threads with a prefix and also sets the threads to daemon. - */ - def namedThreadFactory(prefix: String): ThreadFactory = { - daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() - } - - /** - * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. - */ - def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { - val threadFactory = namedThreadFactory(prefix) - Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] - } - - /** - * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. - */ - def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { - val threadFactory = namedThreadFactory(prefix) - Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] - } - /** * Return the string to tell how long has passed in milliseconds. */ @@ -970,7 +963,7 @@ private[spark] object Utils extends Logging { * Delete a file or directory and its contents recursively. */ def deleteRecursively(dir: TachyonFile, client: TachyonFS) { - if (!client.delete(dir.getPath(), true)) { + if (!client.delete(new TachyonURI(dir.getPath()), true)) { throw new IOException("Failed to delete the tachyon dir: " + dir) } } @@ -1011,21 +1004,64 @@ private[spark] object Utils extends Logging { } /** - * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + def timeStringAsMs(str: String): Long = { + JavaUtils.timeStringAsMs(str) + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + def timeStringAsSeconds(str: String): Long = { + JavaUtils.timeStringAsSec(str) + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for internal use. + * + * If no suffix is provided, the passed number is assumed to be in bytes. + */ + def byteStringAsBytes(str: String): Long = { + JavaUtils.byteStringAsBytes(str) + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for internal use. + * + * If no suffix is provided, the passed number is assumed to be in kibibytes. + */ + def byteStringAsKb(str: String): Long = { + JavaUtils.byteStringAsKb(str) + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for internal use. + * + * If no suffix is provided, the passed number is assumed to be in mebibytes. + */ + def byteStringAsMb(str: String): Long = { + JavaUtils.byteStringAsMb(str) + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m, 500g) to gibibytes for internal use. + * + * If no suffix is provided, the passed number is assumed to be in gibibytes. + */ + def byteStringAsGb(str: String): Long = { + JavaUtils.byteStringAsGb(str) + } + + /** + * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes. */ def memoryStringToMb(str: String): Int = { - val lower = str.toLowerCase - if (lower.endsWith("k")) { - (lower.substring(0, lower.length-1).toLong / 1024).toInt - } else if (lower.endsWith("m")) { - lower.substring(0, lower.length-1).toInt - } else if (lower.endsWith("g")) { - lower.substring(0, lower.length-1).toInt * 1024 - } else if (lower.endsWith("t")) { - lower.substring(0, lower.length-1).toInt * 1024 * 1024 - } else {// no suffix, so it's just a number in bytes - (lower.toLong / 1024 / 1024).toInt - } + // Convert to bytes, rather than directly to MB, because when no units are specified the unit + // is assumed to be bytes + (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt } /** @@ -1146,7 +1182,7 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler - * + * * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { @@ -1159,11 +1195,11 @@ private[spark] object Utils extends Logging { } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught * exception - * - * NOTE: This method is to be called by the driver-side components to avoid stopping the - * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the * spark-started JVM process . */ def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { @@ -1214,17 +1250,67 @@ private[spark] object Utils extends Logging { } } + /** Executes the given block. Log non-fatal errors if any, and only throw fatal errors */ + def tryLogNonFatalError(block: => Unit) { + try { + block + } catch { + case NonFatal(t) => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + } + } + + /** + * Execute a block of code, then a finally block, but if exceptions happen in + * the finally block, do not suppress the original exception. + * + * This is primarily an issue with `finally { out.close() }` blocks, where + * close needs to be called to clean up `out`, but if an exception happened + * in `out.write`, it's likely `out` may be corrupted and `out.close` will + * fail as well. This would then suppress the original/likely more meaningful + * exception from the original `out.write` call. + */ + def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = { + // It would be nice to find a method on Try that did this + var originalThrowable: Throwable = null + try { + block + } catch { + case t: Throwable => + // Purposefully not using NonFatal, because even fatal exceptions + // we don't want to have our finallyBlock suppress + originalThrowable = t + throw originalThrowable + } finally { + try { + finallyBlock + } catch { + case t: Throwable => + if (originalThrowable != null) { + // We could do originalThrowable.addSuppressed(t), but it's + // not available in JDK 1.6. + logWarning(s"Suppressing exception in finally: " + t.getMessage, t) + throw originalThrowable + } else { + throw t + } + } + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ - private def coreExclusionFunction(className: String): Boolean = { - // A regular expression to match classes of the "core" Spark API that we want to skip when - // finding the call site of a method. + private def sparkInternalExclusionFunction(className: String): Boolean = { + // A regular expression to match classes of the internal Spark API's + // that we want to skip when finding the call site of a method. val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r + val SPARK_SQL_CLASS_REGEX = """^org\.apache\.spark\.sql.*""".r val SCALA_CORE_CLASS_PREFIX = "scala" - val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined + val isSparkClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined || + SPARK_SQL_CLASS_REGEX.findFirstIn(className).isDefined val isScalaClass = className.startsWith(SCALA_CORE_CLASS_PREFIX) // If the class is a Spark internal class or a Scala class, then exclude. - isSparkCoreClass || isScalaClass + isSparkClass || isScalaClass } /** @@ -1234,7 +1320,7 @@ private[spark] object Utils extends Logging { * * @param skipClass Function that is used to exclude non-user-code classes. */ - def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { + def getCallSite(skipClass: String => Boolean = sparkInternalExclusionFunction): CallSite = { // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD // transformation, a SparkContext function (such as parallelize), or anything else that leads @@ -1273,9 +1359,17 @@ private[spark] object Utils extends Logging { } val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt - CallSite( - shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", - longForm = callStack.take(callStackDepth).mkString("\n")) + val shortForm = + if (firstUserFile == "HiveSessionImpl.java") { + // To be more user friendly, show a nicer string for queries submitted from the JDBC + // server. + "Spark JDBC Server Query" + } else { + s"$lastSparkMethod at $firstUserFile:$firstUserLine" + } + val longForm = callStack.take(callStackDepth).mkString("\n") + + CallSite(shortForm, longForm) } /** Return a string containing part of a file from byte 'start' to 'end'. */ @@ -1557,7 +1651,7 @@ private[spark] object Utils extends Logging { /** Return the class name of the given object, removing all dollar signs */ - def getFormattedClassName(obj: AnyRef) = { + def getFormattedClassName(obj: AnyRef): String = { obj.getClass.getSimpleName.replace("$", "") } @@ -1570,7 +1664,7 @@ private[spark] object Utils extends Logging { } /** Return an empty JSON object */ - def emptyJson = JObject(List[JField]()) + def emptyJson: JsonAST.JObject = JObject(List[JField]()) /** * Return a Hadoop FileSystem with the scheme encoded in the given path. @@ -1618,7 +1712,7 @@ private[spark] object Utils extends Logging { /** * Indicates whether Spark is currently running unit tests. */ - def isTesting = { + def isTesting: Boolean = { sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") } @@ -1876,6 +1970,10 @@ private[spark] object Utils extends Logging { startService: Int => (T, Int), conf: SparkConf, serviceName: String = ""): (T, Int) = { + + require(startPort == 0 || (1024 <= startPort && startPort < 65536), + "startPort should be between 1024 and 65535 (inclusive), or 0 for a random free port.") + val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" val maxRetries = portMaxRetries(conf) for (offset <- 0 to maxRetries) { @@ -1924,6 +2022,13 @@ private[spark] object Utils extends Logging { } } + /** + * configure a new log4j level + */ + def setLogLevel(l: org.apache.log4j.Level) { + org.apache.log4j.Logger.getRootLogger().setLevel(l) + } + /** * config a log4j properties used for testsuite */ @@ -2051,11 +2156,122 @@ private[spark] object Utils extends Logging { */ def getCurrentUserName(): String = { Option(System.getenv("SPARK_USER")) - .getOrElse(UserGroupInformation.getCurrentUser().getUserName()) + .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) + } + + /** + * Split the comma delimited string of master URLs into a list. + * For instance, "spark://abc,def" becomes [spark://abc, spark://def]. + */ + def parseStandaloneMasterUrls(masterUrls: String): Array[String] = { + masterUrls.stripPrefix("spark://").split(",").map("spark://" + _) + } + + /** An identifier that backup masters use in their responses. */ + val BACKUP_STANDALONE_MASTER_PREFIX = "Current state is not alive" + + /** Return true if the response message is sent from a backup Master on standby. */ + def responseFromBackup(msg: String): Boolean = { + msg.startsWith(BACKUP_STANDALONE_MASTER_PREFIX) + } + + /** + * Adds a shutdown hook with default priority. + * + * @param hook The code to run during shutdown. + * @return A handle that can be used to unregister the shutdown hook. + */ + def addShutdownHook(hook: () => Unit): AnyRef = { + addShutdownHook(DEFAULT_SHUTDOWN_PRIORITY)(hook) + } + + /** + * Adds a shutdown hook with the given priority. Hooks with lower priority values run + * first. + * + * @param hook The code to run during shutdown. + * @return A handle that can be used to unregister the shutdown hook. + */ + def addShutdownHook(priority: Int)(hook: () => Unit): AnyRef = { + shutdownHooks.add(priority, hook) + } + + /** + * Remove a previously installed shutdown hook. + * + * @param ref A handle returned by `addShutdownHook`. + * @return Whether the hook was removed. + */ + def removeShutdownHook(ref: AnyRef): Boolean = { + shutdownHooks.remove(ref) + } + +} + +private [util] class SparkShutdownHookManager { + + private val hooks = new PriorityQueue[SparkShutdownHook]() + 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. + */ + def install(): Unit = { + val hookTask = new Runnable() { + override def run(): Unit = runAll() + } + Try(Class.forName("org.apache.hadoop.util.ShutdownHookManager")) match { + case Success(shmClass) => + val fsPriority = classOf[FileSystem].getField("SHUTDOWN_HOOK_PRIORITY").get() + .asInstanceOf[Int] + val shm = shmClass.getMethod("get").invoke(null) + shm.getClass().getMethod("addShutdownHook", classOf[Runnable], classOf[Int]) + .invoke(shm, hookTask, Integer.valueOf(fsPriority + 30)) + + case Failure(_) => + Runtime.getRuntime.addShutdownHook(new Thread(hookTask, "Spark Shutdown Hook")); + } + } + + def runAll(): Unit = synchronized { + shuttingDown = true + while (!hooks.isEmpty()) { + Try(Utils.logUncaughtExceptions(hooks.poll().run())) + } + } + + def add(priority: Int, hook: () => Unit): AnyRef = synchronized { + checkState() + val hookRef = new SparkShutdownHook(priority, hook) + hooks.add(hookRef) + hookRef + } + + def remove(ref: AnyRef): Boolean = synchronized { + hooks.remove(ref) + } + + private def checkState(): Unit = { + if (shuttingDown) { + throw new IllegalStateException("Shutdown hooks cannot be modified during shutdown.") + } } } +private class SparkShutdownHook(private val priority: Int, hook: () => Unit) + extends Comparable[SparkShutdownHook] { + + override def compareTo(other: SparkShutdownHook): Int = { + other.priority - priority + } + + def run(): Unit = hook() + +} + /** * A utility class to redirect the child process's stdout or stderr. */ @@ -2070,7 +2286,7 @@ private[spark] class RedirectThread( override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - try { + Utils.tryWithSafeFinally { val buf = new Array[Byte](1024) var len = in.read(buf) while (len != -1) { @@ -2078,7 +2294,7 @@ private[spark] class RedirectThread( out.flush() len = in.read(buf) } - } finally { + } { if (propagateEof) { out.close() } diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index af1f64649f354..41cb8cfe2afa3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -39,7 +39,7 @@ class BitSet(numBits: Int) extends Serializable { val wordIndex = bitIndex >> 6 // divide by 64 var i = 0 while(i < wordIndex) { words(i) = -1; i += 1 } - if(wordIndex < words.size) { + if(wordIndex < words.length) { // Set the remaining bits (note that the mask could still be zero) val mask = ~(-1L << (bitIndex & 0x3f)) words(wordIndex) |= mask @@ -156,10 +156,10 @@ class BitSet(numBits: Int) extends Serializable { /** * Get an iterator over the set bits. */ - def iterator = new Iterator[Int] { + def iterator: Iterator[Int] = new Iterator[Int] { var ind = nextSetBit(0) override def hasNext: Boolean = ind >= 0 - override def next() = { + override def next(): Int = { val tmp = ind ind = nextSetBit(ind + 1) tmp diff --git a/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala new file mode 100644 index 0000000000000..a60bffe611f14 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala @@ -0,0 +1,144 @@ +/* + * 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.collection + +import java.io.OutputStream + +import scala.collection.mutable.ArrayBuffer + +/** + * A logical byte buffer that wraps a list of byte arrays. All the byte arrays have equal size. The + * advantage of this over a standard ArrayBuffer is that it can grow without claiming large amounts + * of memory and needing to copy the full contents. The disadvantage is that the contents don't + * occupy a contiguous segment of memory. + */ +private[spark] class ChainedBuffer(chunkSize: Int) { + private val chunkSizeLog2 = (math.log(chunkSize) / math.log(2)).toInt + assert(math.pow(2, chunkSizeLog2).toInt == chunkSize, + s"ChainedBuffer chunk size $chunkSize must be a power of two") + private val chunks: ArrayBuffer[Array[Byte]] = new ArrayBuffer[Array[Byte]]() + private var _size: Int = _ + + /** + * Feed bytes from this buffer into a BlockObjectWriter. + * + * @param pos Offset in the buffer to read from. + * @param os OutputStream to read into. + * @param len Number of bytes to read. + */ + def read(pos: Int, os: OutputStream, len: Int): Unit = { + if (pos + len > _size) { + throw new IndexOutOfBoundsException( + s"Read of $len bytes at position $pos would go past size ${_size} of buffer") + } + var chunkIndex = pos >> chunkSizeLog2 + var posInChunk = pos - (chunkIndex << chunkSizeLog2) + var written = 0 + while (written < len) { + val toRead = math.min(len - written, chunkSize - posInChunk) + os.write(chunks(chunkIndex), posInChunk, toRead) + written += toRead + chunkIndex += 1 + posInChunk = 0 + } + } + + /** + * Read bytes from this buffer into a byte array. + * + * @param pos Offset in the buffer to read from. + * @param bytes Byte array to read into. + * @param offs Offset in the byte array to read to. + * @param len Number of bytes to read. + */ + def read(pos: Int, bytes: Array[Byte], offs: Int, len: Int): Unit = { + if (pos + len > _size) { + throw new IndexOutOfBoundsException( + s"Read of $len bytes at position $pos would go past size of buffer") + } + var chunkIndex = pos >> chunkSizeLog2 + var posInChunk = pos - (chunkIndex << chunkSizeLog2) + var written = 0 + while (written < len) { + val toRead = math.min(len - written, chunkSize - posInChunk) + System.arraycopy(chunks(chunkIndex), posInChunk, bytes, offs + written, toRead) + written += toRead + chunkIndex += 1 + posInChunk = 0 + } + } + + /** + * Write bytes from a byte array into this buffer. + * + * @param pos Offset in the buffer to write to. + * @param bytes Byte array to write from. + * @param offs Offset in the byte array to write from. + * @param len Number of bytes to write. + */ + def write(pos: Int, bytes: Array[Byte], offs: Int, len: Int): Unit = { + if (pos > _size) { + throw new IndexOutOfBoundsException( + s"Write at position $pos starts after end of buffer ${_size}") + } + // Grow if needed + val endChunkIndex = (pos + len - 1) >> chunkSizeLog2 + while (endChunkIndex >= chunks.length) { + chunks += new Array[Byte](chunkSize) + } + + var chunkIndex = pos >> chunkSizeLog2 + var posInChunk = pos - (chunkIndex << chunkSizeLog2) + var written = 0 + while (written < len) { + val toWrite = math.min(len - written, chunkSize - posInChunk) + System.arraycopy(bytes, offs + written, chunks(chunkIndex), posInChunk, toWrite) + written += toWrite + chunkIndex += 1 + posInChunk = 0 + } + + _size = math.max(_size, pos + len) + } + + /** + * Total size of buffer that can be written to without allocating additional memory. + */ + def capacity: Int = chunks.size * chunkSize + + /** + * Size of the logical buffer. + */ + def size: Int = _size +} + +/** + * Output stream that writes to a ChainedBuffer. + */ +private[spark] class ChainedBufferOutputStream(chainedBuffer: ChainedBuffer) extends OutputStream { + private var pos = 0 + + override def write(b: Int): Unit = { + throw new UnsupportedOperationException() + } + + override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { + chainedBuffer.write(pos, bytes, offs, len) + pos += len + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 8a0f5a602de12..b850973145077 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -89,8 +89,10 @@ class ExternalAppendOnlyMap[K, V, C]( // Number of bytes spilled in total private var _diskBytesSpilled = 0L - - private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided + private val fileBufferSize = + sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 // Write metrics for current spill private var curWriteMetrics: ShuffleWriteMetrics = _ @@ -151,15 +153,14 @@ class ExternalAppendOnlyMap[K, V, C]( override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempLocalBlock() curWriteMetrics = new ShuffleWriteMetrics() - var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, - curWriteMetrics) + var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // List of batch sizes (bytes) in the order they are written to disk val batchSizes = new ArrayBuffer[Long] // Flush the disk writer's contents to disk, and update relevant variables - def flush() = { + def flush(): Unit = { val w = writer writer = null w.commitAndClose() @@ -173,14 +174,13 @@ class ExternalAppendOnlyMap[K, V, C]( val it = currentMap.destructiveSortedIterator(keyComparator) while (it.hasNext) { val kv = it.next() - writer.write(kv) + writer.write(kv._1, kv._2) objectsWritten += 1 if (objectsWritten == serializerBatchSize) { flush() curWriteMetrics = new ShuffleWriteMetrics() - writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, - curWriteMetrics) + writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { @@ -355,7 +355,7 @@ class ExternalAppendOnlyMap[K, V, C]( val pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { - def isEmpty = pairs.length == 0 + def isEmpty: Boolean = pairs.length == 0 // Invalid if there are no more pairs in this stream def minKeyHash: Int = { @@ -435,7 +435,9 @@ class ExternalAppendOnlyMap[K, V, C]( */ private def readNextItem(): (K, C) = { try { - val item = deserializeStream.readObject().asInstanceOf[(K, C)] + val k = deserializeStream.readKey().asInstanceOf[K] + val c = deserializeStream.readValue().asInstanceOf[C] + val item = (k, c) objectsRead += 1 if (objectsRead == serializerBatchSize) { objectsRead = 0 diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index d69f2d9048055..b7306cd551918 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import com.google.common.io.ByteStreams import org.apache.spark._ -import org.apache.spark.serializer.{DeserializationStream, Serializer} +import org.apache.spark.serializer._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage.{BlockObjectWriter, BlockId} @@ -53,23 +53,35 @@ import org.apache.spark.storage.{BlockObjectWriter, BlockId} * probably want to pass None as the ordering to avoid extra sorting. On the other hand, if you do * want to do combining, having an Ordering is more efficient than not having it. * - * At a high level, this class works as follows: + * Users interact with this class in the following way: * - * - We repeatedly fill up buffers of in-memory data, using either a SizeTrackingAppendOnlyMap if - * we want to combine by key, or an simple SizeTrackingBuffer if we don't. Inside these buffers, - * we sort elements of type ((Int, K), C) where the Int is the partition ID. This is done to - * avoid calling the partitioner multiple times on the same key (e.g. for RangePartitioner). + * 1. Instantiate an ExternalSorter. + * + * 2. Call insertAll() with a set of records. + * + * 3. Request an iterator() back to traverse sorted/aggregated records. + * - or - + * Invoke writePartitionedFile() to create a file containing sorted/aggregated outputs + * that can be used in Spark's sort shuffle. + * + * At a high level, this class works internally as follows: + * + * - We repeatedly fill up buffers of in-memory data, using either a PartitionedAppendOnlyMap if + * we want to combine by key, or a PartitionedSerializedPairBuffer or PartitionedPairBuffer if we + * don't. Inside these buffers, we sort elements by partition ID and then possibly also by key. + * To avoid calling the partitioner multiple times with each key, we store the partition ID + * alongside each record. * * - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first * by partition ID and possibly second by key or by hash code of the key, if we want to do * aggregation. For each file, we track how many objects were in each partition in memory, so we * don't have to write out the partition ID for every element. * - * - When the user requests an iterator, the spilled files are merged, along with any remaining - * in-memory data, using the same sort order defined above (unless both sorting and aggregation - * are disabled). If we need to aggregate by key, we either use a total ordering from the - * ordering parameter, or read the keys with the same hash code and compare them with each other - * for equality to merge values. + * - When the user requests an iterator or file output, the spilled files are merged, along with + * any remaining in-memory data, using the same sort order defined above (unless both sorting + * and aggregation are disabled). If we need to aggregate by key, we either use a total ordering + * from the ordering parameter, or read the keys with the same hash code and compare them with + * each other for equality to merge values. * * - Users are expected to call stop() at the end to delete all the intermediate files. * @@ -85,7 +97,7 @@ private[spark] class ExternalSorter[K, V, C]( partitioner: Option[Partitioner] = None, ordering: Option[Ordering[K]] = None, serializer: Option[Serializer] = None) - extends Logging with Spillable[SizeTrackingPairCollection[(Int, K), C]] { + extends Logging with Spillable[WritablePartitionedPairCollection[K, C]] { private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) private val shouldPartition = numPartitions > 1 @@ -97,7 +109,9 @@ private[spark] class ExternalSorter[K, V, C]( private val conf = SparkEnv.get.conf private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true) - private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + // Use getSizeAsKb (not bytes) to maintain backwards compatibility of on units are provided + private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true) // Size of object batches when reading/writing from serializers. @@ -113,11 +127,22 @@ private[spark] class ExternalSorter[K, V, C]( if (shouldPartition) partitioner.get.getPartition(key) else 0 } + private val metaInitialRecords = 256 + private val kvChunkSize = conf.getInt("spark.shuffle.sort.kvChunkSize", 1 << 22) // 4 MB + private val useSerializedPairBuffer = + !ordering.isDefined && conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) && + ser.isInstanceOf[KryoSerializer] && + serInstance.asInstanceOf[KryoSerializerInstance].getAutoReset + // Data structures to store in-memory objects before we spill. Depending on whether we have an // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we // store them in an array buffer. - private var map = new SizeTrackingAppendOnlyMap[(Int, K), C] - private var buffer = new SizeTrackingPairBuffer[(Int, K), C] + private var map = new PartitionedAppendOnlyMap[K, C] + private var buffer = if (useSerializedPairBuffer) { + new PartitionedSerializedPairBuffer[K, C](metaInitialRecords, kvChunkSize, serInstance) + } else { + new PartitionedPairBuffer[K, C] + } // Total spilling statistics private var _diskBytesSpilled = 0L @@ -150,33 +175,6 @@ private[spark] class ExternalSorter[K, V, C]( } }) - // A comparator for (Int, K) pairs that orders them by only their partition ID - private val partitionComparator: Comparator[(Int, K)] = new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - a._1 - b._1 - } - } - - // A comparator that orders (Int, K) pairs by partition ID and then possibly by key - private val partitionKeyComparator: Comparator[(Int, K)] = { - if (ordering.isDefined || aggregator.isDefined) { - // Sort by partition ID then key comparator - new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - val partitionDiff = a._1 - b._1 - if (partitionDiff != 0) { - partitionDiff - } else { - keyComparator.compare(a._2, b._2) - } - } - } - } else { - // Just sort it by partition ID - partitionComparator - } - } - // Information about a spilled file. Includes sizes in bytes of "batches" written by the // serializer as we periodically reset its stream, as well as number of elements in each // partition, used to efficiently keep track of partitions when merging. @@ -208,16 +206,18 @@ private[spark] class ExternalSorter[K, V, C]( } else if (bypassMergeSort) { // SPARK-4479: Also bypass buffering if merge sort is bypassed to avoid defensive copies if (records.hasNext) { - spillToPartitionFiles(records.map { kv => - ((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) - }) + spillToPartitionFiles( + WritablePartitionedIterator.fromIterator(records.map { kv => + ((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) + }) + ) } } else { // Stick values into our buffer while (records.hasNext) { addElementsRead() val kv = records.next() - buffer.insert((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) + buffer.insert(getPartition(kv._1), kv._1, kv._2.asInstanceOf[C]) maybeSpillCollection(usingMap = false) } } @@ -235,11 +235,15 @@ private[spark] class ExternalSorter[K, V, C]( if (usingMap) { if (maybeSpill(map, map.estimateSize())) { - map = new SizeTrackingAppendOnlyMap[(Int, K), C] + map = new PartitionedAppendOnlyMap[K, C] } } else { if (maybeSpill(buffer, buffer.estimateSize())) { - buffer = new SizeTrackingPairBuffer[(Int, K), C] + buffer = if (useSerializedPairBuffer) { + new PartitionedSerializedPairBuffer[K, C](metaInitialRecords, kvChunkSize, serInstance) + } else { + new PartitionedPairBuffer[K, C] + } } } } @@ -247,7 +251,7 @@ private[spark] class ExternalSorter[K, V, C]( /** * Spill the current in-memory collection to disk, adding a new file to spills, and clear it. */ - override protected[this] def spill(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = { if (bypassMergeSort) { spillToPartitionFiles(collection) } else { @@ -259,12 +263,12 @@ private[spark] class ExternalSorter[K, V, C]( * Spill our in-memory collection to a sorted file that we can merge later (normal code path). * We add this file into spilledFiles to find it later. * - * Alternatively, if bypassMergeSort is true, we spill to separate files for each partition. - * See spillToPartitionedFiles() for that code path. + * This should not be invoked if bypassMergeSort is true. In that case, spillToPartitionedFiles() + * is used to write files for each partition. * * @param collection whichever collection we're using (map or buffer) */ - private def spillToMergeableFile(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + private def spillToMergeableFile(collection: WritablePartitionedPairCollection[K, C]): Unit = { assert(!bypassMergeSort) // Because these files may be read during shuffle, their compression must be controlled by @@ -272,7 +276,8 @@ private[spark] class ExternalSorter[K, V, C]( // createTempShuffleBlock here; see SPARK-3426 for more context. val (blockId, file) = diskBlockManager.createTempShuffleBlock() curWriteMetrics = new ShuffleWriteMetrics() - var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) + var writer = blockManager.getDiskWriter( + blockId, file, serInstance, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // Objects written since the last flush // List of batch sizes (bytes) in the order they are written to disk @@ -283,7 +288,7 @@ private[spark] class ExternalSorter[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables. // The writer is closed at the end of this process, and cannot be reused. - def flush() = { + def flush(): Unit = { val w = writer writer = null w.commitAndClose() @@ -294,21 +299,18 @@ private[spark] class ExternalSorter[K, V, C]( var success = false try { - val it = collection.destructiveSortedIterator(partitionKeyComparator) + val it = collection.destructiveSortedWritablePartitionedIterator(comparator) while (it.hasNext) { - val elem = it.next() - val partitionId = elem._1._1 - val key = elem._1._2 - val value = elem._2 - writer.write(key) - writer.write(value) + val partitionId = it.nextPartition() + it.writeNext(writer) elementsPerPartition(partitionId) += 1 objectsWritten += 1 if (objectsWritten == serializerBatchSize) { flush() curWriteMetrics = new ShuffleWriteMetrics() - writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) + writer = blockManager.getDiskWriter( + blockId, file, serInstance, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { @@ -342,32 +344,36 @@ private[spark] class ExternalSorter[K, V, C]( * * @param collection whichever collection we're using (map or buffer) */ - private def spillToPartitionFiles(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { - spillToPartitionFiles(collection.iterator) + private def spillToPartitionFiles(collection: WritablePartitionedPairCollection[K, C]): Unit = { + spillToPartitionFiles(collection.writablePartitionedIterator()) } - private def spillToPartitionFiles(iterator: Iterator[((Int, K), C)]): Unit = { + private def spillToPartitionFiles(iterator: WritablePartitionedIterator): Unit = { assert(bypassMergeSort) // Create our file writers if we haven't done so yet if (partitionWriters == null) { curWriteMetrics = new ShuffleWriteMetrics() + val openStartTime = System.nanoTime partitionWriters = Array.fill(numPartitions) { // Because these files may be read during shuffle, their compression must be controlled by // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use // createTempShuffleBlock here; see SPARK-3426 for more context. val (blockId, file) = diskBlockManager.createTempShuffleBlock() - blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics).open() + val writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, + curWriteMetrics) + writer.open() } + // Creating the file to write to and creating a disk writer both involve interacting with + // the disk, and can take a long time in aggregate when we open many files, so should be + // included in the shuffle write time. + curWriteMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) } // No need to sort stuff, just write each element out while (iterator.hasNext) { - val elem = iterator.next() - val partitionId = elem._1._1 - val key = elem._1._2 - val value = elem._2 - partitionWriters(partitionId).write((key, value)) + val partitionId = iterator.nextPartition() + iterator.writeNext(partitionWriters(partitionId)) } } @@ -505,7 +511,8 @@ private[spark] class ExternalSorter[K, V, C]( val k = elem._1 var c = elem._2 while (sorted.hasNext && sorted.head._1 == k) { - c = mergeCombiners(c, sorted.head._2) + val pair = sorted.next() + c = mergeCombiners(c, pair._2) } (k, c) } @@ -595,8 +602,8 @@ private[spark] class ExternalSorter[K, V, C]( if (finished || deserializeStream == null) { return null } - val k = deserializeStream.readObject().asInstanceOf[K] - val c = deserializeStream.readObject().asInstanceOf[C] + val k = deserializeStream.readKey().asInstanceOf[K] + val c = deserializeStream.readValue().asInstanceOf[C] lastPartitionId = partitionId // Start reading the next batch if we're done with this one indexInBatch += 1 @@ -659,6 +666,8 @@ private[spark] class ExternalSorter[K, V, C]( } /** + * Exposed for testing purposes. + * * Return an iterator over all the data written to this object, grouped by partition and * aggregated by the requested aggregator. For each partition we then have an iterator over its * contents, and these are expected to be accessed in order (you can't "skip ahead" to one @@ -668,29 +677,29 @@ private[spark] class ExternalSorter[K, V, C]( * For now, we just merge all the spilled files in once pass, but this can be modified to * support hierarchical merging. */ - def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { + def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { val usingMap = aggregator.isDefined - val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer + val collection: WritablePartitionedPairCollection[K, C] = if (usingMap) map else buffer if (spills.isEmpty && partitionWriters == null) { // Special case: if we have only in-memory data, we don't need to merge streams, and perhaps // we don't even need to sort by anything other than partition ID if (!ordering.isDefined) { // The user hasn't requested sorted keys, so only sort by partition ID, not key - groupByPartition(collection.destructiveSortedIterator(partitionComparator)) + groupByPartition(collection.partitionedDestructiveSortedIterator(None)) } else { // We do need to sort by both partition ID and key - groupByPartition(collection.destructiveSortedIterator(partitionKeyComparator)) + groupByPartition(collection.partitionedDestructiveSortedIterator(Some(keyComparator))) } } else if (bypassMergeSort) { // Read data from each partition file and merge it together with the data in memory; // note that there's no ordering or aggregator in this case -- we just partition objects - val collIter = groupByPartition(collection.destructiveSortedIterator(partitionComparator)) + val collIter = groupByPartition(collection.partitionedDestructiveSortedIterator(None)) collIter.map { case (partitionId, values) => (partitionId, values ++ readPartitionFile(partitionWriters(partitionId))) } } else { // Merge spilled and in-memory data - merge(spills, collection.destructiveSortedIterator(partitionKeyComparator)) + merge(spills, collection.partitionedDestructiveSortedIterator(comparator)) } } @@ -721,37 +730,45 @@ private[spark] class ExternalSorter[K, V, C]( // this simple we spill out the current in-memory collection so that everything is in files. spillToPartitionFiles(if (aggregator.isDefined) map else buffer) partitionWriters.foreach(_.commitAndClose()) - var out: FileOutputStream = null - var in: FileInputStream = null + val out = new FileOutputStream(outputFile, true) val writeStartTime = System.nanoTime - try { - out = new FileOutputStream(outputFile, true) + util.Utils.tryWithSafeFinally { for (i <- 0 until numPartitions) { - in = new FileInputStream(partitionWriters(i).fileSegment().file) - val size = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) - in.close() - in = null - lengths(i) = size - } - } finally { - if (out != null) { - out.close() - } - if (in != null) { - in.close() + val in = new FileInputStream(partitionWriters(i).fileSegment().file) + util.Utils.tryWithSafeFinally { + lengths(i) = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) + } { + in.close() + } } + } { + out.close() context.taskMetrics.shuffleWriteMetrics.foreach( _.incShuffleWriteTime(System.nanoTime - writeStartTime)) } + } else if (spills.isEmpty && partitionWriters == null) { + // Case where we only have in-memory data + val collection = if (aggregator.isDefined) map else buffer + val it = collection.destructiveSortedWritablePartitionedIterator(comparator) + while (it.hasNext) { + val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, + context.taskMetrics.shuffleWriteMetrics.get) + val partitionId = it.nextPartition() + while (it.hasNext && it.nextPartition() == partitionId) { + it.writeNext(writer) + } + writer.commitAndClose() + val segment = writer.fileSegment() + lengths(partitionId) = segment.length + } } else { - // Either we're not bypassing merge-sort or we have only in-memory data; get an iterator by - // partition and just write everything directly. + // Not bypassing merge-sort; get an iterator by partition and just write everything directly. for ((id, elements) <- this.partitionedIterator) { if (elements.hasNext) { - val writer = blockManager.getDiskWriter( - blockId, outputFile, ser, fileBufferSize, context.taskMetrics.shuffleWriteMetrics.get) + val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, + context.taskMetrics.shuffleWriteMetrics.get) for (elem <- elements) { - writer.write(elem) + writer.write(elem._1, elem._2) } writer.commitAndClose() val segment = writer.fileSegment() @@ -776,11 +793,11 @@ private[spark] class ExternalSorter[K, V, C]( /** * Read a partition file back as an iterator (used in our iterator method) */ - def readPartitionFile(writer: BlockObjectWriter): Iterator[Product2[K, C]] = { + private def readPartitionFile(writer: BlockObjectWriter): Iterator[Product2[K, C]] = { if (writer.isOpen) { writer.commitAndClose() } - blockManager.diskStore.getValues(writer.blockId, ser).get.asInstanceOf[Iterator[Product2[K, C]]] + new PairIterator[K, C](blockManager.diskStore.getValues(writer.blockId, ser).get) } def stop(): Unit = { @@ -810,6 +827,14 @@ private[spark] class ExternalSorter[K, V, C]( (0 until numPartitions).iterator.map(p => (p, new IteratorForPartition(p, buffered))) } + private def comparator: Option[Comparator[K]] = { + if (ordering.isDefined || aggregator.isDefined) { + Some(keyComparator) + } else { + None + } + } + /** * An iterator that reads only the elements for a given partition ID from an underlying buffered * stream, assuming this partition is the next one to be read. Used to make it easier to return diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index b8de4ff9aa494..efc2482c74ddf 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -53,6 +53,15 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( override def size: Int = if (haveNullValue) _keySet.size + 1 else _keySet.size + /** Tests whether this map contains a binding for a key. */ + def contains(k: K): Boolean = { + if (k == null) { + haveNullValue + } else { + _keySet.getPos(k) != OpenHashSet.INVALID_POS + } + } + /** Get the value for a given key */ def apply(k: K): V = { if (k == null) { @@ -109,7 +118,7 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = -1 var nextPair: (K, V) = computeNextPair() @@ -132,9 +141,9 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 4e363b74f4bef..1501111a06655 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -85,7 +85,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( protected var _bitset = new BitSet(_capacity) - def getBitSet = _bitset + def getBitSet: BitSet = _bitset // Init of the array in constructor (instead of in declaration) to work around a Scala compiler // specialization bug that would generate two arrays (one for Object and one for specialized T). @@ -122,7 +122,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( */ def addWithoutResize(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask - var i = 1 + var delta = 1 while (true) { if (!_bitset.get(pos)) { // This is a new key. @@ -134,14 +134,12 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( // Found an existing key. return pos } else { - val delta = i + // quadratic probing with values increase by 1, 2, 3, ... pos = (pos + delta) & _mask - i += 1 + delta += 1 } } - // Never reached here - assert(INVALID_POS != INVALID_POS) - INVALID_POS + throw new RuntimeException("Should never reach here.") } /** @@ -163,27 +161,25 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( */ def getPos(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask - var i = 1 - val maxProbe = _data.size - while (i < maxProbe) { + var delta = 1 + while (true) { if (!_bitset.get(pos)) { return INVALID_POS } else if (k == _data(pos)) { return pos } else { - val delta = i + // quadratic probing with values increase by 1, 2, 3, ... pos = (pos + delta) & _mask - i += 1 + delta += 1 } } - // Never reached here - INVALID_POS + throw new RuntimeException("Should never reach here.") } /** Return the value at the specified position. */ def getValue(pos: Int): T = _data(pos) - def iterator = new Iterator[T] { + def iterator: Iterator[T] = new Iterator[T] { var pos = nextPos(0) override def hasNext: Boolean = pos != INVALID_POS override def next(): T = { diff --git a/core/src/main/scala/org/apache/spark/util/collection/PairIterator.scala b/core/src/main/scala/org/apache/spark/util/collection/PairIterator.scala new file mode 100644 index 0000000000000..d75959f480756 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/PairIterator.scala @@ -0,0 +1,24 @@ +/* + * 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.collection + +private[spark] class PairIterator[K, V](iter: Iterator[Any]) extends Iterator[(K, V)] { + def hasNext: Boolean = iter.hasNext + + def next(): (K, V) = (iter.next().asInstanceOf[K], iter.next().asInstanceOf[V]) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedAppendOnlyMap.scala new file mode 100644 index 0000000000000..e2e2f1faae9d1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedAppendOnlyMap.scala @@ -0,0 +1,44 @@ +/* + * 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.collection + +import java.util.Comparator + +import org.apache.spark.util.collection.WritablePartitionedPairCollection._ + +/** + * Implementation of WritablePartitionedPairCollection that wraps a map in which the keys are tuples + * of (partition ID, K) + */ +private[spark] class PartitionedAppendOnlyMap[K, V] + extends SizeTrackingAppendOnlyMap[(Int, K), V] with WritablePartitionedPairCollection[K, V] { + + def partitionedDestructiveSortedIterator(keyComparator: Option[Comparator[K]]) + : Iterator[((Int, K), V)] = { + val comparator = keyComparator.map(partitionKeyComparator).getOrElse(partitionComparator) + destructiveSortedIterator(comparator) + } + + def writablePartitionedIterator(): WritablePartitionedIterator = { + WritablePartitionedIterator.fromIterator(super.iterator) + } + + def insert(partition: Int, key: K, value: V): Unit = { + update((partition, key), value) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala similarity index 66% rename from core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala rename to core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala index 9e9c16c5a2962..e8332e1a87eac 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala @@ -19,11 +19,15 @@ package org.apache.spark.util.collection import java.util.Comparator +import org.apache.spark.storage.BlockObjectWriter +import org.apache.spark.util.collection.WritablePartitionedPairCollection._ + /** - * Append-only buffer of key-value pairs that keeps track of its estimated size in bytes. + * Append-only buffer of key-value pairs, each with a corresponding partition ID, that keeps track + * of its estimated size in bytes. */ -private[spark] class SizeTrackingPairBuffer[K, V](initialCapacity: Int = 64) - extends SizeTracker with SizeTrackingPairCollection[K, V] +private[spark] class PartitionedPairBuffer[K, V](initialCapacity: Int = 64) + extends WritablePartitionedPairCollection[K, V] with SizeTracker { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") @@ -35,35 +39,16 @@ private[spark] class SizeTrackingPairBuffer[K, V](initialCapacity: Int = 64) private var data = new Array[AnyRef](2 * initialCapacity) /** Add an element into the buffer */ - def insert(key: K, value: V): Unit = { + def insert(partition: Int, key: K, value: V): Unit = { if (curSize == capacity) { growArray() } - data(2 * curSize) = key.asInstanceOf[AnyRef] + data(2 * curSize) = (partition, key.asInstanceOf[AnyRef]) data(2 * curSize + 1) = value.asInstanceOf[AnyRef] curSize += 1 afterUpdate() } - /** Total number of elements in buffer */ - override def size: Int = curSize - - /** Iterate over the elements of the buffer */ - override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { - var pos = 0 - - override def hasNext: Boolean = pos < curSize - - override def next(): (K, V) = { - if (!hasNext) { - throw new NoSuchElementException - } - val pair = (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) - pos += 1 - pair - } - } - /** Double the size of the array because we've reached capacity */ private def growArray(): Unit = { if (capacity == (1 << 29)) { @@ -79,8 +64,29 @@ private[spark] class SizeTrackingPairBuffer[K, V](initialCapacity: Int = 64) } /** Iterate through the data in a given order. For this class this is not really destructive. */ - override def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] = { - new Sorter(new KVArraySortDataFormat[K, AnyRef]).sort(data, 0, curSize, keyComparator) + override def partitionedDestructiveSortedIterator(keyComparator: Option[Comparator[K]]) + : Iterator[((Int, K), V)] = { + val comparator = keyComparator.map(partitionKeyComparator).getOrElse(partitionComparator) + new Sorter(new KVArraySortDataFormat[(Int, K), AnyRef]).sort(data, 0, curSize, comparator) iterator } + + override def writablePartitionedIterator(): WritablePartitionedIterator = { + WritablePartitionedIterator.fromIterator(iterator) + } + + private def iterator(): Iterator[((Int, K), V)] = new Iterator[((Int, K), V)] { + var pos = 0 + + override def hasNext: Boolean = pos < curSize + + override def next(): ((Int, K), V) = { + if (!hasNext) { + throw new NoSuchElementException + } + val pair = (data(2 * pos).asInstanceOf[(Int, K)], data(2 * pos + 1).asInstanceOf[V]) + pos += 1 + pair + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala new file mode 100644 index 0000000000000..b5ca0c62a04f2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala @@ -0,0 +1,254 @@ +/* + * 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.collection + +import java.io.InputStream +import java.nio.IntBuffer +import java.util.Comparator + +import org.apache.spark.SparkEnv +import org.apache.spark.serializer.{JavaSerializerInstance, SerializerInstance} +import org.apache.spark.storage.BlockObjectWriter +import org.apache.spark.util.collection.PartitionedSerializedPairBuffer._ + +/** + * Append-only buffer of key-value pairs, each with a corresponding partition ID, that serializes + * its records upon insert and stores them as raw bytes. + * + * We use two data-structures to store the contents. The serialized records are stored in a + * ChainedBuffer that can expand gracefully as records are added. This buffer is accompanied by a + * metadata buffer that stores pointers into the data buffer as well as the partition ID of each + * record. Each entry in the metadata buffer takes up a fixed amount of space. + * + * Sorting the collection means swapping entries in the metadata buffer - the record buffer need not + * be modified at all. Storing the partition IDs in the metadata buffer means that comparisons can + * happen without following any pointers, which should minimize cache misses. + * + * Currently, only sorting by partition is supported. + * + * @param metaInitialRecords The initial number of entries in the metadata buffer. + * @param kvBlockSize The size of each byte buffer in the ChainedBuffer used to store the records. + * @param serializerInstance the serializer used for serializing inserted records. + */ +private[spark] class PartitionedSerializedPairBuffer[K, V]( + metaInitialRecords: Int, + kvBlockSize: Int, + serializerInstance: SerializerInstance) + extends WritablePartitionedPairCollection[K, V] with SizeTracker { + + if (serializerInstance.isInstanceOf[JavaSerializerInstance]) { + throw new IllegalArgumentException("PartitionedSerializedPairBuffer does not support" + + " Java-serialized objects.") + } + + private var metaBuffer = IntBuffer.allocate(metaInitialRecords * RECORD_SIZE) + + private val kvBuffer: ChainedBuffer = new ChainedBuffer(kvBlockSize) + private val kvOutputStream = new ChainedBufferOutputStream(kvBuffer) + private val kvSerializationStream = serializerInstance.serializeStream(kvOutputStream) + + def insert(partition: Int, key: K, value: V): Unit = { + if (metaBuffer.position == metaBuffer.capacity) { + growMetaBuffer() + } + + val keyStart = kvBuffer.size + if (keyStart < 0) { + throw new Exception(s"Can't grow buffer beyond ${1 << 31} bytes") + } + kvSerializationStream.writeObject[Any](key) + kvSerializationStream.flush() + val valueStart = kvBuffer.size + kvSerializationStream.writeObject[Any](value) + kvSerializationStream.flush() + val valueEnd = kvBuffer.size + + metaBuffer.put(keyStart) + metaBuffer.put(valueStart) + metaBuffer.put(valueEnd) + metaBuffer.put(partition) + } + + /** Double the size of the array because we've reached capacity */ + private def growMetaBuffer(): Unit = { + if (metaBuffer.capacity.toLong * 2 > Int.MaxValue) { + // Doubling the capacity would create an array bigger than Int.MaxValue, so don't + throw new Exception(s"Can't grow buffer beyond ${Int.MaxValue} bytes") + } + val newMetaBuffer = IntBuffer.allocate(metaBuffer.capacity * 2) + newMetaBuffer.put(metaBuffer.array) + metaBuffer = newMetaBuffer + } + + /** Iterate through the data in a given order. For this class this is not really destructive. */ + override def partitionedDestructiveSortedIterator(keyComparator: Option[Comparator[K]]) + : Iterator[((Int, K), V)] = { + sort(keyComparator) + val is = orderedInputStream + val deserStream = serializerInstance.deserializeStream(is) + new Iterator[((Int, K), V)] { + var metaBufferPos = 0 + def hasNext: Boolean = metaBufferPos < metaBuffer.position + def next(): ((Int, K), V) = { + val key = deserStream.readKey[Any]().asInstanceOf[K] + val value = deserStream.readValue[Any]().asInstanceOf[V] + val partition = metaBuffer.get(metaBufferPos + PARTITION) + metaBufferPos += RECORD_SIZE + ((partition, key), value) + } + } + } + + override def estimateSize: Long = metaBuffer.capacity * 4 + kvBuffer.capacity + + override def destructiveSortedWritablePartitionedIterator(keyComparator: Option[Comparator[K]]) + : WritablePartitionedIterator = { + sort(keyComparator) + writablePartitionedIterator + } + + override def writablePartitionedIterator(): WritablePartitionedIterator = { + new WritablePartitionedIterator { + // current position in the meta buffer in ints + var pos = 0 + + def writeNext(writer: BlockObjectWriter): Unit = { + val keyStart = metaBuffer.get(pos + KEY_START) + val valueEnd = metaBuffer.get(pos + VAL_END) + pos += RECORD_SIZE + kvBuffer.read(keyStart, writer, valueEnd - keyStart) + writer.recordWritten() + } + def nextPartition(): Int = metaBuffer.get(pos + PARTITION) + def hasNext(): Boolean = pos < metaBuffer.position + } + } + + // Visible for testing + def orderedInputStream: OrderedInputStream = { + new OrderedInputStream(metaBuffer, kvBuffer) + } + + private def sort(keyComparator: Option[Comparator[K]]): Unit = { + val comparator = if (keyComparator.isEmpty) { + new Comparator[Int]() { + def compare(partition1: Int, partition2: Int): Int = { + partition1 - partition2 + } + } + } else { + throw new UnsupportedOperationException() + } + + val sorter = new Sorter(new SerializedSortDataFormat) + sorter.sort(metaBuffer, 0, metaBuffer.position / RECORD_SIZE, comparator) + } +} + +private[spark] class OrderedInputStream(metaBuffer: IntBuffer, kvBuffer: ChainedBuffer) + extends InputStream { + + private var metaBufferPos = 0 + private var kvBufferPos = + if (metaBuffer.position > 0) metaBuffer.get(metaBufferPos + KEY_START) else 0 + + override def read(bytes: Array[Byte]): Int = read(bytes, 0, bytes.length) + + override def read(bytes: Array[Byte], offs: Int, len: Int): Int = { + if (metaBufferPos >= metaBuffer.position) { + return -1 + } + val bytesRemainingInRecord = metaBuffer.get(metaBufferPos + VAL_END) - kvBufferPos + val toRead = math.min(bytesRemainingInRecord, len) + kvBuffer.read(kvBufferPos, bytes, offs, toRead) + if (toRead == bytesRemainingInRecord) { + metaBufferPos += RECORD_SIZE + if (metaBufferPos < metaBuffer.position) { + kvBufferPos = metaBuffer.get(metaBufferPos + KEY_START) + } + } else { + kvBufferPos += toRead + } + toRead + } + + override def read(): Int = { + throw new UnsupportedOperationException() + } +} + +private[spark] class SerializedSortDataFormat extends SortDataFormat[Int, IntBuffer] { + + private val META_BUFFER_TMP = new Array[Int](RECORD_SIZE) + + /** Return the sort key for the element at the given index. */ + override protected def getKey(metaBuffer: IntBuffer, pos: Int): Int = { + metaBuffer.get(pos * RECORD_SIZE + PARTITION) + } + + /** Swap two elements. */ + override def swap(metaBuffer: IntBuffer, pos0: Int, pos1: Int): Unit = { + val iOff = pos0 * RECORD_SIZE + val jOff = pos1 * RECORD_SIZE + System.arraycopy(metaBuffer.array, iOff, META_BUFFER_TMP, 0, RECORD_SIZE) + System.arraycopy(metaBuffer.array, jOff, metaBuffer.array, iOff, RECORD_SIZE) + System.arraycopy(META_BUFFER_TMP, 0, metaBuffer.array, jOff, RECORD_SIZE) + } + + /** Copy a single element from src(srcPos) to dst(dstPos). */ + override def copyElement( + src: IntBuffer, + srcPos: Int, + dst: IntBuffer, + dstPos: Int): Unit = { + val srcOff = srcPos * RECORD_SIZE + val dstOff = dstPos * RECORD_SIZE + System.arraycopy(src.array, srcOff, dst.array, dstOff, RECORD_SIZE) + } + + /** + * Copy a range of elements starting at src(srcPos) to dst, starting at dstPos. + * Overlapping ranges are allowed. + */ + override def copyRange( + src: IntBuffer, + srcPos: Int, + dst: IntBuffer, + dstPos: Int, + length: Int): Unit = { + val srcOff = srcPos * RECORD_SIZE + val dstOff = dstPos * RECORD_SIZE + System.arraycopy(src.array, srcOff, dst.array, dstOff, RECORD_SIZE * length) + } + + /** + * Allocates a Buffer that can hold up to 'length' elements. + * All elements of the buffer should be considered invalid until data is explicitly copied in. + */ + override def allocate(length: Int): IntBuffer = { + IntBuffer.allocate(length * RECORD_SIZE) + } +} + +private[spark] object PartitionedSerializedPairBuffer { + val KEY_START = 0 + val VAL_START = 1 + val VAL_END = 2 + val PARTITION = 3 + val RECORD_SIZE = Seq(KEY_START, VAL_START, VAL_END, PARTITION).size // num ints of metadata +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index 2e1ef06cbc4e1..b4ec4ea521253 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -46,7 +46,12 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, private var _oldValues: Array[V] = null - override def size = _keySet.size + override def size: Int = _keySet.size + + /** Tests whether this map contains a binding for a key. */ + def contains(k: K): Boolean = { + _keySet.getPos(k) != OpenHashSet.INVALID_POS + } /** Get the value for a given key */ def apply(k: K): V = { @@ -87,7 +92,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = 0 var nextPair: (K, V) = computeNextPair() @@ -103,9 +108,9 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index eb4de413867a0..722f78bd15986 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -21,7 +21,7 @@ package org.apache.spark.util.collection * An append-only map that keeps track of its estimated size in bytes. */ private[spark] class SizeTrackingAppendOnlyMap[K, V] - extends AppendOnlyMap[K, V] with SizeTracker with SizeTrackingPairCollection[K, V] + extends AppendOnlyMap[K, V] with SizeTracker { override def update(key: K, value: V): Unit = { super.update(key, value) diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala index c5268c0fae0ef..bdbca00a00622 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala @@ -32,7 +32,7 @@ private[spark] object Utils { */ def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = { val ordering = new GuavaOrdering[T] { - override def compare(l: T, r: T) = ord.compare(l, r) + override def compare(l: T, r: T): Int = ord.compare(l, r) } collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator } diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala new file mode 100644 index 0000000000000..f26d1618c9200 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -0,0 +1,113 @@ +/* + * 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.collection + +import java.util.Comparator + +import org.apache.spark.storage.BlockObjectWriter + +/** + * A common interface for size-tracking collections of key-value pairs that + * - Have an associated partition for each key-value pair. + * - Support a memory-efficient sorted iterator + * - Support a WritablePartitionedIterator for writing the contents directly as bytes. + */ +private[spark] trait WritablePartitionedPairCollection[K, V] { + /** + * Insert a key-value pair with a partition into the collection + */ + def insert(partition: Int, key: K, value: V): Unit + + /** + * Iterate through the data in order of partition ID and then the given comparator. This may + * destroy the underlying collection. + */ + def partitionedDestructiveSortedIterator(keyComparator: Option[Comparator[K]]) + : Iterator[((Int, K), V)] + + /** + * Iterate through the data and write out the elements instead of returning them. Records are + * returned in order of their partition ID and then the given comparator. + * This may destroy the underlying collection. + */ + def destructiveSortedWritablePartitionedIterator(keyComparator: Option[Comparator[K]]) + : WritablePartitionedIterator = { + WritablePartitionedIterator.fromIterator(partitionedDestructiveSortedIterator(keyComparator)) + } + + /** + * Iterate through the data and write out the elements instead of returning them. + */ + def writablePartitionedIterator(): WritablePartitionedIterator +} + +private[spark] object WritablePartitionedPairCollection { + /** + * A comparator for (Int, K) pairs that orders them by only their partition ID. + */ + def partitionComparator[K]: Comparator[(Int, K)] = new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + a._1 - b._1 + } + } + + /** + * A comparator for (Int, K) pairs that orders them both by their partition ID and a key ordering. + */ + def partitionKeyComparator[K](keyComparator: Comparator[K]): Comparator[(Int, K)] = { + new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + val partitionDiff = a._1 - b._1 + if (partitionDiff != 0) { + partitionDiff + } else { + keyComparator.compare(a._2, b._2) + } + } + } + } +} + +/** + * Iterator that writes elements to a BlockObjectWriter instead of returning them. Each element + * has an associated partition. + */ +private[spark] trait WritablePartitionedIterator { + def writeNext(writer: BlockObjectWriter): Unit + + def hasNext(): Boolean + + def nextPartition(): Int +} + +private[spark] object WritablePartitionedIterator { + def fromIterator(it: Iterator[((Int, _), _)]): WritablePartitionedIterator = { + new WritablePartitionedIterator { + var cur = if (it.hasNext) it.next() else null + + def writeNext(writer: BlockObjectWriter): Unit = { + writer.write(cur._1._2, cur._2) + cur = if (it.hasNext) it.next() else null + } + + def hasNext(): Boolean = cur != null + + def nextPartition(): Int = cur._1._1 + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 1d5467060623c..14b6ba4af489a 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -121,7 +121,7 @@ private[spark] object FileAppender extends Logging { val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) - def createTimeBasedAppender() = { + def createTimeBasedAppender(): FileAppender = { val validatedParams: Option[(Long, String)] = rollingInterval match { case "daily" => logInfo(s"Rolling executor logs enabled for $file with daily rolling") @@ -149,7 +149,7 @@ private[spark] object FileAppender extends Logging { } } - def createSizeBasedAppender() = { + def createSizeBasedAppender(): FileAppender = { rollingSizeBytes match { case IntParam(bytes) => logInfo(s"Rolling executor logs enabled for $file with rolling every $bytes bytes") diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index e579421676343..7138b4b8e4533 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -138,7 +138,7 @@ private[spark] object RollingFileAppender { val STRATEGY_DEFAULT = "" val INTERVAL_PROPERTY = "spark.executor.logs.rolling.time.interval" val INTERVAL_DEFAULT = "daily" - val SIZE_PROPERTY = "spark.executor.logs.rolling.size.maxBytes" + val SIZE_PROPERTY = "spark.executor.logs.rolling.maxSize" val SIZE_DEFAULT = (1024 * 1024).toString val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 76e7a2760bcd1..786b97ad7b9ec 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -105,7 +105,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals private val rng: Random = new XORShiftRandom - override def setSeed(seed: Long) = rng.setSeed(seed) + override def setSeed(seed: Long): Unit = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { if (ub - lb <= 0.0) { @@ -131,7 +131,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals def cloneComplement(): BernoulliCellSampler[T] = new BernoulliCellSampler[T](lb, ub, !complement) - override def clone = new BernoulliCellSampler[T](lb, ub, complement) + override def clone: BernoulliCellSampler[T] = new BernoulliCellSampler[T](lb, ub, complement) } @@ -153,7 +153,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T private val rng: Random = RandomSampler.newDefaultRNG - override def setSeed(seed: Long) = rng.setSeed(seed) + override def setSeed(seed: Long): Unit = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { if (fraction <= 0.0) { @@ -167,7 +167,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T } } - override def clone = new BernoulliSampler[T](fraction) + override def clone: BernoulliSampler[T] = new BernoulliSampler[T](fraction) } @@ -209,7 +209,7 @@ class PoissonSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] } } - override def clone = new PoissonSampler[T](fraction) + override def clone: PoissonSampler[T] = new PoissonSampler[T](fraction) } @@ -228,15 +228,18 @@ class GapSamplingIterator[T: ClassTag]( val arrayClass = Array.empty[T].iterator.getClass val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass data.getClass match { - case `arrayClass` => ((n: Int) => { data = data.drop(n) }) - case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) - case _ => ((n: Int) => { + case `arrayClass` => + (n: Int) => { data = data.drop(n) } + case `arrayBufferClass` => + (n: Int) => { data = data.drop(n) } + case _ => + (n: Int) => { var j = 0 while (j < n && data.hasNext) { data.next() j += 1 } - }) + } } } @@ -244,21 +247,21 @@ class GapSamplingIterator[T: ClassTag]( override def next(): T = { val r = data.next() - advance + advance() r } private val lnq = math.log1p(-f) /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ - private def advance: Unit = { + private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) val k = (math.log(u) / lnq).toInt iterDrop(k) } /** advance to first sample as part of object construction. */ - advance + advance() // Attempting to invoke this closer to the top with other object initialization // was causing it to break in strange ways, so I'm invoking it last, which seems to // work reliably. @@ -279,15 +282,18 @@ class GapSamplingReplacementIterator[T: ClassTag]( val arrayClass = Array.empty[T].iterator.getClass val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass data.getClass match { - case `arrayClass` => ((n: Int) => { data = data.drop(n) }) - case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) - case _ => ((n: Int) => { + case `arrayClass` => + (n: Int) => { data = data.drop(n) } + case `arrayBufferClass` => + (n: Int) => { data = data.drop(n) } + case _ => + (n: Int) => { var j = 0 while (j < n && data.hasNext) { data.next() j += 1 } - }) + } } } @@ -300,7 +306,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( override def next(): T = { val r = v rep -= 1 - if (rep <= 0) advance + if (rep <= 0) advance() r } @@ -309,7 +315,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is * q is the probabililty of Poisson(0; f) */ - private def advance: Unit = { + private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) val k = (math.log(u) / (-f)).toInt iterDrop(k) @@ -343,7 +349,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( } /** advance to first sample as part of object construction. */ - advance + advance() // Attempting to invoke this closer to the top with other object initialization // was causing it to break in strange ways, so I'm invoking it last, which seems to // work reliably. diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 2ae308dacf1ae..9e29bf9d61f17 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -311,7 +311,7 @@ private[random] class AcceptanceResult(var numItems: Long = 0L, var numAccepted: var acceptBound: Double = Double.NaN // upper bound for accepting item instantly var waitListBound: Double = Double.NaN // upper bound for adding item to waitlist - def areBoundsEmpty = acceptBound.isNaN || waitListBound.isNaN + def areBoundsEmpty: Boolean = acceptBound.isNaN || waitListBound.isNaN def merge(other: Option[AcceptanceResult]): Unit = { if (other.isDefined) { diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 467b890fb4bb9..c4a7b4441c85c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -83,7 +83,7 @@ private[spark] object XORShiftRandom { * @return Map of execution times for {@link java.util.Random java.util.Random} * and XORShift */ - def benchmark(numIters: Int) = { + def benchmark(numIters: Int): Map[String, Long] = { val seed = 1L val million = 1e6.toInt diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index d4b5bb519157c..c2089b0e56a1f 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -157,11 +157,11 @@ public void sample() { public void randomSplit() { List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); JavaRDD rdd = sc.parallelize(ints); - JavaRDD[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 11); + JavaRDD[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31); Assert.assertEquals(3, splits.length); - Assert.assertEquals(2, splits[0].count()); - Assert.assertEquals(3, splits[1].count()); - Assert.assertEquals(5, splits[2].count()); + Assert.assertEquals(1, splits[0].count()); + Assert.assertEquals(2, splits[1].count()); + Assert.assertEquals(7, splits[2].count()); } @Test @@ -761,6 +761,20 @@ public void min() { Assert.assertEquals(1.0, max, 0.001); } + @Test + public void naturalMax() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(); + Assert.assertTrue(4.0 == max); + } + + @Test + public void naturalMin() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(); + Assert.assertTrue(1.0 == max); + } + @Test public void takeOrdered() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); @@ -995,7 +1009,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContextImpl(0, 0, 0L, 0, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, 0, null, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation new file mode 100644 index 0000000000000..6101177e4bb85 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation @@ -0,0 +1,53 @@ +[ { + "id" : "local-1427397477963", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-03-26T19:17:57.184GMT", + "endTime" : "2015-03-26T19:20:02.949GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation new file mode 100644 index 0000000000000..cb622e147249e --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation @@ -0,0 +1,17 @@ +[ { + "id" : "", + "hostPort" : "localhost:57971", + "rddBlocks" : 8, + "memoryUsed" : 28000128, + "diskUsed" : 0, + "activeTasks" : 0, + "failedTasks" : 1, + "completedTasks" : 31, + "totalTasks" : 32, + "totalDuration" : 8820, + "totalInputBytes" : 28000288, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 13180, + "maxMemory" : 278302556, + "executorLogs" : { } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation new file mode 100644 index 0000000000000..4a29072bdb6e4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation @@ -0,0 +1,15 @@ +{ + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation new file mode 100644 index 0000000000000..cab4750270dfa --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation @@ -0,0 +1,43 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 1, + "name" : "count at :20", + "stageIds" : [ 1, 2 ], + "status" : "FAILED", + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompletedTasks" : 15, + "numSkippedTasks" : 15, + "numFailedTasks" : 1, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 1 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation new file mode 100644 index 0000000000000..cab4750270dfa --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation @@ -0,0 +1,43 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 1, + "name" : "count at :20", + "stageIds" : [ 1, 2 ], + "status" : "FAILED", + "numTasks" : 16, + "numActiveTasks" : 0, + "numCompletedTasks" : 15, + "numSkippedTasks" : 15, + "numFailedTasks" : 1, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 1 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation new file mode 100644 index 0000000000000..6fd25befbf7e8 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation @@ -0,0 +1,29 @@ +[ { + "jobId" : 2, + "name" : "count at :17", + "stageIds" : [ 3 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +}, { + "jobId" : 0, + "name" : "count at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation new file mode 100644 index 0000000000000..07489ad96414a --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation @@ -0,0 +1,10 @@ +{ + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation new file mode 100644 index 0000000000000..111cb8163eb3d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation @@ -0,0 +1,270 @@ +{ + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ], + "tasks" : { + "8" : { + "taskId" : 8, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 94000, + "recordsWritten" : 0 + } + } + }, + "11" : { + "taskId" : 11, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1647, + "writeTime" : 83000, + "recordsWritten" : 0 + } + } + }, + "14" : { + "taskId" : 14, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 88000, + "recordsWritten" : 0 + } + } + }, + "13" : { + "taskId" : 13, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 73000, + "recordsWritten" : 0 + } + } + }, + "10" : { + "taskId" : 10, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 76000, + "recordsWritten" : 0 + } + } + }, + "9" : { + "taskId" : 9, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 436, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 98000, + "recordsWritten" : 0 + } + } + }, + "12" : { + "taskId" : 12, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1645, + "writeTime" : 101000, + "recordsWritten" : 0 + } + } + }, + "15" : { + "taskId" : 15, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.833GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 79000, + "recordsWritten" : 0 + } + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 3624, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 28000128, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 13180, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation new file mode 100644 index 0000000000000..ef339f89afa45 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation @@ -0,0 +1,270 @@ +[ { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ], + "tasks" : { + "8" : { + "taskId" : 8, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 94000, + "recordsWritten" : 0 + } + } + }, + "11" : { + "taskId" : 11, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1647, + "writeTime" : 83000, + "recordsWritten" : 0 + } + } + }, + "14" : { + "taskId" : 14, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 88000, + "recordsWritten" : 0 + } + } + }, + "13" : { + "taskId" : 13, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 73000, + "recordsWritten" : 0 + } + } + }, + "10" : { + "taskId" : 10, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 76000, + "recordsWritten" : 0 + } + } + }, + "9" : { + "taskId" : 9, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 436, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 98000, + "recordsWritten" : 0 + } + } + }, + "12" : { + "taskId" : 12, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorRunTime" : 434, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1645, + "writeTime" : 101000, + "recordsWritten" : 0 + } + } + }, + "15" : { + "taskId" : 15, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-02-03T16:43:05.833GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 435, + "resultSize" : 1902, + "jvmGcTime" : 19, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 3500016, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 1648, + "writeTime" : 79000, + "recordsWritten" : 0 + } + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 3624, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 28000128, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 13180, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation new file mode 100644 index 0000000000000..056fac7088594 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation @@ -0,0 +1,89 @@ +[ { + "status" : "COMPLETE", + "stageId" : 3, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 162, + "inputBytes" : 160, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :17", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 4338, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :15", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "executorRunTime" : 278, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :20", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation new file mode 100644 index 0000000000000..31ac9beea8788 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation @@ -0,0 +1,67 @@ +[ { + "status" : "COMPLETE", + "stageId" : 3, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 162, + "inputBytes" : 160, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :17", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 1, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 3476, + "inputBytes" : 28000128, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 13180, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "map at :14", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +}, { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 4338, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :15", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation new file mode 100644 index 0000000000000..bff6a4f69d077 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation @@ -0,0 +1,23 @@ +[ { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "executorRunTime" : 278, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "count at :20", + "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation new file mode 100644 index 0000000000000..38b5328ffbb03 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation @@ -0,0 +1,64 @@ +{ + "id" : 0, + "name" : "0", + "numPartitions" : 8, + "numCachedPartitions" : 8, + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 28000128, + "diskUsed" : 0, + "dataDistribution" : [ { + "address" : "localhost:57971", + "memoryUsed" : 28000128, + "memoryRemaining" : 250302428, + "diskUsed" : 0 + } ], + "partitions" : [ { + "blockName" : "rdd_0_0", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_1", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_2", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_3", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_4", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_5", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_6", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + }, { + "blockName" : "rdd_0_7", + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 3500016, + "diskUsed" : 0, + "executors" : [ "localhost:57971" ] + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation new file mode 100644 index 0000000000000..f79a31022d214 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation @@ -0,0 +1,9 @@ +[ { + "id" : 0, + "name" : "0", + "numPartitions" : 8, + "numCachedPartitions" : 8, + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 28000128, + "diskUsed" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation new file mode 100644 index 0000000000000..2e92e1fa0ec23 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation @@ -0,0 +1,15 @@ +[ { + "jobId" : 0, + "name" : "foreach at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation new file mode 100644 index 0000000000000..32d5731676ad5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation @@ -0,0 +1,242 @@ +{ + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 120, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "foreach at :15", + "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "value" : "5050" + } ], + "tasks" : { + "2" : { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "5" : { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "4" : { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "7" : { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "1" : { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "3" : { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "6" : { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + }, + "0" : { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } + }, + "executorSummary" : { + "" : { + "taskTime" : 418, + "failedTasks" : 0, + "succeededTasks" : 8, + "inputBytes" : 0, + "outputBytes" : 0, + "shuffleRead" : 0, + "shuffleWrite" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation new file mode 100644 index 0000000000000..c3febc5fc9447 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation @@ -0,0 +1,193 @@ +[ { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-16T19:25:36.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation new file mode 100644 index 0000000000000..79ccacd309693 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation @@ -0,0 +1,27 @@ +[ { + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numActiveTasks" : 0, + "numCompleteTasks" : 8, + "numFailedTasks" : 0, + "executorRunTime" : 120, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "name" : "foreach at :15", + "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", + "schedulingPool" : "default", + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "value" : "5050" + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation new file mode 100644 index 0000000000000..2e92e1fa0ec23 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation @@ -0,0 +1,15 @@ +[ { + "jobId" : 0, + "name" : "foreach at :15", + "stageIds" : [ 0 ], + "status" : "SUCCEEDED", + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompletedTasks" : 8, + "numSkippedTasks" : 8, + "numFailedTasks" : 0, + "numActiveStages" : 0, + "numCompletedStages" : 1, + "numSkippedStages" : 0, + "numFailedStages" : 0 +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation new file mode 100644 index 0000000000000..56d667d88917c --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation @@ -0,0 +1,193 @@ +[ { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.515GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "78", + "value" : "5050" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "247", + "value" : "2175" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 14, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "378", + "value" : "378" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 3, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "572", + "value" : "950" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 13, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 4, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.522GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "678", + "value" : "2853" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 5, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "897", + "value" : "3750" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 6, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.523GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "978", + "value" : "1928" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +}, { + "taskId" : 7, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-17T23:12:16.524GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ { + "id" : 1, + "name" : "my counter", + "update" : "1222", + "value" : "4972" + } ], + "taskMetrics" : { + "executorDeserializeTime" : 12, + "executorRunTime" : 15, + "resultSize" : 697, + "jvmGcTime" : 0, + "resultSerializationTime" : 2, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0 + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation new file mode 100644 index 0000000000000..8f3d7160c723f --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation @@ -0,0 +1,17 @@ +{ + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation new file mode 100644 index 0000000000000..140c76d7e5208 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation @@ -0,0 +1,481 @@ +[ { + "taskId" : 5010, + "index" : 0, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 318464, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5011, + "index" : 1, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 271674, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5012, + "index" : 2, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 274899, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5013, + "index" : 3, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 11, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 315224, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5014, + "index" : 4, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 12, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 316561, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5015, + "index" : 5, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 269443, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5016, + "index" : 6, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 275977, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5017, + "index" : 7, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.571GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 267412, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5018, + "index" : 8, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 310279, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5019, + "index" : 9, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 319028, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5020, + "index" : 10, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 306724, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5021, + "index" : 11, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 280729, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5022, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 273837, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5023, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 282108, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5024, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9604, + "writeTime" : 266308, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5025, + "index" : 15, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 8, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 330830, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5026, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 247775, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5027, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 289503, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5028, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.592GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 355545, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5029, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 339081, + "recordsWritten" : 100 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation new file mode 100644 index 0000000000000..0fa1ac28c9e90 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation @@ -0,0 +1,1201 @@ +[ { + "taskId" : 5020, + "index" : 10, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 306724, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5021, + "index" : 11, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 280729, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5022, + "index" : 12, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 273837, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5023, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 282108, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5024, + "index" : 14, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9604, + "writeTime" : 266308, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5025, + "index" : 15, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 8, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 330830, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5026, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 247775, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5027, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 289503, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5028, + "index" : 18, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.592GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 355545, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5029, + "index" : 19, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 339081, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5030, + "index" : 20, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 9, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 367295, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5031, + "index" : 21, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 9, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 493701, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5032, + "index" : 22, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 345941, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5033, + "index" : 23, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.593GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 301382, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5034, + "index" : 24, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.594GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9603, + "writeTime" : 340547, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5035, + "index" : 25, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.600GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 324374, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5036, + "index" : 26, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.600GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9603, + "writeTime" : 321083, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5037, + "index" : 27, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.601GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 318662, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5038, + "index" : 28, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.601GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 334376, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5039, + "index" : 29, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.601GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 287042, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5040, + "index" : 30, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.601GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 332938, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5041, + "index" : 31, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.602GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9603, + "writeTime" : 255099, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5042, + "index" : 32, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.602GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 297534, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5043, + "index" : 33, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.607GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 265929, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5044, + "index" : 34, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.607GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 279851, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5045, + "index" : 35, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.608GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 278965, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5046, + "index" : 36, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.609GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 340518, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5047, + "index" : 37, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.609GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 402877, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5048, + "index" : 38, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.609GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 265447, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5049, + "index" : 39, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.611GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 23, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 304745, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5050, + "index" : 40, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.615GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 24, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 302371, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5051, + "index" : 41, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.615GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 25, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 296177, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5052, + "index" : 42, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.615GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 24, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 962033, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5053, + "index" : 43, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 26, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 310335, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5054, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 110, + "resultSize" : 930, + "jvmGcTime" : 22, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 299315, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5055, + "index" : 45, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.617GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 25, + "resultSize" : 930, + "jvmGcTime" : 16, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9603, + "writeTime" : 307821, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5056, + "index" : 46, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.617GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 17, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9604, + "writeTime" : 306911, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5057, + "index" : 47, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.639GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 274191, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5058, + "index" : 48, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.640GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 48, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 10442141, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5059, + "index" : 49, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.641GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 300675, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5060, + "index" : 50, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.642GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 24, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 812196, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5061, + "index" : 51, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.642GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 6, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 313090, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5062, + "index" : 52, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.642GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 17, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 1572828, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5063, + "index" : 53, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.643GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 312481, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5064, + "index" : 54, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.646GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 13, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 296976, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5065, + "index" : 55, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.650GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 670271, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5066, + "index" : 56, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.650GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 10, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 1240309, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5067, + "index" : 57, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.650GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 9, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 678632, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5068, + "index" : 58, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.659GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 302888, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5069, + "index" : 59, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.660GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 7, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9610, + "writeTime" : 383153, + "recordsWritten" : 100 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation new file mode 100644 index 0000000000000..63b131621c9c4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation @@ -0,0 +1,481 @@ +[ { + "taskId" : 5186, + "index" : 176, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.880GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 147, + "resultSize" : 930, + "jvmGcTime" : 12, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 317390, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5463, + "index" : 453, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.518GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 134, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 348853, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5461, + "index" : 451, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.516GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 134, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 413676, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5505, + "index" : 495, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 133, + "resultSize" : 930, + "jvmGcTime" : 11, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 384212, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5672, + "index" : 662, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.016GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 131, + "resultSize" : 930, + "jvmGcTime" : 4, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 387955, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5865, + "index" : 855, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.514GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 131, + "resultSize" : 930, + "jvmGcTime" : 10, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 1710924, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5415, + "index" : 405, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.389GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 130, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 377101, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5462, + "index" : 452, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.518GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 128, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 306075, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5411, + "index" : 401, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.385GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 128, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 74684537, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5416, + "index" : 406, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.392GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 123, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 365240, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5924, + "index" : 914, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.648GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 121, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 335682, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5179, + "index" : 169, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.866GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 114, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 276876, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5243, + "index" : 233, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.018GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 113, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 12003053, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5054, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 110, + "resultSize" : 930, + "jvmGcTime" : 22, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 299315, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5465, + "index" : 455, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 108, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 318366, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5614, + "index" : 604, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.893GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 19, + "executorRunTime" : 108, + "resultSize" : 930, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 309135, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5862, + "index" : 852, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.509GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 96, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 1052228, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5566, + "index" : 556, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.762GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 95, + "resultSize" : 930, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 327089, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5567, + "index" : 557, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.768GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 94, + "resultSize" : 930, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 374043, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5191, + "index" : 181, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.885GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 90, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 344199, + "recordsWritten" : 100 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation new file mode 100644 index 0000000000000..63b131621c9c4 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation @@ -0,0 +1,481 @@ +[ { + "taskId" : 5186, + "index" : 176, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.880GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 147, + "resultSize" : 930, + "jvmGcTime" : 12, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 317390, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5463, + "index" : 453, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.518GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 134, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 348853, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5461, + "index" : 451, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.516GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 134, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 413676, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5505, + "index" : 495, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 133, + "resultSize" : 930, + "jvmGcTime" : 11, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 384212, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5672, + "index" : 662, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.016GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 131, + "resultSize" : 930, + "jvmGcTime" : 4, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 387955, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5865, + "index" : 855, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.514GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 131, + "resultSize" : 930, + "jvmGcTime" : 10, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 1710924, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5415, + "index" : 405, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.389GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 130, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 377101, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5462, + "index" : 452, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.518GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 128, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 306075, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5411, + "index" : 401, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.385GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 128, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 74684537, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5416, + "index" : 406, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.392GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 11, + "executorRunTime" : 123, + "resultSize" : 930, + "jvmGcTime" : 14, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 365240, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5924, + "index" : 914, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.648GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 121, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 335682, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5179, + "index" : 169, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.866GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 114, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 276876, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5243, + "index" : 233, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.018GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 113, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 12003053, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5054, + "index" : 44, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.616GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 110, + "resultSize" : 930, + "jvmGcTime" : 22, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 299315, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5465, + "index" : 455, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.521GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 108, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 318366, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5614, + "index" : 604, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.893GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 19, + "executorRunTime" : 108, + "resultSize" : 930, + "jvmGcTime" : 7, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 309135, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5862, + "index" : 852, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.509GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 96, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 1052228, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5566, + "index" : 556, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.762GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 95, + "resultSize" : 930, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 327089, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5567, + "index" : 557, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.768GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 94, + "resultSize" : 930, + "jvmGcTime" : 6, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 374043, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5191, + "index" : 181, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.885GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 90, + "resultSize" : 930, + "jvmGcTime" : 5, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 344199, + "recordsWritten" : 100 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation new file mode 100644 index 0000000000000..329065b835dd0 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation @@ -0,0 +1,481 @@ +[ { + "taskId" : 5610, + "index" : 600, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.884GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 285253, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5511, + "index" : 501, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.634GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 277358, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5027, + "index" : 17, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 289503, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5828, + "index" : 818, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.438GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 291997, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5173, + "index" : 163, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.861GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 267617, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5185, + "index" : 175, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.880GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 285928, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5553, + "index" : 543, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.719GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 298472, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5077, + "index" : 67, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.670GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9614, + "writeTime" : 259332, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5831, + "index" : 821, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:01.442GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9597, + "writeTime" : 281603, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5194, + "index" : 184, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.903GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9600, + "writeTime" : 286315, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5481, + "index" : 471, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.561GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9613, + "writeTime" : 328893, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5035, + "index" : 25, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.600GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 324374, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5134, + "index" : 124, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.782GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 279741, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5044, + "index" : 34, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.607GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 279851, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5026, + "index" : 16, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.586GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9598, + "writeTime" : 247775, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5331, + "index" : 321, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.208GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 313699, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5343, + "index" : 333, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.241GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9601, + "writeTime" : 260259, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5606, + "index" : 596, + "attempt" : 0, + "launchTime" : "2015-03-26T19:20:00.878GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9599, + "writeTime" : 310521, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5041, + "index" : 31, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.602GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 4, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9603, + "writeTime" : 255099, + "recordsWritten" : 100 + } + } +}, { + "taskId" : 5023, + "index" : 13, + "attempt" : 0, + "launchTime" : "2015-03-26T19:19:59.579GMT", + "executorId" : "", + "host" : "localhost", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 6, + "executorRunTime" : 5, + "resultSize" : 930, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "shuffleWriteMetrics" : { + "bytesWritten" : 9602, + "writeTime" : 282108, + "recordsWritten" : 100 + } + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation new file mode 100644 index 0000000000000..46f32c19abed5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation @@ -0,0 +1,15 @@ +{ + "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], + "executorDeserializeTime" : [ 0.0, 0.0, 0.0, 1.0, 3.0 ], + "executorRunTime" : [ 6.0, 6.0, 7.0, 10.0, 48.0 ], + "resultSize" : [ 930.0, 930.0, 930.0, 930.0, 930.0 ], + "jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 7.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "shuffleWriteMetrics" : { + "writeBytes" : [ 9598.0, 9599.0, 9600.0, 9601.0, 9603.0 ], + "writeRecords" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], + "writeTime" : [ 278805.0, 308809.0, 327953.0, 367487.0, 944783.0 ] + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation new file mode 100644 index 0000000000000..65b46388ced87 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation @@ -0,0 +1,15 @@ +{ + "quantiles" : [ 0.01, 0.5, 0.99 ], + "executorDeserializeTime" : [ 0.0, 0.0, 25.0 ], + "executorRunTime" : [ 5.0, 7.0, 123.0 ], + "resultSize" : [ 930.0, 930.0, 930.0 ], + "jvmGcTime" : [ 0.0, 0.0, 14.0 ], + "resultSerializationTime" : [ 0.0, 0.0, 1.0 ], + "memoryBytesSpilled" : [ 0.0, 0.0, 0.0 ], + "diskBytesSpilled" : [ 0.0, 0.0, 0.0 ], + "shuffleWriteMetrics" : { + "writeBytes" : [ 9597.0, 9600.0, 9614.0 ], + "writeRecords" : [ 100.0, 100.0, 100.0 ], + "writeTime" : [ 260991.0, 327953.0, 8980526.0 ] + } +} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation new file mode 100644 index 0000000000000..483632a3956ed --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation @@ -0,0 +1,10 @@ +[ { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation new file mode 100644 index 0000000000000..4b85690fd9199 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation @@ -0,0 +1,19 @@ +[ { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation new file mode 100644 index 0000000000000..e67f25252e934 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation @@ -0,0 +1,35 @@ +[ { + "id" : "local-1427397477963", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-03-26T19:17:57.184GMT", + "endTime" : "2015-03-26T19:20:02.949GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation new file mode 100644 index 0000000000000..6101177e4bb85 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation @@ -0,0 +1,53 @@ +[ { + "id" : "local-1427397477963", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-03-26T19:17:57.184GMT", + "endTime" : "2015-03-26T19:20:02.949GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "sparkUser" : "irashid", + "completed" : true + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "sparkUser" : "irashid", + "completed" : true + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation new file mode 100644 index 0000000000000..8878e547a7984 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation @@ -0,0 +1 @@ +[ ] \ No newline at end of file diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 new file mode 100755 index 0000000000000..4794e56d1107a --- /dev/null +++ b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 @@ -0,0 +1,88 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1422981759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981759269","Timestamp":1422981758277,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981762069,"Completion Time":1422981762637,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981763578,"Completion Time":1422981764005,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764014,"Completion Time":1422981764045,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764396,"Completion Time":1422981764648,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981765026,"Completion Time":1422981765050,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} diff --git a/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 new file mode 100755 index 0000000000000..f14a000bf2c28 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 @@ -0,0 +1,82 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57969","spark.repl.class.uri":"http://192.168.1.103:57968","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57970","spark.tachyonStore.folderName":"spark-3f19daee-844c-41d0-a3fc-5e3e508f9731","spark.app.id":"local-1422981780767"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981780767","Timestamp":1422981779720,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981784228,"Completion Time":1422981784819,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981785829,"Completion Time":1422981786286,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981786296,"Completion Time":1422981786347,"Failure Reason":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Stack Trace":[{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages","File Name":"DAGScheduler.scala","Line Number":1214},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1203},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"scala.collection.mutable.ResizableArray$class","Method Name":"foreach","File Name":"ResizableArray.scala","Line Number":59},{"Declaring Class":"scala.collection.mutable.ArrayBuffer","Method Name":"foreach","File Name":"ArrayBuffer.scala","Line Number":47},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"abortStage","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"scala.Option","Method Name":"foreach","File Name":"Option.scala","Line Number":236},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"handleTaskSetFailed","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2","Method Name":"applyOrElse","File Name":"DAGScheduler.scala","Line Number":1420},{"Declaring Class":"akka.actor.Actor$class","Method Name":"aroundReceive","File Name":"Actor.scala","Line Number":465},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor","Method Name":"aroundReceive","File Name":"DAGScheduler.scala","Line Number":1375},{"Declaring Class":"akka.actor.ActorCell","Method Name":"receiveMessage","File Name":"ActorCell.scala","Line Number":516},{"Declaring Class":"akka.actor.ActorCell","Method Name":"invoke","File Name":"ActorCell.scala","Line Number":487},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"processMailbox","File Name":"Mailbox.scala","Line Number":238},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"run","File Name":"Mailbox.scala","Line Number":220},{"Declaring Class":"akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask","Method Name":"exec","File Name":"AbstractDispatcher.scala","Line Number":393},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinTask","Method Name":"doExec","File Name":"ForkJoinTask.java","Line Number":260},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool$WorkQueue","Method Name":"runTask","File Name":"ForkJoinPool.java","Line Number":1339},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool","Method Name":"runWorker","File Name":"ForkJoinPool.java","Line Number":1979},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinWorkerThread","Method Name":"run","File Name":"ForkJoinWorkerThread.java","Line Number":107}]}}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981787191,"Completion Time":1422981787226,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1422981788731} diff --git a/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 new file mode 100755 index 0000000000000..9745b36b09e44 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 @@ -0,0 +1,88 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1425081759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1425081759269","Timestamp":1425081758277,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081762069,"Completion Time":1425081762637,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081763578,"Completion Time":1425081764005,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764014,"Completion Time":1425081764045,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764396,"Completion Time":1425081764648,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081765026,"Completion Time":1425081765050,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1425081766912} diff --git a/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 new file mode 100755 index 0000000000000..9ef5bd5d92de5 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 @@ -0,0 +1,24 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426533911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426533910242,"User":"irashid","App Attempt ID":"1"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426533936103,"Completion Time":1426533936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1426533945177} diff --git a/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 new file mode 100755 index 0000000000000..e7043282107d3 --- /dev/null +++ b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 @@ -0,0 +1,24 @@ +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426633911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426633910242,"User":"irashid","App Attempt ID":"2"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426633936103,"Completion Time":1426633936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1426633945177} diff --git a/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 new file mode 100755 index 0000000000000..e69de29bb2d1d diff --git a/core/src/test/resources/spark-events/local-1427397477963 b/core/src/test/resources/spark-events/local-1427397477963 new file mode 100755 index 0000000000000..3bd251d48c87b --- /dev/null +++ b/core/src/test/resources/spark-events/local-1427397477963 @@ -0,0 +1,12083 @@ +{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":53827},"Maximum Memory":257918238,"Timestamp":1427397478113} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.101","spark.eventLog.enabled":"true","spark.driver.port":"53825","spark.repl.class.uri":"http://192.168.1.101:53823","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.cleaner.ttl.MAP_OUTPUT_TRACKER":"1","spark.executor.id":"","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.101:53826","spark.tachyonStore.folderName":"spark-ba9af2c0-12a3-4d07-8f0a-2aded3ba3ded","spark.app.id":"local-1427397477963"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.cleaner.ttl.MAP_OUTPUT_TRACKER=1 --class org.apache.spark.repl.Main spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/github/spark/tools/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-continuation-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/yarn/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-http-8.1.14.v20131031.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/launcher/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/streaming/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/sql/hive/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/guava-14.0.1.jar":"System Classpath","/Users/irashid/github/spark/sql/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/repl/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/sql/hive-thriftserver/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/bagel/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-security-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-server-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/sql/catalyst/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-servlet-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-io-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/mllib/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-plus-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/github/spark/graphx/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-6.1.26.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1427397477963","Timestamp":1427397477184,"User":"irashid"} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1427397514437,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Accumulables":[]}],"Stage IDs":[0,1]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2709542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1683487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":273,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2220286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":270,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1664784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":261,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2178902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":274,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4718634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":269,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1786420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":430298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419405,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":433438,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514996,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":91,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":479137,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":79,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":478808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":103,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":525454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":101,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":593414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":112,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":524832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":504502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1349002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":545262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":72,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":561286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":74,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":492272,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":511744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":590793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":712959,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1133685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":647955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":65,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":487315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1173343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":542166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":17403787,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":870321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":859166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":711567,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":506890,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":58,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1552336,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":502961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":713958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":775901,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":77,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":852701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":98,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4651370,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1798770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":411722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":888369,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":525960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1901209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":812239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1742885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":486771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1036134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":733581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":857448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822807,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":597860,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1682586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1918631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":570824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":730395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":885679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1427397515394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":701822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":724051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":99,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1235948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":100,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1249286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1628301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1427397515414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1319579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1427397515435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":84,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":940133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1427397515439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1161930,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1427397515445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":4875074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1427397515452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1422810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1427397515454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1311510,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1427397515459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1180217,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1427397515462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1427397515394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1136518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1427397515486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1427397515414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":774329,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1427397515491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1427397515435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1443592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1427397515503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1427397515445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":21293788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1427397515510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1427397515439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":68,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1488019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1427397515512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1427397515462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1596086,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1427397515521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1427397515452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1645424,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1427397515522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1427397515459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1552735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1427397515526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1427397515454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3022416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1427397515535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1427397515486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":878266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1427397515550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1427397515510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":745599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1427397515552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1427397515491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1183846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1427397515553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1427397515503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1082699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1427397515558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1427397515512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1656735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1427397515571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1427397515521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":705517,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1427397515577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1427397515526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":588365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1427397515580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1427397515522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1437387,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1427397515585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1427397515535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2067069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1427397515595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1427397515550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2560008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":100,"Attempt":0,"Launch Time":1427397515600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1427397515553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1545777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":101,"Attempt":0,"Launch Time":1427397515608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1427397515558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":643678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":102,"Attempt":0,"Launch Time":1427397515628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1427397515571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1091283,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":103,"Attempt":0,"Launch Time":1427397515631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1427397515580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1737597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":104,"Attempt":0,"Launch Time":1427397515637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1427397515552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1004616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":105,"Attempt":0,"Launch Time":1427397515639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1427397515585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":5664832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":106,"Attempt":0,"Launch Time":1427397515649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1427397515577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":835020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":107,"Attempt":0,"Launch Time":1427397515655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":101,"Attempt":0,"Launch Time":1427397515608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2692928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":108,"Attempt":0,"Launch Time":1427397515665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":102,"Attempt":0,"Launch Time":1427397515628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":806822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":109,"Attempt":0,"Launch Time":1427397515667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":104,"Attempt":0,"Launch Time":1427397515637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":629448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":110,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":105,"Attempt":0,"Launch Time":1427397515639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1592879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":111,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":100,"Attempt":0,"Launch Time":1427397515600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":79,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5717688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":112,"Attempt":0,"Launch Time":1427397515683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":103,"Attempt":0,"Launch Time":1427397515631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2110758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":113,"Attempt":0,"Launch Time":1427397515686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1427397515595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2827506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":114,"Attempt":0,"Launch Time":1427397515689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":106,"Attempt":0,"Launch Time":1427397515649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":993259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":115,"Attempt":0,"Launch Time":1427397515703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":107,"Attempt":0,"Launch Time":1427397515655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":901718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":116,"Attempt":0,"Launch Time":1427397515712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":111,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":665220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":117,"Attempt":0,"Launch Time":1427397515719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":109,"Attempt":0,"Launch Time":1427397515667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2188354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":118,"Attempt":0,"Launch Time":1427397515720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":108,"Attempt":0,"Launch Time":1427397515665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515721,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1868304,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":119,"Attempt":0,"Launch Time":1427397515723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":113,"Attempt":0,"Launch Time":1427397515686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1575183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":120,"Attempt":0,"Launch Time":1427397515736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":114,"Attempt":0,"Launch Time":1427397515689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":824387,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":121,"Attempt":0,"Launch Time":1427397515742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":112,"Attempt":0,"Launch Time":1427397515683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":919762,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":122,"Attempt":0,"Launch Time":1427397515756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":117,"Attempt":0,"Launch Time":1427397515719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1330006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":123,"Attempt":0,"Launch Time":1427397515762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":115,"Attempt":0,"Launch Time":1427397515703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1796871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":124,"Attempt":0,"Launch Time":1427397515766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":110,"Attempt":0,"Launch Time":1427397515680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":84,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1387335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":125,"Attempt":0,"Launch Time":1427397515778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":116,"Attempt":0,"Launch Time":1427397515712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1125262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":126,"Attempt":0,"Launch Time":1427397515785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":120,"Attempt":0,"Launch Time":1427397515736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":632892,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":127,"Attempt":0,"Launch Time":1427397515795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":118,"Attempt":0,"Launch Time":1427397515720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1530208,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":128,"Attempt":0,"Launch Time":1427397515796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":119,"Attempt":0,"Launch Time":1427397515723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2231579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":129,"Attempt":0,"Launch Time":1427397515802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":123,"Attempt":0,"Launch Time":1427397515762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1211775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":130,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":121,"Attempt":0,"Launch Time":1427397515742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1650668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":131,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":124,"Attempt":0,"Launch Time":1427397515766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":654729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":132,"Attempt":0,"Launch Time":1427397515819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":122,"Attempt":0,"Launch Time":1427397515756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":626171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":133,"Attempt":0,"Launch Time":1427397515828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":125,"Attempt":0,"Launch Time":1427397515778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":788072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":134,"Attempt":0,"Launch Time":1427397515833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":127,"Attempt":0,"Launch Time":1427397515795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1102978,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":135,"Attempt":0,"Launch Time":1427397515836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":128,"Attempt":0,"Launch Time":1427397515796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":742957,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":136,"Attempt":0,"Launch Time":1427397515843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":126,"Attempt":0,"Launch Time":1427397515785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1350040,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":137,"Attempt":0,"Launch Time":1427397515860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":132,"Attempt":0,"Launch Time":1427397515819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":565467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":138,"Attempt":0,"Launch Time":1427397515862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":131,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2514923,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":139,"Attempt":0,"Launch Time":1427397515871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":133,"Attempt":0,"Launch Time":1427397515828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":762398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":140,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":141,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":130,"Attempt":0,"Launch Time":1427397515804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":136,"Attempt":0,"Launch Time":1427397515843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1970360,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":142,"Attempt":0,"Launch Time":1427397515879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":129,"Attempt":0,"Launch Time":1427397515802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1143246,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":143,"Attempt":0,"Launch Time":1427397515899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":134,"Attempt":0,"Launch Time":1427397515833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":11485852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":144,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":138,"Attempt":0,"Launch Time":1427397515862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":54,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":818784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":145,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":135,"Attempt":0,"Launch Time":1427397515836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1145231,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":146,"Attempt":0,"Launch Time":1427397515927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":142,"Attempt":0,"Launch Time":1427397515879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":456410,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":147,"Attempt":0,"Launch Time":1427397515934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":141,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":5319888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":148,"Attempt":0,"Launch Time":1427397515939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":139,"Attempt":0,"Launch Time":1427397515871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1180285,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":149,"Attempt":0,"Launch Time":1427397515940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":137,"Attempt":0,"Launch Time":1427397515860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":79,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1130140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":150,"Attempt":0,"Launch Time":1427397515955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":146,"Attempt":0,"Launch Time":1427397515927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1146878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":151,"Attempt":0,"Launch Time":1427397515968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":140,"Attempt":0,"Launch Time":1427397515873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":93,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":888469,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":152,"Attempt":0,"Launch Time":1427397515969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":143,"Attempt":0,"Launch Time":1427397515899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":67,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":554532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":153,"Attempt":0,"Launch Time":1427397515979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":150,"Attempt":0,"Launch Time":1427397515955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":880380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":154,"Attempt":0,"Launch Time":1427397515983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":145,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1803512,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":155,"Attempt":0,"Launch Time":1427397515997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":147,"Attempt":0,"Launch Time":1427397515934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":803764,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":156,"Attempt":0,"Launch Time":1427397516001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":148,"Attempt":0,"Launch Time":1427397515939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1352276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":157,"Attempt":0,"Launch Time":1427397516007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":153,"Attempt":0,"Launch Time":1427397515979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1319166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":158,"Attempt":0,"Launch Time":1427397516025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":144,"Attempt":0,"Launch Time":1427397515925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":97,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2039278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":159,"Attempt":0,"Launch Time":1427397516032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":157,"Attempt":0,"Launch Time":1427397516007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":982378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":160,"Attempt":0,"Launch Time":1427397516034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":149,"Attempt":0,"Launch Time":1427397515940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":92,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2072494,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":161,"Attempt":0,"Launch Time":1427397516046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":152,"Attempt":0,"Launch Time":1427397515969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2115526,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":162,"Attempt":0,"Launch Time":1427397516047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":151,"Attempt":0,"Launch Time":1427397515968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":76,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":802352,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":163,"Attempt":0,"Launch Time":1427397516059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":154,"Attempt":0,"Launch Time":1427397515983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3726182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":164,"Attempt":0,"Launch Time":1427397516102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":163,"Attempt":0,"Launch Time":1427397516059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3131755,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":165,"Attempt":0,"Launch Time":1427397516104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":155,"Attempt":0,"Launch Time":1427397515997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":105,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3474912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":166,"Attempt":0,"Launch Time":1427397516107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":159,"Attempt":0,"Launch Time":1427397516032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":73,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":31801636,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":167,"Attempt":0,"Launch Time":1427397516117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":160,"Attempt":0,"Launch Time":1427397516034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8899144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":168,"Attempt":0,"Launch Time":1427397516119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":158,"Attempt":0,"Launch Time":1427397516025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8724838,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":169,"Attempt":0,"Launch Time":1427397516124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":156,"Attempt":0,"Launch Time":1427397516001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":121,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":23115082,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":170,"Attempt":0,"Launch Time":1427397516135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":162,"Attempt":0,"Launch Time":1427397516047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":86,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17488871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":171,"Attempt":0,"Launch Time":1427397516137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":161,"Attempt":0,"Launch Time":1427397516046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":88,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1397930,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":172,"Attempt":0,"Launch Time":1427397516157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":169,"Attempt":0,"Launch Time":1427397516124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1286286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":173,"Attempt":0,"Launch Time":1427397516163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":168,"Attempt":0,"Launch Time":1427397516119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1164757,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":174,"Attempt":0,"Launch Time":1427397516166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":167,"Attempt":0,"Launch Time":1427397516117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1257435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":175,"Attempt":0,"Launch Time":1427397516172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":165,"Attempt":0,"Launch Time":1427397516104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":857534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":176,"Attempt":0,"Launch Time":1427397516175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":170,"Attempt":0,"Launch Time":1427397516135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1930750,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":177,"Attempt":0,"Launch Time":1427397516183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":166,"Attempt":0,"Launch Time":1427397516107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":74,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":918242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":178,"Attempt":0,"Launch Time":1427397516199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":171,"Attempt":0,"Launch Time":1427397516137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":601779,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":179,"Attempt":0,"Launch Time":1427397516200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":164,"Attempt":0,"Launch Time":1427397516102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":97,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1163035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":180,"Attempt":0,"Launch Time":1427397516212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":172,"Attempt":0,"Launch Time":1427397516157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3053113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":181,"Attempt":0,"Launch Time":1427397516221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":182,"Attempt":0,"Launch Time":1427397516222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":177,"Attempt":0,"Launch Time":1427397516183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":729369,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":175,"Attempt":0,"Launch Time":1427397516172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":862204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":183,"Attempt":0,"Launch Time":1427397516236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":173,"Attempt":0,"Launch Time":1427397516163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2386820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":184,"Attempt":0,"Launch Time":1427397516240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":174,"Attempt":0,"Launch Time":1427397516166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":807851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":185,"Attempt":0,"Launch Time":1427397516247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":178,"Attempt":0,"Launch Time":1427397516199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":758218,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":186,"Attempt":0,"Launch Time":1427397516262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":176,"Attempt":0,"Launch Time":1427397516175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":691124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":187,"Attempt":0,"Launch Time":1427397516276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":179,"Attempt":0,"Launch Time":1427397516200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":76,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1328811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":188,"Attempt":0,"Launch Time":1427397516283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":180,"Attempt":0,"Launch Time":1427397516212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":70,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":599086,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":189,"Attempt":0,"Launch Time":1427397516287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":181,"Attempt":0,"Launch Time":1427397516221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1475983,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":190,"Attempt":0,"Launch Time":1427397516299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":183,"Attempt":0,"Launch Time":1427397516236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2833401,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":191,"Attempt":0,"Launch Time":1427397516300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":184,"Attempt":0,"Launch Time":1427397516240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":58,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":739937,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":192,"Attempt":0,"Launch Time":1427397516304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":182,"Attempt":0,"Launch Time":1427397516222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1404765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":193,"Attempt":0,"Launch Time":1427397516313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":185,"Attempt":0,"Launch Time":1427397516247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":561301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":194,"Attempt":0,"Launch Time":1427397516319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":186,"Attempt":0,"Launch Time":1427397516262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1564335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":195,"Attempt":0,"Launch Time":1427397516327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":187,"Attempt":0,"Launch Time":1427397516276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":989979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":196,"Attempt":0,"Launch Time":1427397516334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":188,"Attempt":0,"Launch Time":1427397516283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1382934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":197,"Attempt":0,"Launch Time":1427397516343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":192,"Attempt":0,"Launch Time":1427397516304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":711388,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":198,"Attempt":0,"Launch Time":1427397516344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":190,"Attempt":0,"Launch Time":1427397516299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":495051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":199,"Attempt":0,"Launch Time":1427397516348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":189,"Attempt":0,"Launch Time":1427397516287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":799667,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":200,"Attempt":0,"Launch Time":1427397516360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":191,"Attempt":0,"Launch Time":1427397516300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":663667,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":201,"Attempt":0,"Launch Time":1427397516370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":193,"Attempt":0,"Launch Time":1427397516313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":895416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":202,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":194,"Attempt":0,"Launch Time":1427397516319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":995102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":203,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":199,"Attempt":0,"Launch Time":1427397516348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1007155,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":204,"Attempt":0,"Launch Time":1427397516384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":195,"Attempt":0,"Launch Time":1427397516327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2006866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":205,"Attempt":0,"Launch Time":1427397516399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":196,"Attempt":0,"Launch Time":1427397516334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":994986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":206,"Attempt":0,"Launch Time":1427397516401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":202,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":777981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":207,"Attempt":0,"Launch Time":1427397516422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":197,"Attempt":0,"Launch Time":1427397516343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":77,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":734290,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":208,"Attempt":0,"Launch Time":1427397516427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":200,"Attempt":0,"Launch Time":1427397516360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":890678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":209,"Attempt":0,"Launch Time":1427397516428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":198,"Attempt":0,"Launch Time":1427397516344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":82,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2042848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":210,"Attempt":0,"Launch Time":1427397516436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":206,"Attempt":0,"Launch Time":1427397516401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":915341,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":211,"Attempt":0,"Launch Time":1427397516439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":205,"Attempt":0,"Launch Time":1427397516399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":883846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":212,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":204,"Attempt":0,"Launch Time":1427397516384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":64,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":690408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":213,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":201,"Attempt":0,"Launch Time":1427397516370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2085369,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":214,"Attempt":0,"Launch Time":1427397516454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":203,"Attempt":0,"Launch Time":1427397516377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1600531,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":215,"Attempt":0,"Launch Time":1427397516467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":210,"Attempt":0,"Launch Time":1427397516436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":707402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":216,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":213,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":5862185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":217,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":212,"Attempt":0,"Launch Time":1427397516450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5903505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":218,"Attempt":0,"Launch Time":1427397516479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":214,"Attempt":0,"Launch Time":1427397516454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1108244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":219,"Attempt":0,"Launch Time":1427397516482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":215,"Attempt":0,"Launch Time":1427397516467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1424767,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":220,"Attempt":0,"Launch Time":1427397516497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":221,"Index":221,"Attempt":0,"Launch Time":1427397516498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":216,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2152877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":219,"Attempt":0,"Launch Time":1427397516482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1170174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":222,"Index":222,"Attempt":0,"Launch Time":1427397516502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":218,"Attempt":0,"Launch Time":1427397516479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2103594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":223,"Index":223,"Attempt":0,"Launch Time":1427397516505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":217,"Attempt":0,"Launch Time":1427397516478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2305044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":224,"Index":224,"Attempt":0,"Launch Time":1427397516516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":221,"Index":221,"Attempt":0,"Launch Time":1427397516498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1233130,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":225,"Index":225,"Attempt":0,"Launch Time":1427397516519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":222,"Index":222,"Attempt":0,"Launch Time":1427397516502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1335051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":226,"Index":226,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":220,"Attempt":0,"Launch Time":1427397516497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3418527,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":227,"Index":227,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":223,"Index":223,"Attempt":0,"Launch Time":1427397516505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":914172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":228,"Index":228,"Attempt":0,"Launch Time":1427397516529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":208,"Attempt":0,"Launch Time":1427397516427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":100,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1002051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":229,"Index":229,"Attempt":0,"Launch Time":1427397516531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":207,"Attempt":0,"Launch Time":1427397516422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":106,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3672588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":230,"Index":230,"Attempt":0,"Launch Time":1427397516558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":209,"Attempt":0,"Launch Time":1427397516428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":128,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2462354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":231,"Index":231,"Attempt":0,"Launch Time":1427397516559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":211,"Attempt":0,"Launch Time":1427397516439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":118,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3121161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":232,"Index":232,"Attempt":0,"Launch Time":1427397516565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":228,"Index":228,"Attempt":0,"Launch Time":1427397516529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1682298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":233,"Index":233,"Attempt":0,"Launch Time":1427397516567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":224,"Index":224,"Attempt":0,"Launch Time":1427397516516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":33955650,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":234,"Index":234,"Attempt":0,"Launch Time":1427397516571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":229,"Index":229,"Attempt":0,"Launch Time":1427397516531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1487266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":235,"Index":235,"Attempt":0,"Launch Time":1427397516573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":230,"Index":230,"Attempt":0,"Launch Time":1427397516558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1457248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":236,"Index":236,"Attempt":0,"Launch Time":1427397516575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":231,"Index":231,"Attempt":0,"Launch Time":1427397516559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":929760,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":237,"Index":237,"Attempt":0,"Launch Time":1427397516580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":232,"Index":232,"Attempt":0,"Launch Time":1427397516565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":976217,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":238,"Index":238,"Attempt":0,"Launch Time":1427397516587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":234,"Index":234,"Attempt":0,"Launch Time":1427397516571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1735408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":239,"Index":239,"Attempt":0,"Launch Time":1427397516589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":235,"Index":235,"Attempt":0,"Launch Time":1427397516573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1858962,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":240,"Index":240,"Attempt":0,"Launch Time":1427397516591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":236,"Index":236,"Attempt":0,"Launch Time":1427397516575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1261207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":241,"Index":241,"Attempt":0,"Launch Time":1427397516597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":237,"Index":237,"Attempt":0,"Launch Time":1427397516580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1060138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":242,"Index":242,"Attempt":0,"Launch Time":1427397516598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":233,"Index":233,"Attempt":0,"Launch Time":1427397516567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2912642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":243,"Index":243,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":244,"Index":244,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":225,"Index":225,"Attempt":0,"Launch Time":1427397516519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":82,"Result Size":930,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":40931747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":238,"Index":238,"Attempt":0,"Launch Time":1427397516587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1613089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":245,"Index":245,"Attempt":0,"Launch Time":1427397516604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":239,"Index":239,"Attempt":0,"Launch Time":1427397516589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1335202,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":246,"Index":246,"Attempt":0,"Launch Time":1427397516607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":240,"Index":240,"Attempt":0,"Launch Time":1427397516591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1712197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":247,"Index":247,"Attempt":0,"Launch Time":1427397516612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":242,"Index":242,"Attempt":0,"Launch Time":1427397516598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":893554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":248,"Index":248,"Attempt":0,"Launch Time":1427397516614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":227,"Index":227,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":60,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":18973299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":249,"Index":249,"Attempt":0,"Launch Time":1427397516616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":243,"Index":243,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1141362,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":250,"Index":250,"Attempt":0,"Launch Time":1427397516617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":245,"Index":245,"Attempt":0,"Launch Time":1427397516604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":866325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":251,"Index":251,"Attempt":0,"Launch Time":1427397516619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":241,"Index":241,"Attempt":0,"Launch Time":1427397516597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2477820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":252,"Index":252,"Attempt":0,"Launch Time":1427397516621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":246,"Index":246,"Attempt":0,"Launch Time":1427397516607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1132855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":253,"Index":253,"Attempt":0,"Launch Time":1427397516641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":247,"Index":247,"Attempt":0,"Launch Time":1427397516612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1005231,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":254,"Index":254,"Attempt":0,"Launch Time":1427397516643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":250,"Index":250,"Attempt":0,"Launch Time":1427397516617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":689612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":255,"Index":255,"Attempt":0,"Launch Time":1427397516644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":226,"Index":226,"Attempt":0,"Launch Time":1427397516522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":90,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4229102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":256,"Index":256,"Attempt":0,"Launch Time":1427397516646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":249,"Index":249,"Attempt":0,"Launch Time":1427397516616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1129918,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":257,"Index":257,"Attempt":0,"Launch Time":1427397516655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":244,"Index":244,"Attempt":0,"Launch Time":1427397516602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":38,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1062798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":258,"Index":258,"Attempt":0,"Launch Time":1427397516657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":254,"Index":254,"Attempt":0,"Launch Time":1427397516643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1053040,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":259,"Index":259,"Attempt":0,"Launch Time":1427397516662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":255,"Index":255,"Attempt":0,"Launch Time":1427397516644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1372717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":260,"Index":260,"Attempt":0,"Launch Time":1427397516665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":248,"Index":248,"Attempt":0,"Launch Time":1427397516614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":28,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4140539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":261,"Index":261,"Attempt":0,"Launch Time":1427397516670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":258,"Index":258,"Attempt":0,"Launch Time":1427397516657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":834919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":262,"Index":262,"Attempt":0,"Launch Time":1427397516672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":257,"Index":257,"Attempt":0,"Launch Time":1427397516655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1313635,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":263,"Index":263,"Attempt":0,"Launch Time":1427397516679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":259,"Index":259,"Attempt":0,"Launch Time":1427397516662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1562479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":264,"Index":264,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":260,"Index":260,"Attempt":0,"Launch Time":1427397516665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1094824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":265,"Index":265,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":261,"Index":261,"Attempt":0,"Launch Time":1427397516670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1053413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":266,"Index":266,"Attempt":0,"Launch Time":1427397516688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":262,"Index":262,"Attempt":0,"Launch Time":1427397516672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1082688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":267,"Index":267,"Attempt":0,"Launch Time":1427397516693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":263,"Index":263,"Attempt":0,"Launch Time":1427397516679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":849398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":268,"Index":268,"Attempt":0,"Launch Time":1427397516695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":265,"Index":265,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1144571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":269,"Index":269,"Attempt":0,"Launch Time":1427397516696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":264,"Index":264,"Attempt":0,"Launch Time":1427397516681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1215266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":270,"Index":270,"Attempt":0,"Launch Time":1427397516702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":266,"Index":266,"Attempt":0,"Launch Time":1427397516688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1082701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":271,"Index":271,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":267,"Index":267,"Attempt":0,"Launch Time":1427397516693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":831860,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":272,"Index":272,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":268,"Index":268,"Attempt":0,"Launch Time":1427397516695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":580898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":273,"Index":273,"Attempt":0,"Launch Time":1427397516709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":251,"Index":251,"Attempt":0,"Launch Time":1427397516619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3552935,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":274,"Index":274,"Attempt":0,"Launch Time":1427397516710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":269,"Index":269,"Attempt":0,"Launch Time":1427397516696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1251490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":275,"Index":275,"Attempt":0,"Launch Time":1427397516730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":252,"Index":252,"Attempt":0,"Launch Time":1427397516621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":107,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":44499282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":276,"Index":276,"Attempt":0,"Launch Time":1427397516731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":270,"Index":270,"Attempt":0,"Launch Time":1427397516702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1226028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":277,"Index":277,"Attempt":0,"Launch Time":1427397516734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":271,"Index":271,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":909786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":278,"Index":278,"Attempt":0,"Launch Time":1427397516735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":274,"Index":274,"Attempt":0,"Launch Time":1427397516710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1240993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":279,"Index":279,"Attempt":0,"Launch Time":1427397516736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":272,"Index":272,"Attempt":0,"Launch Time":1427397516707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":626240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":280,"Index":280,"Attempt":0,"Launch Time":1427397516741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":273,"Index":273,"Attempt":0,"Launch Time":1427397516709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2206706,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":281,"Index":281,"Attempt":0,"Launch Time":1427397516744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":276,"Index":276,"Attempt":0,"Launch Time":1427397516731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":877775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":282,"Index":282,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":278,"Index":278,"Attempt":0,"Launch Time":1427397516735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1332523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":283,"Index":283,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":277,"Index":277,"Attempt":0,"Launch Time":1427397516734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":862540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":284,"Index":284,"Attempt":0,"Launch Time":1427397516750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":279,"Index":279,"Attempt":0,"Launch Time":1427397516736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1521596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":285,"Index":285,"Attempt":0,"Launch Time":1427397516758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":281,"Index":281,"Attempt":0,"Launch Time":1427397516744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":976300,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":286,"Index":286,"Attempt":0,"Launch Time":1427397516760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":280,"Index":280,"Attempt":0,"Launch Time":1427397516741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2435944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":287,"Index":287,"Attempt":0,"Launch Time":1427397516761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":282,"Index":282,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":741457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":288,"Index":288,"Attempt":0,"Launch Time":1427397516762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":283,"Index":283,"Attempt":0,"Launch Time":1427397516748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":761089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":289,"Index":289,"Attempt":0,"Launch Time":1427397516764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":284,"Index":284,"Attempt":0,"Launch Time":1427397516750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1384071,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":290,"Index":290,"Attempt":0,"Launch Time":1427397516766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":275,"Index":275,"Attempt":0,"Launch Time":1427397516730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2674775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":291,"Index":291,"Attempt":0,"Launch Time":1427397516770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":285,"Index":285,"Attempt":0,"Launch Time":1427397516758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1468412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":292,"Index":292,"Attempt":0,"Launch Time":1427397516776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":256,"Index":256,"Attempt":0,"Launch Time":1427397516646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":128,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":82497711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":293,"Index":293,"Attempt":0,"Launch Time":1427397516777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":253,"Index":253,"Attempt":0,"Launch Time":1427397516641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":135,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":85013064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":294,"Index":294,"Attempt":0,"Launch Time":1427397516786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":290,"Index":290,"Attempt":0,"Launch Time":1427397516766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1218711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":295,"Index":295,"Attempt":0,"Launch Time":1427397516790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":292,"Index":292,"Attempt":0,"Launch Time":1427397516776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":904333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":296,"Index":296,"Attempt":0,"Launch Time":1427397516794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":293,"Index":293,"Attempt":0,"Launch Time":1427397516777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1532209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":297,"Index":297,"Attempt":0,"Launch Time":1427397516798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":294,"Index":294,"Attempt":0,"Launch Time":1427397516786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":837720,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":298,"Index":298,"Attempt":0,"Launch Time":1427397516801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":286,"Index":286,"Attempt":0,"Launch Time":1427397516760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1204841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":299,"Index":299,"Attempt":0,"Launch Time":1427397516805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":295,"Index":295,"Attempt":0,"Launch Time":1427397516790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":514610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":300,"Index":300,"Attempt":0,"Launch Time":1427397516806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":296,"Index":296,"Attempt":0,"Launch Time":1427397516794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1007479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":301,"Index":301,"Attempt":0,"Launch Time":1427397516819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":297,"Index":297,"Attempt":0,"Launch Time":1427397516798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1566994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":302,"Index":302,"Attempt":0,"Launch Time":1427397516821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":299,"Index":299,"Attempt":0,"Launch Time":1427397516805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":723564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":303,"Index":303,"Attempt":0,"Launch Time":1427397516829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":301,"Index":301,"Attempt":0,"Launch Time":1427397516819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":832310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":304,"Index":304,"Attempt":0,"Launch Time":1427397516833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":302,"Index":302,"Attempt":0,"Launch Time":1427397516821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":948880,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":305,"Index":305,"Attempt":0,"Launch Time":1427397516834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":288,"Index":288,"Attempt":0,"Launch Time":1427397516762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":836825,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":306,"Index":306,"Attempt":0,"Launch Time":1427397516835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":298,"Index":298,"Attempt":0,"Launch Time":1427397516801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":636012,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":307,"Index":307,"Attempt":0,"Launch Time":1427397516843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":308,"Index":308,"Attempt":0,"Launch Time":1427397516844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":300,"Index":300,"Attempt":0,"Launch Time":1427397516806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":604797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":303,"Index":303,"Attempt":0,"Launch Time":1427397516829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":987993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":309,"Index":309,"Attempt":0,"Launch Time":1427397516846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":304,"Index":304,"Attempt":0,"Launch Time":1427397516833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":938716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":310,"Index":310,"Attempt":0,"Launch Time":1427397516851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":306,"Index":306,"Attempt":0,"Launch Time":1427397516835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1013438,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":311,"Index":311,"Attempt":0,"Launch Time":1427397516853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":305,"Index":305,"Attempt":0,"Launch Time":1427397516834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2304832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":312,"Index":312,"Attempt":0,"Launch Time":1427397516856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":307,"Index":307,"Attempt":0,"Launch Time":1427397516843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":881106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":313,"Index":313,"Attempt":0,"Launch Time":1427397516857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":308,"Index":308,"Attempt":0,"Launch Time":1427397516844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1149081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":314,"Index":314,"Attempt":0,"Launch Time":1427397516859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":309,"Index":309,"Attempt":0,"Launch Time":1427397516846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1227905,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":315,"Index":315,"Attempt":0,"Launch Time":1427397516868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":312,"Index":312,"Attempt":0,"Launch Time":1427397516856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":818296,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":316,"Index":316,"Attempt":0,"Launch Time":1427397516869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":310,"Index":310,"Attempt":0,"Launch Time":1427397516851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1294808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":317,"Index":317,"Attempt":0,"Launch Time":1427397516871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":313,"Index":313,"Attempt":0,"Launch Time":1427397516857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1270124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":318,"Index":318,"Attempt":0,"Launch Time":1427397516872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":314,"Index":314,"Attempt":0,"Launch Time":1427397516859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1007462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":319,"Index":319,"Attempt":0,"Launch Time":1427397516881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":311,"Index":311,"Attempt":0,"Launch Time":1427397516853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2263558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":320,"Index":320,"Attempt":0,"Launch Time":1427397516882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":315,"Index":315,"Attempt":0,"Launch Time":1427397516868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516882,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822692,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":321,"Index":321,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":318,"Index":318,"Attempt":0,"Launch Time":1427397516872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":865866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":322,"Index":322,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":316,"Index":316,"Attempt":0,"Launch Time":1427397516869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":625382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":317,"Index":317,"Attempt":0,"Launch Time":1427397516871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":820572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":323,"Index":323,"Attempt":0,"Launch Time":1427397516886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":324,"Index":324,"Attempt":0,"Launch Time":1427397516901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":289,"Index":289,"Attempt":0,"Launch Time":1427397516764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":71692999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":325,"Index":325,"Attempt":0,"Launch Time":1427397516914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":324,"Index":324,"Attempt":0,"Launch Time":1427397516901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":849191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":326,"Index":326,"Attempt":0,"Launch Time":1427397516919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":319,"Index":319,"Attempt":0,"Launch Time":1427397516881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":31,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":970277,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":327,"Index":327,"Attempt":0,"Launch Time":1427397516926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":325,"Index":325,"Attempt":0,"Launch Time":1427397516914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1158253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":328,"Index":328,"Attempt":0,"Launch Time":1427397516930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":326,"Index":326,"Attempt":0,"Launch Time":1427397516919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":861117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":329,"Index":329,"Attempt":0,"Launch Time":1427397516938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":327,"Index":327,"Attempt":0,"Launch Time":1427397516926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516938,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":695280,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":330,"Index":330,"Attempt":0,"Launch Time":1427397516942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":328,"Index":328,"Attempt":0,"Launch Time":1427397516930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":952902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":331,"Index":331,"Attempt":0,"Launch Time":1427397516950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":329,"Index":329,"Attempt":0,"Launch Time":1427397516938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":743062,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":332,"Index":332,"Attempt":0,"Launch Time":1427397516951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":287,"Index":287,"Attempt":0,"Launch Time":1427397516761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":189,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":61275953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":333,"Index":333,"Attempt":0,"Launch Time":1427397516952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":291,"Index":291,"Attempt":0,"Launch Time":1427397516770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":181,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2829169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":334,"Index":334,"Attempt":0,"Launch Time":1427397516957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":330,"Index":330,"Attempt":0,"Launch Time":1427397516942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1859274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":335,"Index":335,"Attempt":0,"Launch Time":1427397516965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":331,"Index":331,"Attempt":0,"Launch Time":1427397516950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1157612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":336,"Index":336,"Attempt":0,"Launch Time":1427397516966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":333,"Index":333,"Attempt":0,"Launch Time":1427397516952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":644566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":332,"Index":332,"Attempt":0,"Launch Time":1427397516951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":724475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":337,"Index":337,"Attempt":0,"Launch Time":1427397516967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":338,"Index":338,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":323,"Index":323,"Attempt":0,"Launch Time":1427397516886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":62525299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":339,"Index":339,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":334,"Index":334,"Attempt":0,"Launch Time":1427397516957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":685650,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":340,"Index":340,"Attempt":0,"Launch Time":1427397516978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":335,"Index":335,"Attempt":0,"Launch Time":1427397516965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":933222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":341,"Index":341,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":336,"Index":336,"Attempt":0,"Launch Time":1427397516966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":843684,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":342,"Index":342,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":337,"Index":337,"Attempt":0,"Launch Time":1427397516967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":1214932,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":343,"Index":343,"Attempt":0,"Launch Time":1427397516986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":338,"Index":338,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":1593390,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":344,"Index":344,"Attempt":0,"Launch Time":1427397516989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":320,"Index":320,"Attempt":0,"Launch Time":1427397516882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":95769818,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":345,"Index":345,"Attempt":0,"Launch Time":1427397516992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":339,"Index":339,"Attempt":0,"Launch Time":1427397516970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":1314676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":346,"Index":346,"Attempt":0,"Launch Time":1427397516993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":341,"Index":341,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1017580,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":347,"Index":347,"Attempt":0,"Launch Time":1427397516994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":340,"Index":340,"Attempt":0,"Launch Time":1427397516978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516995,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":1095444,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":348,"Index":348,"Attempt":0,"Launch Time":1427397516998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":342,"Index":342,"Attempt":0,"Launch Time":1427397516980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397516998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3855566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":349,"Index":349,"Attempt":0,"Launch Time":1427397517000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":343,"Index":343,"Attempt":0,"Launch Time":1427397516986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1152773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":350,"Index":350,"Attempt":0,"Launch Time":1427397517016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":345,"Index":345,"Attempt":0,"Launch Time":1427397516992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1160926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":351,"Index":351,"Attempt":0,"Launch Time":1427397517017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":348,"Index":348,"Attempt":0,"Launch Time":1427397516998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1102575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":352,"Index":352,"Attempt":0,"Launch Time":1427397517020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":344,"Index":344,"Attempt":0,"Launch Time":1427397516989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2292906,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":353,"Index":353,"Attempt":0,"Launch Time":1427397517021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":349,"Index":349,"Attempt":0,"Launch Time":1427397517000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1168034,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":354,"Index":354,"Attempt":0,"Launch Time":1427397517022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":347,"Index":347,"Attempt":0,"Launch Time":1427397516994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":807976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":355,"Index":355,"Attempt":0,"Launch Time":1427397517029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":322,"Index":322,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":142,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1187593,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":356,"Index":356,"Attempt":0,"Launch Time":1427397517031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":321,"Index":321,"Attempt":0,"Launch Time":1427397516885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2162370,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":357,"Index":357,"Attempt":0,"Launch Time":1427397517033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":346,"Index":346,"Attempt":0,"Launch Time":1427397516993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2826934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":358,"Index":358,"Attempt":0,"Launch Time":1427397517034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":353,"Index":353,"Attempt":0,"Launch Time":1427397517021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1055736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":359,"Index":359,"Attempt":0,"Launch Time":1427397517042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":352,"Index":352,"Attempt":0,"Launch Time":1427397517020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5785380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":360,"Index":360,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":361,"Index":361,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":355,"Index":355,"Attempt":0,"Launch Time":1427397517029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3505869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":358,"Index":358,"Attempt":0,"Launch Time":1427397517034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1250631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":362,"Index":362,"Attempt":0,"Launch Time":1427397517055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":356,"Index":356,"Attempt":0,"Launch Time":1427397517031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5128594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":363,"Index":363,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":361,"Index":361,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1167859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":364,"Index":364,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":360,"Index":360,"Attempt":0,"Launch Time":1427397517047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1405558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":365,"Index":365,"Attempt":0,"Launch Time":1427397517064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":359,"Index":359,"Attempt":0,"Launch Time":1427397517042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1831979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":366,"Index":366,"Attempt":0,"Launch Time":1427397517071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":362,"Index":362,"Attempt":0,"Launch Time":1427397517055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1339156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":367,"Index":367,"Attempt":0,"Launch Time":1427397517074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":363,"Index":363,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":749564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":368,"Index":368,"Attempt":0,"Launch Time":1427397517077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":364,"Index":364,"Attempt":0,"Launch Time":1427397517061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1212245,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":369,"Index":369,"Attempt":0,"Launch Time":1427397517082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":365,"Index":365,"Attempt":0,"Launch Time":1427397517064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1658156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":370,"Index":370,"Attempt":0,"Launch Time":1427397517086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":357,"Index":357,"Attempt":0,"Launch Time":1427397517033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":5310680,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":371,"Index":371,"Attempt":0,"Launch Time":1427397517087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":367,"Index":367,"Attempt":0,"Launch Time":1427397517074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1005569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":372,"Index":372,"Attempt":0,"Launch Time":1427397517088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":366,"Index":366,"Attempt":0,"Launch Time":1427397517071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1612039,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":373,"Index":373,"Attempt":0,"Launch Time":1427397517091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":368,"Index":368,"Attempt":0,"Launch Time":1427397517077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":982290,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":374,"Index":374,"Attempt":0,"Launch Time":1427397517098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":369,"Index":369,"Attempt":0,"Launch Time":1427397517082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":962960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":375,"Index":375,"Attempt":0,"Launch Time":1427397517101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":371,"Index":371,"Attempt":0,"Launch Time":1427397517087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1561476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":376,"Index":376,"Attempt":0,"Launch Time":1427397517113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":373,"Index":373,"Attempt":0,"Launch Time":1427397517091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1089926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":377,"Index":377,"Attempt":0,"Launch Time":1427397517117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":372,"Index":372,"Attempt":0,"Launch Time":1427397517088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1476305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":378,"Index":378,"Attempt":0,"Launch Time":1427397517122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":370,"Index":370,"Attempt":0,"Launch Time":1427397517086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1679617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":379,"Index":379,"Attempt":0,"Launch Time":1427397517139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":374,"Index":374,"Attempt":0,"Launch Time":1427397517098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":15480325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":380,"Index":380,"Attempt":0,"Launch Time":1427397517140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":375,"Index":375,"Attempt":0,"Launch Time":1427397517101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517140,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":14202156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":381,"Index":381,"Attempt":0,"Launch Time":1427397517141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":376,"Index":376,"Attempt":0,"Launch Time":1427397517113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":14597620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":382,"Index":382,"Attempt":0,"Launch Time":1427397517145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":377,"Index":377,"Attempt":0,"Launch Time":1427397517117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":13160317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":383,"Index":383,"Attempt":0,"Launch Time":1427397517149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":350,"Index":350,"Attempt":0,"Launch Time":1427397517016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517149,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":132,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":22313020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":351,"Index":351,"Attempt":0,"Launch Time":1427397517017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":36937750,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":384,"Index":384,"Attempt":0,"Launch Time":1427397517150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":385,"Index":385,"Attempt":0,"Launch Time":1427397517162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":386,"Index":386,"Attempt":0,"Launch Time":1427397517163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":383,"Index":383,"Attempt":0,"Launch Time":1427397517149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":732682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":384,"Index":384,"Attempt":0,"Launch Time":1427397517150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":868663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":387,"Index":387,"Attempt":0,"Launch Time":1427397517166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":354,"Index":354,"Attempt":0,"Launch Time":1427397517022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":16656179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":388,"Index":388,"Attempt":0,"Launch Time":1427397517167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":378,"Index":378,"Attempt":0,"Launch Time":1427397517122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7873571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":389,"Index":389,"Attempt":0,"Launch Time":1427397517177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":386,"Index":386,"Attempt":0,"Launch Time":1427397517163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1546840,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":390,"Index":390,"Attempt":0,"Launch Time":1427397517179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":385,"Index":385,"Attempt":0,"Launch Time":1427397517162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1098172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":391,"Index":391,"Attempt":0,"Launch Time":1427397517180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":387,"Index":387,"Attempt":0,"Launch Time":1427397517166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":972993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":392,"Index":392,"Attempt":0,"Launch Time":1427397517188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":388,"Index":388,"Attempt":0,"Launch Time":1427397517167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2451862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":393,"Index":393,"Attempt":0,"Launch Time":1427397517190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":389,"Index":389,"Attempt":0,"Launch Time":1427397517177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1191798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":394,"Index":394,"Attempt":0,"Launch Time":1427397517192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":390,"Index":390,"Attempt":0,"Launch Time":1427397517179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":807803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":395,"Index":395,"Attempt":0,"Launch Time":1427397517195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":391,"Index":391,"Attempt":0,"Launch Time":1427397517180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1708633,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":396,"Index":396,"Attempt":0,"Launch Time":1427397517204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":393,"Index":393,"Attempt":0,"Launch Time":1427397517190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1253358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":397,"Index":397,"Attempt":0,"Launch Time":1427397517205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":394,"Index":394,"Attempt":0,"Launch Time":1427397517192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":773985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":398,"Index":398,"Attempt":0,"Launch Time":1427397517207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":392,"Index":392,"Attempt":0,"Launch Time":1427397517188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1344108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":399,"Index":399,"Attempt":0,"Launch Time":1427397517208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":395,"Index":395,"Attempt":0,"Launch Time":1427397517195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1477012,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":400,"Index":400,"Attempt":0,"Launch Time":1427397517210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":380,"Index":380,"Attempt":0,"Launch Time":1427397517140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":14664841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":401,"Index":401,"Attempt":0,"Launch Time":1427397517217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":402,"Index":402,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":403,"Index":403,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":396,"Index":396,"Attempt":0,"Launch Time":1427397517204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1002641,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":399,"Index":399,"Attempt":0,"Launch Time":1427397517208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":590442,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":397,"Index":397,"Attempt":0,"Launch Time":1427397517205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":545047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":404,"Index":404,"Attempt":0,"Launch Time":1427397517232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":398,"Index":398,"Attempt":0,"Launch Time":1427397517207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1789100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":405,"Index":405,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":382,"Index":382,"Attempt":0,"Launch Time":1427397517145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":2981032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":406,"Index":406,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":400,"Index":400,"Attempt":0,"Launch Time":1427397517210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1841304,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":407,"Index":407,"Attempt":0,"Launch Time":1427397517241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":401,"Index":401,"Attempt":0,"Launch Time":1427397517217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1328705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":408,"Index":408,"Attempt":0,"Launch Time":1427397517242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":403,"Index":403,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1814322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":409,"Index":409,"Attempt":0,"Launch Time":1427397517245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":402,"Index":402,"Attempt":0,"Launch Time":1427397517228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2628000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":410,"Index":410,"Attempt":0,"Launch Time":1427397517252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":404,"Index":404,"Attempt":0,"Launch Time":1427397517232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3967154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":411,"Index":411,"Attempt":0,"Launch Time":1427397517254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":407,"Index":407,"Attempt":0,"Launch Time":1427397517241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517254,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1215156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":412,"Index":412,"Attempt":0,"Launch Time":1427397517255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":408,"Index":408,"Attempt":0,"Launch Time":1427397517242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1252701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":413,"Index":413,"Attempt":0,"Launch Time":1427397517260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":409,"Index":409,"Attempt":0,"Launch Time":1427397517245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1187345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":414,"Index":414,"Attempt":0,"Launch Time":1427397517261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":405,"Index":405,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":3154171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":415,"Index":415,"Attempt":0,"Launch Time":1427397517264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":379,"Index":379,"Attempt":0,"Launch Time":1427397517139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":122,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4896741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":416,"Index":416,"Attempt":0,"Launch Time":1427397517266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":406,"Index":406,"Attempt":0,"Launch Time":1427397517237,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1972281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":417,"Index":417,"Attempt":0,"Launch Time":1427397517267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":411,"Index":411,"Attempt":0,"Launch Time":1427397517254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1000609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":418,"Index":418,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":410,"Index":410,"Attempt":0,"Launch Time":1427397517252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1269698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":419,"Index":419,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":412,"Index":412,"Attempt":0,"Launch Time":1427397517255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":862428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":420,"Index":420,"Attempt":0,"Launch Time":1427397517278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":381,"Index":381,"Attempt":0,"Launch Time":1427397517141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517278,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":78728345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":421,"Index":421,"Attempt":0,"Launch Time":1427397517282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":415,"Index":415,"Attempt":0,"Launch Time":1427397517264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1819081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":422,"Index":422,"Attempt":0,"Launch Time":1427397517290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":420,"Index":420,"Attempt":0,"Launch Time":1427397517278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":592846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":423,"Index":423,"Attempt":0,"Launch Time":1427397517301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":421,"Index":421,"Attempt":0,"Launch Time":1427397517282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1116267,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":424,"Index":424,"Attempt":0,"Launch Time":1427397517304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":416,"Index":416,"Attempt":0,"Launch Time":1427397517266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2064986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":425,"Index":425,"Attempt":0,"Launch Time":1427397517305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":422,"Index":422,"Attempt":0,"Launch Time":1427397517290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":930744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":426,"Index":426,"Attempt":0,"Launch Time":1427397517307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":413,"Index":413,"Attempt":0,"Launch Time":1427397517260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2592970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":427,"Index":427,"Attempt":0,"Launch Time":1427397517314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":423,"Index":423,"Attempt":0,"Launch Time":1427397517301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":828874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":428,"Index":428,"Attempt":0,"Launch Time":1427397517329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":425,"Index":425,"Attempt":0,"Launch Time":1427397517305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":621117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":429,"Index":429,"Attempt":0,"Launch Time":1427397517332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":424,"Index":424,"Attempt":0,"Launch Time":1427397517304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1106143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":430,"Index":430,"Attempt":0,"Launch Time":1427397517335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":427,"Index":427,"Attempt":0,"Launch Time":1427397517314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1018339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":431,"Index":431,"Attempt":0,"Launch Time":1427397517336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":426,"Index":426,"Attempt":0,"Launch Time":1427397517307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2062339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":432,"Index":432,"Attempt":0,"Launch Time":1427397517344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":428,"Index":428,"Attempt":0,"Launch Time":1427397517329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1170879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":433,"Index":433,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":434,"Index":434,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":430,"Index":430,"Attempt":0,"Launch Time":1427397517335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1270765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":429,"Index":429,"Attempt":0,"Launch Time":1427397517332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1533391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":435,"Index":435,"Attempt":0,"Launch Time":1427397517354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":418,"Index":418,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":85,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":22223345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":436,"Index":436,"Attempt":0,"Launch Time":1427397517357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":432,"Index":432,"Attempt":0,"Launch Time":1427397517344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1123679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":437,"Index":437,"Attempt":0,"Launch Time":1427397517358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":431,"Index":431,"Attempt":0,"Launch Time":1427397517336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2475096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":438,"Index":438,"Attempt":0,"Launch Time":1427397517362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":433,"Index":433,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1086255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":439,"Index":439,"Attempt":0,"Launch Time":1427397517363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":434,"Index":434,"Attempt":0,"Launch Time":1427397517348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1953708,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":440,"Index":440,"Attempt":0,"Launch Time":1427397517372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":436,"Index":436,"Attempt":0,"Launch Time":1427397517357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1512789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":441,"Index":441,"Attempt":0,"Launch Time":1427397517373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":435,"Index":435,"Attempt":0,"Launch Time":1427397517354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4674473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":442,"Index":442,"Attempt":0,"Launch Time":1427397517375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":438,"Index":438,"Attempt":0,"Launch Time":1427397517362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":821201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":443,"Index":443,"Attempt":0,"Launch Time":1427397517376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":437,"Index":437,"Attempt":0,"Launch Time":1427397517358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1911090,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":444,"Index":444,"Attempt":0,"Launch Time":1427397517378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":439,"Index":439,"Attempt":0,"Launch Time":1427397517363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1110677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":445,"Index":445,"Attempt":0,"Launch Time":1427397517382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":419,"Index":419,"Attempt":0,"Launch Time":1427397517268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4749022,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":446,"Index":446,"Attempt":0,"Launch Time":1427397517385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":440,"Index":440,"Attempt":0,"Launch Time":1427397517372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1566548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":447,"Index":447,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":417,"Index":417,"Attempt":0,"Launch Time":1427397517267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":119,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":59794479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":442,"Index":442,"Attempt":0,"Launch Time":1427397517375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":945759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":448,"Index":448,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":449,"Index":449,"Attempt":0,"Launch Time":1427397517390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":443,"Index":443,"Attempt":0,"Launch Time":1427397517376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1451393,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":450,"Index":450,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":451,"Index":451,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":444,"Index":444,"Attempt":0,"Launch Time":1427397517378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1603745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":441,"Index":441,"Attempt":0,"Launch Time":1427397517373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2114817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":452,"Index":452,"Attempt":0,"Launch Time":1427397517396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":446,"Index":446,"Attempt":0,"Launch Time":1427397517385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1180410,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":453,"Index":453,"Attempt":0,"Launch Time":1427397517398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":414,"Index":414,"Attempt":0,"Launch Time":1427397517261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":57639582,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":454,"Index":454,"Attempt":0,"Launch Time":1427397517410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":453,"Index":453,"Attempt":0,"Launch Time":1427397517398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":896715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":455,"Index":455,"Attempt":0,"Launch Time":1427397517419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":445,"Index":445,"Attempt":0,"Launch Time":1427397517382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":759758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":456,"Index":456,"Attempt":0,"Launch Time":1427397517422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":448,"Index":448,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1277715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":457,"Index":457,"Attempt":0,"Launch Time":1427397517439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":454,"Index":454,"Attempt":0,"Launch Time":1427397517410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1485312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":458,"Index":458,"Attempt":0,"Launch Time":1427397517448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":450,"Index":450,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1569023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":459,"Index":459,"Attempt":0,"Launch Time":1427397517452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":456,"Index":456,"Attempt":0,"Launch Time":1427397517422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":923565,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":460,"Index":460,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":461,"Index":461,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":455,"Index":455,"Attempt":0,"Launch Time":1427397517419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1717503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":457,"Index":457,"Attempt":0,"Launch Time":1427397517439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1074176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":462,"Index":462,"Attempt":0,"Launch Time":1427397517462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":449,"Index":449,"Attempt":0,"Launch Time":1427397517390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1947813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":463,"Index":463,"Attempt":0,"Launch Time":1427397517466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":458,"Index":458,"Attempt":0,"Launch Time":1427397517448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2931621,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":464,"Index":464,"Attempt":0,"Launch Time":1427397517468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":459,"Index":459,"Attempt":0,"Launch Time":1427397517452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2098780,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":465,"Index":465,"Attempt":0,"Launch Time":1427397517469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":461,"Index":461,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":878672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":466,"Index":466,"Attempt":0,"Launch Time":1427397517472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":460,"Index":460,"Attempt":0,"Launch Time":1427397517453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1298778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":467,"Index":467,"Attempt":0,"Launch Time":1427397517480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":464,"Index":464,"Attempt":0,"Launch Time":1427397517468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":950545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":468,"Index":468,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":469,"Index":469,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":465,"Index":465,"Attempt":0,"Launch Time":1427397517469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1116822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":470,"Index":470,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":462,"Index":462,"Attempt":0,"Launch Time":1427397517462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3199779,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":463,"Index":463,"Attempt":0,"Launch Time":1427397517466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":1651032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":471,"Index":471,"Attempt":0,"Launch Time":1427397517485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":466,"Index":466,"Attempt":0,"Launch Time":1427397517472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1679057,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":472,"Index":472,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":468,"Index":468,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1324475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":467,"Index":467,"Attempt":0,"Launch Time":1427397517480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1127154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":473,"Index":473,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":474,"Index":474,"Attempt":0,"Launch Time":1427397517499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":471,"Index":471,"Attempt":0,"Launch Time":1427397517485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":1414588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":475,"Index":475,"Attempt":0,"Launch Time":1427397517502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":451,"Index":451,"Attempt":0,"Launch Time":1427397517392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":44143768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":476,"Index":476,"Attempt":0,"Launch Time":1427397517504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":470,"Index":470,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2894428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":477,"Index":477,"Attempt":0,"Launch Time":1427397517511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":469,"Index":469,"Attempt":0,"Launch Time":1427397517482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3328936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":478,"Index":478,"Attempt":0,"Launch Time":1427397517512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":473,"Index":473,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1347895,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":479,"Index":479,"Attempt":0,"Launch Time":1427397517513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":472,"Index":472,"Attempt":0,"Launch Time":1427397517496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1948031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":480,"Index":480,"Attempt":0,"Launch Time":1427397517516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":474,"Index":474,"Attempt":0,"Launch Time":1427397517499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2632678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":481,"Index":481,"Attempt":0,"Launch Time":1427397517531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":452,"Index":452,"Attempt":0,"Launch Time":1427397517396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":35069714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":482,"Index":482,"Attempt":0,"Launch Time":1427397517532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":447,"Index":447,"Attempt":0,"Launch Time":1427397517388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":144,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1747600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":483,"Index":483,"Attempt":0,"Launch Time":1427397517558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":477,"Index":477,"Attempt":0,"Launch Time":1427397517511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1945702,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":484,"Index":484,"Attempt":0,"Launch Time":1427397517559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":481,"Index":481,"Attempt":0,"Launch Time":1427397517531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1438399,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":485,"Index":485,"Attempt":0,"Launch Time":1427397517562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":482,"Index":482,"Attempt":0,"Launch Time":1427397517532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1777334,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":486,"Index":486,"Attempt":0,"Launch Time":1427397517564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":475,"Index":475,"Attempt":0,"Launch Time":1427397517502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3432026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":487,"Index":487,"Attempt":0,"Launch Time":1427397517566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":476,"Index":476,"Attempt":0,"Launch Time":1427397517504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":39636620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":488,"Index":488,"Attempt":0,"Launch Time":1427397517572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":484,"Index":484,"Attempt":0,"Launch Time":1427397517559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1328408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":489,"Index":489,"Attempt":0,"Launch Time":1427397517574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":485,"Index":485,"Attempt":0,"Launch Time":1427397517562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":935001,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":490,"Index":490,"Attempt":0,"Launch Time":1427397517576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":483,"Index":483,"Attempt":0,"Launch Time":1427397517558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1745492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":491,"Index":491,"Attempt":0,"Launch Time":1427397517578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":486,"Index":486,"Attempt":0,"Launch Time":1427397517564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":759501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":492,"Index":492,"Attempt":0,"Launch Time":1427397517585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":488,"Index":488,"Attempt":0,"Launch Time":1427397517572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1067417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":493,"Index":493,"Attempt":0,"Launch Time":1427397517586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":489,"Index":489,"Attempt":0,"Launch Time":1427397517574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":911402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":494,"Index":494,"Attempt":0,"Launch Time":1427397517591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":490,"Index":490,"Attempt":0,"Launch Time":1427397517576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1978609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":495,"Index":495,"Attempt":0,"Launch Time":1427397517593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":491,"Index":491,"Attempt":0,"Launch Time":1427397517578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1918613,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":496,"Index":496,"Attempt":0,"Launch Time":1427397517595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":487,"Index":487,"Attempt":0,"Launch Time":1427397517566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3093626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":497,"Index":497,"Attempt":0,"Launch Time":1427397517599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":492,"Index":492,"Attempt":0,"Launch Time":1427397517585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1353839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":498,"Index":498,"Attempt":0,"Launch Time":1427397517600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":493,"Index":493,"Attempt":0,"Launch Time":1427397517586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1277568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":499,"Index":499,"Attempt":0,"Launch Time":1427397517612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":495,"Index":495,"Attempt":0,"Launch Time":1427397517593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1271262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":500,"Index":500,"Attempt":0,"Launch Time":1427397517614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":494,"Index":494,"Attempt":0,"Launch Time":1427397517591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2791509,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":501,"Index":501,"Attempt":0,"Launch Time":1427397517616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":497,"Index":497,"Attempt":0,"Launch Time":1427397517599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2494651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":502,"Index":502,"Attempt":0,"Launch Time":1427397517617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":498,"Index":498,"Attempt":0,"Launch Time":1427397517600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1547112,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":503,"Index":503,"Attempt":0,"Launch Time":1427397517619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":480,"Index":480,"Attempt":0,"Launch Time":1427397517516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":101,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":45142493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":504,"Index":504,"Attempt":0,"Launch Time":1427397517621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":496,"Index":496,"Attempt":0,"Launch Time":1427397517595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1485327,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":505,"Index":505,"Attempt":0,"Launch Time":1427397517626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":499,"Index":499,"Attempt":0,"Launch Time":1427397517612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1637739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":506,"Index":506,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":501,"Index":501,"Attempt":0,"Launch Time":1427397517616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":925581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":507,"Index":507,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":502,"Index":502,"Attempt":0,"Launch Time":1427397517617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1169102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":508,"Index":508,"Attempt":0,"Launch Time":1427397517632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":500,"Index":500,"Attempt":0,"Launch Time":1427397517614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2860778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":509,"Index":509,"Attempt":0,"Launch Time":1427397517639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":505,"Index":505,"Attempt":0,"Launch Time":1427397517626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":979974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":510,"Index":510,"Attempt":0,"Launch Time":1427397517642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":506,"Index":506,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":922942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":511,"Index":511,"Attempt":0,"Launch Time":1427397517644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":507,"Index":507,"Attempt":0,"Launch Time":1427397517630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":821882,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":512,"Index":512,"Attempt":0,"Launch Time":1427397517688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":513,"Index":513,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":514,"Index":514,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":479,"Index":479,"Attempt":0,"Launch Time":1427397517513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":173,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":12042883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":509,"Index":509,"Attempt":0,"Launch Time":1427397517639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":579786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":478,"Index":478,"Attempt":0,"Launch Time":1427397517512,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":175,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":77089428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":515,"Index":515,"Attempt":0,"Launch Time":1427397517693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":504,"Index":504,"Attempt":0,"Launch Time":1427397517621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2167756,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":516,"Index":516,"Attempt":0,"Launch Time":1427397517696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":503,"Index":503,"Attempt":0,"Launch Time":1427397517619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":73,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2669207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":517,"Index":517,"Attempt":0,"Launch Time":1427397517702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":513,"Index":513,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":713938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":518,"Index":518,"Attempt":0,"Launch Time":1427397517703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":512,"Index":512,"Attempt":0,"Launch Time":1427397517688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1123595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":519,"Index":519,"Attempt":0,"Launch Time":1427397517704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":508,"Index":508,"Attempt":0,"Launch Time":1427397517632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2084143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":520,"Index":520,"Attempt":0,"Launch Time":1427397517708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":511,"Index":511,"Attempt":0,"Launch Time":1427397517644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2772446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":521,"Index":521,"Attempt":0,"Launch Time":1427397517710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":516,"Index":516,"Attempt":0,"Launch Time":1427397517696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1417141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":522,"Index":522,"Attempt":0,"Launch Time":1427397517712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":515,"Index":515,"Attempt":0,"Launch Time":1427397517693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1576742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":523,"Index":523,"Attempt":0,"Launch Time":1427397517713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":517,"Index":517,"Attempt":0,"Launch Time":1427397517702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":867706,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":524,"Index":524,"Attempt":0,"Launch Time":1427397517714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":518,"Index":518,"Attempt":0,"Launch Time":1427397517703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":849661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":525,"Index":525,"Attempt":0,"Launch Time":1427397517725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":521,"Index":521,"Attempt":0,"Launch Time":1427397517710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":622848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":526,"Index":526,"Attempt":0,"Launch Time":1427397517726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":523,"Index":523,"Attempt":0,"Launch Time":1427397517713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":636357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":527,"Index":527,"Attempt":0,"Launch Time":1427397517727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":524,"Index":524,"Attempt":0,"Launch Time":1427397517714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":939009,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":528,"Index":528,"Attempt":0,"Launch Time":1427397517730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":522,"Index":522,"Attempt":0,"Launch Time":1427397517712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1322126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":529,"Index":529,"Attempt":0,"Launch Time":1427397517731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":519,"Index":519,"Attempt":0,"Launch Time":1427397517704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":11354138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":530,"Index":530,"Attempt":0,"Launch Time":1427397517738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":520,"Index":520,"Attempt":0,"Launch Time":1427397517708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2552117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":531,"Index":531,"Attempt":0,"Launch Time":1427397517739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":526,"Index":526,"Attempt":0,"Launch Time":1427397517726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":724623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":532,"Index":532,"Attempt":0,"Launch Time":1427397517740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":527,"Index":527,"Attempt":0,"Launch Time":1427397517727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":533,"Index":533,"Attempt":0,"Launch Time":1427397517741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":525,"Index":525,"Attempt":0,"Launch Time":1427397517725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1728950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":534,"Index":534,"Attempt":0,"Launch Time":1427397517749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":528,"Index":528,"Attempt":0,"Launch Time":1427397517730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1537752,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":535,"Index":535,"Attempt":0,"Launch Time":1427397517752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":531,"Index":531,"Attempt":0,"Launch Time":1427397517739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":740608,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":536,"Index":536,"Attempt":0,"Launch Time":1427397517753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":529,"Index":529,"Attempt":0,"Launch Time":1427397517731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2650037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":537,"Index":537,"Attempt":0,"Launch Time":1427397517754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":533,"Index":533,"Attempt":0,"Launch Time":1427397517741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1026946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":538,"Index":538,"Attempt":0,"Launch Time":1427397517756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":532,"Index":532,"Attempt":0,"Launch Time":1427397517740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1401216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":539,"Index":539,"Attempt":0,"Launch Time":1427397517758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":530,"Index":530,"Attempt":0,"Launch Time":1427397517738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3609756,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":540,"Index":540,"Attempt":0,"Launch Time":1427397517785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":534,"Index":534,"Attempt":0,"Launch Time":1427397517749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1385805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":541,"Index":541,"Attempt":0,"Launch Time":1427397517789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":510,"Index":510,"Attempt":0,"Launch Time":1427397517642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":28,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":998406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":542,"Index":542,"Attempt":0,"Launch Time":1427397517798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":514,"Index":514,"Attempt":0,"Launch Time":1427397517689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":29675408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":543,"Index":543,"Attempt":0,"Launch Time":1427397517801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":536,"Index":536,"Attempt":0,"Launch Time":1427397517753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1845877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":544,"Index":544,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":541,"Index":541,"Attempt":0,"Launch Time":1427397517789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1460863,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":545,"Index":545,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":540,"Index":540,"Attempt":0,"Launch Time":1427397517785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1974499,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":546,"Index":546,"Attempt":0,"Launch Time":1427397517812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":542,"Index":542,"Attempt":0,"Launch Time":1427397517798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1264019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":547,"Index":547,"Attempt":0,"Launch Time":1427397517814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":537,"Index":537,"Attempt":0,"Launch Time":1427397517754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":58,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1409574,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":548,"Index":548,"Attempt":0,"Launch Time":1427397517815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":535,"Index":535,"Attempt":0,"Launch Time":1427397517752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":62,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1780476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":549,"Index":549,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":543,"Index":543,"Attempt":0,"Launch Time":1427397517801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1393992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":550,"Index":550,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":544,"Index":544,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":873682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":551,"Index":551,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":552,"Index":552,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":538,"Index":538,"Attempt":0,"Launch Time":1427397517756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":61,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1169305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":545,"Index":545,"Attempt":0,"Launch Time":1427397517804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":955737,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":553,"Index":553,"Attempt":0,"Launch Time":1427397517826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":546,"Index":546,"Attempt":0,"Launch Time":1427397517812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2221833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":554,"Index":554,"Attempt":0,"Launch Time":1427397517830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":549,"Index":549,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1045177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":555,"Index":555,"Attempt":0,"Launch Time":1427397517835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":550,"Index":550,"Attempt":0,"Launch Time":1427397517816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":966618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":556,"Index":556,"Attempt":0,"Launch Time":1427397517836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":551,"Index":551,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2494317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":557,"Index":557,"Attempt":0,"Launch Time":1427397517839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":558,"Index":558,"Attempt":0,"Launch Time":1427397517840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":553,"Index":553,"Attempt":0,"Launch Time":1427397517826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1104010,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":547,"Index":547,"Attempt":0,"Launch Time":1427397517814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3828304,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":559,"Index":559,"Attempt":0,"Launch Time":1427397517843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":552,"Index":552,"Attempt":0,"Launch Time":1427397517819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3461056,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":560,"Index":560,"Attempt":0,"Launch Time":1427397517845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":554,"Index":554,"Attempt":0,"Launch Time":1427397517830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517845,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1242374,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":561,"Index":561,"Attempt":0,"Launch Time":1427397517850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":539,"Index":539,"Attempt":0,"Launch Time":1427397517758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":92,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":45116282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":562,"Index":562,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":563,"Index":563,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":556,"Index":556,"Attempt":0,"Launch Time":1427397517836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":891521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":555,"Index":555,"Attempt":0,"Launch Time":1427397517835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1044145,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":564,"Index":564,"Attempt":0,"Launch Time":1427397517853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":557,"Index":557,"Attempt":0,"Launch Time":1427397517839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1165734,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":565,"Index":565,"Attempt":0,"Launch Time":1427397517856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":558,"Index":558,"Attempt":0,"Launch Time":1427397517840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1039595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":566,"Index":566,"Attempt":0,"Launch Time":1427397517857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":560,"Index":560,"Attempt":0,"Launch Time":1427397517845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":901364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":567,"Index":567,"Attempt":0,"Launch Time":1427397517859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":548,"Index":548,"Attempt":0,"Launch Time":1427397517815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4263226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":568,"Index":568,"Attempt":0,"Launch Time":1427397517863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":559,"Index":559,"Attempt":0,"Launch Time":1427397517843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1936077,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":569,"Index":569,"Attempt":0,"Launch Time":1427397517865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":563,"Index":563,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1132866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":570,"Index":570,"Attempt":0,"Launch Time":1427397517867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":562,"Index":562,"Attempt":0,"Launch Time":1427397517851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2376573,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":571,"Index":571,"Attempt":0,"Launch Time":1427397517868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":564,"Index":564,"Attempt":0,"Launch Time":1427397517853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":888490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":572,"Index":572,"Attempt":0,"Launch Time":1427397517879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":566,"Index":566,"Attempt":0,"Launch Time":1427397517857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":837492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":573,"Index":573,"Attempt":0,"Launch Time":1427397517884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":565,"Index":565,"Attempt":0,"Launch Time":1427397517856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517884,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1247308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":574,"Index":574,"Attempt":0,"Launch Time":1427397517887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":561,"Index":561,"Attempt":0,"Launch Time":1427397517850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2283669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":575,"Index":575,"Attempt":0,"Launch Time":1427397517893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":570,"Index":570,"Attempt":0,"Launch Time":1427397517867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1096851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":576,"Index":576,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":571,"Index":571,"Attempt":0,"Launch Time":1427397517868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1167550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":577,"Index":577,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":578,"Index":578,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":569,"Index":569,"Attempt":0,"Launch Time":1427397517865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1779674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":572,"Index":572,"Attempt":0,"Launch Time":1427397517879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":1173313,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":579,"Index":579,"Attempt":0,"Launch Time":1427397517896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":568,"Index":568,"Attempt":0,"Launch Time":1427397517863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1198971,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":580,"Index":580,"Attempt":0,"Launch Time":1427397517907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":567,"Index":567,"Attempt":0,"Launch Time":1427397517859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2599856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":581,"Index":581,"Attempt":0,"Launch Time":1427397517911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":574,"Index":574,"Attempt":0,"Launch Time":1427397517887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2396275,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":582,"Index":582,"Attempt":0,"Launch Time":1427397517922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":580,"Index":580,"Attempt":0,"Launch Time":1427397517907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":817378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":583,"Index":583,"Attempt":0,"Launch Time":1427397517930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":581,"Index":581,"Attempt":0,"Launch Time":1427397517911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4658037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":584,"Index":584,"Attempt":0,"Launch Time":1427397517937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":582,"Index":582,"Attempt":0,"Launch Time":1427397517922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":694222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":585,"Index":585,"Attempt":0,"Launch Time":1427397517939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":573,"Index":573,"Attempt":0,"Launch Time":1427397517884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2198659,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":586,"Index":586,"Attempt":0,"Launch Time":1427397517945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":583,"Index":583,"Attempt":0,"Launch Time":1427397517930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1060856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":587,"Index":587,"Attempt":0,"Launch Time":1427397517948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":579,"Index":579,"Attempt":0,"Launch Time":1427397517896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1228089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":588,"Index":588,"Attempt":0,"Launch Time":1427397517950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":584,"Index":584,"Attempt":0,"Launch Time":1427397517937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1171815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":589,"Index":589,"Attempt":0,"Launch Time":1427397517955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":585,"Index":585,"Attempt":0,"Launch Time":1427397517939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1039888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":590,"Index":590,"Attempt":0,"Launch Time":1427397517957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":586,"Index":586,"Attempt":0,"Launch Time":1427397517945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1126608,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":591,"Index":591,"Attempt":0,"Launch Time":1427397517962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":588,"Index":588,"Attempt":0,"Launch Time":1427397517950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":945936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":592,"Index":592,"Attempt":0,"Launch Time":1427397517963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":587,"Index":587,"Attempt":0,"Launch Time":1427397517948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":791497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":593,"Index":593,"Attempt":0,"Launch Time":1427397517970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":590,"Index":590,"Attempt":0,"Launch Time":1427397517957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1111792,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":594,"Index":594,"Attempt":0,"Launch Time":1427397517976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":589,"Index":589,"Attempt":0,"Launch Time":1427397517955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1326906,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":595,"Index":595,"Attempt":0,"Launch Time":1427397517977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":591,"Index":591,"Attempt":0,"Launch Time":1427397517962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1148382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":596,"Index":596,"Attempt":0,"Launch Time":1427397517979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":592,"Index":592,"Attempt":0,"Launch Time":1427397517963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":948654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":597,"Index":597,"Attempt":0,"Launch Time":1427397517984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":593,"Index":593,"Attempt":0,"Launch Time":1427397517970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1199277,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":598,"Index":598,"Attempt":0,"Launch Time":1427397517985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":577,"Index":577,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3660664,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":599,"Index":599,"Attempt":0,"Launch Time":1427397517989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":578,"Index":578,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":93,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":60387585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":600,"Index":600,"Attempt":0,"Launch Time":1427397517991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":594,"Index":594,"Attempt":0,"Launch Time":1427397517976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397517991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1896265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":601,"Index":601,"Attempt":0,"Launch Time":1427397518005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":595,"Index":595,"Attempt":0,"Launch Time":1427397517977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1705638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":602,"Index":602,"Attempt":0,"Launch Time":1427397518008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":596,"Index":596,"Attempt":0,"Launch Time":1427397517979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2170830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":603,"Index":603,"Attempt":0,"Launch Time":1427397518009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":597,"Index":597,"Attempt":0,"Launch Time":1427397517984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1182619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":604,"Index":604,"Attempt":0,"Launch Time":1427397518011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":600,"Index":600,"Attempt":0,"Launch Time":1427397517991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1296646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":598,"Index":598,"Attempt":0,"Launch Time":1427397517985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1348627,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":605,"Index":605,"Attempt":0,"Launch Time":1427397518012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":606,"Index":606,"Attempt":0,"Launch Time":1427397518017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":599,"Index":599,"Attempt":0,"Launch Time":1427397517989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2200244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":607,"Index":607,"Attempt":0,"Launch Time":1427397518020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":601,"Index":601,"Attempt":0,"Launch Time":1427397518005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1589678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":608,"Index":608,"Attempt":0,"Launch Time":1427397518023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":603,"Index":603,"Attempt":0,"Launch Time":1427397518009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1143496,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":609,"Index":609,"Attempt":0,"Launch Time":1427397518028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":575,"Index":575,"Attempt":0,"Launch Time":1427397517893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":133,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1710141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":610,"Index":610,"Attempt":0,"Launch Time":1427397518030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":576,"Index":576,"Attempt":0,"Launch Time":1427397517894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518030,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":85014522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":611,"Index":611,"Attempt":0,"Launch Time":1427397518040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":606,"Index":606,"Attempt":0,"Launch Time":1427397518017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1345139,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":612,"Index":612,"Attempt":0,"Launch Time":1427397518042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":609,"Index":609,"Attempt":0,"Launch Time":1427397518028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":890819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":613,"Index":613,"Attempt":0,"Launch Time":1427397518044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":610,"Index":610,"Attempt":0,"Launch Time":1427397518030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":614,"Index":614,"Attempt":0,"Launch Time":1427397518046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":604,"Index":604,"Attempt":0,"Launch Time":1427397518011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1579826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":615,"Index":615,"Attempt":0,"Launch Time":1427397518048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":602,"Index":602,"Attempt":0,"Launch Time":1427397518008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":26499615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":616,"Index":616,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":611,"Index":611,"Attempt":0,"Launch Time":1427397518040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1263345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":617,"Index":617,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":613,"Index":613,"Attempt":0,"Launch Time":1427397518044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":967919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":618,"Index":618,"Attempt":0,"Launch Time":1427397518059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":612,"Index":612,"Attempt":0,"Launch Time":1427397518042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1265857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":619,"Index":619,"Attempt":0,"Launch Time":1427397518069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":614,"Index":614,"Attempt":0,"Launch Time":1427397518046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2108801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":620,"Index":620,"Attempt":0,"Launch Time":1427397518071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":617,"Index":617,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":935883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":621,"Index":621,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":618,"Index":618,"Attempt":0,"Launch Time":1427397518059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":893958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":622,"Index":622,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":616,"Index":616,"Attempt":0,"Launch Time":1427397518058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1553353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":623,"Index":623,"Attempt":0,"Launch Time":1427397518082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":607,"Index":607,"Attempt":0,"Launch Time":1427397518020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":19894084,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":624,"Index":624,"Attempt":0,"Launch Time":1427397518085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":620,"Index":620,"Attempt":0,"Launch Time":1427397518071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1025996,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":625,"Index":625,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":619,"Index":619,"Attempt":0,"Launch Time":1427397518069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1406978,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":626,"Index":626,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":621,"Index":621,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1144566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":627,"Index":627,"Attempt":0,"Launch Time":1427397518089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":615,"Index":615,"Attempt":0,"Launch Time":1427397518048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1749583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":628,"Index":628,"Attempt":0,"Launch Time":1427397518090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":622,"Index":622,"Attempt":0,"Launch Time":1427397518073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":966380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":629,"Index":629,"Attempt":0,"Launch Time":1427397518098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":624,"Index":624,"Attempt":0,"Launch Time":1427397518085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1178464,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":630,"Index":630,"Attempt":0,"Launch Time":1427397518101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":626,"Index":626,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1000357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":631,"Index":631,"Attempt":0,"Launch Time":1427397518102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":625,"Index":625,"Attempt":0,"Launch Time":1427397518087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1223620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":632,"Index":632,"Attempt":0,"Launch Time":1427397518114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":628,"Index":628,"Attempt":0,"Launch Time":1427397518090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":996216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":633,"Index":633,"Attempt":0,"Launch Time":1427397518121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":623,"Index":623,"Attempt":0,"Launch Time":1427397518082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":6756900,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":629,"Index":629,"Attempt":0,"Launch Time":1427397518098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":10260216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":634,"Index":634,"Attempt":0,"Launch Time":1427397518122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":635,"Index":635,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":636,"Index":636,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":627,"Index":627,"Attempt":0,"Launch Time":1427397518089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3981548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":630,"Index":630,"Attempt":0,"Launch Time":1427397518101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1004190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":637,"Index":637,"Attempt":0,"Launch Time":1427397518131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":632,"Index":632,"Attempt":0,"Launch Time":1427397518114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1181604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":638,"Index":638,"Attempt":0,"Launch Time":1427397518133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":631,"Index":631,"Attempt":0,"Launch Time":1427397518102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1519273,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":639,"Index":639,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":636,"Index":636,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":797823,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":640,"Index":640,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":633,"Index":633,"Attempt":0,"Launch Time":1427397518121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1472339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":641,"Index":641,"Attempt":0,"Launch Time":1427397518144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":637,"Index":637,"Attempt":0,"Launch Time":1427397518131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1257855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":642,"Index":642,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":643,"Index":643,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":639,"Index":639,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1008363,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":640,"Index":640,"Attempt":0,"Launch Time":1427397518136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1081518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":644,"Index":644,"Attempt":0,"Launch Time":1427397518149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":638,"Index":638,"Attempt":0,"Launch Time":1427397518133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518149,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1243824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":645,"Index":645,"Attempt":0,"Launch Time":1427397518153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":605,"Index":605,"Attempt":0,"Launch Time":1427397518012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":141,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1971241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":646,"Index":646,"Attempt":0,"Launch Time":1427397518161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":644,"Index":644,"Attempt":0,"Launch Time":1427397518149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":815227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":647,"Index":647,"Attempt":0,"Launch Time":1427397518167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":648,"Index":648,"Attempt":0,"Launch Time":1427397518168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":608,"Index":608,"Attempt":0,"Launch Time":1427397518023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1199441,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":645,"Index":645,"Attempt":0,"Launch Time":1427397518153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":854553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":649,"Index":649,"Attempt":0,"Launch Time":1427397518173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":646,"Index":646,"Attempt":0,"Launch Time":1427397518161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":919453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":650,"Index":650,"Attempt":0,"Launch Time":1427397518185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":648,"Index":648,"Attempt":0,"Launch Time":1427397518168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2238891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":651,"Index":651,"Attempt":0,"Launch Time":1427397518186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":649,"Index":649,"Attempt":0,"Launch Time":1427397518173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":802232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":652,"Index":652,"Attempt":0,"Launch Time":1427397518188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":647,"Index":647,"Attempt":0,"Launch Time":1427397518167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1489241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":653,"Index":653,"Attempt":0,"Launch Time":1427397518197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":651,"Index":651,"Attempt":0,"Launch Time":1427397518186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":725637,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":654,"Index":654,"Attempt":0,"Launch Time":1427397518198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":650,"Index":650,"Attempt":0,"Launch Time":1427397518185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":715885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":655,"Index":655,"Attempt":0,"Launch Time":1427397518201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":652,"Index":652,"Attempt":0,"Launch Time":1427397518188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":942925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":656,"Index":656,"Attempt":0,"Launch Time":1427397518208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":653,"Index":653,"Attempt":0,"Launch Time":1427397518197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":578479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":657,"Index":657,"Attempt":0,"Launch Time":1427397518212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":654,"Index":654,"Attempt":0,"Launch Time":1427397518198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":826411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":658,"Index":658,"Attempt":0,"Launch Time":1427397518213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":634,"Index":634,"Attempt":0,"Launch Time":1427397518122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":89,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3560632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":659,"Index":659,"Attempt":0,"Launch Time":1427397518215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":655,"Index":655,"Attempt":0,"Launch Time":1427397518201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1279654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":660,"Index":660,"Attempt":0,"Launch Time":1427397518219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":656,"Index":656,"Attempt":0,"Launch Time":1427397518208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518219,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":756789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":661,"Index":661,"Attempt":0,"Launch Time":1427397518227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":659,"Index":659,"Attempt":0,"Launch Time":1427397518215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":943039,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":662,"Index":662,"Attempt":0,"Launch Time":1427397518228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":657,"Index":657,"Attempt":0,"Launch Time":1427397518212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":600657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":663,"Index":663,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":635,"Index":635,"Attempt":0,"Launch Time":1427397518123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":105,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":62925029,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":664,"Index":664,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":660,"Index":660,"Attempt":0,"Launch Time":1427397518219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":705793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":665,"Index":665,"Attempt":0,"Launch Time":1427397518247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":661,"Index":661,"Attempt":0,"Launch Time":1427397518227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518247,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":655679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":666,"Index":666,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":664,"Index":664,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":932495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":667,"Index":667,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":662,"Index":662,"Attempt":0,"Launch Time":1427397518228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17776927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":668,"Index":668,"Attempt":0,"Launch Time":1427397518262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":658,"Index":658,"Attempt":0,"Launch Time":1427397518213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1274355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":669,"Index":669,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":666,"Index":666,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":704091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":670,"Index":670,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":665,"Index":665,"Attempt":0,"Launch Time":1427397518247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":998002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":671,"Index":671,"Attempt":0,"Launch Time":1427397518272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":667,"Index":667,"Attempt":0,"Launch Time":1427397518257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":893909,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":672,"Index":672,"Attempt":0,"Launch Time":1427397518284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":663,"Index":663,"Attempt":0,"Launch Time":1427397518231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":833424,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":673,"Index":673,"Attempt":0,"Launch Time":1427397518289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":668,"Index":668,"Attempt":0,"Launch Time":1427397518262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3139482,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":674,"Index":674,"Attempt":0,"Launch Time":1427397518291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":641,"Index":641,"Attempt":0,"Launch Time":1427397518144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":146,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96445661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":675,"Index":675,"Attempt":0,"Launch Time":1427397518292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":643,"Index":643,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2798441,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":676,"Index":676,"Attempt":0,"Launch Time":1427397518298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":672,"Index":672,"Attempt":0,"Launch Time":1427397518284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":940764,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":677,"Index":677,"Attempt":0,"Launch Time":1427397518302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":673,"Index":673,"Attempt":0,"Launch Time":1427397518289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1678220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":678,"Index":678,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":675,"Index":675,"Attempt":0,"Launch Time":1427397518292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":9873260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":674,"Index":674,"Attempt":0,"Launch Time":1427397518291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":9068714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":679,"Index":679,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":680,"Index":680,"Attempt":0,"Launch Time":1427397518320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":676,"Index":676,"Attempt":0,"Launch Time":1427397518298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8198893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":681,"Index":681,"Attempt":0,"Launch Time":1427397518326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":677,"Index":677,"Attempt":0,"Launch Time":1427397518302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10627703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":682,"Index":682,"Attempt":0,"Launch Time":1427397518329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":642,"Index":642,"Attempt":0,"Launch Time":1427397518148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":3,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10668028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":683,"Index":683,"Attempt":0,"Launch Time":1427397518334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":680,"Index":680,"Attempt":0,"Launch Time":1427397518320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822562,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":684,"Index":684,"Attempt":0,"Launch Time":1427397518335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":678,"Index":678,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1101769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":685,"Index":685,"Attempt":0,"Launch Time":1427397518336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":679,"Index":679,"Attempt":0,"Launch Time":1427397518318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1655047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":686,"Index":686,"Attempt":0,"Launch Time":1427397518339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":681,"Index":681,"Attempt":0,"Launch Time":1427397518326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":869198,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":687,"Index":687,"Attempt":0,"Launch Time":1427397518345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":682,"Index":682,"Attempt":0,"Launch Time":1427397518329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":851806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":688,"Index":688,"Attempt":0,"Launch Time":1427397518346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":683,"Index":683,"Attempt":0,"Launch Time":1427397518334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":759627,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":689,"Index":689,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":685,"Index":685,"Attempt":0,"Launch Time":1427397518336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1055604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":690,"Index":690,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":684,"Index":684,"Attempt":0,"Launch Time":1427397518335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1246863,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":691,"Index":691,"Attempt":0,"Launch Time":1427397518354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":686,"Index":686,"Attempt":0,"Launch Time":1427397518339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1089776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":692,"Index":692,"Attempt":0,"Launch Time":1427397518358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":688,"Index":688,"Attempt":0,"Launch Time":1427397518346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":776475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":693,"Index":693,"Attempt":0,"Launch Time":1427397518360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":687,"Index":687,"Attempt":0,"Launch Time":1427397518345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1385121,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":694,"Index":694,"Attempt":0,"Launch Time":1427397518362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":689,"Index":689,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1052519,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":695,"Index":695,"Attempt":0,"Launch Time":1427397518375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":690,"Index":690,"Attempt":0,"Launch Time":1427397518350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":901887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":696,"Index":696,"Attempt":0,"Launch Time":1427397518376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":692,"Index":692,"Attempt":0,"Launch Time":1427397518358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":957042,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":697,"Index":697,"Attempt":0,"Launch Time":1427397518378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":691,"Index":691,"Attempt":0,"Launch Time":1427397518354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1165536,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":698,"Index":698,"Attempt":0,"Launch Time":1427397518380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":669,"Index":669,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":44998455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":699,"Index":699,"Attempt":0,"Launch Time":1427397518384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":693,"Index":693,"Attempt":0,"Launch Time":1427397518360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1261925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":700,"Index":700,"Attempt":0,"Launch Time":1427397518388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":696,"Index":696,"Attempt":0,"Launch Time":1427397518376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":917106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":701,"Index":701,"Attempt":0,"Launch Time":1427397518389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":695,"Index":695,"Attempt":0,"Launch Time":1427397518375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1332842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":702,"Index":702,"Attempt":0,"Launch Time":1427397518390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":694,"Index":694,"Attempt":0,"Launch Time":1427397518362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1789898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":703,"Index":703,"Attempt":0,"Launch Time":1427397518392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":697,"Index":697,"Attempt":0,"Launch Time":1427397518378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":1368981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":704,"Index":704,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":705,"Index":705,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":670,"Index":670,"Attempt":0,"Launch Time":1427397518271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":6364313,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":698,"Index":698,"Attempt":0,"Launch Time":1427397518380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1019458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":671,"Index":671,"Attempt":0,"Launch Time":1427397518272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4275349,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":706,"Index":706,"Attempt":0,"Launch Time":1427397518407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":707,"Index":707,"Attempt":0,"Launch Time":1427397518419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":704,"Index":704,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518419,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":782736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":708,"Index":708,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":706,"Index":706,"Attempt":0,"Launch Time":1427397518407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":950640,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":709,"Index":709,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":705,"Index":705,"Attempt":0,"Launch Time":1427397518406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518421,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":539270,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":710,"Index":710,"Attempt":0,"Launch Time":1427397518422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":699,"Index":699,"Attempt":0,"Launch Time":1427397518384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1752342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":711,"Index":711,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":707,"Index":707,"Attempt":0,"Launch Time":1427397518419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1043565,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":712,"Index":712,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":708,"Index":708,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":614836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":713,"Index":713,"Attempt":0,"Launch Time":1427397518433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":709,"Index":709,"Attempt":0,"Launch Time":1427397518420,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":728789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":714,"Index":714,"Attempt":0,"Launch Time":1427397518444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":711,"Index":711,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":777714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":715,"Index":715,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":716,"Index":716,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":713,"Index":713,"Attempt":0,"Launch Time":1427397518433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":627763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":712,"Index":712,"Attempt":0,"Launch Time":1427397518432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1426603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":717,"Index":717,"Attempt":0,"Launch Time":1427397518448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":710,"Index":710,"Attempt":0,"Launch Time":1427397518422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2362929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":718,"Index":718,"Attempt":0,"Launch Time":1427397518458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":714,"Index":714,"Attempt":0,"Launch Time":1427397518444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1277226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":719,"Index":719,"Attempt":0,"Launch Time":1427397518459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":716,"Index":716,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1138865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":720,"Index":720,"Attempt":0,"Launch Time":1427397518460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":715,"Index":715,"Attempt":0,"Launch Time":1427397518446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":804645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":721,"Index":721,"Attempt":0,"Launch Time":1427397518467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":717,"Index":717,"Attempt":0,"Launch Time":1427397518448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2278262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":722,"Index":722,"Attempt":0,"Launch Time":1427397518470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":701,"Index":701,"Attempt":0,"Launch Time":1427397518389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3457251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":723,"Index":723,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":719,"Index":719,"Attempt":0,"Launch Time":1427397518459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1142979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":724,"Index":724,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":725,"Index":725,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":718,"Index":718,"Attempt":0,"Launch Time":1427397518458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1282930,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":720,"Index":720,"Attempt":0,"Launch Time":1427397518460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1157078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":726,"Index":726,"Attempt":0,"Launch Time":1427397518476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":702,"Index":702,"Attempt":0,"Launch Time":1427397518390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":85,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1628826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":727,"Index":727,"Attempt":0,"Launch Time":1427397518499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":723,"Index":723,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1492933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":728,"Index":728,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":725,"Index":725,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":15954768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":729,"Index":729,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":724,"Index":724,"Attempt":0,"Launch Time":1427397518473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":845647,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":730,"Index":730,"Attempt":0,"Launch Time":1427397518503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":721,"Index":721,"Attempt":0,"Launch Time":1427397518467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2278855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":731,"Index":731,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":722,"Index":722,"Attempt":0,"Launch Time":1427397518470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":34,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1318500,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":732,"Index":732,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":726,"Index":726,"Attempt":0,"Launch Time":1427397518476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1911498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":733,"Index":733,"Attempt":0,"Launch Time":1427397518511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":727,"Index":727,"Attempt":0,"Launch Time":1427397518499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":748735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":734,"Index":734,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":728,"Index":728,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":934742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":735,"Index":735,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":729,"Index":729,"Attempt":0,"Launch Time":1427397518501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1185458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":736,"Index":736,"Attempt":0,"Launch Time":1427397518520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":730,"Index":730,"Attempt":0,"Launch Time":1427397518503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518520,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1838332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":737,"Index":737,"Attempt":0,"Launch Time":1427397518526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":700,"Index":700,"Attempt":0,"Launch Time":1427397518388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":124144554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":738,"Index":738,"Attempt":0,"Launch Time":1427397518529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":703,"Index":703,"Attempt":0,"Launch Time":1427397518392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":83709622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":739,"Index":739,"Attempt":0,"Launch Time":1427397518532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":732,"Index":732,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3204630,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":740,"Index":740,"Attempt":0,"Launch Time":1427397518533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":731,"Index":731,"Attempt":0,"Launch Time":1427397518509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4633871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":741,"Index":741,"Attempt":0,"Launch Time":1427397518541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":737,"Index":737,"Attempt":0,"Launch Time":1427397518526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1839177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":742,"Index":742,"Attempt":0,"Launch Time":1427397518542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":738,"Index":738,"Attempt":0,"Launch Time":1427397518529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":837096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":743,"Index":743,"Attempt":0,"Launch Time":1427397518551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":739,"Index":739,"Attempt":0,"Launch Time":1427397518532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1931678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":744,"Index":744,"Attempt":0,"Launch Time":1427397518554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":741,"Index":741,"Attempt":0,"Launch Time":1427397518541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1546163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":745,"Index":745,"Attempt":0,"Launch Time":1427397518557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":742,"Index":742,"Attempt":0,"Launch Time":1427397518542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1294777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":746,"Index":746,"Attempt":0,"Launch Time":1427397518565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":740,"Index":740,"Attempt":0,"Launch Time":1427397518533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4071116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":747,"Index":747,"Attempt":0,"Launch Time":1427397518568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":744,"Index":744,"Attempt":0,"Launch Time":1427397518554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1443843,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":748,"Index":748,"Attempt":0,"Launch Time":1427397518569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":743,"Index":743,"Attempt":0,"Launch Time":1427397518551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1339172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":749,"Index":749,"Attempt":0,"Launch Time":1427397518572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":745,"Index":745,"Attempt":0,"Launch Time":1427397518557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1124569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":750,"Index":750,"Attempt":0,"Launch Time":1427397518575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":736,"Index":736,"Attempt":0,"Launch Time":1427397518520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":30819021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":751,"Index":751,"Attempt":0,"Launch Time":1427397518585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":747,"Index":747,"Attempt":0,"Launch Time":1427397518568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1216389,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":752,"Index":752,"Attempt":0,"Launch Time":1427397518587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":746,"Index":746,"Attempt":0,"Launch Time":1427397518565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1275108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":753,"Index":753,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":754,"Index":754,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":750,"Index":750,"Attempt":0,"Launch Time":1427397518575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":915188,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":749,"Index":749,"Attempt":0,"Launch Time":1427397518572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2672594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":755,"Index":755,"Attempt":0,"Launch Time":1427397518589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":748,"Index":748,"Attempt":0,"Launch Time":1427397518569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4094700,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":756,"Index":756,"Attempt":0,"Launch Time":1427397518600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":754,"Index":754,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":750210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":757,"Index":757,"Attempt":0,"Launch Time":1427397518601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":751,"Index":751,"Attempt":0,"Launch Time":1427397518585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1109654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":758,"Index":758,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":759,"Index":759,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":752,"Index":752,"Attempt":0,"Launch Time":1427397518587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2207779,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":753,"Index":753,"Attempt":0,"Launch Time":1427397518588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":898989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":760,"Index":760,"Attempt":0,"Launch Time":1427397518605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":755,"Index":755,"Attempt":0,"Launch Time":1427397518589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1237156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":761,"Index":761,"Attempt":0,"Launch Time":1427397518619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":756,"Index":756,"Attempt":0,"Launch Time":1427397518600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":908289,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":762,"Index":762,"Attempt":0,"Launch Time":1427397518621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":733,"Index":733,"Attempt":0,"Launch Time":1427397518511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1058745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":763,"Index":763,"Attempt":0,"Launch Time":1427397518624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":758,"Index":758,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2470317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":764,"Index":764,"Attempt":0,"Launch Time":1427397518626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":757,"Index":757,"Attempt":0,"Launch Time":1427397518601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1687037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":765,"Index":765,"Attempt":0,"Launch Time":1427397518630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":761,"Index":761,"Attempt":0,"Launch Time":1427397518619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1104025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":766,"Index":766,"Attempt":0,"Launch Time":1427397518632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":760,"Index":760,"Attempt":0,"Launch Time":1427397518605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2279316,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":767,"Index":767,"Attempt":0,"Launch Time":1427397518636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":759,"Index":759,"Attempt":0,"Launch Time":1427397518602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2308802,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":768,"Index":768,"Attempt":0,"Launch Time":1427397518637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":763,"Index":763,"Attempt":0,"Launch Time":1427397518624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1582336,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":769,"Index":769,"Attempt":0,"Launch Time":1427397518641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":765,"Index":765,"Attempt":0,"Launch Time":1427397518630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":932416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":770,"Index":770,"Attempt":0,"Launch Time":1427397518642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":764,"Index":764,"Attempt":0,"Launch Time":1427397518626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1362826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":771,"Index":771,"Attempt":0,"Launch Time":1427397518650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":734,"Index":734,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":107479596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":772,"Index":772,"Attempt":0,"Launch Time":1427397518651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":735,"Index":735,"Attempt":0,"Launch Time":1427397518515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96079025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":773,"Index":773,"Attempt":0,"Launch Time":1427397518652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":766,"Index":766,"Attempt":0,"Launch Time":1427397518632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":904156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":774,"Index":774,"Attempt":0,"Launch Time":1427397518656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":770,"Index":770,"Attempt":0,"Launch Time":1427397518642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1207802,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":775,"Index":775,"Attempt":0,"Launch Time":1427397518665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":762,"Index":762,"Attempt":0,"Launch Time":1427397518621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1648965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":776,"Index":776,"Attempt":0,"Launch Time":1427397518670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":772,"Index":772,"Attempt":0,"Launch Time":1427397518651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1263199,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":777,"Index":777,"Attempt":0,"Launch Time":1427397518671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":771,"Index":771,"Attempt":0,"Launch Time":1427397518650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1350605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":778,"Index":778,"Attempt":0,"Launch Time":1427397518672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":774,"Index":774,"Attempt":0,"Launch Time":1427397518656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1097031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":779,"Index":779,"Attempt":0,"Launch Time":1427397518673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":773,"Index":773,"Attempt":0,"Launch Time":1427397518652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2225170,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":780,"Index":780,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":778,"Index":778,"Attempt":0,"Launch Time":1427397518672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1291359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":781,"Index":781,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":775,"Index":775,"Attempt":0,"Launch Time":1427397518665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1990356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":782,"Index":782,"Attempt":0,"Launch Time":1427397518688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":776,"Index":776,"Attempt":0,"Launch Time":1427397518670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1054865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":783,"Index":783,"Attempt":0,"Launch Time":1427397518693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":777,"Index":777,"Attempt":0,"Launch Time":1427397518671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1631425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":784,"Index":784,"Attempt":0,"Launch Time":1427397518699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":779,"Index":779,"Attempt":0,"Launch Time":1427397518673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2096569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":785,"Index":785,"Attempt":0,"Launch Time":1427397518700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":780,"Index":780,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":882714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":786,"Index":786,"Attempt":0,"Launch Time":1427397518706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":782,"Index":782,"Attempt":0,"Launch Time":1427397518688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2184213,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":787,"Index":787,"Attempt":0,"Launch Time":1427397518709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":781,"Index":781,"Attempt":0,"Launch Time":1427397518686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2846770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":788,"Index":788,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":785,"Index":785,"Attempt":0,"Launch Time":1427397518700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":880165,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":789,"Index":789,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":783,"Index":783,"Attempt":0,"Launch Time":1427397518693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1300604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":790,"Index":790,"Attempt":0,"Launch Time":1427397518716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":784,"Index":784,"Attempt":0,"Launch Time":1427397518699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518716,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1535742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":791,"Index":791,"Attempt":0,"Launch Time":1427397518721,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":786,"Index":786,"Attempt":0,"Launch Time":1427397518706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518721,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1538497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":792,"Index":792,"Attempt":0,"Launch Time":1427397518727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":788,"Index":788,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":945529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":793,"Index":793,"Attempt":0,"Launch Time":1427397518732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":790,"Index":790,"Attempt":0,"Launch Time":1427397518716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1405261,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":794,"Index":794,"Attempt":0,"Launch Time":1427397518733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":789,"Index":789,"Attempt":0,"Launch Time":1427397518714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1107323,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":795,"Index":795,"Attempt":0,"Launch Time":1427397518757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":793,"Index":793,"Attempt":0,"Launch Time":1427397518732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3140894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":796,"Index":796,"Attempt":0,"Launch Time":1427397518761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":768,"Index":768,"Attempt":0,"Launch Time":1427397518637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":123,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":112233137,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":797,"Index":797,"Attempt":0,"Launch Time":1427397518768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":791,"Index":791,"Attempt":0,"Launch Time":1427397518721,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":922417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":798,"Index":798,"Attempt":0,"Launch Time":1427397518769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":792,"Index":792,"Attempt":0,"Launch Time":1427397518727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1099371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":799,"Index":799,"Attempt":0,"Launch Time":1427397518774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":767,"Index":767,"Attempt":0,"Launch Time":1427397518636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":793017,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":800,"Index":800,"Attempt":0,"Launch Time":1427397518783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":787,"Index":787,"Attempt":0,"Launch Time":1427397518709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":5559239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":801,"Index":801,"Attempt":0,"Launch Time":1427397518785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":769,"Index":769,"Attempt":0,"Launch Time":1427397518641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":114225589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":802,"Index":802,"Attempt":0,"Launch Time":1427397518790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":799,"Index":799,"Attempt":0,"Launch Time":1427397518774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2083689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":803,"Index":803,"Attempt":0,"Launch Time":1427397518797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":800,"Index":800,"Attempt":0,"Launch Time":1427397518783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1595268,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":804,"Index":804,"Attempt":0,"Launch Time":1427397518802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":801,"Index":801,"Attempt":0,"Launch Time":1427397518785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1301262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":805,"Index":805,"Attempt":0,"Launch Time":1427397518806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":802,"Index":802,"Attempt":0,"Launch Time":1427397518790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1706403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":806,"Index":806,"Attempt":0,"Launch Time":1427397518808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":796,"Index":796,"Attempt":0,"Launch Time":1427397518761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4205936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":807,"Index":807,"Attempt":0,"Launch Time":1427397518810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":803,"Index":803,"Attempt":0,"Launch Time":1427397518797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1312456,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":808,"Index":808,"Attempt":0,"Launch Time":1427397518816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":804,"Index":804,"Attempt":0,"Launch Time":1427397518802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1043761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":809,"Index":809,"Attempt":0,"Launch Time":1427397518820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":805,"Index":805,"Attempt":0,"Launch Time":1427397518806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":970271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":810,"Index":810,"Attempt":0,"Launch Time":1427397518826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":807,"Index":807,"Attempt":0,"Launch Time":1427397518810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2036358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":811,"Index":811,"Attempt":0,"Launch Time":1427397518831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":808,"Index":808,"Attempt":0,"Launch Time":1427397518816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1027524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":812,"Index":812,"Attempt":0,"Launch Time":1427397518832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":806,"Index":806,"Attempt":0,"Launch Time":1427397518808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1752067,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":813,"Index":813,"Attempt":0,"Launch Time":1427397518834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":809,"Index":809,"Attempt":0,"Launch Time":1427397518820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1703839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":814,"Index":814,"Attempt":0,"Launch Time":1427397518839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":810,"Index":810,"Attempt":0,"Launch Time":1427397518826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1457884,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":815,"Index":815,"Attempt":0,"Launch Time":1427397518845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":794,"Index":794,"Attempt":0,"Launch Time":1427397518733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":55781568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":816,"Index":816,"Attempt":0,"Launch Time":1427397518846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":811,"Index":811,"Attempt":0,"Launch Time":1427397518831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1934016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":817,"Index":817,"Attempt":0,"Launch Time":1427397518848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":813,"Index":813,"Attempt":0,"Launch Time":1427397518834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1063968,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":818,"Index":818,"Attempt":0,"Launch Time":1427397518849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":812,"Index":812,"Attempt":0,"Launch Time":1427397518832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1367111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":819,"Index":819,"Attempt":0,"Launch Time":1427397518851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":814,"Index":814,"Attempt":0,"Launch Time":1427397518839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":987153,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":820,"Index":820,"Attempt":0,"Launch Time":1427397518859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":816,"Index":816,"Attempt":0,"Launch Time":1427397518846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1355285,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":821,"Index":821,"Attempt":0,"Launch Time":1427397518863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":817,"Index":817,"Attempt":0,"Launch Time":1427397518848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1200886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":822,"Index":822,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":823,"Index":823,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":819,"Index":819,"Attempt":0,"Launch Time":1427397518851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1497703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":818,"Index":818,"Attempt":0,"Launch Time":1427397518849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1508487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":824,"Index":824,"Attempt":0,"Launch Time":1427397518873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":820,"Index":820,"Attempt":0,"Launch Time":1427397518859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1799066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":825,"Index":825,"Attempt":0,"Launch Time":1427397518876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":821,"Index":821,"Attempt":0,"Launch Time":1427397518863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1195661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":826,"Index":826,"Attempt":0,"Launch Time":1427397518878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":823,"Index":823,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1152157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":827,"Index":827,"Attempt":0,"Launch Time":1427397518879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":822,"Index":822,"Attempt":0,"Launch Time":1427397518865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":950730,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":828,"Index":828,"Attempt":0,"Launch Time":1427397518881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":815,"Index":815,"Attempt":0,"Launch Time":1427397518845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4058005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":829,"Index":829,"Attempt":0,"Launch Time":1427397518892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":795,"Index":795,"Attempt":0,"Launch Time":1427397518757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1310626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":830,"Index":830,"Attempt":0,"Launch Time":1427397518901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":797,"Index":797,"Attempt":0,"Launch Time":1427397518768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":132,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":9358833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":831,"Index":831,"Attempt":0,"Launch Time":1427397518905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":829,"Index":829,"Attempt":0,"Launch Time":1427397518892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518905,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1580258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":832,"Index":832,"Attempt":0,"Launch Time":1427397518908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":798,"Index":798,"Attempt":0,"Launch Time":1427397518769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":88878597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":833,"Index":833,"Attempt":0,"Launch Time":1427397518919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":830,"Index":830,"Attempt":0,"Launch Time":1427397518901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1750894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":834,"Index":834,"Attempt":0,"Launch Time":1427397518920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":831,"Index":831,"Attempt":0,"Launch Time":1427397518905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1427789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":835,"Index":835,"Attempt":0,"Launch Time":1427397518925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":832,"Index":832,"Attempt":0,"Launch Time":1427397518908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1318879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":836,"Index":836,"Attempt":0,"Launch Time":1427397518927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":828,"Index":828,"Attempt":0,"Launch Time":1427397518881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8082551,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":837,"Index":837,"Attempt":0,"Launch Time":1427397518933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":834,"Index":834,"Attempt":0,"Launch Time":1427397518920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":993122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":838,"Index":838,"Attempt":0,"Launch Time":1427397518937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":833,"Index":833,"Attempt":0,"Launch Time":1427397518919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1927955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":839,"Index":839,"Attempt":0,"Launch Time":1427397518940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":835,"Index":835,"Attempt":0,"Launch Time":1427397518925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2263830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":840,"Index":840,"Attempt":0,"Launch Time":1427397518949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":837,"Index":837,"Attempt":0,"Launch Time":1427397518933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1182333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":841,"Index":841,"Attempt":0,"Launch Time":1427397518950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":836,"Index":836,"Attempt":0,"Launch Time":1427397518927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4335355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":842,"Index":842,"Attempt":0,"Launch Time":1427397518951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":838,"Index":838,"Attempt":0,"Launch Time":1427397518937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":954364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":843,"Index":843,"Attempt":0,"Launch Time":1427397518953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":839,"Index":839,"Attempt":0,"Launch Time":1427397518940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1201631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":844,"Index":844,"Attempt":0,"Launch Time":1427397518960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":840,"Index":840,"Attempt":0,"Launch Time":1427397518949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":906265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":845,"Index":845,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":842,"Index":842,"Attempt":0,"Launch Time":1427397518951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1738240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":841,"Index":841,"Attempt":0,"Launch Time":1427397518950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1216948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":846,"Index":846,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":847,"Index":847,"Attempt":0,"Launch Time":1427397518967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":843,"Index":843,"Attempt":0,"Launch Time":1427397518953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518967,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1280963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":848,"Index":848,"Attempt":0,"Launch Time":1427397518972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":844,"Index":844,"Attempt":0,"Launch Time":1427397518960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1296904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":849,"Index":849,"Attempt":0,"Launch Time":1427397518978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":846,"Index":846,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":867400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":850,"Index":850,"Attempt":0,"Launch Time":1427397518979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":847,"Index":847,"Attempt":0,"Launch Time":1427397518967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1175717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":851,"Index":851,"Attempt":0,"Launch Time":1427397518984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":845,"Index":845,"Attempt":0,"Launch Time":1427397518965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1023589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":852,"Index":852,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":827,"Index":827,"Attempt":0,"Launch Time":1427397518879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":61313887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":853,"Index":853,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":848,"Index":848,"Attempt":0,"Launch Time":1427397518972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":688805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":854,"Index":854,"Attempt":0,"Launch Time":1427397518993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":849,"Index":849,"Attempt":0,"Launch Time":1427397518978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1224563,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":855,"Index":855,"Attempt":0,"Launch Time":1427397518995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":850,"Index":850,"Attempt":0,"Launch Time":1427397518979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397518995,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":856,"Index":856,"Attempt":0,"Launch Time":1427397519000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":853,"Index":853,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1942802,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":857,"Index":857,"Attempt":0,"Launch Time":1427397519009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":854,"Index":854,"Attempt":0,"Launch Time":1427397518993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1441182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":858,"Index":858,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":856,"Index":856,"Attempt":0,"Launch Time":1427397519000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1974025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":859,"Index":859,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":855,"Index":855,"Attempt":0,"Launch Time":1427397518995,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3056146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":860,"Index":860,"Attempt":0,"Launch Time":1427397519021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":851,"Index":851,"Attempt":0,"Launch Time":1427397518984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":6308582,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":861,"Index":861,"Attempt":0,"Launch Time":1427397519024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":824,"Index":824,"Attempt":0,"Launch Time":1427397518873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":150,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1917666,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":862,"Index":862,"Attempt":0,"Launch Time":1427397519029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":826,"Index":826,"Attempt":0,"Launch Time":1427397518878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":150,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":965692,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":863,"Index":863,"Attempt":0,"Launch Time":1427397519045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":825,"Index":825,"Attempt":0,"Launch Time":1427397518876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":169,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":80313429,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":864,"Index":864,"Attempt":0,"Launch Time":1427397519050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":852,"Index":852,"Attempt":0,"Launch Time":1427397518985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2529511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":865,"Index":865,"Attempt":0,"Launch Time":1427397519059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":863,"Index":863,"Attempt":0,"Launch Time":1427397519045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1177505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":866,"Index":866,"Attempt":0,"Launch Time":1427397519062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":861,"Index":861,"Attempt":0,"Launch Time":1427397519024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1290395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":867,"Index":867,"Attempt":0,"Launch Time":1427397519063,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":864,"Index":864,"Attempt":0,"Launch Time":1427397519050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519063,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1113045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":868,"Index":868,"Attempt":0,"Launch Time":1427397519067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":862,"Index":862,"Attempt":0,"Launch Time":1427397519029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1950693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":869,"Index":869,"Attempt":0,"Launch Time":1427397519071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":865,"Index":865,"Attempt":0,"Launch Time":1427397519059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1117194,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":870,"Index":870,"Attempt":0,"Launch Time":1427397519074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":866,"Index":866,"Attempt":0,"Launch Time":1427397519062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1217767,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":871,"Index":871,"Attempt":0,"Launch Time":1427397519075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":860,"Index":860,"Attempt":0,"Launch Time":1427397519021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4879142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":872,"Index":872,"Attempt":0,"Launch Time":1427397519076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":867,"Index":867,"Attempt":0,"Launch Time":1427397519063,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1065814,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":873,"Index":873,"Attempt":0,"Launch Time":1427397519080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":868,"Index":868,"Attempt":0,"Launch Time":1427397519067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1742448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":874,"Index":874,"Attempt":0,"Launch Time":1427397519084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":869,"Index":869,"Attempt":0,"Launch Time":1427397519071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1087908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":875,"Index":875,"Attempt":0,"Launch Time":1427397519087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":870,"Index":870,"Attempt":0,"Launch Time":1427397519074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1590139,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":876,"Index":876,"Attempt":0,"Launch Time":1427397519097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":872,"Index":872,"Attempt":0,"Launch Time":1427397519076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":7000798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":877,"Index":877,"Attempt":0,"Launch Time":1427397519103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":873,"Index":873,"Attempt":0,"Launch Time":1427397519080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8261893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":878,"Index":878,"Attempt":0,"Launch Time":1427397519104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":874,"Index":874,"Attempt":0,"Launch Time":1427397519084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7813253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":879,"Index":879,"Attempt":0,"Launch Time":1427397519106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":875,"Index":875,"Attempt":0,"Launch Time":1427397519087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5494837,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":880,"Index":880,"Attempt":0,"Launch Time":1427397519109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":876,"Index":876,"Attempt":0,"Launch Time":1427397519097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":894716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":881,"Index":881,"Attempt":0,"Launch Time":1427397519115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":871,"Index":871,"Attempt":0,"Launch Time":1427397519075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10207502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":882,"Index":882,"Attempt":0,"Launch Time":1427397519116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":878,"Index":878,"Attempt":0,"Launch Time":1427397519104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":884149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":883,"Index":883,"Attempt":0,"Launch Time":1427397519119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":884,"Index":884,"Attempt":0,"Launch Time":1427397519122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":880,"Index":880,"Attempt":0,"Launch Time":1427397519109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":769454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":885,"Index":885,"Attempt":0,"Launch Time":1427397519130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":882,"Index":882,"Attempt":0,"Launch Time":1427397519116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":934314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":879,"Index":879,"Attempt":0,"Launch Time":1427397519106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":920887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":886,"Index":886,"Attempt":0,"Launch Time":1427397519131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":881,"Index":881,"Attempt":0,"Launch Time":1427397519115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1003615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":887,"Index":887,"Attempt":0,"Launch Time":1427397519132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":883,"Index":883,"Attempt":0,"Launch Time":1427397519119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1219642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":888,"Index":888,"Attempt":0,"Launch Time":1427397519137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":858,"Index":858,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4918663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":889,"Index":889,"Attempt":0,"Launch Time":1427397519138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":857,"Index":857,"Attempt":0,"Launch Time":1427397519009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":127,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":26559974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":890,"Index":890,"Attempt":0,"Launch Time":1427397519139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":884,"Index":884,"Attempt":0,"Launch Time":1427397519122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":795694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":891,"Index":891,"Attempt":0,"Launch Time":1427397519142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":885,"Index":885,"Attempt":0,"Launch Time":1427397519130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":879291,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":892,"Index":892,"Attempt":0,"Launch Time":1427397519144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":877,"Index":877,"Attempt":0,"Launch Time":1427397519103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":27,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1515766,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":887,"Index":887,"Attempt":0,"Launch Time":1427397519132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1184134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":893,"Index":893,"Attempt":0,"Launch Time":1427397519145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":894,"Index":894,"Attempt":0,"Launch Time":1427397519149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":859,"Index":859,"Attempt":0,"Launch Time":1427397519014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":13382710,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":895,"Index":895,"Attempt":0,"Launch Time":1427397519163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":889,"Index":889,"Attempt":0,"Launch Time":1427397519138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":922664,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":896,"Index":896,"Attempt":0,"Launch Time":1427397519165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":894,"Index":894,"Attempt":0,"Launch Time":1427397519149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":794150,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":897,"Index":897,"Attempt":0,"Launch Time":1427397519166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":888,"Index":888,"Attempt":0,"Launch Time":1427397519137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1308678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":898,"Index":898,"Attempt":0,"Launch Time":1427397519185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":895,"Index":895,"Attempt":0,"Launch Time":1427397519163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1006447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":899,"Index":899,"Attempt":0,"Launch Time":1427397519188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":896,"Index":896,"Attempt":0,"Launch Time":1427397519165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1575459,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":900,"Index":900,"Attempt":0,"Launch Time":1427397519197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":898,"Index":898,"Attempt":0,"Launch Time":1427397519185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1148129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":901,"Index":901,"Attempt":0,"Launch Time":1427397519204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":899,"Index":899,"Attempt":0,"Launch Time":1427397519188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2151250,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":902,"Index":902,"Attempt":0,"Launch Time":1427397519212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":900,"Index":900,"Attempt":0,"Launch Time":1427397519197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":906787,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":903,"Index":903,"Attempt":0,"Launch Time":1427397519217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":901,"Index":901,"Attempt":0,"Launch Time":1427397519204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1383964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":904,"Index":904,"Attempt":0,"Launch Time":1427397519221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":897,"Index":897,"Attempt":0,"Launch Time":1427397519166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1178254,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":905,"Index":905,"Attempt":0,"Launch Time":1427397519224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":893,"Index":893,"Attempt":0,"Launch Time":1427397519145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":78,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":13126730,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":906,"Index":906,"Attempt":0,"Launch Time":1427397519227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":902,"Index":902,"Attempt":0,"Launch Time":1427397519212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1890599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":907,"Index":907,"Attempt":0,"Launch Time":1427397519231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":903,"Index":903,"Attempt":0,"Launch Time":1427397519217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":1499259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":908,"Index":908,"Attempt":0,"Launch Time":1427397519233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":886,"Index":886,"Attempt":0,"Launch Time":1427397519131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":100,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":5805222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":909,"Index":909,"Attempt":0,"Launch Time":1427397519234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":904,"Index":904,"Attempt":0,"Launch Time":1427397519221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":905666,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":910,"Index":910,"Attempt":0,"Launch Time":1427397519238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":906,"Index":906,"Attempt":0,"Launch Time":1427397519227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1425421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":911,"Index":911,"Attempt":0,"Launch Time":1427397519241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":905,"Index":905,"Attempt":0,"Launch Time":1427397519224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1165764,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":912,"Index":912,"Attempt":0,"Launch Time":1427397519243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":907,"Index":907,"Attempt":0,"Launch Time":1427397519231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":710233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":913,"Index":913,"Attempt":0,"Launch Time":1427397519246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":909,"Index":909,"Attempt":0,"Launch Time":1427397519234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1029149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":914,"Index":914,"Attempt":0,"Launch Time":1427397519250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":910,"Index":910,"Attempt":0,"Launch Time":1427397519238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1440044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":915,"Index":915,"Attempt":0,"Launch Time":1427397519257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":912,"Index":912,"Attempt":0,"Launch Time":1427397519243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1749249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":916,"Index":916,"Attempt":0,"Launch Time":1427397519260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":913,"Index":913,"Attempt":0,"Launch Time":1427397519246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1174355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":917,"Index":917,"Attempt":0,"Launch Time":1427397519262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":918,"Index":918,"Attempt":0,"Launch Time":1427397519263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":911,"Index":911,"Attempt":0,"Launch Time":1427397519241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1553845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":914,"Index":914,"Attempt":0,"Launch Time":1427397519250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1317904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":919,"Index":919,"Attempt":0,"Launch Time":1427397519264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":890,"Index":890,"Attempt":0,"Launch Time":1427397519139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1963146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":920,"Index":920,"Attempt":0,"Launch Time":1427397519265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":908,"Index":908,"Attempt":0,"Launch Time":1427397519233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8876284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":921,"Index":921,"Attempt":0,"Launch Time":1427397519270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":915,"Index":915,"Attempt":0,"Launch Time":1427397519257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1215728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":922,"Index":922,"Attempt":0,"Launch Time":1427397519273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":916,"Index":916,"Attempt":0,"Launch Time":1427397519260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1076483,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":923,"Index":923,"Attempt":0,"Launch Time":1427397519278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":891,"Index":891,"Attempt":0,"Launch Time":1427397519142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":112037179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":924,"Index":924,"Attempt":0,"Launch Time":1427397519279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":892,"Index":892,"Attempt":0,"Launch Time":1427397519144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":65342858,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":925,"Index":925,"Attempt":0,"Launch Time":1427397519280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":920,"Index":920,"Attempt":0,"Launch Time":1427397519265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2541286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":926,"Index":926,"Attempt":0,"Launch Time":1427397519284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":919,"Index":919,"Attempt":0,"Launch Time":1427397519264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1072626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":927,"Index":927,"Attempt":0,"Launch Time":1427397519294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":925,"Index":925,"Attempt":0,"Launch Time":1427397519280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1202529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":928,"Index":928,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":923,"Index":923,"Attempt":0,"Launch Time":1427397519278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1798236,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":929,"Index":929,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":924,"Index":924,"Attempt":0,"Launch Time":1427397519279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1607043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":930,"Index":930,"Attempt":0,"Launch Time":1427397519304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":926,"Index":926,"Attempt":0,"Launch Time":1427397519284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1237148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":931,"Index":931,"Attempt":0,"Launch Time":1427397519312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":927,"Index":927,"Attempt":0,"Launch Time":1427397519294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1487969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":932,"Index":932,"Attempt":0,"Launch Time":1427397519314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":918,"Index":918,"Attempt":0,"Launch Time":1427397519263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":25308838,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":933,"Index":933,"Attempt":0,"Launch Time":1427397519315,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":928,"Index":928,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1097581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":934,"Index":934,"Attempt":0,"Launch Time":1427397519338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":929,"Index":929,"Attempt":0,"Launch Time":1427397519301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2645126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":935,"Index":935,"Attempt":0,"Launch Time":1427397519340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":930,"Index":930,"Attempt":0,"Launch Time":1427397519304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1505657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":936,"Index":936,"Attempt":0,"Launch Time":1427397519345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":931,"Index":931,"Attempt":0,"Launch Time":1427397519312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":854520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":937,"Index":937,"Attempt":0,"Launch Time":1427397519347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":933,"Index":933,"Attempt":0,"Launch Time":1427397519315,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1013305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":938,"Index":938,"Attempt":0,"Launch Time":1427397519354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":934,"Index":934,"Attempt":0,"Launch Time":1427397519338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1443023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":939,"Index":939,"Attempt":0,"Launch Time":1427397519355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":935,"Index":935,"Attempt":0,"Launch Time":1427397519340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1933750,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":940,"Index":940,"Attempt":0,"Launch Time":1427397519357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":932,"Index":932,"Attempt":0,"Launch Time":1427397519314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1918124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":936,"Index":936,"Attempt":0,"Launch Time":1427397519345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1195916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":941,"Index":941,"Attempt":0,"Launch Time":1427397519358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":942,"Index":942,"Attempt":0,"Launch Time":1427397519361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":937,"Index":937,"Attempt":0,"Launch Time":1427397519347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1342989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":943,"Index":943,"Attempt":0,"Launch Time":1427397519367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":938,"Index":938,"Attempt":0,"Launch Time":1427397519354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":817165,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":944,"Index":944,"Attempt":0,"Launch Time":1427397519368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":939,"Index":939,"Attempt":0,"Launch Time":1427397519355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1009118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":945,"Index":945,"Attempt":0,"Launch Time":1427397519371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":940,"Index":940,"Attempt":0,"Launch Time":1427397519357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1475243,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":946,"Index":946,"Attempt":0,"Launch Time":1427397519374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":941,"Index":941,"Attempt":0,"Launch Time":1427397519358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2299128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":947,"Index":947,"Attempt":0,"Launch Time":1427397519376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":942,"Index":942,"Attempt":0,"Launch Time":1427397519361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2450414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":948,"Index":948,"Attempt":0,"Launch Time":1427397519381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":943,"Index":943,"Attempt":0,"Launch Time":1427397519367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1585642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":949,"Index":949,"Attempt":0,"Launch Time":1427397519382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":944,"Index":944,"Attempt":0,"Launch Time":1427397519368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1093025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":950,"Index":950,"Attempt":0,"Launch Time":1427397519385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":945,"Index":945,"Attempt":0,"Launch Time":1427397519371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1184163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":951,"Index":951,"Attempt":0,"Launch Time":1427397519388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":946,"Index":946,"Attempt":0,"Launch Time":1427397519374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1777402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":952,"Index":952,"Attempt":0,"Launch Time":1427397519389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":947,"Index":947,"Attempt":0,"Launch Time":1427397519376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1305569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":953,"Index":953,"Attempt":0,"Launch Time":1427397519396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":948,"Index":948,"Attempt":0,"Launch Time":1427397519381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1084889,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":954,"Index":954,"Attempt":0,"Launch Time":1427397519398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":949,"Index":949,"Attempt":0,"Launch Time":1427397519382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1373943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":955,"Index":955,"Attempt":0,"Launch Time":1427397519401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":917,"Index":917,"Attempt":0,"Launch Time":1427397519262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1342133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":956,"Index":956,"Attempt":0,"Launch Time":1427397519404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":922,"Index":922,"Attempt":0,"Launch Time":1427397519273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5965962,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":957,"Index":957,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":921,"Index":921,"Attempt":0,"Launch Time":1427397519270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1382575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":958,"Index":958,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":956,"Index":956,"Attempt":0,"Launch Time":1427397519404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1225133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":959,"Index":959,"Attempt":0,"Launch Time":1427397519422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":955,"Index":955,"Attempt":0,"Launch Time":1427397519401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1378843,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":960,"Index":960,"Attempt":0,"Launch Time":1427397519429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":957,"Index":957,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1248427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":961,"Index":961,"Attempt":0,"Launch Time":1427397519431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":958,"Index":958,"Attempt":0,"Launch Time":1427397519417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":840158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":962,"Index":962,"Attempt":0,"Launch Time":1427397519437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":959,"Index":959,"Attempt":0,"Launch Time":1427397519422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1598225,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":963,"Index":963,"Attempt":0,"Launch Time":1427397519442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":960,"Index":960,"Attempt":0,"Launch Time":1427397519429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":882797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":964,"Index":964,"Attempt":0,"Launch Time":1427397519447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":961,"Index":961,"Attempt":0,"Launch Time":1427397519431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2003577,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":965,"Index":965,"Attempt":0,"Launch Time":1427397519453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":963,"Index":963,"Attempt":0,"Launch Time":1427397519442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1212515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":966,"Index":966,"Attempt":0,"Launch Time":1427397519454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":962,"Index":962,"Attempt":0,"Launch Time":1427397519437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1077814,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":967,"Index":967,"Attempt":0,"Launch Time":1427397519462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":964,"Index":964,"Attempt":0,"Launch Time":1427397519447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1405353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":968,"Index":968,"Attempt":0,"Launch Time":1427397519467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":965,"Index":965,"Attempt":0,"Launch Time":1427397519453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":798357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":969,"Index":969,"Attempt":0,"Launch Time":1427397519482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":950,"Index":950,"Attempt":0,"Launch Time":1427397519385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1706590,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":970,"Index":970,"Attempt":0,"Launch Time":1427397519491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":968,"Index":968,"Attempt":0,"Launch Time":1427397519467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3033800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":971,"Index":971,"Attempt":0,"Launch Time":1427397519501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":966,"Index":966,"Attempt":0,"Launch Time":1427397519454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":29708714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":972,"Index":972,"Attempt":0,"Launch Time":1427397519504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":970,"Index":970,"Attempt":0,"Launch Time":1427397519491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1158147,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":973,"Index":973,"Attempt":0,"Launch Time":1427397519508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":967,"Index":967,"Attempt":0,"Launch Time":1427397519462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1435798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":974,"Index":974,"Attempt":0,"Launch Time":1427397519514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":951,"Index":951,"Attempt":0,"Launch Time":1427397519388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":66091869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":975,"Index":975,"Attempt":0,"Launch Time":1427397519516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":972,"Index":972,"Attempt":0,"Launch Time":1427397519504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1278260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":976,"Index":976,"Attempt":0,"Launch Time":1427397519518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":971,"Index":971,"Attempt":0,"Launch Time":1427397519501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1822579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":977,"Index":977,"Attempt":0,"Launch Time":1427397519526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":952,"Index":952,"Attempt":0,"Launch Time":1427397519389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":100350714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":978,"Index":978,"Attempt":0,"Launch Time":1427397519530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":953,"Index":953,"Attempt":0,"Launch Time":1427397519396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":104511846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":979,"Index":979,"Attempt":0,"Launch Time":1427397519533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":974,"Index":974,"Attempt":0,"Launch Time":1427397519514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1387747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":980,"Index":980,"Attempt":0,"Launch Time":1427397519540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":977,"Index":977,"Attempt":0,"Launch Time":1427397519526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1978782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":981,"Index":981,"Attempt":0,"Launch Time":1427397519543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":954,"Index":954,"Attempt":0,"Launch Time":1427397519398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":96706049,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":982,"Index":982,"Attempt":0,"Launch Time":1427397519544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":978,"Index":978,"Attempt":0,"Launch Time":1427397519530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1339015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":983,"Index":983,"Attempt":0,"Launch Time":1427397519546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":979,"Index":979,"Attempt":0,"Launch Time":1427397519533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1120076,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":984,"Index":984,"Attempt":0,"Launch Time":1427397519553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":980,"Index":980,"Attempt":0,"Launch Time":1427397519540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1200804,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":985,"Index":985,"Attempt":0,"Launch Time":1427397519560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":982,"Index":982,"Attempt":0,"Launch Time":1427397519544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1810000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":986,"Index":986,"Attempt":0,"Launch Time":1427397519561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":983,"Index":983,"Attempt":0,"Launch Time":1427397519546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1767969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":987,"Index":987,"Attempt":0,"Launch Time":1427397519563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":981,"Index":981,"Attempt":0,"Launch Time":1427397519543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1171712,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":988,"Index":988,"Attempt":0,"Launch Time":1427397519567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":984,"Index":984,"Attempt":0,"Launch Time":1427397519553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1703771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":989,"Index":989,"Attempt":0,"Launch Time":1427397519569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":969,"Index":969,"Attempt":0,"Launch Time":1427397519482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":76,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2271587,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":990,"Index":990,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":985,"Index":985,"Attempt":0,"Launch Time":1427397519560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1425232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":991,"Index":991,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":986,"Index":986,"Attempt":0,"Launch Time":1427397519561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1133122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":992,"Index":992,"Attempt":0,"Launch Time":1427397519578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":987,"Index":987,"Attempt":0,"Launch Time":1427397519563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1185286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":993,"Index":993,"Attempt":0,"Launch Time":1427397519580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":988,"Index":988,"Attempt":0,"Launch Time":1427397519567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1989353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":994,"Index":994,"Attempt":0,"Launch Time":1427397519585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":991,"Index":991,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1599759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":995,"Index":995,"Attempt":0,"Launch Time":1427397519587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":990,"Index":990,"Attempt":0,"Launch Time":1427397519572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1603063,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":996,"Index":996,"Attempt":0,"Launch Time":1427397519593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":993,"Index":993,"Attempt":0,"Launch Time":1427397519580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":840060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":997,"Index":997,"Attempt":0,"Launch Time":1427397519596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":989,"Index":989,"Attempt":0,"Launch Time":1427397519569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5183705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":998,"Index":998,"Attempt":0,"Launch Time":1427397519597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":994,"Index":994,"Attempt":0,"Launch Time":1427397519585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1001529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":999,"Index":999,"Attempt":0,"Launch Time":1427397519598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":992,"Index":992,"Attempt":0,"Launch Time":1427397519578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2489437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":995,"Index":995,"Attempt":0,"Launch Time":1427397519587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1020616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":996,"Index":996,"Attempt":0,"Launch Time":1427397519593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1542373,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":997,"Index":997,"Attempt":0,"Launch Time":1427397519596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1718136,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":999,"Index":999,"Attempt":0,"Launch Time":1427397519598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1222086,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":998,"Index":998,"Attempt":0,"Launch Time":1427397519597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1240678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":973,"Index":973,"Attempt":0,"Launch Time":1427397519508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1283929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":976,"Index":976,"Attempt":0,"Launch Time":1427397519518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":102,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":69851795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":975,"Index":975,"Attempt":0,"Launch Time":1427397519516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":94564853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397514574,"Completion Time":1427397519624,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1000,"Index":0,"Attempt":0,"Launch Time":1427397519638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1000,"Index":0,"Attempt":0,"Launch Time":1427397519638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397519918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":275,"Result Size":1060,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":4,"Remote Bytes Read":0,"Local Bytes Read":192026,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at :10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.(:10)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Submission Time":1427397519638,"Completion Time":1427397519919,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1427397519923,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1427397526089,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[2,3]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":1001,"Index":0,"Attempt":0,"Launch Time":1427397526091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1001,"Index":0,"Attempt":0,"Launch Time":1427397526091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397526183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192021,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line22.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line22.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line22.$read$$iwC$$iwC$$iwC.(:39)\n$line22.$read$$iwC$$iwC.(:41)\n$line22.$read$$iwC.(:43)\n$line22.$read.(:45)\n$line22.$read$.(:49)\n$line22.$read$.()\n$line22.$eval$.(:7)\n$line22.$eval$.()\n$line22.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397526091,"Completion Time":1427397526184,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1427397526184,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1427397532114,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[5,4]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1002,"Index":0,"Attempt":0,"Launch Time":1427397532127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1003,"Index":1,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1004,"Index":2,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1005,"Index":3,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1006,"Index":4,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1007,"Index":5,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1008,"Index":6,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1009,"Index":7,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1010,"Index":8,"Attempt":0,"Launch Time":1427397532137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1004,"Index":2,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1011,"Index":9,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1012,"Index":10,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1013,"Index":11,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1014,"Index":12,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1003,"Index":1,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305159,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1006,"Index":4,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":517009,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1009,"Index":7,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1007,"Index":5,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1015,"Index":13,"Attempt":0,"Launch Time":1427397532139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1005,"Index":3,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":310558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1016,"Index":14,"Attempt":0,"Launch Time":1427397532145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1017,"Index":15,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1018,"Index":16,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1019,"Index":17,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532156,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1014,"Index":12,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1010,"Index":8,"Attempt":0,"Launch Time":1427397532137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351670,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1020,"Index":18,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532157,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1013,"Index":11,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1012,"Index":10,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1011,"Index":9,"Attempt":0,"Launch Time":1427397532138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1021,"Index":19,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1022,"Index":20,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532173,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1017,"Index":15,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1018,"Index":16,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341136,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1023,"Index":21,"Attempt":0,"Launch Time":1427397532155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1019,"Index":17,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386762,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1024,"Index":22,"Attempt":0,"Launch Time":1427397532157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1020,"Index":18,"Attempt":0,"Launch Time":1427397532146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1025,"Index":23,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1026,"Index":24,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1027,"Index":25,"Attempt":0,"Launch Time":1427397532172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1002,"Index":0,"Attempt":0,"Launch Time":1427397532127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1022,"Index":20,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1016,"Index":14,"Attempt":0,"Launch Time":1427397532145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":368695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1028,"Index":26,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1029,"Index":27,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1023,"Index":21,"Attempt":0,"Launch Time":1427397532155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":387339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1021,"Index":19,"Attempt":0,"Launch Time":1427397532154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1030,"Index":28,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1031,"Index":29,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1032,"Index":30,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1033,"Index":31,"Attempt":0,"Launch Time":1427397532182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1008,"Index":6,"Attempt":0,"Launch Time":1427397532128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1015,"Index":13,"Attempt":0,"Launch Time":1427397532139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1024,"Index":22,"Attempt":0,"Launch Time":1427397532157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459160,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1026,"Index":24,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":415900,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1034,"Index":32,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1028,"Index":26,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":311771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1035,"Index":33,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1029,"Index":27,"Attempt":0,"Launch Time":1427397532177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1036,"Index":34,"Attempt":0,"Launch Time":1427397532187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1025,"Index":23,"Attempt":0,"Launch Time":1427397532170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1037,"Index":35,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1030,"Index":28,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1038,"Index":36,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1032,"Index":30,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1039,"Index":37,"Attempt":0,"Launch Time":1427397532191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1033,"Index":31,"Attempt":0,"Launch Time":1427397532182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532192,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":309664,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1040,"Index":38,"Attempt":0,"Launch Time":1427397532195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1027,"Index":25,"Attempt":0,"Launch Time":1427397532172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1041,"Index":39,"Attempt":0,"Launch Time":1427397532197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1031,"Index":29,"Attempt":0,"Launch Time":1427397532181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":408299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1042,"Index":40,"Attempt":0,"Launch Time":1427397532198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1036,"Index":34,"Attempt":0,"Launch Time":1427397532187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364367,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1043,"Index":41,"Attempt":0,"Launch Time":1427397532201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1039,"Index":37,"Attempt":0,"Launch Time":1427397532191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":367495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1044,"Index":42,"Attempt":0,"Launch Time":1427397532202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1040,"Index":38,"Attempt":0,"Launch Time":1427397532195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532202,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1045,"Index":43,"Attempt":0,"Launch Time":1427397532204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1041,"Index":39,"Attempt":0,"Launch Time":1427397532197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532204,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318181,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1046,"Index":44,"Attempt":0,"Launch Time":1427397532205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1042,"Index":40,"Attempt":0,"Launch Time":1427397532198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282565,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1047,"Index":45,"Attempt":0,"Launch Time":1427397532209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1044,"Index":42,"Attempt":0,"Launch Time":1427397532202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1048,"Index":46,"Attempt":0,"Launch Time":1427397532210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1043,"Index":41,"Attempt":0,"Launch Time":1427397532201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1049,"Index":47,"Attempt":0,"Launch Time":1427397532213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1046,"Index":44,"Attempt":0,"Launch Time":1427397532205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1050,"Index":48,"Attempt":0,"Launch Time":1427397532215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1045,"Index":43,"Attempt":0,"Launch Time":1427397532204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1051,"Index":49,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1047,"Index":45,"Attempt":0,"Launch Time":1427397532209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1052,"Index":50,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1048,"Index":46,"Attempt":0,"Launch Time":1427397532210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":316677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1053,"Index":51,"Attempt":0,"Launch Time":1427397532221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1037,"Index":35,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1054,"Index":52,"Attempt":0,"Launch Time":1427397532223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1049,"Index":47,"Attempt":0,"Launch Time":1427397532213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360510,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1055,"Index":53,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1056,"Index":54,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1034,"Index":32,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353633,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1050,"Index":48,"Attempt":0,"Launch Time":1427397532215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347823,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1057,"Index":55,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1051,"Index":49,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379551,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1058,"Index":56,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1052,"Index":50,"Attempt":0,"Launch Time":1427397532217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1059,"Index":57,"Attempt":0,"Launch Time":1427397532230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1053,"Index":51,"Attempt":0,"Launch Time":1427397532221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1060,"Index":58,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1057,"Index":55,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1061,"Index":59,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1062,"Index":60,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1054,"Index":52,"Attempt":0,"Launch Time":1427397532223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":413134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1055,"Index":53,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328512,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1063,"Index":61,"Attempt":0,"Launch Time":1427397532235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1058,"Index":56,"Attempt":0,"Launch Time":1427397532225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532235,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377670,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1064,"Index":62,"Attempt":0,"Launch Time":1427397532236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1056,"Index":54,"Attempt":0,"Launch Time":1427397532224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532237,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1065,"Index":63,"Attempt":0,"Launch Time":1427397532239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1035,"Index":33,"Attempt":0,"Launch Time":1427397532186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":3097121,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1066,"Index":64,"Attempt":0,"Launch Time":1427397532248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1059,"Index":57,"Attempt":0,"Launch Time":1427397532230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294046,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1067,"Index":65,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1061,"Index":59,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":320439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1068,"Index":66,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1062,"Index":60,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344267,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1069,"Index":67,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1060,"Index":58,"Attempt":0,"Launch Time":1427397532233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":12183325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1070,"Index":68,"Attempt":0,"Launch Time":1427397532256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1063,"Index":61,"Attempt":0,"Launch Time":1427397532235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1071,"Index":69,"Attempt":0,"Launch Time":1427397532257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1064,"Index":62,"Attempt":0,"Launch Time":1427397532236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1072,"Index":70,"Attempt":0,"Launch Time":1427397532261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1066,"Index":64,"Attempt":0,"Launch Time":1427397532248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1073,"Index":71,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1068,"Index":66,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346438,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1074,"Index":72,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1067,"Index":65,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329537,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1075,"Index":73,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1070,"Index":68,"Attempt":0,"Launch Time":1427397532256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1076,"Index":74,"Attempt":0,"Launch Time":1427397532265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1069,"Index":67,"Attempt":0,"Launch Time":1427397532255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":351657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1077,"Index":75,"Attempt":0,"Launch Time":1427397532267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1071,"Index":69,"Attempt":0,"Launch Time":1427397532257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":306511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1078,"Index":76,"Attempt":0,"Launch Time":1427397532270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1072,"Index":70,"Attempt":0,"Launch Time":1427397532261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310319,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1079,"Index":77,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1073,"Index":71,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1080,"Index":78,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1075,"Index":73,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1081,"Index":79,"Attempt":0,"Launch Time":1427397532272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1074,"Index":72,"Attempt":0,"Launch Time":1427397532263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1082,"Index":80,"Attempt":0,"Launch Time":1427397532276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1076,"Index":74,"Attempt":0,"Launch Time":1427397532265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1083,"Index":81,"Attempt":0,"Launch Time":1427397532277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1078,"Index":76,"Attempt":0,"Launch Time":1427397532270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1084,"Index":82,"Attempt":0,"Launch Time":1427397532278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1079,"Index":77,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1085,"Index":83,"Attempt":0,"Launch Time":1427397532279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1080,"Index":78,"Attempt":0,"Launch Time":1427397532271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368844,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1086,"Index":84,"Attempt":0,"Launch Time":1427397532280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1038,"Index":36,"Attempt":0,"Launch Time":1427397532189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332193,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1087,"Index":85,"Attempt":0,"Launch Time":1427397532286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1082,"Index":80,"Attempt":0,"Launch Time":1427397532276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1088,"Index":86,"Attempt":0,"Launch Time":1427397532287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1084,"Index":82,"Attempt":0,"Launch Time":1427397532278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1089,"Index":87,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1085,"Index":83,"Attempt":0,"Launch Time":1427397532279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":634558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1090,"Index":88,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1081,"Index":79,"Attempt":0,"Launch Time":1427397532272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364350,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1091,"Index":89,"Attempt":0,"Launch Time":1427397532289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1077,"Index":75,"Attempt":0,"Launch Time":1427397532267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":401494,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1092,"Index":90,"Attempt":0,"Launch Time":1427397532293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1087,"Index":85,"Attempt":0,"Launch Time":1427397532286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1093,"Index":91,"Attempt":0,"Launch Time":1427397532294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1088,"Index":86,"Attempt":0,"Launch Time":1427397532287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1094,"Index":92,"Attempt":0,"Launch Time":1427397532297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1089,"Index":87,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354671,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1095,"Index":93,"Attempt":0,"Launch Time":1427397532298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1090,"Index":88,"Attempt":0,"Launch Time":1427397532288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":631722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1096,"Index":94,"Attempt":0,"Launch Time":1427397532300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1083,"Index":81,"Attempt":0,"Launch Time":1427397532277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1097,"Index":95,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1093,"Index":91,"Attempt":0,"Launch Time":1427397532294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1098,"Index":96,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1099,"Index":97,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1091,"Index":89,"Attempt":0,"Launch Time":1427397532289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":540797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1092,"Index":90,"Attempt":0,"Launch Time":1427397532293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308791,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1100,"Index":98,"Attempt":0,"Launch Time":1427397532305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1095,"Index":93,"Attempt":0,"Launch Time":1427397532298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532305,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1101,"Index":99,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1102,"Index":100,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1096,"Index":94,"Attempt":0,"Launch Time":1427397532300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1094,"Index":92,"Attempt":0,"Launch Time":1427397532297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1103,"Index":101,"Attempt":0,"Launch Time":1427397532310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1098,"Index":96,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304450,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1104,"Index":102,"Attempt":0,"Launch Time":1427397532311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1065,"Index":63,"Attempt":0,"Launch Time":1427397532239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311712,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1105,"Index":103,"Attempt":0,"Launch Time":1427397532312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1097,"Index":95,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1106,"Index":104,"Attempt":0,"Launch Time":1427397532314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1100,"Index":98,"Attempt":0,"Launch Time":1427397532305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532314,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1107,"Index":105,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1102,"Index":100,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1108,"Index":106,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1086,"Index":84,"Attempt":0,"Launch Time":1427397532280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364062,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1109,"Index":107,"Attempt":0,"Launch Time":1427397532328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1104,"Index":102,"Attempt":0,"Launch Time":1427397532311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532328,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1110,"Index":108,"Attempt":0,"Launch Time":1427397532334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1108,"Index":106,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374513,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1111,"Index":109,"Attempt":0,"Launch Time":1427397532335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1109,"Index":107,"Attempt":0,"Launch Time":1427397532328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1112,"Index":110,"Attempt":0,"Launch Time":1427397532336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1099,"Index":97,"Attempt":0,"Launch Time":1427397532302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369192,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1113,"Index":111,"Attempt":0,"Launch Time":1427397532341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1110,"Index":108,"Attempt":0,"Launch Time":1427397532334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1114,"Index":112,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1111,"Index":109,"Attempt":0,"Launch Time":1427397532335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1112,"Index":110,"Attempt":0,"Launch Time":1427397532336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1115,"Index":113,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1116,"Index":114,"Attempt":0,"Launch Time":1427397532349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1113,"Index":111,"Attempt":0,"Launch Time":1427397532341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1117,"Index":115,"Attempt":0,"Launch Time":1427397532351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1114,"Index":112,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353178,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1118,"Index":116,"Attempt":0,"Launch Time":1427397532353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1115,"Index":113,"Attempt":0,"Launch Time":1427397532343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1119,"Index":117,"Attempt":0,"Launch Time":1427397532356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1116,"Index":114,"Attempt":0,"Launch Time":1427397532349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289671,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1120,"Index":118,"Attempt":0,"Launch Time":1427397532359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1117,"Index":115,"Attempt":0,"Launch Time":1427397532351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324601,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1121,"Index":119,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1118,"Index":116,"Attempt":0,"Launch Time":1427397532353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1122,"Index":120,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1119,"Index":117,"Attempt":0,"Launch Time":1427397532356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":333314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1123,"Index":121,"Attempt":0,"Launch Time":1427397532366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1120,"Index":118,"Attempt":0,"Launch Time":1427397532359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1124,"Index":122,"Attempt":0,"Launch Time":1427397532371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1125,"Index":123,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1126,"Index":124,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1122,"Index":120,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1123,"Index":121,"Attempt":0,"Launch Time":1427397532366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368007,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1121,"Index":119,"Attempt":0,"Launch Time":1427397532363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1127,"Index":125,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1124,"Index":122,"Attempt":0,"Launch Time":1427397532371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391385,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1128,"Index":126,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1125,"Index":123,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1129,"Index":127,"Attempt":0,"Launch Time":1427397532387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1126,"Index":124,"Attempt":0,"Launch Time":1427397532374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1130,"Index":128,"Attempt":0,"Launch Time":1427397532391,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1127,"Index":125,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532391,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":294821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1131,"Index":129,"Attempt":0,"Launch Time":1427397532392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1128,"Index":126,"Attempt":0,"Launch Time":1427397532384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":310892,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1132,"Index":130,"Attempt":0,"Launch Time":1427397532395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1129,"Index":127,"Attempt":0,"Launch Time":1427397532387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1133,"Index":131,"Attempt":0,"Launch Time":1427397532398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1130,"Index":128,"Attempt":0,"Launch Time":1427397532391,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1134,"Index":132,"Attempt":0,"Launch Time":1427397532399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1131,"Index":129,"Attempt":0,"Launch Time":1427397532392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1135,"Index":133,"Attempt":0,"Launch Time":1427397532402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1132,"Index":130,"Attempt":0,"Launch Time":1427397532395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1136,"Index":134,"Attempt":0,"Launch Time":1427397532407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1133,"Index":131,"Attempt":0,"Launch Time":1427397532398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316250,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1137,"Index":135,"Attempt":0,"Launch Time":1427397532409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1134,"Index":132,"Attempt":0,"Launch Time":1427397532399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1138,"Index":136,"Attempt":0,"Launch Time":1427397532410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1135,"Index":133,"Attempt":0,"Launch Time":1427397532402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1139,"Index":137,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1137,"Index":135,"Attempt":0,"Launch Time":1427397532409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1140,"Index":138,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1138,"Index":136,"Attempt":0,"Launch Time":1427397532410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1141,"Index":139,"Attempt":0,"Launch Time":1427397532418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1136,"Index":134,"Attempt":0,"Launch Time":1427397532407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1142,"Index":140,"Attempt":0,"Launch Time":1427397532425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1139,"Index":137,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1143,"Index":141,"Attempt":0,"Launch Time":1427397532426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1140,"Index":138,"Attempt":0,"Launch Time":1427397532417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":388004,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1144,"Index":142,"Attempt":0,"Launch Time":1427397532429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1141,"Index":139,"Attempt":0,"Launch Time":1427397532418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2594911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1145,"Index":143,"Attempt":0,"Launch Time":1427397532434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1143,"Index":141,"Attempt":0,"Launch Time":1427397532426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1146,"Index":144,"Attempt":0,"Launch Time":1427397532435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1142,"Index":140,"Attempt":0,"Launch Time":1427397532425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":317795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1147,"Index":145,"Attempt":0,"Launch Time":1427397532436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1144,"Index":142,"Attempt":0,"Launch Time":1427397532429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1148,"Index":146,"Attempt":0,"Launch Time":1427397532462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1103,"Index":101,"Attempt":0,"Launch Time":1427397532310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1149,"Index":147,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1150,"Index":148,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1107,"Index":105,"Attempt":0,"Launch Time":1427397532316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1145,"Index":143,"Attempt":0,"Launch Time":1427397532434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1151,"Index":149,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1152,"Index":150,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1101,"Index":99,"Attempt":0,"Launch Time":1427397532307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":161,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1146,"Index":144,"Attempt":0,"Launch Time":1427397532435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1153,"Index":151,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1105,"Index":103,"Attempt":0,"Launch Time":1427397532312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1154,"Index":152,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1106,"Index":104,"Attempt":0,"Launch Time":1427397532314,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":738861,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1155,"Index":153,"Attempt":0,"Launch Time":1427397532483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1151,"Index":149,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":348594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1156,"Index":154,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1152,"Index":150,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":374266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1157,"Index":155,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1150,"Index":148,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1158,"Index":156,"Attempt":0,"Launch Time":1427397532485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1154,"Index":152,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1159,"Index":157,"Attempt":0,"Launch Time":1427397532489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1149,"Index":147,"Attempt":0,"Launch Time":1427397532472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1160,"Index":158,"Attempt":0,"Launch Time":1427397532491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1155,"Index":153,"Attempt":0,"Launch Time":1427397532483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1161,"Index":159,"Attempt":0,"Launch Time":1427397532492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1156,"Index":154,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1162,"Index":160,"Attempt":0,"Launch Time":1427397532493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1157,"Index":155,"Attempt":0,"Launch Time":1427397532484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1163,"Index":161,"Attempt":0,"Launch Time":1427397532495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1148,"Index":146,"Attempt":0,"Launch Time":1427397532462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":664614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1164,"Index":162,"Attempt":0,"Launch Time":1427397532496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1158,"Index":156,"Attempt":0,"Launch Time":1427397532485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1238925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1165,"Index":163,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1160,"Index":158,"Attempt":0,"Launch Time":1427397532491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327843,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1166,"Index":164,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1159,"Index":157,"Attempt":0,"Launch Time":1427397532489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301485,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1167,"Index":165,"Attempt":0,"Launch Time":1427397532500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1161,"Index":159,"Attempt":0,"Launch Time":1427397532492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263381,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1168,"Index":166,"Attempt":0,"Launch Time":1427397532502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1163,"Index":161,"Attempt":0,"Launch Time":1427397532495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284390,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1169,"Index":167,"Attempt":0,"Launch Time":1427397532503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1164,"Index":162,"Attempt":0,"Launch Time":1427397532496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532503,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287640,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1170,"Index":168,"Attempt":0,"Launch Time":1427397532505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1165,"Index":163,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":265607,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1171,"Index":169,"Attempt":0,"Launch Time":1427397532508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1166,"Index":164,"Attempt":0,"Launch Time":1427397532499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1172,"Index":170,"Attempt":0,"Launch Time":1427397532510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1167,"Index":165,"Attempt":0,"Launch Time":1427397532500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1173,"Index":171,"Attempt":0,"Launch Time":1427397532511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1168,"Index":166,"Attempt":0,"Launch Time":1427397532502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319746,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1174,"Index":172,"Attempt":0,"Launch Time":1427397532513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1153,"Index":151,"Attempt":0,"Launch Time":1427397532473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":8517822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1170,"Index":168,"Attempt":0,"Launch Time":1427397532505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1175,"Index":173,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1176,"Index":174,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1162,"Index":160,"Attempt":0,"Launch Time":1427397532493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3122989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1177,"Index":175,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1178,"Index":176,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1171,"Index":169,"Attempt":0,"Launch Time":1427397532508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1169,"Index":167,"Attempt":0,"Launch Time":1427397532503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":437685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1179,"Index":177,"Attempt":0,"Launch Time":1427397532518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1172,"Index":170,"Attempt":0,"Launch Time":1427397532510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1180,"Index":178,"Attempt":0,"Launch Time":1427397532522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1176,"Index":174,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297324,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1181,"Index":179,"Attempt":0,"Launch Time":1427397532524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1175,"Index":173,"Attempt":0,"Launch Time":1427397532514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532524,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321764,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1182,"Index":180,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1183,"Index":181,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1184,"Index":182,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1177,"Index":175,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1178,"Index":176,"Attempt":0,"Launch Time":1427397532517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1174,"Index":172,"Attempt":0,"Launch Time":1427397532513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1185,"Index":183,"Attempt":0,"Launch Time":1427397532531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1179,"Index":177,"Attempt":0,"Launch Time":1427397532518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1186,"Index":184,"Attempt":0,"Launch Time":1427397532532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1180,"Index":178,"Attempt":0,"Launch Time":1427397532522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1187,"Index":185,"Attempt":0,"Launch Time":1427397532533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1184,"Index":182,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1188,"Index":186,"Attempt":0,"Launch Time":1427397532534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1182,"Index":180,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1189,"Index":187,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1181,"Index":179,"Attempt":0,"Launch Time":1427397532524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":555430,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1190,"Index":188,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1183,"Index":181,"Attempt":0,"Launch Time":1427397532525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1191,"Index":189,"Attempt":0,"Launch Time":1427397532546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1187,"Index":185,"Attempt":0,"Launch Time":1427397532533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298482,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1192,"Index":190,"Attempt":0,"Launch Time":1427397532553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1191,"Index":189,"Attempt":0,"Launch Time":1427397532546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1193,"Index":191,"Attempt":0,"Launch Time":1427397532561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1192,"Index":190,"Attempt":0,"Launch Time":1427397532553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1194,"Index":192,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1186,"Index":184,"Attempt":0,"Launch Time":1427397532532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1195,"Index":193,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1188,"Index":186,"Attempt":0,"Launch Time":1427397532534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284054,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1196,"Index":194,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1190,"Index":188,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1197,"Index":195,"Attempt":0,"Launch Time":1427397532563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1173,"Index":171,"Attempt":0,"Launch Time":1427397532511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":44,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":19745151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1198,"Index":196,"Attempt":0,"Launch Time":1427397532565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1185,"Index":183,"Attempt":0,"Launch Time":1427397532531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335039,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1199,"Index":197,"Attempt":0,"Launch Time":1427397532566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1189,"Index":187,"Attempt":0,"Launch Time":1427397532535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1200,"Index":198,"Attempt":0,"Launch Time":1427397532568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1147,"Index":145,"Attempt":0,"Launch Time":1427397532436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":96,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":5252046,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1201,"Index":199,"Attempt":0,"Launch Time":1427397532571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1197,"Index":195,"Attempt":0,"Launch Time":1427397532563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307748,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1202,"Index":200,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1199,"Index":197,"Attempt":0,"Launch Time":1427397532566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":364824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1203,"Index":201,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1198,"Index":196,"Attempt":0,"Launch Time":1427397532565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1204,"Index":202,"Attempt":0,"Launch Time":1427397532578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1200,"Index":198,"Attempt":0,"Launch Time":1427397532568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532579,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394834,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1205,"Index":203,"Attempt":0,"Launch Time":1427397532580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1201,"Index":199,"Attempt":0,"Launch Time":1427397532571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":459168,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1206,"Index":204,"Attempt":0,"Launch Time":1427397532582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1202,"Index":200,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1207,"Index":205,"Attempt":0,"Launch Time":1427397532587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1203,"Index":201,"Attempt":0,"Launch Time":1427397532574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1208,"Index":206,"Attempt":0,"Launch Time":1427397532588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1205,"Index":203,"Attempt":0,"Launch Time":1427397532580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1209,"Index":207,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1206,"Index":204,"Attempt":0,"Launch Time":1427397532582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":349340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1210,"Index":208,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1204,"Index":202,"Attempt":0,"Launch Time":1427397532578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322496,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1211,"Index":209,"Attempt":0,"Launch Time":1427397532596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1208,"Index":206,"Attempt":0,"Launch Time":1427397532588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":408585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1212,"Index":210,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1213,"Index":211,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1209,"Index":207,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1207,"Index":205,"Attempt":0,"Launch Time":1427397532587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1214,"Index":212,"Attempt":0,"Launch Time":1427397532598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1210,"Index":208,"Attempt":0,"Launch Time":1427397532590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1215,"Index":213,"Attempt":0,"Launch Time":1427397532605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1211,"Index":209,"Attempt":0,"Launch Time":1427397532596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1216,"Index":214,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1213,"Index":211,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1217,"Index":215,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1214,"Index":212,"Attempt":0,"Launch Time":1427397532598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":359494,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1218,"Index":216,"Attempt":0,"Launch Time":1427397532608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1212,"Index":210,"Attempt":0,"Launch Time":1427397532597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":564813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1219,"Index":217,"Attempt":0,"Launch Time":1427397532612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1194,"Index":192,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":540404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1220,"Index":218,"Attempt":0,"Launch Time":1427397532614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1215,"Index":213,"Attempt":0,"Launch Time":1427397532605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":369813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1221,"Index":219,"Attempt":0,"Launch Time":1427397532617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1216,"Index":214,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":401448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1222,"Index":220,"Attempt":0,"Launch Time":1427397532619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1218,"Index":216,"Attempt":0,"Launch Time":1427397532608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350873,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1223,"Index":221,"Attempt":0,"Launch Time":1427397532620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1219,"Index":217,"Attempt":0,"Launch Time":1427397532612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1224,"Index":222,"Attempt":0,"Launch Time":1427397532622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1220,"Index":218,"Attempt":0,"Launch Time":1427397532614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1225,"Index":223,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1221,"Index":219,"Attempt":0,"Launch Time":1427397532617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1226,"Index":224,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1217,"Index":215,"Attempt":0,"Launch Time":1427397532607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":531759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1227,"Index":225,"Attempt":0,"Launch Time":1427397532626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1222,"Index":220,"Attempt":0,"Launch Time":1427397532619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316374,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1228,"Index":226,"Attempt":0,"Launch Time":1427397532627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1223,"Index":221,"Attempt":0,"Launch Time":1427397532620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1229,"Index":227,"Attempt":0,"Launch Time":1427397532631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1224,"Index":222,"Attempt":0,"Launch Time":1427397532622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341870,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1230,"Index":228,"Attempt":0,"Launch Time":1427397532633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1225,"Index":223,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328956,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1231,"Index":229,"Attempt":0,"Launch Time":1427397532634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1227,"Index":225,"Attempt":0,"Launch Time":1427397532626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1232,"Index":230,"Attempt":0,"Launch Time":1427397532648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1229,"Index":227,"Attempt":0,"Launch Time":1427397532631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1233,"Index":231,"Attempt":0,"Launch Time":1427397532651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1228,"Index":226,"Attempt":0,"Launch Time":1427397532627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":611848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1234,"Index":232,"Attempt":0,"Launch Time":1427397532653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1226,"Index":224,"Attempt":0,"Launch Time":1427397532625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":12767861,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1235,"Index":233,"Attempt":0,"Launch Time":1427397532655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1230,"Index":228,"Attempt":0,"Launch Time":1427397532633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":618579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1231,"Index":229,"Attempt":0,"Launch Time":1427397532634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":374977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1236,"Index":234,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1237,"Index":235,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1232,"Index":230,"Attempt":0,"Launch Time":1427397532648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1238,"Index":236,"Attempt":0,"Launch Time":1427397532660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1233,"Index":231,"Attempt":0,"Launch Time":1427397532651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":466555,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1239,"Index":237,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1240,"Index":238,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1235,"Index":233,"Attempt":0,"Launch Time":1427397532655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1196,"Index":194,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":591905,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1241,"Index":239,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1236,"Index":234,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372544,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1242,"Index":240,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1234,"Index":232,"Attempt":0,"Launch Time":1427397532653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434915,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1243,"Index":241,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1237,"Index":235,"Attempt":0,"Launch Time":1427397532656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1244,"Index":242,"Attempt":0,"Launch Time":1427397532667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1195,"Index":193,"Attempt":0,"Launch Time":1427397532562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":39,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1816125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1245,"Index":243,"Attempt":0,"Launch Time":1427397532672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1239,"Index":237,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1246,"Index":244,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1241,"Index":239,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1247,"Index":245,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1248,"Index":246,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1240,"Index":238,"Attempt":0,"Launch Time":1427397532664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317237,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1243,"Index":241,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1249,"Index":247,"Attempt":0,"Launch Time":1427397532675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1238,"Index":236,"Attempt":0,"Launch Time":1427397532660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1250,"Index":248,"Attempt":0,"Launch Time":1427397532678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1242,"Index":240,"Attempt":0,"Launch Time":1427397532665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1251,"Index":249,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1252,"Index":250,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1253,"Index":251,"Attempt":0,"Launch Time":1427397532681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1247,"Index":245,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":337849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1246,"Index":244,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1248,"Index":246,"Attempt":0,"Launch Time":1427397532673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319346,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1254,"Index":252,"Attempt":0,"Launch Time":1427397532687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1250,"Index":248,"Attempt":0,"Launch Time":1427397532678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1255,"Index":253,"Attempt":0,"Launch Time":1427397532693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1244,"Index":242,"Attempt":0,"Launch Time":1427397532667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1256,"Index":254,"Attempt":0,"Launch Time":1427397532694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1193,"Index":191,"Attempt":0,"Launch Time":1427397532561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":39,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1257,"Index":255,"Attempt":0,"Launch Time":1427397532696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1245,"Index":243,"Attempt":0,"Launch Time":1427397532672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335077,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1258,"Index":256,"Attempt":0,"Launch Time":1427397532701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1256,"Index":254,"Attempt":0,"Launch Time":1427397532694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1259,"Index":257,"Attempt":0,"Launch Time":1427397532702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1255,"Index":253,"Attempt":0,"Launch Time":1427397532693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1260,"Index":258,"Attempt":0,"Launch Time":1427397532704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1257,"Index":255,"Attempt":0,"Launch Time":1427397532696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1261,"Index":259,"Attempt":0,"Launch Time":1427397532710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1262,"Index":260,"Attempt":0,"Launch Time":1427397532711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1258,"Index":256,"Attempt":0,"Launch Time":1427397532701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1259,"Index":257,"Attempt":0,"Launch Time":1427397532702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":393368,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1263,"Index":261,"Attempt":0,"Launch Time":1427397532713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1260,"Index":258,"Attempt":0,"Launch Time":1427397532704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":465471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1264,"Index":262,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1265,"Index":263,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1262,"Index":260,"Attempt":0,"Launch Time":1427397532711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1249,"Index":247,"Attempt":0,"Launch Time":1427397532675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2135654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1266,"Index":264,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1265,"Index":263,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1267,"Index":265,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1264,"Index":262,"Attempt":0,"Launch Time":1427397532719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323379,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1268,"Index":266,"Attempt":0,"Launch Time":1427397532729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1263,"Index":261,"Attempt":0,"Launch Time":1427397532713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1269,"Index":267,"Attempt":0,"Launch Time":1427397532730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1261,"Index":259,"Attempt":0,"Launch Time":1427397532710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":432272,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1270,"Index":268,"Attempt":0,"Launch Time":1427397532733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1266,"Index":264,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1271,"Index":269,"Attempt":0,"Launch Time":1427397532736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1268,"Index":266,"Attempt":0,"Launch Time":1427397532729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1272,"Index":270,"Attempt":0,"Launch Time":1427397532737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1269,"Index":267,"Attempt":0,"Launch Time":1427397532730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319713,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1273,"Index":271,"Attempt":0,"Launch Time":1427397532767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1270,"Index":268,"Attempt":0,"Launch Time":1427397532733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327160,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1274,"Index":272,"Attempt":0,"Launch Time":1427397532769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1267,"Index":265,"Attempt":0,"Launch Time":1427397532726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":487730,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1275,"Index":273,"Attempt":0,"Launch Time":1427397532774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1251,"Index":249,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":93,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":375765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1276,"Index":274,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1271,"Index":269,"Attempt":0,"Launch Time":1427397532736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1277,"Index":275,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1273,"Index":271,"Attempt":0,"Launch Time":1427397532767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1278,"Index":276,"Attempt":0,"Launch Time":1427397532777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1274,"Index":272,"Attempt":0,"Launch Time":1427397532769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349932,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1279,"Index":277,"Attempt":0,"Launch Time":1427397532779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1272,"Index":270,"Attempt":0,"Launch Time":1427397532737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1280,"Index":278,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1276,"Index":274,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1277,"Index":275,"Attempt":0,"Launch Time":1427397532775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1281,"Index":279,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1282,"Index":280,"Attempt":0,"Launch Time":1427397532786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1279,"Index":277,"Attempt":0,"Launch Time":1427397532779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":278799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1283,"Index":281,"Attempt":0,"Launch Time":1427397532791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1281,"Index":279,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1284,"Index":282,"Attempt":0,"Launch Time":1427397532792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1280,"Index":278,"Attempt":0,"Launch Time":1427397532783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":417842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1285,"Index":283,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1286,"Index":284,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1278,"Index":276,"Attempt":0,"Launch Time":1427397532777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":917389,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1282,"Index":280,"Attempt":0,"Launch Time":1427397532786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1287,"Index":285,"Attempt":0,"Launch Time":1427397532795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1254,"Index":252,"Attempt":0,"Launch Time":1427397532687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":53,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":854377,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1288,"Index":286,"Attempt":0,"Launch Time":1427397532798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1283,"Index":281,"Attempt":0,"Launch Time":1427397532791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309781,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1289,"Index":287,"Attempt":0,"Launch Time":1427397532800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1284,"Index":282,"Attempt":0,"Launch Time":1427397532792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1290,"Index":288,"Attempt":0,"Launch Time":1427397532802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1285,"Index":283,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317079,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1291,"Index":289,"Attempt":0,"Launch Time":1427397532803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1286,"Index":284,"Attempt":0,"Launch Time":1427397532794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1292,"Index":290,"Attempt":0,"Launch Time":1427397532805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1288,"Index":286,"Attempt":0,"Launch Time":1427397532798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1293,"Index":291,"Attempt":0,"Launch Time":1427397532809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1252,"Index":250,"Attempt":0,"Launch Time":1427397532680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":377423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1294,"Index":292,"Attempt":0,"Launch Time":1427397532810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1289,"Index":287,"Attempt":0,"Launch Time":1427397532800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":407204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1295,"Index":293,"Attempt":0,"Launch Time":1427397532812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1253,"Index":251,"Attempt":0,"Launch Time":1427397532681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1296,"Index":294,"Attempt":0,"Launch Time":1427397532813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1290,"Index":288,"Attempt":0,"Launch Time":1427397532802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1297,"Index":295,"Attempt":0,"Launch Time":1427397532815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1275,"Index":273,"Attempt":0,"Launch Time":1427397532774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":420051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1298,"Index":296,"Attempt":0,"Launch Time":1427397532818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1293,"Index":291,"Attempt":0,"Launch Time":1427397532809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1299,"Index":297,"Attempt":0,"Launch Time":1427397532819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1287,"Index":285,"Attempt":0,"Launch Time":1427397532795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1300,"Index":298,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1301,"Index":299,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1297,"Index":295,"Attempt":0,"Launch Time":1427397532815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327543,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1295,"Index":293,"Attempt":0,"Launch Time":1427397532812,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1302,"Index":300,"Attempt":0,"Launch Time":1427397532825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1298,"Index":296,"Attempt":0,"Launch Time":1427397532818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1303,"Index":301,"Attempt":0,"Launch Time":1427397532832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1300,"Index":298,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1304,"Index":302,"Attempt":0,"Launch Time":1427397532833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1302,"Index":300,"Attempt":0,"Launch Time":1427397532825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1305,"Index":303,"Attempt":0,"Launch Time":1427397532835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1301,"Index":299,"Attempt":0,"Launch Time":1427397532822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302159,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1306,"Index":304,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1307,"Index":305,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1304,"Index":302,"Attempt":0,"Launch Time":1427397532833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317738,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1303,"Index":301,"Attempt":0,"Launch Time":1427397532832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347409,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1308,"Index":306,"Attempt":0,"Launch Time":1427397532846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1305,"Index":303,"Attempt":0,"Launch Time":1427397532835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400863,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1309,"Index":307,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1299,"Index":297,"Attempt":0,"Launch Time":1427397532819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2422447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1310,"Index":308,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1306,"Index":304,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1311,"Index":309,"Attempt":0,"Launch Time":1427397532852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1307,"Index":305,"Attempt":0,"Launch Time":1427397532840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352671,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1312,"Index":310,"Attempt":0,"Launch Time":1427397532854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1308,"Index":306,"Attempt":0,"Launch Time":1427397532846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1313,"Index":311,"Attempt":0,"Launch Time":1427397532861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1310,"Index":308,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1314,"Index":312,"Attempt":0,"Launch Time":1427397532862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1309,"Index":307,"Attempt":0,"Launch Time":1427397532851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1315,"Index":313,"Attempt":0,"Launch Time":1427397532863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1311,"Index":309,"Attempt":0,"Launch Time":1427397532852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":386906,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1316,"Index":314,"Attempt":0,"Launch Time":1427397532864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1296,"Index":294,"Attempt":0,"Launch Time":1427397532813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377329,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1317,"Index":315,"Attempt":0,"Launch Time":1427397532879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1314,"Index":312,"Attempt":0,"Launch Time":1427397532862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1318,"Index":316,"Attempt":0,"Launch Time":1427397532880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1312,"Index":310,"Attempt":0,"Launch Time":1427397532854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":705379,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1319,"Index":317,"Attempt":0,"Launch Time":1427397532882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1313,"Index":311,"Attempt":0,"Launch Time":1427397532861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326653,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1320,"Index":318,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1319,"Index":317,"Attempt":0,"Launch Time":1427397532882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532891,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1318,"Index":316,"Attempt":0,"Launch Time":1427397532880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532891,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":443144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1321,"Index":319,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1322,"Index":320,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1316,"Index":314,"Attempt":0,"Launch Time":1427397532864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":439879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1323,"Index":321,"Attempt":0,"Launch Time":1427397532894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1315,"Index":313,"Attempt":0,"Launch Time":1427397532863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":526743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1324,"Index":322,"Attempt":0,"Launch Time":1427397532898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1317,"Index":315,"Attempt":0,"Launch Time":1427397532879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491754,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1325,"Index":323,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1322,"Index":320,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376957,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1326,"Index":324,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1320,"Index":318,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370996,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1327,"Index":325,"Attempt":0,"Launch Time":1427397532901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1321,"Index":319,"Attempt":0,"Launch Time":1427397532891,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435376,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1328,"Index":326,"Attempt":0,"Launch Time":1427397532903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1323,"Index":321,"Attempt":0,"Launch Time":1427397532894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":513472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1329,"Index":327,"Attempt":0,"Launch Time":1427397532904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1291,"Index":289,"Attempt":0,"Launch Time":1427397532803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":100,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1330,"Index":328,"Attempt":0,"Launch Time":1427397532908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1326,"Index":324,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1331,"Index":329,"Attempt":0,"Launch Time":1427397532909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1325,"Index":323,"Attempt":0,"Launch Time":1427397532900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":604949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1332,"Index":330,"Attempt":0,"Launch Time":1427397532910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1327,"Index":325,"Attempt":0,"Launch Time":1427397532901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":323699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1333,"Index":331,"Attempt":0,"Launch Time":1427397532912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1328,"Index":326,"Attempt":0,"Launch Time":1427397532903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1334,"Index":332,"Attempt":0,"Launch Time":1427397532915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1330,"Index":328,"Attempt":0,"Launch Time":1427397532908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532915,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329104,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1335,"Index":333,"Attempt":0,"Launch Time":1427397532916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1292,"Index":290,"Attempt":0,"Launch Time":1427397532805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":110,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1336,"Index":334,"Attempt":0,"Launch Time":1427397532917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1331,"Index":329,"Attempt":0,"Launch Time":1427397532909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1337,"Index":335,"Attempt":0,"Launch Time":1427397532920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1333,"Index":331,"Attempt":0,"Launch Time":1427397532912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1338,"Index":336,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1332,"Index":330,"Attempt":0,"Launch Time":1427397532910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353868,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1339,"Index":337,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1335,"Index":333,"Attempt":0,"Launch Time":1427397532916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1340,"Index":338,"Attempt":0,"Launch Time":1427397532928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1336,"Index":334,"Attempt":0,"Launch Time":1427397532917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":349449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1341,"Index":339,"Attempt":0,"Launch Time":1427397532929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1337,"Index":335,"Attempt":0,"Launch Time":1427397532920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":337146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1342,"Index":340,"Attempt":0,"Launch Time":1427397532930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1338,"Index":336,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":314161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1343,"Index":341,"Attempt":0,"Launch Time":1427397532931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1339,"Index":337,"Attempt":0,"Launch Time":1427397532924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":312281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1344,"Index":342,"Attempt":0,"Launch Time":1427397532935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1340,"Index":338,"Attempt":0,"Launch Time":1427397532928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532935,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":337520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1345,"Index":343,"Attempt":0,"Launch Time":1427397532936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1329,"Index":327,"Attempt":0,"Launch Time":1427397532904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1346,"Index":344,"Attempt":0,"Launch Time":1427397532940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1294,"Index":292,"Attempt":0,"Launch Time":1427397532810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":405322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1347,"Index":345,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1334,"Index":332,"Attempt":0,"Launch Time":1427397532915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":380556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1348,"Index":346,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1341,"Index":339,"Attempt":0,"Launch Time":1427397532929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":389321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1349,"Index":347,"Attempt":0,"Launch Time":1427397532948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1346,"Index":344,"Attempt":0,"Launch Time":1427397532940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1350,"Index":348,"Attempt":0,"Launch Time":1427397532952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1345,"Index":343,"Attempt":0,"Launch Time":1427397532936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1351,"Index":349,"Attempt":0,"Launch Time":1427397532956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1348,"Index":346,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1352,"Index":350,"Attempt":0,"Launch Time":1427397532957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1349,"Index":347,"Attempt":0,"Launch Time":1427397532948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1353,"Index":351,"Attempt":0,"Launch Time":1427397532964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1351,"Index":349,"Attempt":0,"Launch Time":1427397532956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":361912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1354,"Index":352,"Attempt":0,"Launch Time":1427397532965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1350,"Index":348,"Attempt":0,"Launch Time":1427397532952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372746,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1344,"Index":342,"Attempt":0,"Launch Time":1427397532935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1355,"Index":353,"Attempt":0,"Launch Time":1427397532967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1356,"Index":354,"Attempt":0,"Launch Time":1427397532970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1352,"Index":350,"Attempt":0,"Launch Time":1427397532957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":330282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1357,"Index":355,"Attempt":0,"Launch Time":1427397532972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1354,"Index":352,"Attempt":0,"Launch Time":1427397532965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1358,"Index":356,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1353,"Index":351,"Attempt":0,"Launch Time":1427397532964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351254,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1359,"Index":357,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1355,"Index":353,"Attempt":0,"Launch Time":1427397532967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1360,"Index":358,"Attempt":0,"Launch Time":1427397532986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1359,"Index":357,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":299740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1361,"Index":359,"Attempt":0,"Launch Time":1427397532999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1347,"Index":345,"Attempt":0,"Launch Time":1427397532942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397532999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338751,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1362,"Index":360,"Attempt":0,"Launch Time":1427397533002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1343,"Index":341,"Attempt":0,"Launch Time":1427397532931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":55125458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1363,"Index":361,"Attempt":0,"Launch Time":1427397533005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1342,"Index":340,"Attempt":0,"Launch Time":1427397532930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":74,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":67252055,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1364,"Index":362,"Attempt":0,"Launch Time":1427397533008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1358,"Index":356,"Attempt":0,"Launch Time":1427397532973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324013,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1365,"Index":363,"Attempt":0,"Launch Time":1427397533010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1357,"Index":355,"Attempt":0,"Launch Time":1427397532972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341063,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1366,"Index":364,"Attempt":0,"Launch Time":1427397533012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1363,"Index":361,"Attempt":0,"Launch Time":1427397533005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322208,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1367,"Index":365,"Attempt":0,"Launch Time":1427397533013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1356,"Index":354,"Attempt":0,"Launch Time":1427397532970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":382433,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1368,"Index":366,"Attempt":0,"Launch Time":1427397533016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1324,"Index":322,"Attempt":0,"Launch Time":1427397532898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":116,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2076105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1369,"Index":367,"Attempt":0,"Launch Time":1427397533018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1365,"Index":363,"Attempt":0,"Launch Time":1427397533010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321011,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1370,"Index":368,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1366,"Index":364,"Attempt":0,"Launch Time":1427397533012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1371,"Index":369,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1367,"Index":365,"Attempt":0,"Launch Time":1427397533013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1372,"Index":370,"Attempt":0,"Launch Time":1427397533023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1368,"Index":366,"Attempt":0,"Launch Time":1427397533016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1373,"Index":371,"Attempt":0,"Launch Time":1427397533028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1370,"Index":368,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1374,"Index":372,"Attempt":0,"Launch Time":1427397533031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1364,"Index":362,"Attempt":0,"Launch Time":1427397533008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1375,"Index":373,"Attempt":0,"Launch Time":1427397533038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1369,"Index":367,"Attempt":0,"Launch Time":1427397533018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1376,"Index":374,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1371,"Index":369,"Attempt":0,"Launch Time":1427397533021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452377,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1377,"Index":375,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1372,"Index":370,"Attempt":0,"Launch Time":1427397533023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":5054761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1378,"Index":376,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1374,"Index":372,"Attempt":0,"Launch Time":1427397533031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1379,"Index":377,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1376,"Index":374,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":355781,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1380,"Index":378,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1381,"Index":379,"Attempt":0,"Launch Time":1427397533059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1382,"Index":380,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1383,"Index":381,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1384,"Index":382,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1385,"Index":383,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1386,"Index":384,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1377,"Index":375,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1362,"Index":360,"Attempt":0,"Launch Time":1427397533002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1361,"Index":359,"Attempt":0,"Launch Time":1427397532999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":646712,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1373,"Index":371,"Attempt":0,"Launch Time":1427397533028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":817743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1378,"Index":376,"Attempt":0,"Launch Time":1427397533039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":414813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1375,"Index":373,"Attempt":0,"Launch Time":1427397533038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348309,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1360,"Index":358,"Attempt":0,"Launch Time":1427397532986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1387,"Index":385,"Attempt":0,"Launch Time":1427397533065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1379,"Index":377,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326027,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1388,"Index":386,"Attempt":0,"Launch Time":1427397533069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1381,"Index":379,"Attempt":0,"Launch Time":1427397533059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1389,"Index":387,"Attempt":0,"Launch Time":1427397533075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1386,"Index":384,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":365336,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1390,"Index":388,"Attempt":0,"Launch Time":1427397533082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1388,"Index":386,"Attempt":0,"Launch Time":1427397533069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":644995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1391,"Index":389,"Attempt":0,"Launch Time":1427397533084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1389,"Index":387,"Attempt":0,"Launch Time":1427397533075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1392,"Index":390,"Attempt":0,"Launch Time":1427397533089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1382,"Index":380,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353101,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1393,"Index":391,"Attempt":0,"Launch Time":1427397533090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1390,"Index":388,"Attempt":0,"Launch Time":1427397533082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1394,"Index":392,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1391,"Index":389,"Attempt":0,"Launch Time":1427397533084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":346948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1395,"Index":393,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1384,"Index":382,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":816812,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1396,"Index":394,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1393,"Index":391,"Attempt":0,"Launch Time":1427397533090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1397,"Index":395,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1392,"Index":390,"Attempt":0,"Launch Time":1427397533089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":363196,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1398,"Index":396,"Attempt":0,"Launch Time":1427397533100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1394,"Index":392,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1399,"Index":397,"Attempt":0,"Launch Time":1427397533104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1395,"Index":393,"Attempt":0,"Launch Time":1427397533092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":455851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1400,"Index":398,"Attempt":0,"Launch Time":1427397533106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1397,"Index":395,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":384131,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1401,"Index":399,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1402,"Index":400,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1398,"Index":396,"Attempt":0,"Launch Time":1427397533100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1396,"Index":394,"Attempt":0,"Launch Time":1427397533098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1403,"Index":401,"Attempt":0,"Launch Time":1427397533121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1402,"Index":400,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1404,"Index":402,"Attempt":0,"Launch Time":1427397533122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1399,"Index":397,"Attempt":0,"Launch Time":1427397533104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":542014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1405,"Index":403,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1406,"Index":404,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1380,"Index":378,"Attempt":0,"Launch Time":1427397533058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1403,"Index":401,"Attempt":0,"Launch Time":1427397533121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":398458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1407,"Index":405,"Attempt":0,"Launch Time":1427397533131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1404,"Index":402,"Attempt":0,"Launch Time":1427397533122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":388487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1408,"Index":406,"Attempt":0,"Launch Time":1427397533137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1405,"Index":403,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1409,"Index":407,"Attempt":0,"Launch Time":1427397533138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1407,"Index":405,"Attempt":0,"Launch Time":1427397533131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":362557,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1410,"Index":408,"Attempt":0,"Launch Time":1427397533139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1406,"Index":404,"Attempt":0,"Launch Time":1427397533130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318738,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1411,"Index":409,"Attempt":0,"Launch Time":1427397533146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1409,"Index":407,"Attempt":0,"Launch Time":1427397533138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":400622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1412,"Index":410,"Attempt":0,"Launch Time":1427397533147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1410,"Index":408,"Attempt":0,"Launch Time":1427397533139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435193,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1413,"Index":411,"Attempt":0,"Launch Time":1427397533156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1412,"Index":410,"Attempt":0,"Launch Time":1427397533147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362480,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1414,"Index":412,"Attempt":0,"Launch Time":1427397533158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1411,"Index":409,"Attempt":0,"Launch Time":1427397533146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533158,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359313,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1415,"Index":413,"Attempt":0,"Launch Time":1427397533159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1408,"Index":406,"Attempt":0,"Launch Time":1427397533137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":915274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1416,"Index":414,"Attempt":0,"Launch Time":1427397533164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1413,"Index":411,"Attempt":0,"Launch Time":1427397533156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1417,"Index":415,"Attempt":0,"Launch Time":1427397533166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1414,"Index":412,"Attempt":0,"Launch Time":1427397533158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361990,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1418,"Index":416,"Attempt":0,"Launch Time":1427397533169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1383,"Index":381,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3208002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1419,"Index":417,"Attempt":0,"Launch Time":1427397533173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1416,"Index":414,"Attempt":0,"Launch Time":1427397533164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368790,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1420,"Index":418,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1387,"Index":385,"Attempt":0,"Launch Time":1427397533065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":17534191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1421,"Index":419,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1400,"Index":398,"Attempt":0,"Launch Time":1427397533106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1422,"Index":420,"Attempt":0,"Launch Time":1427397533179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1401,"Index":399,"Attempt":0,"Launch Time":1427397533107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":71,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390159,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1423,"Index":421,"Attempt":0,"Launch Time":1427397533184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1415,"Index":413,"Attempt":0,"Launch Time":1427397533159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1424,"Index":422,"Attempt":0,"Launch Time":1427397533186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1421,"Index":419,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1425,"Index":423,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1426,"Index":424,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1422,"Index":420,"Attempt":0,"Launch Time":1427397533179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1385,"Index":383,"Attempt":0,"Launch Time":1427397533060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364829,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1427,"Index":425,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1418,"Index":416,"Attempt":0,"Launch Time":1427397533169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1428,"Index":426,"Attempt":0,"Launch Time":1427397533195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1420,"Index":418,"Attempt":0,"Launch Time":1427397533178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":627265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1429,"Index":427,"Attempt":0,"Launch Time":1427397533196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1424,"Index":422,"Attempt":0,"Launch Time":1427397533186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390670,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1430,"Index":428,"Attempt":0,"Launch Time":1427397533197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1426,"Index":424,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":437632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1431,"Index":429,"Attempt":0,"Launch Time":1427397533199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1427,"Index":425,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1432,"Index":430,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1433,"Index":431,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1429,"Index":427,"Attempt":0,"Launch Time":1427397533196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":487273,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1430,"Index":428,"Attempt":0,"Launch Time":1427397533197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1434,"Index":432,"Attempt":0,"Launch Time":1427397533206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1428,"Index":426,"Attempt":0,"Launch Time":1427397533195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1435,"Index":433,"Attempt":0,"Launch Time":1427397533212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1425,"Index":423,"Attempt":0,"Launch Time":1427397533188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350065,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1436,"Index":434,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1437,"Index":435,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1434,"Index":432,"Attempt":0,"Launch Time":1427397533206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1432,"Index":430,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382027,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1438,"Index":436,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1433,"Index":431,"Attempt":0,"Launch Time":1427397533205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1439,"Index":437,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1417,"Index":415,"Attempt":0,"Launch Time":1427397533166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352461,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1440,"Index":438,"Attempt":0,"Launch Time":1427397533222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1435,"Index":433,"Attempt":0,"Launch Time":1427397533212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":393461,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1441,"Index":439,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1438,"Index":436,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":371597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1442,"Index":440,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1439,"Index":437,"Attempt":0,"Launch Time":1427397533216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":392440,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1443,"Index":441,"Attempt":0,"Launch Time":1427397533227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1431,"Index":429,"Attempt":0,"Launch Time":1427397533199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":369102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1444,"Index":442,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1436,"Index":434,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402180,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1445,"Index":443,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1437,"Index":435,"Attempt":0,"Launch Time":1427397533215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":461008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1446,"Index":444,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1423,"Index":421,"Attempt":0,"Launch Time":1427397533184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":364333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1447,"Index":445,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1442,"Index":440,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":329833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1448,"Index":446,"Attempt":0,"Launch Time":1427397533247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1443,"Index":441,"Attempt":0,"Launch Time":1427397533227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1449,"Index":447,"Attempt":0,"Launch Time":1427397533248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1444,"Index":442,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327816,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1450,"Index":448,"Attempt":0,"Launch Time":1427397533249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1447,"Index":445,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":334842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1451,"Index":449,"Attempt":0,"Launch Time":1427397533250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1445,"Index":443,"Attempt":0,"Launch Time":1427397533228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":582785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1452,"Index":450,"Attempt":0,"Launch Time":1427397533256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1448,"Index":446,"Attempt":0,"Launch Time":1427397533247,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1453,"Index":451,"Attempt":0,"Launch Time":1427397533258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1449,"Index":447,"Attempt":0,"Launch Time":1427397533248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1454,"Index":452,"Attempt":0,"Launch Time":1427397533259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1455,"Index":453,"Attempt":0,"Launch Time":1427397533260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1451,"Index":449,"Attempt":0,"Launch Time":1427397533250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":535263,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1441,"Index":439,"Attempt":0,"Launch Time":1427397533224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":597186,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1456,"Index":454,"Attempt":0,"Launch Time":1427397533264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1452,"Index":450,"Attempt":0,"Launch Time":1427397533256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":352354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1457,"Index":455,"Attempt":0,"Launch Time":1427397533265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1446,"Index":444,"Attempt":0,"Launch Time":1427397533242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":654543,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1458,"Index":456,"Attempt":0,"Launch Time":1427397533267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1454,"Index":452,"Attempt":0,"Launch Time":1427397533259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1459,"Index":457,"Attempt":0,"Launch Time":1427397533268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1453,"Index":451,"Attempt":0,"Launch Time":1427397533258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1460,"Index":458,"Attempt":0,"Launch Time":1427397533269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1455,"Index":453,"Attempt":0,"Launch Time":1427397533260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":366404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1461,"Index":459,"Attempt":0,"Launch Time":1427397533273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1456,"Index":454,"Attempt":0,"Launch Time":1427397533264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1462,"Index":460,"Attempt":0,"Launch Time":1427397533277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1459,"Index":457,"Attempt":0,"Launch Time":1427397533268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":505518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1463,"Index":461,"Attempt":0,"Launch Time":1427397533279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1457,"Index":455,"Attempt":0,"Launch Time":1427397533265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1275280,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1464,"Index":462,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1465,"Index":463,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1460,"Index":458,"Attempt":0,"Launch Time":1427397533269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":525255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1440,"Index":438,"Attempt":0,"Launch Time":1427397533222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1840075,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1466,"Index":464,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1458,"Index":456,"Attempt":0,"Launch Time":1427397533267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":983522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1467,"Index":465,"Attempt":0,"Launch Time":1427397533283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1461,"Index":459,"Attempt":0,"Launch Time":1427397533273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":414864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1468,"Index":466,"Attempt":0,"Launch Time":1427397533285,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1450,"Index":448,"Attempt":0,"Launch Time":1427397533249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":694355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1462,"Index":460,"Attempt":0,"Launch Time":1427397533277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1469,"Index":467,"Attempt":0,"Launch Time":1427397533286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1470,"Index":468,"Attempt":0,"Launch Time":1427397533289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1463,"Index":461,"Attempt":0,"Launch Time":1427397533279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":365229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1471,"Index":469,"Attempt":0,"Launch Time":1427397533290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1466,"Index":464,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309474,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1472,"Index":470,"Attempt":0,"Launch Time":1427397533292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1467,"Index":465,"Attempt":0,"Launch Time":1427397533283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1473,"Index":471,"Attempt":0,"Launch Time":1427397533293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1468,"Index":466,"Attempt":0,"Launch Time":1427397533285,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1474,"Index":472,"Attempt":0,"Launch Time":1427397533297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1470,"Index":468,"Attempt":0,"Launch Time":1427397533289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339872,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1475,"Index":473,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1464,"Index":462,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1476,"Index":474,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1477,"Index":475,"Attempt":0,"Launch Time":1427397533300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1471,"Index":469,"Attempt":0,"Launch Time":1427397533290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317562,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1472,"Index":470,"Attempt":0,"Launch Time":1427397533292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325825,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1478,"Index":476,"Attempt":0,"Launch Time":1427397533301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1473,"Index":471,"Attempt":0,"Launch Time":1427397533293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":318456,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1479,"Index":477,"Attempt":0,"Launch Time":1427397533304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1474,"Index":472,"Attempt":0,"Launch Time":1427397533297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1480,"Index":478,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1481,"Index":479,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1476,"Index":474,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1477,"Index":475,"Attempt":0,"Launch Time":1427397533300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1482,"Index":480,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1465,"Index":463,"Attempt":0,"Launch Time":1427397533282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":4263497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1483,"Index":481,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533323,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1478,"Index":476,"Attempt":0,"Launch Time":1427397533301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1484,"Index":482,"Attempt":0,"Launch Time":1427397533312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1479,"Index":477,"Attempt":0,"Launch Time":1427397533304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1485,"Index":483,"Attempt":0,"Launch Time":1427397533313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533321,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1419,"Index":417,"Attempt":0,"Launch Time":1427397533173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":140,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1486,"Index":484,"Attempt":0,"Launch Time":1427397533321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1485,"Index":483,"Attempt":0,"Launch Time":1427397533313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373196,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1487,"Index":485,"Attempt":0,"Launch Time":1427397533323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533352,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1483,"Index":481,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356131,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1488,"Index":486,"Attempt":0,"Launch Time":1427397533326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1475,"Index":473,"Attempt":0,"Launch Time":1427397533299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369607,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1489,"Index":487,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1487,"Index":485,"Attempt":0,"Launch Time":1427397533323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":2121695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1490,"Index":488,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1491,"Index":489,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1492,"Index":490,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1493,"Index":491,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1481,"Index":479,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348056,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1469,"Index":467,"Attempt":0,"Launch Time":1427397533286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1480,"Index":478,"Attempt":0,"Launch Time":1427397533307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1486,"Index":484,"Attempt":0,"Launch Time":1427397533321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":520761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1494,"Index":492,"Attempt":0,"Launch Time":1427397533355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1484,"Index":482,"Attempt":0,"Launch Time":1427397533312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":33,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1495,"Index":493,"Attempt":0,"Launch Time":1427397533360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1489,"Index":487,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321373,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1496,"Index":494,"Attempt":0,"Launch Time":1427397533361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1493,"Index":491,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":588233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1497,"Index":495,"Attempt":0,"Launch Time":1427397533362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1492,"Index":490,"Attempt":0,"Launch Time":1427397533353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1498,"Index":496,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1494,"Index":492,"Attempt":0,"Launch Time":1427397533355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1499,"Index":497,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1488,"Index":486,"Attempt":0,"Launch Time":1427397533326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364094,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1500,"Index":498,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1490,"Index":488,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491123,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1495,"Index":493,"Attempt":0,"Launch Time":1427397533360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355489,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1501,"Index":499,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1502,"Index":500,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1496,"Index":494,"Attempt":0,"Launch Time":1427397533361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":363718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1503,"Index":501,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1497,"Index":495,"Attempt":0,"Launch Time":1427397533362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1504,"Index":502,"Attempt":0,"Launch Time":1427397533374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1498,"Index":496,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1540396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1505,"Index":503,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1500,"Index":498,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376360,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1502,"Index":500,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":495888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1506,"Index":504,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1507,"Index":505,"Attempt":0,"Launch Time":1427397533380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1503,"Index":501,"Attempt":0,"Launch Time":1427397533370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1508,"Index":506,"Attempt":0,"Launch Time":1427397533381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1501,"Index":499,"Attempt":0,"Launch Time":1427397533369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357547,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1509,"Index":507,"Attempt":0,"Launch Time":1427397533383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1504,"Index":502,"Attempt":0,"Launch Time":1427397533374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533383,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1510,"Index":508,"Attempt":0,"Launch Time":1427397533386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1506,"Index":504,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1511,"Index":509,"Attempt":0,"Launch Time":1427397533387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1505,"Index":503,"Attempt":0,"Launch Time":1427397533378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1512,"Index":510,"Attempt":0,"Launch Time":1427397533388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1499,"Index":497,"Attempt":0,"Launch Time":1427397533364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1513,"Index":511,"Attempt":0,"Launch Time":1427397533390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1507,"Index":505,"Attempt":0,"Launch Time":1427397533380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1514,"Index":512,"Attempt":0,"Launch Time":1427397533393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1509,"Index":507,"Attempt":0,"Launch Time":1427397533383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533393,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":493261,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1515,"Index":513,"Attempt":0,"Launch Time":1427397533394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1510,"Index":508,"Attempt":0,"Launch Time":1427397533386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1516,"Index":514,"Attempt":0,"Launch Time":1427397533395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1508,"Index":506,"Attempt":0,"Launch Time":1427397533381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1517,"Index":515,"Attempt":0,"Launch Time":1427397533396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1511,"Index":509,"Attempt":0,"Launch Time":1427397533387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337496,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1518,"Index":516,"Attempt":0,"Launch Time":1427397533401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1515,"Index":513,"Attempt":0,"Launch Time":1427397533394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297753,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1519,"Index":517,"Attempt":0,"Launch Time":1427397533402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1513,"Index":511,"Attempt":0,"Launch Time":1427397533390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":627412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1520,"Index":518,"Attempt":0,"Launch Time":1427397533404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1517,"Index":515,"Attempt":0,"Launch Time":1427397533396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1521,"Index":519,"Attempt":0,"Launch Time":1427397533406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1491,"Index":489,"Attempt":0,"Launch Time":1427397533352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1455317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1522,"Index":520,"Attempt":0,"Launch Time":1427397533408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1518,"Index":516,"Attempt":0,"Launch Time":1427397533401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1523,"Index":521,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1516,"Index":514,"Attempt":0,"Launch Time":1427397533395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1003257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1524,"Index":522,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1514,"Index":512,"Attempt":0,"Launch Time":1427397533393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1525,"Index":523,"Attempt":0,"Launch Time":1427397533411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1512,"Index":510,"Attempt":0,"Launch Time":1427397533388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1526,"Index":524,"Attempt":0,"Launch Time":1427397533412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1520,"Index":518,"Attempt":0,"Launch Time":1427397533404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1527,"Index":525,"Attempt":0,"Launch Time":1427397533415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1519,"Index":517,"Attempt":0,"Launch Time":1427397533402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":415972,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1528,"Index":526,"Attempt":0,"Launch Time":1427397533417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1522,"Index":520,"Attempt":0,"Launch Time":1427397533408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1529,"Index":527,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1530,"Index":528,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1524,"Index":522,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":826079,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1523,"Index":521,"Attempt":0,"Launch Time":1427397533409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382574,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1531,"Index":529,"Attempt":0,"Launch Time":1427397533421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1532,"Index":530,"Attempt":0,"Launch Time":1427397533423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533457,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1533,"Index":531,"Attempt":0,"Launch Time":1427397533430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533479,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1525,"Index":523,"Attempt":0,"Launch Time":1427397533411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352285,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1526,"Index":524,"Attempt":0,"Launch Time":1427397533412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368361,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1534,"Index":532,"Attempt":0,"Launch Time":1427397533434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533477,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1529,"Index":527,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":369849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1535,"Index":533,"Attempt":0,"Launch Time":1427397533437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1530,"Index":528,"Attempt":0,"Launch Time":1427397533418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1536,"Index":534,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533453,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1521,"Index":519,"Attempt":0,"Launch Time":1427397533406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348897,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1537,"Index":535,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533451,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1482,"Index":480,"Attempt":0,"Launch Time":1427397533310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":110,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1538,"Index":536,"Attempt":0,"Launch Time":1427397533451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533459,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1537,"Index":535,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360173,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1539,"Index":537,"Attempt":0,"Launch Time":1427397533453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533460,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1536,"Index":534,"Attempt":0,"Launch Time":1427397533443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392809,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1540,"Index":538,"Attempt":0,"Launch Time":1427397533457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1532,"Index":530,"Attempt":0,"Launch Time":1427397533423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352641,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1541,"Index":539,"Attempt":0,"Launch Time":1427397533459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1538,"Index":536,"Attempt":0,"Launch Time":1427397533451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1542,"Index":540,"Attempt":0,"Launch Time":1427397533460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533473,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1539,"Index":537,"Attempt":0,"Launch Time":1427397533453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1543,"Index":541,"Attempt":0,"Launch Time":1427397533466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533474,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1544,"Index":542,"Attempt":0,"Launch Time":1427397533467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533478,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1540,"Index":538,"Attempt":0,"Launch Time":1427397533457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1541,"Index":539,"Attempt":0,"Launch Time":1427397533459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1545,"Index":543,"Attempt":0,"Launch Time":1427397533473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533482,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1542,"Index":540,"Attempt":0,"Launch Time":1427397533460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1546,"Index":544,"Attempt":0,"Launch Time":1427397533474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533483,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1543,"Index":541,"Attempt":0,"Launch Time":1427397533466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359733,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1547,"Index":545,"Attempt":0,"Launch Time":1427397533476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533493,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1534,"Index":532,"Attempt":0,"Launch Time":1427397533434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533477,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":472626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1548,"Index":546,"Attempt":0,"Launch Time":1427397533478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533486,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1544,"Index":542,"Attempt":0,"Launch Time":1427397533467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":800000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1549,"Index":547,"Attempt":0,"Launch Time":1427397533479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533489,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1533,"Index":531,"Attempt":0,"Launch Time":1427397533430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386153,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1550,"Index":548,"Attempt":0,"Launch Time":1427397533482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533491,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1545,"Index":543,"Attempt":0,"Launch Time":1427397533473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":426912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1551,"Index":549,"Attempt":0,"Launch Time":1427397533483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533492,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1546,"Index":544,"Attempt":0,"Launch Time":1427397533474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":390265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1552,"Index":550,"Attempt":0,"Launch Time":1427397533486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533495,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1548,"Index":546,"Attempt":0,"Launch Time":1427397533478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1553,"Index":551,"Attempt":0,"Launch Time":1427397533489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533507,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1549,"Index":547,"Attempt":0,"Launch Time":1427397533479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1554,"Index":552,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1550,"Index":548,"Attempt":0,"Launch Time":1427397533482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1555,"Index":553,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1551,"Index":549,"Attempt":0,"Launch Time":1427397533483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":415748,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1556,"Index":554,"Attempt":0,"Launch Time":1427397533493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533500,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1547,"Index":545,"Attempt":0,"Launch Time":1427397533476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":837592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1557,"Index":555,"Attempt":0,"Launch Time":1427397533494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533502,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1552,"Index":550,"Attempt":0,"Launch Time":1427397533486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334254,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1558,"Index":556,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1555,"Index":553,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341120,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1559,"Index":557,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1554,"Index":552,"Attempt":0,"Launch Time":1427397533491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323104,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1560,"Index":558,"Attempt":0,"Launch Time":1427397533500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1556,"Index":554,"Attempt":0,"Launch Time":1427397533493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1561,"Index":559,"Attempt":0,"Launch Time":1427397533502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533510,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1557,"Index":555,"Attempt":0,"Launch Time":1427397533494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1562,"Index":560,"Attempt":0,"Launch Time":1427397533506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533519,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1553,"Index":551,"Attempt":0,"Launch Time":1427397533489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":394559,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1563,"Index":561,"Attempt":0,"Launch Time":1427397533507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1559,"Index":557,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1564,"Index":562,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1560,"Index":558,"Attempt":0,"Launch Time":1427397533500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1565,"Index":563,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1558,"Index":556,"Attempt":0,"Launch Time":1427397533499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":508780,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1566,"Index":564,"Attempt":0,"Launch Time":1427397533510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533518,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1561,"Index":559,"Attempt":0,"Launch Time":1427397533502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1567,"Index":565,"Attempt":0,"Launch Time":1427397533514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1528,"Index":526,"Attempt":0,"Launch Time":1427397533417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1070386,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1568,"Index":566,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1569,"Index":567,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1563,"Index":561,"Attempt":0,"Launch Time":1427397533507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":443773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1564,"Index":562,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383173,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1570,"Index":568,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1565,"Index":563,"Attempt":0,"Launch Time":1427397533508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":347911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1571,"Index":569,"Attempt":0,"Launch Time":1427397533518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1566,"Index":564,"Attempt":0,"Launch Time":1427397533510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335235,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1572,"Index":570,"Attempt":0,"Launch Time":1427397533519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1562,"Index":560,"Attempt":0,"Launch Time":1427397533506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":448241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1573,"Index":571,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1574,"Index":572,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1575,"Index":573,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1576,"Index":574,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1577,"Index":575,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1570,"Index":568,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1569,"Index":567,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336027,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1571,"Index":569,"Attempt":0,"Launch Time":1427397533518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333192,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1568,"Index":566,"Attempt":0,"Launch Time":1427397533516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1572,"Index":570,"Attempt":0,"Launch Time":1427397533519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1527,"Index":525,"Attempt":0,"Launch Time":1427397533415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1578,"Index":576,"Attempt":0,"Launch Time":1427397533550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1574,"Index":572,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":2362919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1579,"Index":577,"Attempt":0,"Launch Time":1427397533561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1578,"Index":576,"Attempt":0,"Launch Time":1427397533550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":923128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1580,"Index":578,"Attempt":0,"Launch Time":1427397533562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1576,"Index":574,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":432327,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1581,"Index":579,"Attempt":0,"Launch Time":1427397533568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533576,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1531,"Index":529,"Attempt":0,"Launch Time":1427397533421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1582,"Index":580,"Attempt":0,"Launch Time":1427397533569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1535,"Index":533,"Attempt":0,"Launch Time":1427397533437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1583,"Index":581,"Attempt":0,"Launch Time":1427397533572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1567,"Index":565,"Attempt":0,"Launch Time":1427397533514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1584,"Index":582,"Attempt":0,"Launch Time":1427397533576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1581,"Index":579,"Attempt":0,"Launch Time":1427397533568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1585,"Index":583,"Attempt":0,"Launch Time":1427397533593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1582,"Index":580,"Attempt":0,"Launch Time":1427397533569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":337520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1586,"Index":584,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1583,"Index":581,"Attempt":0,"Launch Time":1427397533572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1587,"Index":585,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1588,"Index":586,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1575,"Index":573,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":706428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1580,"Index":578,"Attempt":0,"Launch Time":1427397533562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1584,"Index":582,"Attempt":0,"Launch Time":1427397533576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1589,"Index":587,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1590,"Index":588,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1591,"Index":589,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1573,"Index":571,"Attempt":0,"Launch Time":1427397533523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1577,"Index":575,"Attempt":0,"Launch Time":1427397533530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1592,"Index":590,"Attempt":0,"Launch Time":1427397533602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1585,"Index":583,"Attempt":0,"Launch Time":1427397533593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340352,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1593,"Index":591,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1589,"Index":587,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1594,"Index":592,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1588,"Index":586,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":986589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1595,"Index":593,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1590,"Index":588,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":331220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1596,"Index":594,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1591,"Index":589,"Attempt":0,"Launch Time":1427397533595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":514371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1597,"Index":595,"Attempt":0,"Launch Time":1427397533609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1592,"Index":590,"Attempt":0,"Launch Time":1427397533602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1598,"Index":596,"Attempt":0,"Launch Time":1427397533614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1594,"Index":592,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1599,"Index":597,"Attempt":0,"Launch Time":1427397533615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1593,"Index":591,"Attempt":0,"Launch Time":1427397533604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":447900,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1600,"Index":598,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1595,"Index":593,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408202,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1601,"Index":599,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1596,"Index":594,"Attempt":0,"Launch Time":1427397533605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424230,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1602,"Index":600,"Attempt":0,"Launch Time":1427397533617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1597,"Index":595,"Attempt":0,"Launch Time":1427397533609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1603,"Index":601,"Attempt":0,"Launch Time":1427397533623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1598,"Index":596,"Attempt":0,"Launch Time":1427397533614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339519,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1604,"Index":602,"Attempt":0,"Launch Time":1427397533624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1600,"Index":598,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1605,"Index":603,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1602,"Index":600,"Attempt":0,"Launch Time":1427397533617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370658,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1606,"Index":604,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1599,"Index":597,"Attempt":0,"Launch Time":1427397533615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":362640,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1607,"Index":605,"Attempt":0,"Launch Time":1427397533628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1601,"Index":599,"Attempt":0,"Launch Time":1427397533616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1608,"Index":606,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1603,"Index":601,"Attempt":0,"Launch Time":1427397533623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1609,"Index":607,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1604,"Index":602,"Attempt":0,"Launch Time":1427397533624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1610,"Index":608,"Attempt":0,"Launch Time":1427397533634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1606,"Index":604,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331170,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1611,"Index":609,"Attempt":0,"Launch Time":1427397533636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1607,"Index":605,"Attempt":0,"Launch Time":1427397533628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1612,"Index":610,"Attempt":0,"Launch Time":1427397533640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1609,"Index":607,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1613,"Index":611,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1610,"Index":608,"Attempt":0,"Launch Time":1427397533634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297136,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1614,"Index":612,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1608,"Index":606,"Attempt":0,"Launch Time":1427397533632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1615,"Index":613,"Attempt":0,"Launch Time":1427397533646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1611,"Index":609,"Attempt":0,"Launch Time":1427397533636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343063,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1616,"Index":614,"Attempt":0,"Launch Time":1427397533648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1612,"Index":610,"Attempt":0,"Launch Time":1427397533640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1617,"Index":615,"Attempt":0,"Launch Time":1427397533649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1587,"Index":585,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1359105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1618,"Index":616,"Attempt":0,"Launch Time":1427397533650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1613,"Index":611,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":361411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1619,"Index":617,"Attempt":0,"Launch Time":1427397533651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1614,"Index":612,"Attempt":0,"Launch Time":1427397533642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1620,"Index":618,"Attempt":0,"Launch Time":1427397533661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1616,"Index":614,"Attempt":0,"Launch Time":1427397533648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1621,"Index":619,"Attempt":0,"Launch Time":1427397533664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1619,"Index":617,"Attempt":0,"Launch Time":1427397533651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1622,"Index":620,"Attempt":0,"Launch Time":1427397533666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1615,"Index":613,"Attempt":0,"Launch Time":1427397533646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":380709,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1623,"Index":621,"Attempt":0,"Launch Time":1427397533667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1617,"Index":615,"Attempt":0,"Launch Time":1427397533649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1624,"Index":622,"Attempt":0,"Launch Time":1427397533669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1618,"Index":616,"Attempt":0,"Launch Time":1427397533650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1625,"Index":623,"Attempt":0,"Launch Time":1427397533670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1620,"Index":618,"Attempt":0,"Launch Time":1427397533661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1626,"Index":624,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1622,"Index":620,"Attempt":0,"Launch Time":1427397533666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1627,"Index":625,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1605,"Index":603,"Attempt":0,"Launch Time":1427397533627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329375,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1628,"Index":626,"Attempt":0,"Launch Time":1427397533676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1621,"Index":619,"Attempt":0,"Launch Time":1427397533664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1629,"Index":627,"Attempt":0,"Launch Time":1427397533677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1624,"Index":622,"Attempt":0,"Launch Time":1427397533669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1630,"Index":628,"Attempt":0,"Launch Time":1427397533679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1625,"Index":623,"Attempt":0,"Launch Time":1427397533670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":433068,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1631,"Index":629,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1632,"Index":630,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1627,"Index":625,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":422579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1628,"Index":626,"Attempt":0,"Launch Time":1427397533676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1633,"Index":631,"Attempt":0,"Launch Time":1427397533685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1629,"Index":627,"Attempt":0,"Launch Time":1427397533677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1634,"Index":632,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1635,"Index":633,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1586,"Index":584,"Attempt":0,"Launch Time":1427397533594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338560,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1632,"Index":630,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347538,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1636,"Index":634,"Attempt":0,"Launch Time":1427397533699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1634,"Index":632,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1637,"Index":635,"Attempt":0,"Launch Time":1427397533700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1579,"Index":577,"Attempt":0,"Launch Time":1427397533561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1638,"Index":636,"Attempt":0,"Launch Time":1427397533702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1626,"Index":624,"Attempt":0,"Launch Time":1427397533674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":882527,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1639,"Index":637,"Attempt":0,"Launch Time":1427397533709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1636,"Index":634,"Attempt":0,"Launch Time":1427397533699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1640,"Index":638,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1641,"Index":639,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1637,"Index":635,"Attempt":0,"Launch Time":1427397533700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1638,"Index":636,"Attempt":0,"Launch Time":1427397533702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":429006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1642,"Index":640,"Attempt":0,"Launch Time":1427397533715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1635,"Index":633,"Attempt":0,"Launch Time":1427397533691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1643,"Index":641,"Attempt":0,"Launch Time":1427397533718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1639,"Index":637,"Attempt":0,"Launch Time":1427397533709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1644,"Index":642,"Attempt":0,"Launch Time":1427397533719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1641,"Index":639,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1645,"Index":643,"Attempt":0,"Launch Time":1427397533720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1623,"Index":621,"Attempt":0,"Launch Time":1427397533667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":18708393,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1646,"Index":644,"Attempt":0,"Launch Time":1427397533723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1642,"Index":640,"Attempt":0,"Launch Time":1427397533715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1647,"Index":645,"Attempt":0,"Launch Time":1427397533724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1640,"Index":638,"Attempt":0,"Launch Time":1427397533711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533724,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1648,"Index":646,"Attempt":0,"Launch Time":1427397533725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1643,"Index":641,"Attempt":0,"Launch Time":1427397533718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1649,"Index":647,"Attempt":0,"Launch Time":1427397533726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1644,"Index":642,"Attempt":0,"Launch Time":1427397533719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1650,"Index":648,"Attempt":0,"Launch Time":1427397533730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1646,"Index":644,"Attempt":0,"Launch Time":1427397533723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355324,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1651,"Index":649,"Attempt":0,"Launch Time":1427397533731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1645,"Index":643,"Attempt":0,"Launch Time":1427397533720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1652,"Index":650,"Attempt":0,"Launch Time":1427397533732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1648,"Index":646,"Attempt":0,"Launch Time":1427397533725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321327,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1653,"Index":651,"Attempt":0,"Launch Time":1427397533733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1649,"Index":647,"Attempt":0,"Launch Time":1427397533726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1654,"Index":652,"Attempt":0,"Launch Time":1427397533736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1650,"Index":648,"Attempt":0,"Launch Time":1427397533730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1655,"Index":653,"Attempt":0,"Launch Time":1427397533739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1652,"Index":650,"Attempt":0,"Launch Time":1427397533732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289058,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1656,"Index":654,"Attempt":0,"Launch Time":1427397533740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1653,"Index":651,"Attempt":0,"Launch Time":1427397533733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1657,"Index":655,"Attempt":0,"Launch Time":1427397533743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1654,"Index":652,"Attempt":0,"Launch Time":1427397533736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299655,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1658,"Index":656,"Attempt":0,"Launch Time":1427397533745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1647,"Index":645,"Attempt":0,"Launch Time":1427397533724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":676487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1659,"Index":657,"Attempt":0,"Launch Time":1427397533746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1655,"Index":653,"Attempt":0,"Launch Time":1427397533739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344328,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1660,"Index":658,"Attempt":0,"Launch Time":1427397533748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1656,"Index":654,"Attempt":0,"Launch Time":1427397533740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1661,"Index":659,"Attempt":0,"Launch Time":1427397533751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1657,"Index":655,"Attempt":0,"Launch Time":1427397533743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347683,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1662,"Index":660,"Attempt":0,"Launch Time":1427397533762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1651,"Index":649,"Attempt":0,"Launch Time":1427397533731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":439780,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1663,"Index":661,"Attempt":0,"Launch Time":1427397533767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1661,"Index":659,"Attempt":0,"Launch Time":1427397533751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":446106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1664,"Index":662,"Attempt":0,"Launch Time":1427397533768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1659,"Index":657,"Attempt":0,"Launch Time":1427397533746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":403687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1665,"Index":663,"Attempt":0,"Launch Time":1427397533769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1660,"Index":658,"Attempt":0,"Launch Time":1427397533748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":11272360,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1666,"Index":664,"Attempt":0,"Launch Time":1427397533771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1631,"Index":629,"Attempt":0,"Launch Time":1427397533684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":87,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1667,"Index":665,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1664,"Index":662,"Attempt":0,"Launch Time":1427397533768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1668,"Index":666,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1665,"Index":663,"Attempt":0,"Launch Time":1427397533769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351463,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1669,"Index":667,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1662,"Index":660,"Attempt":0,"Launch Time":1427397533762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4437275,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1670,"Index":668,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1666,"Index":664,"Attempt":0,"Launch Time":1427397533771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338145,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1671,"Index":669,"Attempt":0,"Launch Time":1427397533784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1668,"Index":666,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1672,"Index":670,"Attempt":0,"Launch Time":1427397533785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1667,"Index":665,"Attempt":0,"Launch Time":1427397533776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":372530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1673,"Index":671,"Attempt":0,"Launch Time":1427397533786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1670,"Index":668,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373641,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1674,"Index":672,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1675,"Index":673,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1671,"Index":669,"Attempt":0,"Launch Time":1427397533784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361456,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1672,"Index":670,"Attempt":0,"Launch Time":1427397533785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":385924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1676,"Index":674,"Attempt":0,"Launch Time":1427397533797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1669,"Index":667,"Attempt":0,"Launch Time":1427397533778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":474887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1677,"Index":675,"Attempt":0,"Launch Time":1427397533798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1673,"Index":671,"Attempt":0,"Launch Time":1427397533786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1678,"Index":676,"Attempt":0,"Launch Time":1427397533802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1674,"Index":672,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":387847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1679,"Index":677,"Attempt":0,"Launch Time":1427397533803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1675,"Index":673,"Attempt":0,"Launch Time":1427397533794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":599603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1680,"Index":678,"Attempt":0,"Launch Time":1427397533807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1663,"Index":661,"Attempt":0,"Launch Time":1427397533767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533807,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":389111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1681,"Index":679,"Attempt":0,"Launch Time":1427397533811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1676,"Index":674,"Attempt":0,"Launch Time":1427397533797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":447602,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1682,"Index":680,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1677,"Index":675,"Attempt":0,"Launch Time":1427397533798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":498335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1683,"Index":681,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1679,"Index":677,"Attempt":0,"Launch Time":1427397533803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":471103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1684,"Index":682,"Attempt":0,"Launch Time":1427397533814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1630,"Index":628,"Attempt":0,"Launch Time":1427397533679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1685,"Index":683,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1633,"Index":631,"Attempt":0,"Launch Time":1427397533685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":587009,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1686,"Index":684,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1658,"Index":656,"Attempt":0,"Launch Time":1427397533745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":70,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":727149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1687,"Index":685,"Attempt":0,"Launch Time":1427397533823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1684,"Index":682,"Attempt":0,"Launch Time":1427397533814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1683,"Index":681,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":402620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1688,"Index":686,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1689,"Index":687,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1686,"Index":684,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1690,"Index":688,"Attempt":0,"Launch Time":1427397533830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1685,"Index":683,"Attempt":0,"Launch Time":1427397533816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1691,"Index":689,"Attempt":0,"Launch Time":1427397533834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1689,"Index":687,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1692,"Index":690,"Attempt":0,"Launch Time":1427397533838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1687,"Index":685,"Attempt":0,"Launch Time":1427397533823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":378262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1693,"Index":691,"Attempt":0,"Launch Time":1427397533839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1688,"Index":686,"Attempt":0,"Launch Time":1427397533825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1694,"Index":692,"Attempt":0,"Launch Time":1427397533842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1691,"Index":689,"Attempt":0,"Launch Time":1427397533834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372085,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1695,"Index":693,"Attempt":0,"Launch Time":1427397533847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1692,"Index":690,"Attempt":0,"Launch Time":1427397533838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":368353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1696,"Index":694,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1693,"Index":691,"Attempt":0,"Launch Time":1427397533839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":395556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1697,"Index":695,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1690,"Index":688,"Attempt":0,"Launch Time":1427397533830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381691,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1698,"Index":696,"Attempt":0,"Launch Time":1427397533851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1694,"Index":692,"Attempt":0,"Launch Time":1427397533842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373137,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1699,"Index":697,"Attempt":0,"Launch Time":1427397533857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1695,"Index":693,"Attempt":0,"Launch Time":1427397533847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":410577,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1700,"Index":698,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1697,"Index":695,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":410688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1701,"Index":699,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1698,"Index":696,"Attempt":0,"Launch Time":1427397533851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":385146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1702,"Index":700,"Attempt":0,"Launch Time":1427397533864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1696,"Index":694,"Attempt":0,"Launch Time":1427397533850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":428999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1703,"Index":701,"Attempt":0,"Launch Time":1427397533865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1699,"Index":697,"Attempt":0,"Launch Time":1427397533857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":410550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1704,"Index":702,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1681,"Index":679,"Attempt":0,"Launch Time":1427397533811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":655699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1705,"Index":703,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1700,"Index":698,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375957,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1706,"Index":704,"Attempt":0,"Launch Time":1427397533870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1701,"Index":699,"Attempt":0,"Launch Time":1427397533860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1707,"Index":705,"Attempt":0,"Launch Time":1427397533879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1702,"Index":700,"Attempt":0,"Launch Time":1427397533864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365098,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1708,"Index":706,"Attempt":0,"Launch Time":1427397533880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1703,"Index":701,"Attempt":0,"Launch Time":1427397533865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1709,"Index":707,"Attempt":0,"Launch Time":1427397533892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1705,"Index":703,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533892,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362270,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1710,"Index":708,"Attempt":0,"Launch Time":1427397533894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1706,"Index":704,"Attempt":0,"Launch Time":1427397533870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":535855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1711,"Index":709,"Attempt":0,"Launch Time":1427397533895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1708,"Index":706,"Attempt":0,"Launch Time":1427397533880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533895,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":378036,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1712,"Index":710,"Attempt":0,"Launch Time":1427397533896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1707,"Index":705,"Attempt":0,"Launch Time":1427397533879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":380196,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1713,"Index":711,"Attempt":0,"Launch Time":1427397533897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1704,"Index":702,"Attempt":0,"Launch Time":1427397533868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533898,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1714,"Index":712,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1715,"Index":713,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1682,"Index":680,"Attempt":0,"Launch Time":1427397533813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":33,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4347703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1680,"Index":678,"Attempt":0,"Launch Time":1427397533807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":91,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1716,"Index":714,"Attempt":0,"Launch Time":1427397533904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1710,"Index":708,"Attempt":0,"Launch Time":1427397533894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383082,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1717,"Index":715,"Attempt":0,"Launch Time":1427397533905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1712,"Index":710,"Attempt":0,"Launch Time":1427397533896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533905,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1718,"Index":716,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1713,"Index":711,"Attempt":0,"Launch Time":1427397533897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393533,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1719,"Index":717,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1714,"Index":712,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1720,"Index":718,"Attempt":0,"Launch Time":1427397533911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1709,"Index":707,"Attempt":0,"Launch Time":1427397533892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":662437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1721,"Index":719,"Attempt":0,"Launch Time":1427397533915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1719,"Index":717,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1722,"Index":720,"Attempt":0,"Launch Time":1427397533919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1717,"Index":715,"Attempt":0,"Launch Time":1427397533905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4765372,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1723,"Index":721,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1716,"Index":714,"Attempt":0,"Launch Time":1427397533904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1724,"Index":722,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1720,"Index":718,"Attempt":0,"Launch Time":1427397533911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346825,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1725,"Index":723,"Attempt":0,"Launch Time":1427397533922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1718,"Index":716,"Attempt":0,"Launch Time":1427397533906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1122903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1726,"Index":724,"Attempt":0,"Launch Time":1427397533923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1715,"Index":713,"Attempt":0,"Launch Time":1427397533899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":373998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1727,"Index":725,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1724,"Index":722,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1728,"Index":726,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1723,"Index":721,"Attempt":0,"Launch Time":1427397533920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388573,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1729,"Index":727,"Attempt":0,"Launch Time":1427397533930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1725,"Index":723,"Attempt":0,"Launch Time":1427397533922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":382960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1730,"Index":728,"Attempt":0,"Launch Time":1427397533932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1722,"Index":720,"Attempt":0,"Launch Time":1427397533919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370867,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1731,"Index":729,"Attempt":0,"Launch Time":1427397533946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1721,"Index":719,"Attempt":0,"Launch Time":1427397533915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1732,"Index":730,"Attempt":0,"Launch Time":1427397533958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1711,"Index":709,"Attempt":0,"Launch Time":1427397533895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1733,"Index":731,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1734,"Index":732,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1729,"Index":727,"Attempt":0,"Launch Time":1427397533930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5456710,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1726,"Index":724,"Attempt":0,"Launch Time":1427397533923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":800673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1735,"Index":733,"Attempt":0,"Launch Time":1427397533960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1678,"Index":676,"Attempt":0,"Launch Time":1427397533802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":150,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":498693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1736,"Index":734,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1732,"Index":730,"Attempt":0,"Launch Time":1427397533958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1737,"Index":735,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533978,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1731,"Index":729,"Attempt":0,"Launch Time":1427397533946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1738,"Index":736,"Attempt":0,"Launch Time":1427397533970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1733,"Index":731,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386577,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1739,"Index":737,"Attempt":0,"Launch Time":1427397533971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1735,"Index":733,"Attempt":0,"Launch Time":1427397533960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1740,"Index":738,"Attempt":0,"Launch Time":1427397533972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533985,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1734,"Index":732,"Attempt":0,"Launch Time":1427397533959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":418603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1741,"Index":739,"Attempt":0,"Launch Time":1427397533977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1736,"Index":734,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1742,"Index":740,"Attempt":0,"Launch Time":1427397533978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1737,"Index":735,"Attempt":0,"Launch Time":1427397533969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1743,"Index":741,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1738,"Index":736,"Attempt":0,"Launch Time":1427397533970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":430070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1744,"Index":742,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1739,"Index":737,"Attempt":0,"Launch Time":1427397533971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378962,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1745,"Index":743,"Attempt":0,"Launch Time":1427397533985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534008,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1740,"Index":738,"Attempt":0,"Launch Time":1427397533972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375816,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1746,"Index":744,"Attempt":0,"Launch Time":1427397533987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1747,"Index":745,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1741,"Index":739,"Attempt":0,"Launch Time":1427397533977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":785295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1743,"Index":741,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1748,"Index":746,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534010,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1744,"Index":742,"Attempt":0,"Launch Time":1427397533980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397533988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1749,"Index":747,"Attempt":0,"Launch Time":1427397534006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534015,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1742,"Index":740,"Attempt":0,"Launch Time":1427397533978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":893073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1750,"Index":748,"Attempt":0,"Launch Time":1427397534008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1745,"Index":743,"Attempt":0,"Launch Time":1427397533985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":410379,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1751,"Index":749,"Attempt":0,"Launch Time":1427397534010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1748,"Index":746,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1752,"Index":750,"Attempt":0,"Launch Time":1427397534015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1749,"Index":747,"Attempt":0,"Launch Time":1427397534006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":389610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1753,"Index":751,"Attempt":0,"Launch Time":1427397534017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1750,"Index":748,"Attempt":0,"Launch Time":1427397534008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":404996,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1754,"Index":752,"Attempt":0,"Launch Time":1427397534018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1747,"Index":745,"Attempt":0,"Launch Time":1427397533988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":410396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1755,"Index":753,"Attempt":0,"Launch Time":1427397534021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1751,"Index":749,"Attempt":0,"Launch Time":1427397534010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1756,"Index":754,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1757,"Index":755,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1753,"Index":751,"Attempt":0,"Launch Time":1427397534017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383935,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1754,"Index":752,"Attempt":0,"Launch Time":1427397534018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1758,"Index":756,"Attempt":0,"Launch Time":1427397534028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1752,"Index":750,"Attempt":0,"Launch Time":1427397534015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":434779,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1759,"Index":757,"Attempt":0,"Launch Time":1427397534030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1755,"Index":753,"Attempt":0,"Launch Time":1427397534021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534030,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":376569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1760,"Index":758,"Attempt":0,"Launch Time":1427397534034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1756,"Index":754,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1761,"Index":759,"Attempt":0,"Launch Time":1427397534038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1758,"Index":756,"Attempt":0,"Launch Time":1427397534028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":543255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1762,"Index":760,"Attempt":0,"Launch Time":1427397534040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1759,"Index":757,"Attempt":0,"Launch Time":1427397534030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1763,"Index":761,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1760,"Index":758,"Attempt":0,"Launch Time":1427397534034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360346,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1764,"Index":762,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1757,"Index":755,"Attempt":0,"Launch Time":1427397534026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":786785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1765,"Index":763,"Attempt":0,"Launch Time":1427397534046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1761,"Index":759,"Attempt":0,"Launch Time":1427397534038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1766,"Index":764,"Attempt":0,"Launch Time":1427397534048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1762,"Index":760,"Attempt":0,"Launch Time":1427397534040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1767,"Index":765,"Attempt":0,"Launch Time":1427397534051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1763,"Index":761,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534051,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1768,"Index":766,"Attempt":0,"Launch Time":1427397534054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1765,"Index":763,"Attempt":0,"Launch Time":1427397534046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1769,"Index":767,"Attempt":0,"Launch Time":1427397534055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1764,"Index":762,"Attempt":0,"Launch Time":1427397534042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":653560,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1770,"Index":768,"Attempt":0,"Launch Time":1427397534056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1766,"Index":764,"Attempt":0,"Launch Time":1427397534048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1771,"Index":769,"Attempt":0,"Launch Time":1427397534058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1767,"Index":765,"Attempt":0,"Launch Time":1427397534051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1772,"Index":770,"Attempt":0,"Launch Time":1427397534061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1730,"Index":728,"Attempt":0,"Launch Time":1427397533932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":129,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":425225,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1773,"Index":771,"Attempt":0,"Launch Time":1427397534064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1728,"Index":726,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2642563,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1774,"Index":772,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1770,"Index":768,"Attempt":0,"Launch Time":1427397534056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424451,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1775,"Index":773,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1727,"Index":725,"Attempt":0,"Launch Time":1427397533929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":441656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1776,"Index":774,"Attempt":0,"Launch Time":1427397534073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1772,"Index":770,"Attempt":0,"Launch Time":1427397534061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":374395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1777,"Index":775,"Attempt":0,"Launch Time":1427397534076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1773,"Index":771,"Attempt":0,"Launch Time":1427397534064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":446719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1778,"Index":776,"Attempt":0,"Launch Time":1427397534077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1774,"Index":772,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1397794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1779,"Index":777,"Attempt":0,"Launch Time":1427397534082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1746,"Index":744,"Attempt":0,"Launch Time":1427397533987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":93,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":466742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1780,"Index":778,"Attempt":0,"Launch Time":1427397534085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1776,"Index":774,"Attempt":0,"Launch Time":1427397534073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1781,"Index":779,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1778,"Index":776,"Attempt":0,"Launch Time":1427397534077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":367520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1782,"Index":780,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1777,"Index":775,"Attempt":0,"Launch Time":1427397534076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386816,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1783,"Index":781,"Attempt":0,"Launch Time":1427397534092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1780,"Index":778,"Attempt":0,"Launch Time":1427397534085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":381397,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1784,"Index":782,"Attempt":0,"Launch Time":1427397534095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1781,"Index":779,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":416193,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1785,"Index":783,"Attempt":0,"Launch Time":1427397534102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1783,"Index":781,"Attempt":0,"Launch Time":1427397534092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":447791,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1786,"Index":784,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1787,"Index":785,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1784,"Index":782,"Attempt":0,"Launch Time":1427397534095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1782,"Index":780,"Attempt":0,"Launch Time":1427397534086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1788,"Index":786,"Attempt":0,"Launch Time":1427397534107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1779,"Index":777,"Attempt":0,"Launch Time":1427397534082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1132439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1789,"Index":787,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1785,"Index":783,"Attempt":0,"Launch Time":1427397534102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534111,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345063,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1790,"Index":788,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1791,"Index":789,"Attempt":0,"Launch Time":1427397534112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1787,"Index":785,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1786,"Index":784,"Attempt":0,"Launch Time":1427397534103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1792,"Index":790,"Attempt":0,"Launch Time":1427397534118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1788,"Index":786,"Attempt":0,"Launch Time":1427397534107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365875,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1793,"Index":791,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1789,"Index":787,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1794,"Index":792,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1791,"Index":789,"Attempt":0,"Launch Time":1427397534112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350956,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1795,"Index":793,"Attempt":0,"Launch Time":1427397534121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1790,"Index":788,"Attempt":0,"Launch Time":1427397534111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1346837,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1796,"Index":794,"Attempt":0,"Launch Time":1427397534142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1795,"Index":793,"Attempt":0,"Launch Time":1427397534121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":567504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1797,"Index":795,"Attempt":0,"Launch Time":1427397534146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1798,"Index":796,"Attempt":0,"Launch Time":1427397534147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1794,"Index":792,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452054,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1793,"Index":791,"Attempt":0,"Launch Time":1427397534119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414450,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1799,"Index":797,"Attempt":0,"Launch Time":1427397534152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1796,"Index":794,"Attempt":0,"Launch Time":1427397534142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534152,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1800,"Index":798,"Attempt":0,"Launch Time":1427397534155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1797,"Index":795,"Attempt":0,"Launch Time":1427397534146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1798,"Index":796,"Attempt":0,"Launch Time":1427397534147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1801,"Index":799,"Attempt":0,"Launch Time":1427397534156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1802,"Index":800,"Attempt":0,"Launch Time":1427397534161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1799,"Index":797,"Attempt":0,"Launch Time":1427397534152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348909,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1803,"Index":801,"Attempt":0,"Launch Time":1427397534166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1800,"Index":798,"Attempt":0,"Launch Time":1427397534155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1804,"Index":802,"Attempt":0,"Launch Time":1427397534167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1801,"Index":799,"Attempt":0,"Launch Time":1427397534156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":402088,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1805,"Index":803,"Attempt":0,"Launch Time":1427397534169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1802,"Index":800,"Attempt":0,"Launch Time":1427397534161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":400290,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1806,"Index":804,"Attempt":0,"Launch Time":1427397534173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1803,"Index":801,"Attempt":0,"Launch Time":1427397534166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362576,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1807,"Index":805,"Attempt":0,"Launch Time":1427397534177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1805,"Index":803,"Attempt":0,"Launch Time":1427397534169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335633,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1808,"Index":806,"Attempt":0,"Launch Time":1427397534180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1804,"Index":802,"Attempt":0,"Launch Time":1427397534167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":387482,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1809,"Index":807,"Attempt":0,"Launch Time":1427397534181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1806,"Index":804,"Attempt":0,"Launch Time":1427397534173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345430,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1810,"Index":808,"Attempt":0,"Launch Time":1427397534187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1775,"Index":773,"Attempt":0,"Launch Time":1427397534065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":117,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":777772,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1811,"Index":809,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1768,"Index":766,"Attempt":0,"Launch Time":1427397534054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":45244826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1812,"Index":810,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1771,"Index":769,"Attempt":0,"Launch Time":1427397534058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":136,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":862993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1813,"Index":811,"Attempt":0,"Launch Time":1427397534196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1792,"Index":790,"Attempt":0,"Launch Time":1427397534118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":75,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1814,"Index":812,"Attempt":0,"Launch Time":1427397534197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1810,"Index":808,"Attempt":0,"Launch Time":1427397534187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":406893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1815,"Index":813,"Attempt":0,"Launch Time":1427397534199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1769,"Index":767,"Attempt":0,"Launch Time":1427397534055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3810966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1816,"Index":814,"Attempt":0,"Launch Time":1427397534202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1811,"Index":809,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534202,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1817,"Index":815,"Attempt":0,"Launch Time":1427397534203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1812,"Index":810,"Attempt":0,"Launch Time":1427397534195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1818,"Index":816,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1814,"Index":812,"Attempt":0,"Launch Time":1427397534197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467627,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1819,"Index":817,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1813,"Index":811,"Attempt":0,"Launch Time":1427397534196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336095,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1820,"Index":818,"Attempt":0,"Launch Time":1427397534208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1815,"Index":813,"Attempt":0,"Launch Time":1427397534199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":351815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1821,"Index":819,"Attempt":0,"Launch Time":1427397534210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1816,"Index":814,"Attempt":0,"Launch Time":1427397534202,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319362,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1817,"Index":815,"Attempt":0,"Launch Time":1427397534203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1822,"Index":820,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1823,"Index":821,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1819,"Index":817,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1824,"Index":822,"Attempt":0,"Launch Time":1427397534219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1825,"Index":823,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1822,"Index":820,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1818,"Index":816,"Attempt":0,"Launch Time":1427397534205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1826,"Index":824,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1821,"Index":819,"Attempt":0,"Launch Time":1427397534210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1827,"Index":825,"Attempt":0,"Launch Time":1427397534224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1823,"Index":821,"Attempt":0,"Launch Time":1427397534212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":438821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1828,"Index":826,"Attempt":0,"Launch Time":1427397534225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1820,"Index":818,"Attempt":0,"Launch Time":1427397534208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":461853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1829,"Index":827,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1824,"Index":822,"Attempt":0,"Launch Time":1427397534219,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":342253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1830,"Index":828,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1826,"Index":824,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335085,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1831,"Index":829,"Attempt":0,"Launch Time":1427397534233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1825,"Index":823,"Attempt":0,"Launch Time":1427397534220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1832,"Index":830,"Attempt":0,"Launch Time":1427397534235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1833,"Index":831,"Attempt":0,"Launch Time":1427397534236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1829,"Index":827,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":419490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1830,"Index":828,"Attempt":0,"Launch Time":1427397534227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":401931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1834,"Index":832,"Attempt":0,"Launch Time":1427397534238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1828,"Index":826,"Attempt":0,"Launch Time":1427397534225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386199,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1835,"Index":833,"Attempt":0,"Launch Time":1427397534241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1827,"Index":825,"Attempt":0,"Launch Time":1427397534224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":385252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1836,"Index":834,"Attempt":0,"Launch Time":1427397534242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1831,"Index":829,"Attempt":0,"Launch Time":1427397534233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1837,"Index":835,"Attempt":0,"Launch Time":1427397534243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1832,"Index":830,"Attempt":0,"Launch Time":1427397534235,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1838,"Index":836,"Attempt":0,"Launch Time":1427397534245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1833,"Index":831,"Attempt":0,"Launch Time":1427397534236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":436663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1839,"Index":837,"Attempt":0,"Launch Time":1427397534258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1836,"Index":834,"Attempt":0,"Launch Time":1427397534242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":474590,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1840,"Index":838,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1841,"Index":839,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1835,"Index":833,"Attempt":0,"Launch Time":1427397534241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1834,"Index":832,"Attempt":0,"Launch Time":1427397534238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1842,"Index":840,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1843,"Index":841,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1840,"Index":838,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1841,"Index":839,"Attempt":0,"Launch Time":1427397534259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1844,"Index":842,"Attempt":0,"Launch Time":1427397534273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1839,"Index":837,"Attempt":0,"Launch Time":1427397534258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":396341,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1845,"Index":843,"Attempt":0,"Launch Time":1427397534275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1842,"Index":840,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1846,"Index":844,"Attempt":0,"Launch Time":1427397534281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1844,"Index":842,"Attempt":0,"Launch Time":1427397534273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":739907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1847,"Index":845,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1843,"Index":841,"Attempt":0,"Launch Time":1427397534267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1848,"Index":846,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1845,"Index":843,"Attempt":0,"Launch Time":1427397534275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1849,"Index":847,"Attempt":0,"Launch Time":1427397534290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1847,"Index":845,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1850,"Index":848,"Attempt":0,"Launch Time":1427397534293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1848,"Index":846,"Attempt":0,"Launch Time":1427397534282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":427163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1851,"Index":849,"Attempt":0,"Launch Time":1427397534294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1846,"Index":844,"Attempt":0,"Launch Time":1427397534281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":494247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1852,"Index":850,"Attempt":0,"Launch Time":1427397534300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1808,"Index":806,"Attempt":0,"Launch Time":1427397534180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":120,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1853,"Index":851,"Attempt":0,"Launch Time":1427397534301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1849,"Index":847,"Attempt":0,"Launch Time":1427397534290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1854,"Index":852,"Attempt":0,"Launch Time":1427397534303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1850,"Index":848,"Attempt":0,"Launch Time":1427397534293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1855,"Index":853,"Attempt":0,"Launch Time":1427397534304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1851,"Index":849,"Attempt":0,"Launch Time":1427397534294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343459,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1856,"Index":854,"Attempt":0,"Launch Time":1427397534308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1852,"Index":850,"Attempt":0,"Launch Time":1427397534300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1857,"Index":855,"Attempt":0,"Launch Time":1427397534309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1853,"Index":851,"Attempt":0,"Launch Time":1427397534301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1858,"Index":856,"Attempt":0,"Launch Time":1427397534310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1854,"Index":852,"Attempt":0,"Launch Time":1427397534303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1859,"Index":857,"Attempt":0,"Launch Time":1427397534313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1807,"Index":805,"Attempt":0,"Launch Time":1427397534177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":417549,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1860,"Index":858,"Attempt":0,"Launch Time":1427397534321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1859,"Index":857,"Attempt":0,"Launch Time":1427397534313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":389762,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1861,"Index":859,"Attempt":0,"Launch Time":1427397534329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1809,"Index":807,"Attempt":0,"Launch Time":1427397534181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":148,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1862,"Index":860,"Attempt":0,"Launch Time":1427397534331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1860,"Index":858,"Attempt":0,"Launch Time":1427397534321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1863,"Index":861,"Attempt":0,"Launch Time":1427397534335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1856,"Index":854,"Attempt":0,"Launch Time":1427397534308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1864,"Index":862,"Attempt":0,"Launch Time":1427397534341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1862,"Index":860,"Attempt":0,"Launch Time":1427397534331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":396675,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1865,"Index":863,"Attempt":0,"Launch Time":1427397534342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1861,"Index":859,"Attempt":0,"Launch Time":1427397534329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":621066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1866,"Index":864,"Attempt":0,"Launch Time":1427397534344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1863,"Index":861,"Attempt":0,"Launch Time":1427397534335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":445369,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1867,"Index":865,"Attempt":0,"Launch Time":1427397534349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1864,"Index":862,"Attempt":0,"Launch Time":1427397534341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1868,"Index":866,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1838,"Index":836,"Attempt":0,"Launch Time":1427397534245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":105,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353139,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1869,"Index":867,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1865,"Index":863,"Attempt":0,"Launch Time":1427397534342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1870,"Index":868,"Attempt":0,"Launch Time":1427397534352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1866,"Index":864,"Attempt":0,"Launch Time":1427397534344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334635,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1871,"Index":869,"Attempt":0,"Launch Time":1427397534357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1867,"Index":865,"Attempt":0,"Launch Time":1427397534349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330024,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1872,"Index":870,"Attempt":0,"Launch Time":1427397534358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1869,"Index":867,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355173,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1873,"Index":871,"Attempt":0,"Launch Time":1427397534361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1870,"Index":868,"Attempt":0,"Launch Time":1427397534352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":420870,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1874,"Index":872,"Attempt":0,"Launch Time":1427397534364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1871,"Index":869,"Attempt":0,"Launch Time":1427397534357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":312942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1875,"Index":873,"Attempt":0,"Launch Time":1427397534367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1872,"Index":870,"Attempt":0,"Launch Time":1427397534358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1876,"Index":874,"Attempt":0,"Launch Time":1427397534368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1868,"Index":866,"Attempt":0,"Launch Time":1427397534350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":471193,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1877,"Index":875,"Attempt":0,"Launch Time":1427397534369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1873,"Index":871,"Attempt":0,"Launch Time":1427397534361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1878,"Index":876,"Attempt":0,"Launch Time":1427397534372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1837,"Index":835,"Attempt":0,"Launch Time":1427397534243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":387801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1879,"Index":877,"Attempt":0,"Launch Time":1427397534373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1874,"Index":872,"Attempt":0,"Launch Time":1427397534364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534373,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":354789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1880,"Index":878,"Attempt":0,"Launch Time":1427397534375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1875,"Index":873,"Attempt":0,"Launch Time":1427397534367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1881,"Index":879,"Attempt":0,"Launch Time":1427397534377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1876,"Index":874,"Attempt":0,"Launch Time":1427397534368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1882,"Index":880,"Attempt":0,"Launch Time":1427397534380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1877,"Index":875,"Attempt":0,"Launch Time":1427397534369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1883,"Index":881,"Attempt":0,"Launch Time":1427397534382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1879,"Index":877,"Attempt":0,"Launch Time":1427397534373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":444666,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1884,"Index":882,"Attempt":0,"Launch Time":1427397534394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1880,"Index":878,"Attempt":0,"Launch Time":1427397534375,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1885,"Index":883,"Attempt":0,"Launch Time":1427397534398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1882,"Index":880,"Attempt":0,"Launch Time":1427397534380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1886,"Index":884,"Attempt":0,"Launch Time":1427397534400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1881,"Index":879,"Attempt":0,"Launch Time":1427397534377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":445912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1887,"Index":885,"Attempt":0,"Launch Time":1427397534408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1886,"Index":884,"Attempt":0,"Launch Time":1427397534400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1888,"Index":886,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1883,"Index":881,"Attempt":0,"Launch Time":1427397534382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1889,"Index":887,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1884,"Index":882,"Attempt":0,"Launch Time":1427397534394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":815597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1890,"Index":888,"Attempt":0,"Launch Time":1427397534415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1878,"Index":876,"Attempt":0,"Launch Time":1427397534372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534416,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":8429185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1891,"Index":889,"Attempt":0,"Launch Time":1427397534418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1888,"Index":886,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301230,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1892,"Index":890,"Attempt":0,"Launch Time":1427397534419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1889,"Index":887,"Attempt":0,"Launch Time":1427397534412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534420,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332304,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1893,"Index":891,"Attempt":0,"Launch Time":1427397534421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1887,"Index":885,"Attempt":0,"Launch Time":1427397534408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534421,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1894,"Index":892,"Attempt":0,"Launch Time":1427397534422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1890,"Index":888,"Attempt":0,"Launch Time":1427397534415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1895,"Index":893,"Attempt":0,"Launch Time":1427397534425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1885,"Index":883,"Attempt":0,"Launch Time":1427397534398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2945781,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1896,"Index":894,"Attempt":0,"Launch Time":1427397534427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1891,"Index":889,"Attempt":0,"Launch Time":1427397534418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1897,"Index":895,"Attempt":0,"Launch Time":1427397534428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1892,"Index":890,"Attempt":0,"Launch Time":1427397534419,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1898,"Index":896,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1893,"Index":891,"Attempt":0,"Launch Time":1427397534421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1899,"Index":897,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1894,"Index":892,"Attempt":0,"Launch Time":1427397534422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312546,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1900,"Index":898,"Attempt":0,"Launch Time":1427397534430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1855,"Index":853,"Attempt":0,"Launch Time":1427397534304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":28161927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1901,"Index":899,"Attempt":0,"Launch Time":1427397534435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1897,"Index":895,"Attempt":0,"Launch Time":1427397534428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364433,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1902,"Index":900,"Attempt":0,"Launch Time":1427397534436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1896,"Index":894,"Attempt":0,"Launch Time":1427397534427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1903,"Index":901,"Attempt":0,"Launch Time":1427397534437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1895,"Index":893,"Attempt":0,"Launch Time":1427397534425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1904,"Index":902,"Attempt":0,"Launch Time":1427397534438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1858,"Index":856,"Attempt":0,"Launch Time":1427397534310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1905,"Index":903,"Attempt":0,"Launch Time":1427397534439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1898,"Index":896,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1906,"Index":904,"Attempt":0,"Launch Time":1427397534444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1857,"Index":855,"Attempt":0,"Launch Time":1427397534309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1907,"Index":905,"Attempt":0,"Launch Time":1427397534445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1900,"Index":898,"Attempt":0,"Launch Time":1427397534430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":397871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1908,"Index":906,"Attempt":0,"Launch Time":1427397534449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1904,"Index":902,"Attempt":0,"Launch Time":1427397534438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":585569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1909,"Index":907,"Attempt":0,"Launch Time":1427397534453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1906,"Index":904,"Attempt":0,"Launch Time":1427397534444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":377628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1910,"Index":908,"Attempt":0,"Launch Time":1427397534454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1907,"Index":905,"Attempt":0,"Launch Time":1427397534445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1911,"Index":909,"Attempt":0,"Launch Time":1427397534459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1899,"Index":897,"Attempt":0,"Launch Time":1427397534429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":657981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1912,"Index":910,"Attempt":0,"Launch Time":1427397534460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1908,"Index":906,"Attempt":0,"Launch Time":1427397534449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1913,"Index":911,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1909,"Index":907,"Attempt":0,"Launch Time":1427397534453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1914,"Index":912,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1910,"Index":908,"Attempt":0,"Launch Time":1427397534454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356363,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1915,"Index":913,"Attempt":0,"Launch Time":1427397534469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1912,"Index":910,"Attempt":0,"Launch Time":1427397534460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1916,"Index":914,"Attempt":0,"Launch Time":1427397534471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1913,"Index":911,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1917,"Index":915,"Attempt":0,"Launch Time":1427397534472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1914,"Index":912,"Attempt":0,"Launch Time":1427397534462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1918,"Index":916,"Attempt":0,"Launch Time":1427397534475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1911,"Index":909,"Attempt":0,"Launch Time":1427397534459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":367741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1919,"Index":917,"Attempt":0,"Launch Time":1427397534478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1915,"Index":913,"Attempt":0,"Launch Time":1427397534469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1920,"Index":918,"Attempt":0,"Launch Time":1427397534480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1916,"Index":914,"Attempt":0,"Launch Time":1427397534471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":418814,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1921,"Index":919,"Attempt":0,"Launch Time":1427397534481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1917,"Index":915,"Attempt":0,"Launch Time":1427397534472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":489944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1922,"Index":920,"Attempt":0,"Launch Time":1427397534486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1918,"Index":916,"Attempt":0,"Launch Time":1427397534475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1923,"Index":921,"Attempt":0,"Launch Time":1427397534487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1919,"Index":917,"Attempt":0,"Launch Time":1427397534478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1924,"Index":922,"Attempt":0,"Launch Time":1427397534489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1920,"Index":918,"Attempt":0,"Launch Time":1427397534480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1925,"Index":923,"Attempt":0,"Launch Time":1427397534490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1921,"Index":919,"Attempt":0,"Launch Time":1427397534481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1385840,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1926,"Index":924,"Attempt":0,"Launch Time":1427397534494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1923,"Index":921,"Attempt":0,"Launch Time":1427397534487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1927,"Index":925,"Attempt":0,"Launch Time":1427397534496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1922,"Index":920,"Attempt":0,"Launch Time":1427397534486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1928,"Index":926,"Attempt":0,"Launch Time":1427397534497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1924,"Index":922,"Attempt":0,"Launch Time":1427397534489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1929,"Index":927,"Attempt":0,"Launch Time":1427397534499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1925,"Index":923,"Attempt":0,"Launch Time":1427397534490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":501427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1930,"Index":928,"Attempt":0,"Launch Time":1427397534514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1926,"Index":924,"Attempt":0,"Launch Time":1427397534494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332346,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1931,"Index":929,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1905,"Index":903,"Attempt":0,"Launch Time":1427397534439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":353643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1932,"Index":930,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1927,"Index":925,"Attempt":0,"Launch Time":1427397534496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1933,"Index":931,"Attempt":0,"Launch Time":1427397534522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1934,"Index":932,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1929,"Index":927,"Attempt":0,"Launch Time":1427397534499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344812,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1928,"Index":926,"Attempt":0,"Launch Time":1427397534497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":462711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1935,"Index":933,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1930,"Index":928,"Attempt":0,"Launch Time":1427397534514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373195,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1936,"Index":934,"Attempt":0,"Launch Time":1427397534529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1931,"Index":929,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":465886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1937,"Index":935,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1932,"Index":930,"Attempt":0,"Launch Time":1427397534519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":439219,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1938,"Index":936,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1934,"Index":932,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":469687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1939,"Index":937,"Attempt":0,"Launch Time":1427397534533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1935,"Index":933,"Attempt":0,"Launch Time":1427397534523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":448794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1940,"Index":938,"Attempt":0,"Launch Time":1427397534534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1903,"Index":901,"Attempt":0,"Launch Time":1427397534437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":454155,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1941,"Index":939,"Attempt":0,"Launch Time":1427397534535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1933,"Index":931,"Attempt":0,"Launch Time":1427397534522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":635229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1942,"Index":940,"Attempt":0,"Launch Time":1427397534536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1936,"Index":934,"Attempt":0,"Launch Time":1427397534529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1943,"Index":941,"Attempt":0,"Launch Time":1427397534542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1937,"Index":935,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":399594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1944,"Index":942,"Attempt":0,"Launch Time":1427397534543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1941,"Index":939,"Attempt":0,"Launch Time":1427397534535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361707,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1945,"Index":943,"Attempt":0,"Launch Time":1427397534544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1942,"Index":940,"Attempt":0,"Launch Time":1427397534536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1946,"Index":944,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1940,"Index":938,"Attempt":0,"Launch Time":1427397534534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356563,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1947,"Index":945,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1948,"Index":946,"Attempt":0,"Launch Time":1427397534547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1939,"Index":937,"Attempt":0,"Launch Time":1427397534533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336760,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1938,"Index":936,"Attempt":0,"Launch Time":1427397534532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":855395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1949,"Index":947,"Attempt":0,"Launch Time":1427397534552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1945,"Index":943,"Attempt":0,"Launch Time":1427397534544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":422436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1950,"Index":948,"Attempt":0,"Launch Time":1427397534555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1946,"Index":944,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340968,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1951,"Index":949,"Attempt":0,"Launch Time":1427397534556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1948,"Index":946,"Attempt":0,"Launch Time":1427397534547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1952,"Index":950,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1944,"Index":942,"Attempt":0,"Launch Time":1427397534543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1953,"Index":951,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1902,"Index":900,"Attempt":0,"Launch Time":1427397534436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":42,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1954,"Index":952,"Attempt":0,"Launch Time":1427397534559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1943,"Index":941,"Attempt":0,"Launch Time":1427397534542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1955,"Index":953,"Attempt":0,"Launch Time":1427397534560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1949,"Index":947,"Attempt":0,"Launch Time":1427397534552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1956,"Index":954,"Attempt":0,"Launch Time":1427397534563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1947,"Index":945,"Attempt":0,"Launch Time":1427397534546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326052,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1957,"Index":955,"Attempt":0,"Launch Time":1427397534566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1901,"Index":899,"Attempt":0,"Launch Time":1427397534435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4236414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1958,"Index":956,"Attempt":0,"Launch Time":1427397534567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1955,"Index":953,"Attempt":0,"Launch Time":1427397534560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1959,"Index":957,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1958,"Index":956,"Attempt":0,"Launch Time":1427397534567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383290,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1960,"Index":958,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1957,"Index":955,"Attempt":0,"Launch Time":1427397534566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1961,"Index":959,"Attempt":0,"Launch Time":1427397534581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1951,"Index":949,"Attempt":0,"Launch Time":1427397534556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":430177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1962,"Index":960,"Attempt":0,"Launch Time":1427397534583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1959,"Index":957,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392041,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1963,"Index":961,"Attempt":0,"Launch Time":1427397534584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1953,"Index":951,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":458344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1964,"Index":962,"Attempt":0,"Launch Time":1427397534585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1960,"Index":958,"Attempt":0,"Launch Time":1427397534575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383063,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1965,"Index":963,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1966,"Index":964,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1962,"Index":960,"Attempt":0,"Launch Time":1427397534583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1961,"Index":959,"Attempt":0,"Launch Time":1427397534581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1967,"Index":965,"Attempt":0,"Launch Time":1427397534592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1964,"Index":962,"Attempt":0,"Launch Time":1427397534585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1968,"Index":966,"Attempt":0,"Launch Time":1427397534594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1963,"Index":961,"Attempt":0,"Launch Time":1427397534584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1969,"Index":967,"Attempt":0,"Launch Time":1427397534599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1966,"Index":964,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340757,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1970,"Index":968,"Attempt":0,"Launch Time":1427397534600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1967,"Index":965,"Attempt":0,"Launch Time":1427397534592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338434,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1971,"Index":969,"Attempt":0,"Launch Time":1427397534601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1965,"Index":963,"Attempt":0,"Launch Time":1427397534591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1972,"Index":970,"Attempt":0,"Launch Time":1427397534602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1968,"Index":966,"Attempt":0,"Launch Time":1427397534594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1973,"Index":971,"Attempt":0,"Launch Time":1427397534625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1969,"Index":967,"Attempt":0,"Launch Time":1427397534599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1974,"Index":972,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1970,"Index":968,"Attempt":0,"Launch Time":1427397534600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1975,"Index":973,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1971,"Index":969,"Attempt":0,"Launch Time":1427397534601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1976,"Index":974,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1973,"Index":971,"Attempt":0,"Launch Time":1427397534625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1977,"Index":975,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1974,"Index":972,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332727,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1978,"Index":976,"Attempt":0,"Launch Time":1427397534636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1975,"Index":973,"Attempt":0,"Launch Time":1427397534628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335048,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1979,"Index":977,"Attempt":0,"Launch Time":1427397534638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1972,"Index":970,"Attempt":0,"Launch Time":1427397534602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330062,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1980,"Index":978,"Attempt":0,"Launch Time":1427397534639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1956,"Index":954,"Attempt":0,"Launch Time":1427397534563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314348,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1981,"Index":979,"Attempt":0,"Launch Time":1427397534643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1976,"Index":974,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":704497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1982,"Index":980,"Attempt":0,"Launch Time":1427397534644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1977,"Index":975,"Attempt":0,"Launch Time":1427397534635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330702,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1983,"Index":981,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1980,"Index":978,"Attempt":0,"Launch Time":1427397534639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321206,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1984,"Index":982,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1979,"Index":977,"Attempt":0,"Launch Time":1427397534638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1985,"Index":983,"Attempt":0,"Launch Time":1427397534648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1978,"Index":976,"Attempt":0,"Launch Time":1427397534636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332707,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1986,"Index":984,"Attempt":0,"Launch Time":1427397534651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1982,"Index":980,"Attempt":0,"Launch Time":1427397534644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350326,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1987,"Index":985,"Attempt":0,"Launch Time":1427397534652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1981,"Index":979,"Attempt":0,"Launch Time":1427397534643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410055,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1988,"Index":986,"Attempt":0,"Launch Time":1427397534656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1985,"Index":983,"Attempt":0,"Launch Time":1427397534648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364294,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1989,"Index":987,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1984,"Index":982,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":608205,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1990,"Index":988,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1991,"Index":989,"Attempt":0,"Launch Time":1427397534660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1987,"Index":985,"Attempt":0,"Launch Time":1427397534652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1983,"Index":981,"Attempt":0,"Launch Time":1427397534647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":389688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1992,"Index":990,"Attempt":0,"Launch Time":1427397534663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1986,"Index":984,"Attempt":0,"Launch Time":1427397534651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1993,"Index":991,"Attempt":0,"Launch Time":1427397534664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1988,"Index":986,"Attempt":0,"Launch Time":1427397534656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1994,"Index":992,"Attempt":0,"Launch Time":1427397534665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1950,"Index":948,"Attempt":0,"Launch Time":1427397534555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1767356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1995,"Index":993,"Attempt":0,"Launch Time":1427397534667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1990,"Index":988,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379662,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1996,"Index":994,"Attempt":0,"Launch Time":1427397534668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1991,"Index":989,"Attempt":0,"Launch Time":1427397534660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":520453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1997,"Index":995,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1998,"Index":996,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1992,"Index":990,"Attempt":0,"Launch Time":1427397534663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":368161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1993,"Index":991,"Attempt":0,"Launch Time":1427397534664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368150,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":1999,"Index":997,"Attempt":0,"Launch Time":1427397534676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":2000,"Index":998,"Attempt":0,"Launch Time":1427397534677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1994,"Index":992,"Attempt":0,"Launch Time":1427397534665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1996,"Index":994,"Attempt":0,"Launch Time":1427397534668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358850,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":2001,"Index":999,"Attempt":0,"Launch Time":1427397534684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1989,"Index":987,"Attempt":0,"Launch Time":1427397534659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":404274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1954,"Index":952,"Attempt":0,"Launch Time":1427397534559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":125,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":595541,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1997,"Index":995,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1998,"Index":996,"Attempt":0,"Launch Time":1427397534672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1952,"Index":950,"Attempt":0,"Launch Time":1427397534557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2918877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2001,"Index":999,"Attempt":0,"Launch Time":1427397534684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377136,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2000,"Index":998,"Attempt":0,"Launch Time":1427397534677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1999,"Index":997,"Attempt":0,"Launch Time":1427397534676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":537520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1995,"Index":993,"Attempt":0,"Launch Time":1427397534667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397532127,"Completion Time":1427397534702,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":2002,"Index":0,"Attempt":0,"Launch Time":1427397534707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2002,"Index":0,"Attempt":0,"Launch Time":1427397534707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397534876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":164,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192025,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line24.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line24.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line24.$read$$iwC$$iwC$$iwC.(:39)\n$line24.$read$$iwC$$iwC.(:41)\n$line24.$read$$iwC.(:43)\n$line24.$read.(:45)\n$line24.$read$.(:49)\n$line24.$read$.()\n$line24.$eval$.(:7)\n$line24.$eval$.()\n$line24.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397534707,"Completion Time":1427397534876,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1427397534876,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1427397541484,"Stage Infos":[{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[6,7]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2003,"Index":0,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2004,"Index":1,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2005,"Index":2,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2006,"Index":3,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2007,"Index":4,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2008,"Index":5,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2009,"Index":6,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2010,"Index":7,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2011,"Index":8,"Attempt":0,"Launch Time":1427397541504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2009,"Index":6,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2012,"Index":9,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2013,"Index":10,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2003,"Index":0,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2004,"Index":1,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2014,"Index":11,"Attempt":0,"Launch Time":1427397541508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2015,"Index":12,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2016,"Index":13,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2005,"Index":2,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325092,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2006,"Index":3,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":292197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2017,"Index":14,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2018,"Index":15,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2007,"Index":4,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2010,"Index":7,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":651252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2019,"Index":16,"Attempt":0,"Launch Time":1427397541515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2020,"Index":17,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2021,"Index":18,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2022,"Index":19,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2023,"Index":20,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2012,"Index":9,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2015,"Index":12,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321937,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2013,"Index":10,"Attempt":0,"Launch Time":1427397541506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2008,"Index":5,"Attempt":0,"Launch Time":1427397541495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2024,"Index":21,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2014,"Index":11,"Attempt":0,"Launch Time":1427397541508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2016,"Index":13,"Attempt":0,"Launch Time":1427397541509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2011,"Index":8,"Attempt":0,"Launch Time":1427397541504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2025,"Index":22,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2026,"Index":23,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2027,"Index":24,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2028,"Index":25,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2029,"Index":26,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2030,"Index":27,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2031,"Index":28,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2017,"Index":14,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":272820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2021,"Index":18,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2018,"Index":15,"Attempt":0,"Launch Time":1427397541510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2022,"Index":19,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2023,"Index":20,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2019,"Index":16,"Attempt":0,"Launch Time":1427397541515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":251904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2024,"Index":21,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2032,"Index":29,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2033,"Index":30,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2034,"Index":31,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2026,"Index":23,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340640,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2020,"Index":17,"Attempt":0,"Launch Time":1427397541516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2035,"Index":32,"Attempt":0,"Launch Time":1427397541532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2036,"Index":33,"Attempt":0,"Launch Time":1427397541533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2027,"Index":24,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2025,"Index":22,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2037,"Index":34,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2038,"Index":35,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2028,"Index":25,"Attempt":0,"Launch Time":1427397541524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2031,"Index":28,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2039,"Index":36,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2029,"Index":26,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2030,"Index":27,"Attempt":0,"Launch Time":1427397541525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2040,"Index":37,"Attempt":0,"Launch Time":1427397541538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2041,"Index":38,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2033,"Index":30,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2042,"Index":39,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2043,"Index":40,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2034,"Index":31,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":288716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2036,"Index":33,"Attempt":0,"Launch Time":1427397541533,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2032,"Index":29,"Attempt":0,"Launch Time":1427397541531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":273262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2044,"Index":41,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2045,"Index":42,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2037,"Index":34,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280055,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2035,"Index":32,"Attempt":0,"Launch Time":1427397541532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268880,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2046,"Index":43,"Attempt":0,"Launch Time":1427397541544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2047,"Index":44,"Attempt":0,"Launch Time":1427397541545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2038,"Index":35,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2048,"Index":45,"Attempt":0,"Launch Time":1427397541547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2049,"Index":46,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2050,"Index":47,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2051,"Index":48,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2041,"Index":38,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2043,"Index":40,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2042,"Index":39,"Attempt":0,"Launch Time":1427397541539,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":393226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2039,"Index":36,"Attempt":0,"Launch Time":1427397541534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2040,"Index":37,"Attempt":0,"Launch Time":1427397541538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":365613,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2052,"Index":49,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2053,"Index":50,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2054,"Index":51,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2055,"Index":52,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2056,"Index":53,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2049,"Index":46,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":278951,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2050,"Index":47,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":248392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2044,"Index":41,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":560782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2045,"Index":42,"Attempt":0,"Launch Time":1427397541540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2057,"Index":54,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2058,"Index":55,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2047,"Index":44,"Attempt":0,"Launch Time":1427397541545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":421223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2046,"Index":43,"Attempt":0,"Launch Time":1427397541544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260547,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2059,"Index":56,"Attempt":0,"Launch Time":1427397541571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2048,"Index":45,"Attempt":0,"Launch Time":1427397541547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":287921,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2051,"Index":48,"Attempt":0,"Launch Time":1427397541555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2060,"Index":57,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2061,"Index":58,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2053,"Index":50,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306360,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2052,"Index":49,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2054,"Index":51,"Attempt":0,"Launch Time":1427397541565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2062,"Index":59,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2063,"Index":60,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2064,"Index":61,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2056,"Index":53,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2055,"Index":52,"Attempt":0,"Launch Time":1427397541566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2065,"Index":62,"Attempt":0,"Launch Time":1427397541577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2057,"Index":54,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333918,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2066,"Index":63,"Attempt":0,"Launch Time":1427397541579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2059,"Index":56,"Attempt":0,"Launch Time":1427397541571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337549,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2067,"Index":64,"Attempt":0,"Launch Time":1427397541580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2058,"Index":55,"Attempt":0,"Launch Time":1427397541570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":431716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2068,"Index":65,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2069,"Index":66,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2070,"Index":67,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2071,"Index":68,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2060,"Index":57,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2062,"Index":59,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":340081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2064,"Index":61,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2061,"Index":58,"Attempt":0,"Launch Time":1427397541573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386734,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2072,"Index":69,"Attempt":0,"Launch Time":1427397541585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2063,"Index":60,"Attempt":0,"Launch Time":1427397541575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321268,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2073,"Index":70,"Attempt":0,"Launch Time":1427397541589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2068,"Index":65,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2074,"Index":71,"Attempt":0,"Launch Time":1427397541590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2075,"Index":72,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2076,"Index":73,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2066,"Index":63,"Attempt":0,"Launch Time":1427397541579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":278341,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2069,"Index":66,"Attempt":0,"Launch Time":1427397541582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2070,"Index":67,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":276864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2077,"Index":74,"Attempt":0,"Launch Time":1427397541592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2067,"Index":64,"Attempt":0,"Launch Time":1427397541580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2065,"Index":62,"Attempt":0,"Launch Time":1427397541577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":291411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2078,"Index":75,"Attempt":0,"Launch Time":1427397541593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2079,"Index":76,"Attempt":0,"Launch Time":1427397541595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2071,"Index":68,"Attempt":0,"Launch Time":1427397541583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317700,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2080,"Index":77,"Attempt":0,"Launch Time":1427397541597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2073,"Index":70,"Attempt":0,"Launch Time":1427397541589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2081,"Index":78,"Attempt":0,"Launch Time":1427397541600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2072,"Index":69,"Attempt":0,"Launch Time":1427397541585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1048143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2082,"Index":79,"Attempt":0,"Launch Time":1427397541601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2078,"Index":75,"Attempt":0,"Launch Time":1427397541593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2083,"Index":80,"Attempt":0,"Launch Time":1427397541603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2075,"Index":72,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308018,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2084,"Index":81,"Attempt":0,"Launch Time":1427397541606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2080,"Index":77,"Attempt":0,"Launch Time":1427397541597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2085,"Index":82,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2082,"Index":79,"Attempt":0,"Launch Time":1427397541601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295670,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2086,"Index":83,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2081,"Index":78,"Attempt":0,"Launch Time":1427397541600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333637,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2087,"Index":84,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2085,"Index":82,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2088,"Index":85,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2086,"Index":83,"Attempt":0,"Launch Time":1427397541607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2089,"Index":86,"Attempt":0,"Launch Time":1427397541619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2088,"Index":85,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2090,"Index":87,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2083,"Index":80,"Attempt":0,"Launch Time":1427397541603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2091,"Index":88,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2087,"Index":84,"Attempt":0,"Launch Time":1427397541613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410155,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2092,"Index":89,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2076,"Index":73,"Attempt":0,"Launch Time":1427397541591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541622,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2093,"Index":90,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2084,"Index":81,"Attempt":0,"Launch Time":1427397541606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2094,"Index":91,"Attempt":0,"Launch Time":1427397541632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2090,"Index":87,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328543,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2095,"Index":92,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2089,"Index":86,"Attempt":0,"Launch Time":1427397541619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2096,"Index":93,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2097,"Index":94,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2092,"Index":89,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301097,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2079,"Index":76,"Attempt":0,"Launch Time":1427397541595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2098,"Index":95,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2093,"Index":90,"Attempt":0,"Launch Time":1427397541622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":552776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2099,"Index":96,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2077,"Index":74,"Attempt":0,"Launch Time":1427397541592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":47,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346052,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2100,"Index":97,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2074,"Index":71,"Attempt":0,"Launch Time":1427397541590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":8493928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2101,"Index":98,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2095,"Index":92,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2102,"Index":99,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2097,"Index":94,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2103,"Index":100,"Attempt":0,"Launch Time":1427397541644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2098,"Index":95,"Attempt":0,"Launch Time":1427397541637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2104,"Index":101,"Attempt":0,"Launch Time":1427397541645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2094,"Index":91,"Attempt":0,"Launch Time":1427397541632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":5541151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2105,"Index":102,"Attempt":0,"Launch Time":1427397541646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2091,"Index":88,"Attempt":0,"Launch Time":1427397541620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":655760,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2106,"Index":103,"Attempt":0,"Launch Time":1427397541650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2101,"Index":98,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2107,"Index":104,"Attempt":0,"Launch Time":1427397541652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2104,"Index":101,"Attempt":0,"Launch Time":1427397541645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":268008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2108,"Index":105,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2105,"Index":102,"Attempt":0,"Launch Time":1427397541646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2109,"Index":106,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2102,"Index":99,"Attempt":0,"Launch Time":1427397541643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2110,"Index":107,"Attempt":0,"Launch Time":1427397541658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2106,"Index":103,"Attempt":0,"Launch Time":1427397541650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":234805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2111,"Index":108,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2112,"Index":109,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2108,"Index":105,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296499,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2103,"Index":100,"Attempt":0,"Launch Time":1427397541644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":275726,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2113,"Index":110,"Attempt":0,"Launch Time":1427397541663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2100,"Index":97,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2114,"Index":111,"Attempt":0,"Launch Time":1427397541664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2109,"Index":106,"Attempt":0,"Launch Time":1427397541656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302224,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2115,"Index":112,"Attempt":0,"Launch Time":1427397541665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2110,"Index":107,"Attempt":0,"Launch Time":1427397541658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338381,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2116,"Index":113,"Attempt":0,"Launch Time":1427397541666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2099,"Index":96,"Attempt":0,"Launch Time":1427397541641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2117,"Index":114,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2112,"Index":109,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317920,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2118,"Index":115,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2111,"Index":108,"Attempt":0,"Launch Time":1427397541661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2119,"Index":116,"Attempt":0,"Launch Time":1427397541671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2114,"Index":111,"Attempt":0,"Launch Time":1427397541664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2120,"Index":117,"Attempt":0,"Launch Time":1427397541672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2115,"Index":112,"Attempt":0,"Launch Time":1427397541665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2121,"Index":118,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2117,"Index":114,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2122,"Index":119,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2118,"Index":115,"Attempt":0,"Launch Time":1427397541668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2123,"Index":120,"Attempt":0,"Launch Time":1427397541677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2107,"Index":104,"Attempt":0,"Launch Time":1427397541652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2124,"Index":121,"Attempt":0,"Launch Time":1427397541679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2120,"Index":117,"Attempt":0,"Launch Time":1427397541672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2125,"Index":122,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2116,"Index":113,"Attempt":0,"Launch Time":1427397541666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1947325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2119,"Index":116,"Attempt":0,"Launch Time":1427397541671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305331,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2126,"Index":123,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2127,"Index":124,"Attempt":0,"Launch Time":1427397541683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2122,"Index":119,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":682299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2128,"Index":125,"Attempt":0,"Launch Time":1427397541684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2123,"Index":120,"Attempt":0,"Launch Time":1427397541677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2129,"Index":126,"Attempt":0,"Launch Time":1427397541685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2121,"Index":118,"Attempt":0,"Launch Time":1427397541675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2130,"Index":127,"Attempt":0,"Launch Time":1427397541688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2126,"Index":123,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276277,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2131,"Index":128,"Attempt":0,"Launch Time":1427397541689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2124,"Index":121,"Attempt":0,"Launch Time":1427397541679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2132,"Index":129,"Attempt":0,"Launch Time":1427397541690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2127,"Index":124,"Attempt":0,"Launch Time":1427397541683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2133,"Index":130,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2134,"Index":131,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2128,"Index":125,"Attempt":0,"Launch Time":1427397541684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2130,"Index":127,"Attempt":0,"Launch Time":1427397541688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":285980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2135,"Index":132,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2136,"Index":133,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2131,"Index":128,"Attempt":0,"Launch Time":1427397541689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":503242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2132,"Index":129,"Attempt":0,"Launch Time":1427397541690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2137,"Index":134,"Attempt":0,"Launch Time":1427397541699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2129,"Index":126,"Attempt":0,"Launch Time":1427397541685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":292813,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2138,"Index":135,"Attempt":0,"Launch Time":1427397541711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2134,"Index":131,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333075,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2139,"Index":136,"Attempt":0,"Launch Time":1427397541712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2133,"Index":130,"Attempt":0,"Launch Time":1427397541694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":258059,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2140,"Index":137,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2125,"Index":122,"Attempt":0,"Launch Time":1427397541681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2002187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2141,"Index":138,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2096,"Index":93,"Attempt":0,"Launch Time":1427397541636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":67,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2142,"Index":139,"Attempt":0,"Launch Time":1427397541717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2137,"Index":134,"Attempt":0,"Launch Time":1427397541699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2143,"Index":140,"Attempt":0,"Launch Time":1427397541719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2113,"Index":110,"Attempt":0,"Launch Time":1427397541663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2144,"Index":141,"Attempt":0,"Launch Time":1427397541722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2140,"Index":137,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2145,"Index":142,"Attempt":0,"Launch Time":1427397541723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2142,"Index":139,"Attempt":0,"Launch Time":1427397541717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316648,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2146,"Index":143,"Attempt":0,"Launch Time":1427397541727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2143,"Index":140,"Attempt":0,"Launch Time":1427397541719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":310701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2147,"Index":144,"Attempt":0,"Launch Time":1427397541728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2141,"Index":138,"Attempt":0,"Launch Time":1427397541715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2148,"Index":145,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2149,"Index":146,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2145,"Index":142,"Attempt":0,"Launch Time":1427397541723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":377831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2144,"Index":141,"Attempt":0,"Launch Time":1427397541722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":672517,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2150,"Index":147,"Attempt":0,"Launch Time":1427397541734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2147,"Index":144,"Attempt":0,"Launch Time":1427397541728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2151,"Index":148,"Attempt":0,"Launch Time":1427397541735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2146,"Index":143,"Attempt":0,"Launch Time":1427397541727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2152,"Index":149,"Attempt":0,"Launch Time":1427397541737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2149,"Index":146,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":270722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2153,"Index":150,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2151,"Index":148,"Attempt":0,"Launch Time":1427397541735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2154,"Index":151,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2150,"Index":147,"Attempt":0,"Launch Time":1427397541734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":257965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2155,"Index":152,"Attempt":0,"Launch Time":1427397541743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2152,"Index":149,"Attempt":0,"Launch Time":1427397541737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":290612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2156,"Index":153,"Attempt":0,"Launch Time":1427397541747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2153,"Index":150,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297791,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2157,"Index":154,"Attempt":0,"Launch Time":1427397541748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2154,"Index":151,"Attempt":0,"Launch Time":1427397541741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2158,"Index":155,"Attempt":0,"Launch Time":1427397541757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2157,"Index":154,"Attempt":0,"Launch Time":1427397541748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309058,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2159,"Index":156,"Attempt":0,"Launch Time":1427397541766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2158,"Index":155,"Attempt":0,"Launch Time":1427397541757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327319,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2160,"Index":157,"Attempt":0,"Launch Time":1427397541767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2148,"Index":145,"Attempt":0,"Launch Time":1427397541731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":338544,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2161,"Index":158,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2155,"Index":152,"Attempt":0,"Launch Time":1427397541743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338755,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2162,"Index":159,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2156,"Index":153,"Attempt":0,"Launch Time":1427397541747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363485,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2163,"Index":160,"Attempt":0,"Launch Time":1427397541773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2159,"Index":156,"Attempt":0,"Launch Time":1427397541766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2164,"Index":161,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2160,"Index":157,"Attempt":0,"Launch Time":1427397541767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303647,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2165,"Index":162,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2161,"Index":158,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2166,"Index":163,"Attempt":0,"Launch Time":1427397541776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2162,"Index":159,"Attempt":0,"Launch Time":1427397541768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":305405,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2167,"Index":164,"Attempt":0,"Launch Time":1427397541777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2138,"Index":135,"Attempt":0,"Launch Time":1427397541711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3571032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2168,"Index":165,"Attempt":0,"Launch Time":1427397541780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2163,"Index":160,"Attempt":0,"Launch Time":1427397541773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347409,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2169,"Index":166,"Attempt":0,"Launch Time":1427397541781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2164,"Index":161,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338101,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2170,"Index":167,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2165,"Index":162,"Attempt":0,"Launch Time":1427397541774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2171,"Index":168,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2167,"Index":164,"Attempt":0,"Launch Time":1427397541777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347463,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2172,"Index":169,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2166,"Index":163,"Attempt":0,"Launch Time":1427397541776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320097,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2173,"Index":170,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2169,"Index":166,"Attempt":0,"Launch Time":1427397541781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2174,"Index":171,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2168,"Index":165,"Attempt":0,"Launch Time":1427397541780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":306228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2175,"Index":172,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2171,"Index":168,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2176,"Index":173,"Attempt":0,"Launch Time":1427397541794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2173,"Index":170,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":263125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2177,"Index":174,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2170,"Index":167,"Attempt":0,"Launch Time":1427397541784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":640643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2178,"Index":175,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2174,"Index":171,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2179,"Index":176,"Attempt":0,"Launch Time":1427397541797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2175,"Index":172,"Attempt":0,"Launch Time":1427397541790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2180,"Index":177,"Attempt":0,"Launch Time":1427397541800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2176,"Index":173,"Attempt":0,"Launch Time":1427397541794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2181,"Index":178,"Attempt":0,"Launch Time":1427397541802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2177,"Index":174,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2182,"Index":179,"Attempt":0,"Launch Time":1427397541803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2178,"Index":175,"Attempt":0,"Launch Time":1427397541796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2183,"Index":180,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2181,"Index":178,"Attempt":0,"Launch Time":1427397541802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323029,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2184,"Index":181,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2179,"Index":176,"Attempt":0,"Launch Time":1427397541797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2185,"Index":182,"Attempt":0,"Launch Time":1427397541820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2180,"Index":177,"Attempt":0,"Launch Time":1427397541800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2186,"Index":183,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2139,"Index":136,"Attempt":0,"Launch Time":1427397541712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541823,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":108,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2187,"Index":184,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2188,"Index":185,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2189,"Index":186,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2182,"Index":179,"Attempt":0,"Launch Time":1427397541803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2172,"Index":169,"Attempt":0,"Launch Time":1427397541787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399774,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2184,"Index":181,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2190,"Index":187,"Attempt":0,"Launch Time":1427397541825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2183,"Index":180,"Attempt":0,"Launch Time":1427397541816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2191,"Index":188,"Attempt":0,"Launch Time":1427397541826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2185,"Index":182,"Attempt":0,"Launch Time":1427397541820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2192,"Index":189,"Attempt":0,"Launch Time":1427397541830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2187,"Index":184,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2193,"Index":190,"Attempt":0,"Launch Time":1427397541831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2186,"Index":183,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1228549,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2194,"Index":191,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2195,"Index":192,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2191,"Index":188,"Attempt":0,"Launch Time":1427397541826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2188,"Index":185,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":390348,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2196,"Index":193,"Attempt":0,"Launch Time":1427397541835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2135,"Index":132,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":137,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2197,"Index":194,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2189,"Index":186,"Attempt":0,"Launch Time":1427397541823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2198,"Index":195,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2136,"Index":133,"Attempt":0,"Launch Time":1427397541696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":110603586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2199,"Index":196,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2190,"Index":187,"Attempt":0,"Launch Time":1427397541825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":499027,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2200,"Index":197,"Attempt":0,"Launch Time":1427397541841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2193,"Index":190,"Attempt":0,"Launch Time":1427397541831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2201,"Index":198,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2199,"Index":196,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":574202,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2202,"Index":199,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2196,"Index":193,"Attempt":0,"Launch Time":1427397541835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":309753,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2203,"Index":200,"Attempt":0,"Launch Time":1427397541850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2197,"Index":194,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2204,"Index":201,"Attempt":0,"Launch Time":1427397541853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2195,"Index":192,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":304244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2205,"Index":202,"Attempt":0,"Launch Time":1427397541854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2201,"Index":198,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2206,"Index":203,"Attempt":0,"Launch Time":1427397541855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2202,"Index":199,"Attempt":0,"Launch Time":1427397541848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2207,"Index":204,"Attempt":0,"Launch Time":1427397541859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2203,"Index":200,"Attempt":0,"Launch Time":1427397541850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":397446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2208,"Index":205,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2205,"Index":202,"Attempt":0,"Launch Time":1427397541854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364576,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2209,"Index":206,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2204,"Index":201,"Attempt":0,"Launch Time":1427397541853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2210,"Index":207,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2206,"Index":203,"Attempt":0,"Launch Time":1427397541855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313555,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2211,"Index":208,"Attempt":0,"Launch Time":1427397541866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2207,"Index":204,"Attempt":0,"Launch Time":1427397541859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306816,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2212,"Index":209,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2209,"Index":206,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326738,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2213,"Index":210,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2210,"Index":207,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324580,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2214,"Index":211,"Attempt":0,"Launch Time":1427397541870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2208,"Index":205,"Attempt":0,"Launch Time":1427397541862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2215,"Index":212,"Attempt":0,"Launch Time":1427397541872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2211,"Index":208,"Attempt":0,"Launch Time":1427397541866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2216,"Index":213,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2212,"Index":209,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302200,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2217,"Index":214,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2214,"Index":211,"Attempt":0,"Launch Time":1427397541870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2218,"Index":215,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2213,"Index":210,"Attempt":0,"Launch Time":1427397541869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2219,"Index":216,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2215,"Index":212,"Attempt":0,"Launch Time":1427397541872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2220,"Index":217,"Attempt":0,"Launch Time":1427397541886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2217,"Index":214,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2221,"Index":218,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2222,"Index":219,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2218,"Index":215,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348881,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2216,"Index":213,"Attempt":0,"Launch Time":1427397541878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2223,"Index":220,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2219,"Index":216,"Attempt":0,"Launch Time":1427397541879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":636809,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2224,"Index":221,"Attempt":0,"Launch Time":1427397541893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2221,"Index":218,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316372,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2225,"Index":222,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2223,"Index":220,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2226,"Index":223,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2220,"Index":217,"Attempt":0,"Launch Time":1427397541886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2227,"Index":224,"Attempt":0,"Launch Time":1427397541896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2198,"Index":195,"Attempt":0,"Launch Time":1427397541840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2228,"Index":225,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2222,"Index":219,"Attempt":0,"Launch Time":1427397541887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":851978,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2229,"Index":226,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2225,"Index":222,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2230,"Index":227,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2224,"Index":221,"Attempt":0,"Launch Time":1427397541893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2231,"Index":228,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2226,"Index":223,"Attempt":0,"Launch Time":1427397541894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355807,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2232,"Index":229,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2229,"Index":226,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2233,"Index":230,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2228,"Index":225,"Attempt":0,"Launch Time":1427397541909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":437916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2234,"Index":231,"Attempt":0,"Launch Time":1427397541918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2227,"Index":224,"Attempt":0,"Launch Time":1427397541896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1112626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2235,"Index":232,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2230,"Index":227,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2236,"Index":233,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2231,"Index":228,"Attempt":0,"Launch Time":1427397541912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2237,"Index":234,"Attempt":0,"Launch Time":1427397541924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2232,"Index":229,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":320983,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2238,"Index":235,"Attempt":0,"Launch Time":1427397541926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2233,"Index":230,"Attempt":0,"Launch Time":1427397541917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2239,"Index":236,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2235,"Index":232,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2240,"Index":237,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2236,"Index":233,"Attempt":0,"Launch Time":1427397541919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2241,"Index":238,"Attempt":0,"Launch Time":1427397541929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2234,"Index":231,"Attempt":0,"Launch Time":1427397541918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2242,"Index":239,"Attempt":0,"Launch Time":1427397541931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2237,"Index":234,"Attempt":0,"Launch Time":1427397541924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":395576,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2243,"Index":240,"Attempt":0,"Launch Time":1427397541932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2238,"Index":235,"Attempt":0,"Launch Time":1427397541926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2244,"Index":241,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2240,"Index":237,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":411742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2245,"Index":242,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2242,"Index":239,"Attempt":0,"Launch Time":1427397541931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2246,"Index":243,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2247,"Index":244,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2192,"Index":189,"Attempt":0,"Launch Time":1427397541830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":110,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2239,"Index":236,"Attempt":0,"Launch Time":1427397541927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2248,"Index":245,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2249,"Index":246,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2243,"Index":240,"Attempt":0,"Launch Time":1427397541932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":478244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2241,"Index":238,"Attempt":0,"Launch Time":1427397541929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2250,"Index":247,"Attempt":0,"Launch Time":1427397541951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2249,"Index":246,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2251,"Index":248,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2246,"Index":243,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":469396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2252,"Index":249,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2244,"Index":241,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":409163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2253,"Index":250,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2248,"Index":245,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":511449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2254,"Index":251,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2200,"Index":197,"Attempt":0,"Launch Time":1427397541841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":47,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":652169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2245,"Index":242,"Attempt":0,"Launch Time":1427397541941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2255,"Index":252,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2256,"Index":253,"Attempt":0,"Launch Time":1427397541958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2250,"Index":247,"Attempt":0,"Launch Time":1427397541951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2257,"Index":254,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2252,"Index":249,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":344191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2258,"Index":255,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2251,"Index":248,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2259,"Index":256,"Attempt":0,"Launch Time":1427397541962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2253,"Index":250,"Attempt":0,"Launch Time":1427397541952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":458451,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2260,"Index":257,"Attempt":0,"Launch Time":1427397541963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2254,"Index":251,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2261,"Index":258,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2256,"Index":253,"Attempt":0,"Launch Time":1427397541958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392625,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2262,"Index":259,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2257,"Index":254,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2263,"Index":260,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2258,"Index":255,"Attempt":0,"Launch Time":1427397541959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2264,"Index":261,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2255,"Index":252,"Attempt":0,"Launch Time":1427397541954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290937,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2265,"Index":262,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2247,"Index":244,"Attempt":0,"Launch Time":1427397541942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285444,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2266,"Index":263,"Attempt":0,"Launch Time":1427397541972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2194,"Index":191,"Attempt":0,"Launch Time":1427397541833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2267,"Index":264,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2268,"Index":265,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2265,"Index":262,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2264,"Index":261,"Attempt":0,"Launch Time":1427397541970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2269,"Index":266,"Attempt":0,"Launch Time":1427397541983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2270,"Index":267,"Attempt":0,"Launch Time":1427397541984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2267,"Index":264,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2268,"Index":265,"Attempt":0,"Launch Time":1427397541977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":309345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2271,"Index":268,"Attempt":0,"Launch Time":1427397541985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2266,"Index":263,"Attempt":0,"Launch Time":1427397541972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2272,"Index":269,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2269,"Index":266,"Attempt":0,"Launch Time":1427397541983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335486,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2273,"Index":270,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2270,"Index":267,"Attempt":0,"Launch Time":1427397541984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397541992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2274,"Index":271,"Attempt":0,"Launch Time":1427397542004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2271,"Index":268,"Attempt":0,"Launch Time":1427397541985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419921,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2275,"Index":272,"Attempt":0,"Launch Time":1427397542005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2273,"Index":270,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2276,"Index":273,"Attempt":0,"Launch Time":1427397542008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2272,"Index":269,"Attempt":0,"Launch Time":1427397541992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542008,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8147200,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2277,"Index":274,"Attempt":0,"Launch Time":1427397542014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2275,"Index":272,"Attempt":0,"Launch Time":1427397542005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2278,"Index":275,"Attempt":0,"Launch Time":1427397542015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2274,"Index":271,"Attempt":0,"Launch Time":1427397542004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2279,"Index":276,"Attempt":0,"Launch Time":1427397542016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2276,"Index":273,"Attempt":0,"Launch Time":1427397542008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":330288,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2280,"Index":277,"Attempt":0,"Launch Time":1427397542019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2260,"Index":257,"Attempt":0,"Launch Time":1427397541963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":566607,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2281,"Index":278,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2282,"Index":279,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2277,"Index":274,"Attempt":0,"Launch Time":1427397542014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2278,"Index":275,"Attempt":0,"Launch Time":1427397542015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2283,"Index":280,"Attempt":0,"Launch Time":1427397542023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2279,"Index":276,"Attempt":0,"Launch Time":1427397542016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2284,"Index":281,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2285,"Index":282,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2280,"Index":277,"Attempt":0,"Launch Time":1427397542019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":311465,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2281,"Index":278,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371639,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2286,"Index":283,"Attempt":0,"Launch Time":1427397542030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2283,"Index":280,"Attempt":0,"Launch Time":1427397542023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2287,"Index":284,"Attempt":0,"Launch Time":1427397542035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2285,"Index":282,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2288,"Index":285,"Attempt":0,"Launch Time":1427397542037,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2284,"Index":281,"Attempt":0,"Launch Time":1427397542028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542037,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2289,"Index":286,"Attempt":0,"Launch Time":1427397542039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2286,"Index":283,"Attempt":0,"Launch Time":1427397542030,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2290,"Index":287,"Attempt":0,"Launch Time":1427397542043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2291,"Index":288,"Attempt":0,"Launch Time":1427397542044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2287,"Index":284,"Attempt":0,"Launch Time":1427397542035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":381503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2259,"Index":256,"Attempt":0,"Launch Time":1427397541962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":606222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2292,"Index":289,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2293,"Index":290,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2289,"Index":286,"Attempt":0,"Launch Time":1427397542039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2288,"Index":285,"Attempt":0,"Launch Time":1427397542037,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":530244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2294,"Index":291,"Attempt":0,"Launch Time":1427397542052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2291,"Index":288,"Attempt":0,"Launch Time":1427397542044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":920799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2295,"Index":292,"Attempt":0,"Launch Time":1427397542053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2293,"Index":290,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2296,"Index":293,"Attempt":0,"Launch Time":1427397542054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2292,"Index":289,"Attempt":0,"Launch Time":1427397542046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307930,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2297,"Index":294,"Attempt":0,"Launch Time":1427397542059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2290,"Index":287,"Attempt":0,"Launch Time":1427397542043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378068,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2298,"Index":295,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2295,"Index":292,"Attempt":0,"Launch Time":1427397542053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327275,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2299,"Index":296,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2294,"Index":291,"Attempt":0,"Launch Time":1427397542052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":359523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2300,"Index":297,"Attempt":0,"Launch Time":1427397542061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2296,"Index":293,"Attempt":0,"Launch Time":1427397542054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2301,"Index":298,"Attempt":0,"Launch Time":1427397542067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2298,"Index":295,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2302,"Index":299,"Attempt":0,"Launch Time":1427397542068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2300,"Index":297,"Attempt":0,"Launch Time":1427397542061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2303,"Index":300,"Attempt":0,"Launch Time":1427397542070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2299,"Index":296,"Attempt":0,"Launch Time":1427397542060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322139,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2304,"Index":301,"Attempt":0,"Launch Time":1427397542074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2301,"Index":298,"Attempt":0,"Launch Time":1427397542067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323713,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2305,"Index":302,"Attempt":0,"Launch Time":1427397542077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2302,"Index":299,"Attempt":0,"Launch Time":1427397542068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":396110,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2306,"Index":303,"Attempt":0,"Launch Time":1427397542078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2303,"Index":300,"Attempt":0,"Launch Time":1427397542070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343527,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2307,"Index":304,"Attempt":0,"Launch Time":1427397542081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2304,"Index":301,"Attempt":0,"Launch Time":1427397542074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2308,"Index":305,"Attempt":0,"Launch Time":1427397542085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2306,"Index":303,"Attempt":0,"Launch Time":1427397542078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2309,"Index":306,"Attempt":0,"Launch Time":1427397542087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2305,"Index":302,"Attempt":0,"Launch Time":1427397542077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336300,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2310,"Index":307,"Attempt":0,"Launch Time":1427397542088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2307,"Index":304,"Attempt":0,"Launch Time":1427397542081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328348,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2311,"Index":308,"Attempt":0,"Launch Time":1427397542089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2263,"Index":260,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2312,"Index":309,"Attempt":0,"Launch Time":1427397542094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2262,"Index":259,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":69,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2313,"Index":310,"Attempt":0,"Launch Time":1427397542095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2261,"Index":258,"Attempt":0,"Launch Time":1427397541966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":442406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2314,"Index":311,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2297,"Index":294,"Attempt":0,"Launch Time":1427397542059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2315,"Index":312,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2282,"Index":279,"Attempt":0,"Launch Time":1427397542021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1147999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2316,"Index":313,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2317,"Index":314,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2311,"Index":308,"Attempt":0,"Launch Time":1427397542089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324538,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2312,"Index":309,"Attempt":0,"Launch Time":1427397542094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":282174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2318,"Index":315,"Attempt":0,"Launch Time":1427397542110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2315,"Index":312,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342022,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2319,"Index":316,"Attempt":0,"Launch Time":1427397542112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2314,"Index":311,"Attempt":0,"Launch Time":1427397542097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2320,"Index":317,"Attempt":0,"Launch Time":1427397542114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2313,"Index":310,"Attempt":0,"Launch Time":1427397542095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2321,"Index":318,"Attempt":0,"Launch Time":1427397542116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2318,"Index":315,"Attempt":0,"Launch Time":1427397542110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322110,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2322,"Index":319,"Attempt":0,"Launch Time":1427397542118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2310,"Index":307,"Attempt":0,"Launch Time":1427397542088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2323,"Index":320,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2324,"Index":321,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2316,"Index":313,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2319,"Index":316,"Attempt":0,"Launch Time":1427397542112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318881,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2325,"Index":322,"Attempt":0,"Launch Time":1427397542121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2320,"Index":317,"Attempt":0,"Launch Time":1427397542114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321218,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2326,"Index":323,"Attempt":0,"Launch Time":1427397542123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2317,"Index":314,"Attempt":0,"Launch Time":1427397542101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2327,"Index":324,"Attempt":0,"Launch Time":1427397542124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2321,"Index":318,"Attempt":0,"Launch Time":1427397542116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":408981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2328,"Index":325,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2323,"Index":320,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296441,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2329,"Index":326,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2324,"Index":321,"Attempt":0,"Launch Time":1427397542119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379292,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2330,"Index":327,"Attempt":0,"Launch Time":1427397542132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2327,"Index":324,"Attempt":0,"Launch Time":1427397542124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2331,"Index":328,"Attempt":0,"Launch Time":1427397542133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2326,"Index":323,"Attempt":0,"Launch Time":1427397542123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":497812,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2332,"Index":329,"Attempt":0,"Launch Time":1427397542134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2329,"Index":326,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2333,"Index":330,"Attempt":0,"Launch Time":1427397542135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2328,"Index":325,"Attempt":0,"Launch Time":1427397542126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":769044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2334,"Index":331,"Attempt":0,"Launch Time":1427397542136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2325,"Index":322,"Attempt":0,"Launch Time":1427397542121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":455181,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2335,"Index":332,"Attempt":0,"Launch Time":1427397542141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2331,"Index":328,"Attempt":0,"Launch Time":1427397542133,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324059,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2336,"Index":333,"Attempt":0,"Launch Time":1427397542142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2332,"Index":329,"Attempt":0,"Launch Time":1427397542134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542142,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":468594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2337,"Index":334,"Attempt":0,"Launch Time":1427397542145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2330,"Index":327,"Attempt":0,"Launch Time":1427397542132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":475012,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2338,"Index":335,"Attempt":0,"Launch Time":1427397542146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2334,"Index":331,"Attempt":0,"Launch Time":1427397542136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2339,"Index":336,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2335,"Index":332,"Attempt":0,"Launch Time":1427397542141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411684,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2340,"Index":337,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2336,"Index":333,"Attempt":0,"Launch Time":1427397542142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2341,"Index":338,"Attempt":0,"Launch Time":1427397542153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2333,"Index":330,"Attempt":0,"Launch Time":1427397542135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362935,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2342,"Index":339,"Attempt":0,"Launch Time":1427397542154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2322,"Index":319,"Attempt":0,"Launch Time":1427397542118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2343,"Index":340,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2344,"Index":341,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2345,"Index":342,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2338,"Index":335,"Attempt":0,"Launch Time":1427397542146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":332504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2337,"Index":334,"Attempt":0,"Launch Time":1427397542145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":616768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2346,"Index":343,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2339,"Index":336,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":305089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2340,"Index":337,"Attempt":0,"Launch Time":1427397542150,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":310392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2347,"Index":344,"Attempt":0,"Launch Time":1427397542160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2342,"Index":339,"Attempt":0,"Launch Time":1427397542154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":290805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2348,"Index":345,"Attempt":0,"Launch Time":1427397542164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2344,"Index":341,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2349,"Index":346,"Attempt":0,"Launch Time":1427397542165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2345,"Index":342,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542165,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2350,"Index":347,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2346,"Index":343,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351367,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2351,"Index":348,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2341,"Index":338,"Attempt":0,"Launch Time":1427397542153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":403849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2352,"Index":349,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2309,"Index":306,"Attempt":0,"Launch Time":1427397542087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":22,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374327,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2353,"Index":350,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2354,"Index":351,"Attempt":0,"Launch Time":1427397542174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2349,"Index":346,"Attempt":0,"Launch Time":1427397542165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2348,"Index":345,"Attempt":0,"Launch Time":1427397542164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398153,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2355,"Index":352,"Attempt":0,"Launch Time":1427397542175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2343,"Index":340,"Attempt":0,"Launch Time":1427397542157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":499926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2356,"Index":353,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2354,"Index":351,"Attempt":0,"Launch Time":1427397542174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2353,"Index":350,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":498006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2357,"Index":354,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2358,"Index":355,"Attempt":0,"Launch Time":1427397542183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2350,"Index":347,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2359,"Index":356,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2351,"Index":348,"Attempt":0,"Launch Time":1427397542170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2360,"Index":357,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2355,"Index":352,"Attempt":0,"Launch Time":1427397542175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2361,"Index":358,"Attempt":0,"Launch Time":1427397542187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2347,"Index":344,"Attempt":0,"Launch Time":1427397542160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435519,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2362,"Index":359,"Attempt":0,"Launch Time":1427397542190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2357,"Index":354,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":324809,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2363,"Index":360,"Attempt":0,"Launch Time":1427397542205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2359,"Index":356,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2364,"Index":361,"Attempt":0,"Launch Time":1427397542208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2362,"Index":359,"Attempt":0,"Launch Time":1427397542190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2365,"Index":362,"Attempt":0,"Launch Time":1427397542209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2361,"Index":358,"Attempt":0,"Launch Time":1427397542187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1346128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2366,"Index":363,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2367,"Index":364,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2368,"Index":365,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2356,"Index":353,"Attempt":0,"Launch Time":1427397542182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":370058,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2360,"Index":357,"Attempt":0,"Launch Time":1427397542186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":346610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2358,"Index":355,"Attempt":0,"Launch Time":1427397542183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":507468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2369,"Index":366,"Attempt":0,"Launch Time":1427397542225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2308,"Index":305,"Attempt":0,"Launch Time":1427397542085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":139,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2370,"Index":367,"Attempt":0,"Launch Time":1427397542232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2369,"Index":366,"Attempt":0,"Launch Time":1427397542225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2371,"Index":368,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2364,"Index":361,"Attempt":0,"Launch Time":1427397542208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352170,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2372,"Index":369,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2366,"Index":363,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2370,"Index":367,"Attempt":0,"Launch Time":1427397542232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310812,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2373,"Index":370,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2374,"Index":371,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2352,"Index":349,"Attempt":0,"Launch Time":1427397542173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":342303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2365,"Index":362,"Attempt":0,"Launch Time":1427397542209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341559,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2375,"Index":372,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2376,"Index":373,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2377,"Index":374,"Attempt":0,"Launch Time":1427397542244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2363,"Index":360,"Attempt":0,"Launch Time":1427397542205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347017,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2368,"Index":365,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2378,"Index":375,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2373,"Index":370,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":365629,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2379,"Index":376,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2374,"Index":371,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":383856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2380,"Index":377,"Attempt":0,"Launch Time":1427397542253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2375,"Index":372,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":883979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2381,"Index":378,"Attempt":0,"Launch Time":1427397542255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2377,"Index":374,"Attempt":0,"Launch Time":1427397542244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":352312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2382,"Index":379,"Attempt":0,"Launch Time":1427397542258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2371,"Index":368,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":579560,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2383,"Index":380,"Attempt":0,"Launch Time":1427397542259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2378,"Index":375,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2384,"Index":381,"Attempt":0,"Launch Time":1427397542260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2379,"Index":376,"Attempt":0,"Launch Time":1427397542251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2385,"Index":382,"Attempt":0,"Launch Time":1427397542262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2381,"Index":378,"Attempt":0,"Launch Time":1427397542255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322385,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2386,"Index":383,"Attempt":0,"Launch Time":1427397542264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2380,"Index":377,"Attempt":0,"Launch Time":1427397542253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2387,"Index":384,"Attempt":0,"Launch Time":1427397542266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2383,"Index":380,"Attempt":0,"Launch Time":1427397542259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321635,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2388,"Index":385,"Attempt":0,"Launch Time":1427397542267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2384,"Index":381,"Attempt":0,"Launch Time":1427397542260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335935,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2389,"Index":386,"Attempt":0,"Launch Time":1427397542269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2385,"Index":382,"Attempt":0,"Launch Time":1427397542262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":290965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2390,"Index":387,"Attempt":0,"Launch Time":1427397542272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2387,"Index":384,"Attempt":0,"Launch Time":1427397542266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285313,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2391,"Index":388,"Attempt":0,"Launch Time":1427397542274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2388,"Index":385,"Attempt":0,"Launch Time":1427397542267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":277760,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2392,"Index":389,"Attempt":0,"Launch Time":1427397542275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2386,"Index":383,"Attempt":0,"Launch Time":1427397542264,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":415511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2393,"Index":390,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2394,"Index":391,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2390,"Index":387,"Attempt":0,"Launch Time":1427397542272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2389,"Index":386,"Attempt":0,"Launch Time":1427397542269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2395,"Index":392,"Attempt":0,"Launch Time":1427397542280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2391,"Index":388,"Attempt":0,"Launch Time":1427397542274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":295716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2396,"Index":393,"Attempt":0,"Launch Time":1427397542282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2392,"Index":389,"Attempt":0,"Launch Time":1427397542275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2397,"Index":394,"Attempt":0,"Launch Time":1427397542286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2393,"Index":390,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":440435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2398,"Index":395,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2382,"Index":379,"Attempt":0,"Launch Time":1427397542258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":20792857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2399,"Index":396,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2395,"Index":392,"Attempt":0,"Launch Time":1427397542280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":399357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2400,"Index":397,"Attempt":0,"Launch Time":1427397542289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2396,"Index":393,"Attempt":0,"Launch Time":1427397542282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327474,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2401,"Index":398,"Attempt":0,"Launch Time":1427397542291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2394,"Index":391,"Attempt":0,"Launch Time":1427397542279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":445602,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2402,"Index":399,"Attempt":0,"Launch Time":1427397542296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2397,"Index":394,"Attempt":0,"Launch Time":1427397542286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542296,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2403,"Index":400,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2401,"Index":398,"Attempt":0,"Launch Time":1427397542291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2404,"Index":401,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2400,"Index":397,"Attempt":0,"Launch Time":1427397542289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":527832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2405,"Index":402,"Attempt":0,"Launch Time":1427397542301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2372,"Index":369,"Attempt":0,"Launch Time":1427397542242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2406,"Index":403,"Attempt":0,"Launch Time":1427397542303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2399,"Index":396,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1037119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2407,"Index":404,"Attempt":0,"Launch Time":1427397542313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2404,"Index":401,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347748,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2408,"Index":405,"Attempt":0,"Launch Time":1427397542317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2367,"Index":364,"Attempt":0,"Launch Time":1427397542210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542317,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":22,"Executor Run Time":84,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2409,"Index":406,"Attempt":0,"Launch Time":1427397542319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2402,"Index":399,"Attempt":0,"Launch Time":1427397542296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":648398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2410,"Index":407,"Attempt":0,"Launch Time":1427397542320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2407,"Index":404,"Attempt":0,"Launch Time":1427397542313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2411,"Index":408,"Attempt":0,"Launch Time":1427397542324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2408,"Index":405,"Attempt":0,"Launch Time":1427397542317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":326599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2412,"Index":409,"Attempt":0,"Launch Time":1427397542326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2403,"Index":400,"Attempt":0,"Launch Time":1427397542299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":19950449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2413,"Index":410,"Attempt":0,"Launch Time":1427397542327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2409,"Index":406,"Attempt":0,"Launch Time":1427397542319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329296,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2414,"Index":411,"Attempt":0,"Launch Time":1427397542332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2410,"Index":407,"Attempt":0,"Launch Time":1427397542320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316230,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2415,"Index":412,"Attempt":0,"Launch Time":1427397542333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2412,"Index":409,"Attempt":0,"Launch Time":1427397542326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2416,"Index":413,"Attempt":0,"Launch Time":1427397542339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2417,"Index":414,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2415,"Index":412,"Attempt":0,"Launch Time":1427397542333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2414,"Index":411,"Attempt":0,"Launch Time":1427397542332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":522391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2418,"Index":415,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2406,"Index":403,"Attempt":0,"Launch Time":1427397542303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":584072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2419,"Index":416,"Attempt":0,"Launch Time":1427397542345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2376,"Index":373,"Attempt":0,"Launch Time":1427397542243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":101,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":19188514,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2420,"Index":417,"Attempt":0,"Launch Time":1427397542348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2416,"Index":413,"Attempt":0,"Launch Time":1427397542339,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324268,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2421,"Index":418,"Attempt":0,"Launch Time":1427397542349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2405,"Index":402,"Attempt":0,"Launch Time":1427397542301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":384859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2422,"Index":419,"Attempt":0,"Launch Time":1427397542350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2411,"Index":408,"Attempt":0,"Launch Time":1427397542324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":574971,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2423,"Index":420,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2419,"Index":416,"Attempt":0,"Launch Time":1427397542345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350254,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2424,"Index":421,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2420,"Index":417,"Attempt":0,"Launch Time":1427397542348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2425,"Index":422,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2422,"Index":419,"Attempt":0,"Launch Time":1427397542350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338440,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2426,"Index":423,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2421,"Index":418,"Attempt":0,"Launch Time":1427397542349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2427,"Index":424,"Attempt":0,"Launch Time":1427397542363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2423,"Index":420,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2428,"Index":425,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2429,"Index":426,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2426,"Index":423,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2424,"Index":421,"Attempt":0,"Launch Time":1427397542355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":406148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2430,"Index":427,"Attempt":0,"Launch Time":1427397542370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2425,"Index":422,"Attempt":0,"Launch Time":1427397542358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2431,"Index":428,"Attempt":0,"Launch Time":1427397542371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2427,"Index":424,"Attempt":0,"Launch Time":1427397542363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2432,"Index":429,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2429,"Index":426,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303552,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2433,"Index":430,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2428,"Index":425,"Attempt":0,"Launch Time":1427397542368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2434,"Index":431,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2435,"Index":432,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2398,"Index":395,"Attempt":0,"Launch Time":1427397542287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":552343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2431,"Index":428,"Attempt":0,"Launch Time":1427397542371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":487922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2436,"Index":433,"Attempt":0,"Launch Time":1427397542381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2430,"Index":427,"Attempt":0,"Launch Time":1427397542370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2437,"Index":434,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2432,"Index":429,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2438,"Index":435,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2433,"Index":430,"Attempt":0,"Launch Time":1427397542377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2439,"Index":436,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2434,"Index":431,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334178,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2440,"Index":437,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2436,"Index":433,"Attempt":0,"Launch Time":1427397542381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322231,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2441,"Index":438,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2442,"Index":439,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2438,"Index":435,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2435,"Index":432,"Attempt":0,"Launch Time":1427397542380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":568601,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2443,"Index":440,"Attempt":0,"Launch Time":1427397542395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2437,"Index":434,"Attempt":0,"Launch Time":1427397542384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":550818,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2444,"Index":441,"Attempt":0,"Launch Time":1427397542397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2440,"Index":437,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":337933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2445,"Index":442,"Attempt":0,"Launch Time":1427397542398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2439,"Index":436,"Attempt":0,"Launch Time":1427397542388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2446,"Index":443,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2442,"Index":439,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322573,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2447,"Index":444,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2441,"Index":438,"Attempt":0,"Launch Time":1427397542392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542400,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309104,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2448,"Index":445,"Attempt":0,"Launch Time":1427397542403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2443,"Index":440,"Attempt":0,"Launch Time":1427397542395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":311109,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2449,"Index":446,"Attempt":0,"Launch Time":1427397542405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2445,"Index":442,"Attempt":0,"Launch Time":1427397542398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2450,"Index":447,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2451,"Index":448,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2447,"Index":444,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317602,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2444,"Index":441,"Attempt":0,"Launch Time":1427397542397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2452,"Index":449,"Attempt":0,"Launch Time":1427397542408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2446,"Index":443,"Attempt":0,"Launch Time":1427397542400,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":496726,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2453,"Index":450,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2448,"Index":445,"Attempt":0,"Launch Time":1427397542403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":322010,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2454,"Index":451,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2450,"Index":447,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2455,"Index":452,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2451,"Index":448,"Attempt":0,"Launch Time":1427397542407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2456,"Index":453,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2457,"Index":454,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2458,"Index":455,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2413,"Index":410,"Attempt":0,"Launch Time":1427397542327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":95,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":716138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2452,"Index":449,"Attempt":0,"Launch Time":1427397542408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2449,"Index":446,"Attempt":0,"Launch Time":1427397542405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310292,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2459,"Index":456,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2455,"Index":452,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2460,"Index":457,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2456,"Index":453,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320834,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2461,"Index":458,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2457,"Index":454,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331381,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2462,"Index":459,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2454,"Index":451,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":363713,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2463,"Index":460,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2458,"Index":455,"Attempt":0,"Launch Time":1427397542428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":527876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2464,"Index":461,"Attempt":0,"Launch Time":1427397542441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2418,"Index":415,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":439436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2465,"Index":462,"Attempt":0,"Launch Time":1427397542442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2459,"Index":456,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333792,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2466,"Index":463,"Attempt":0,"Launch Time":1427397542444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2467,"Index":464,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2463,"Index":460,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376533,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2461,"Index":458,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351217,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2462,"Index":459,"Attempt":0,"Launch Time":1427397542437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2468,"Index":465,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2469,"Index":466,"Attempt":0,"Launch Time":1427397542446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2460,"Index":457,"Attempt":0,"Launch Time":1427397542434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391038,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2470,"Index":467,"Attempt":0,"Launch Time":1427397542450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2465,"Index":462,"Attempt":0,"Launch Time":1427397542442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2471,"Index":468,"Attempt":0,"Launch Time":1427397542452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2467,"Index":464,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2472,"Index":469,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2466,"Index":463,"Attempt":0,"Launch Time":1427397542444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":346686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2473,"Index":470,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2468,"Index":465,"Attempt":0,"Launch Time":1427397542445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349101,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2474,"Index":471,"Attempt":0,"Launch Time":1427397542456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2469,"Index":466,"Attempt":0,"Launch Time":1427397542446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":549663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2475,"Index":472,"Attempt":0,"Launch Time":1427397542458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2470,"Index":467,"Attempt":0,"Launch Time":1427397542450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320713,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2476,"Index":473,"Attempt":0,"Launch Time":1427397542459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2471,"Index":468,"Attempt":0,"Launch Time":1427397542452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357909,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2477,"Index":474,"Attempt":0,"Launch Time":1427397542460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2464,"Index":461,"Attempt":0,"Launch Time":1427397542441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":496141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2478,"Index":475,"Attempt":0,"Launch Time":1427397542461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2473,"Index":470,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":410225,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2479,"Index":476,"Attempt":0,"Launch Time":1427397542463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2472,"Index":469,"Attempt":0,"Launch Time":1427397542454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":391733,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2480,"Index":477,"Attempt":0,"Launch Time":1427397542466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2477,"Index":474,"Attempt":0,"Launch Time":1427397542460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2481,"Index":478,"Attempt":0,"Launch Time":1427397542468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2453,"Index":450,"Attempt":0,"Launch Time":1427397542427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":316955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2482,"Index":479,"Attempt":0,"Launch Time":1427397542473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2417,"Index":414,"Attempt":0,"Launch Time":1427397542340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2483,"Index":480,"Attempt":0,"Launch Time":1427397542475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2481,"Index":478,"Attempt":0,"Launch Time":1427397542468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":410467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2484,"Index":481,"Attempt":0,"Launch Time":1427397542481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2482,"Index":479,"Attempt":0,"Launch Time":1427397542473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2485,"Index":482,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2483,"Index":480,"Attempt":0,"Launch Time":1427397542475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":549921,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2486,"Index":483,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2480,"Index":477,"Attempt":0,"Launch Time":1427397542466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394959,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2487,"Index":484,"Attempt":0,"Launch Time":1427397542490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2484,"Index":481,"Attempt":0,"Launch Time":1427397542481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":420379,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2488,"Index":485,"Attempt":0,"Launch Time":1427397542491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2485,"Index":482,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2489,"Index":486,"Attempt":0,"Launch Time":1427397542495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2486,"Index":483,"Attempt":0,"Launch Time":1427397542484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":391466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2490,"Index":487,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2491,"Index":488,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2487,"Index":484,"Attempt":0,"Launch Time":1427397542490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2479,"Index":476,"Attempt":0,"Launch Time":1427397542463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":434325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2492,"Index":489,"Attempt":0,"Launch Time":1427397542499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2488,"Index":485,"Attempt":0,"Launch Time":1427397542491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":359385,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2493,"Index":490,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2494,"Index":491,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2489,"Index":486,"Attempt":0,"Launch Time":1427397542495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340097,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2474,"Index":471,"Attempt":0,"Launch Time":1427397542456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":348155,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2495,"Index":492,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2496,"Index":493,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2492,"Index":489,"Attempt":0,"Launch Time":1427397542499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2491,"Index":488,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":380941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2497,"Index":494,"Attempt":0,"Launch Time":1427397542516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2494,"Index":491,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2498,"Index":495,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2476,"Index":473,"Attempt":0,"Launch Time":1427397542459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":700197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2499,"Index":496,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2496,"Index":493,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":7,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2500,"Index":497,"Attempt":0,"Launch Time":1427397542525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2497,"Index":494,"Attempt":0,"Launch Time":1427397542516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":292971,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2501,"Index":498,"Attempt":0,"Launch Time":1427397542529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2498,"Index":495,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328189,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2502,"Index":499,"Attempt":0,"Launch Time":1427397542530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2499,"Index":496,"Attempt":0,"Launch Time":1427397542523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2503,"Index":500,"Attempt":0,"Launch Time":1427397542532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2500,"Index":497,"Attempt":0,"Launch Time":1427397542525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273329,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2504,"Index":501,"Attempt":0,"Launch Time":1427397542536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2493,"Index":490,"Attempt":0,"Launch Time":1427397542502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":624484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2505,"Index":502,"Attempt":0,"Launch Time":1427397542538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2502,"Index":499,"Attempt":0,"Launch Time":1427397542530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348431,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2506,"Index":503,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2490,"Index":487,"Attempt":0,"Launch Time":1427397542497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":510094,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2495,"Index":492,"Attempt":0,"Launch Time":1427397542507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368092,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2507,"Index":504,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2508,"Index":505,"Attempt":0,"Launch Time":1427397542545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2504,"Index":501,"Attempt":0,"Launch Time":1427397542536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415621,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2509,"Index":506,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2478,"Index":475,"Attempt":0,"Launch Time":1427397542461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":9590263,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2510,"Index":507,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2505,"Index":502,"Attempt":0,"Launch Time":1427397542538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2511,"Index":508,"Attempt":0,"Launch Time":1427397542548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2507,"Index":504,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2512,"Index":509,"Attempt":0,"Launch Time":1427397542549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2506,"Index":503,"Attempt":0,"Launch Time":1427397542541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309781,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2513,"Index":510,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2514,"Index":511,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2510,"Index":507,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2501,"Index":498,"Attempt":0,"Launch Time":1427397542529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":421173,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2515,"Index":512,"Attempt":0,"Launch Time":1427397542555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2503,"Index":500,"Attempt":0,"Launch Time":1427397542532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2516,"Index":513,"Attempt":0,"Launch Time":1427397542559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2508,"Index":505,"Attempt":0,"Launch Time":1427397542545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2517,"Index":514,"Attempt":0,"Launch Time":1427397542561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2515,"Index":512,"Attempt":0,"Launch Time":1427397542555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322419,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2518,"Index":515,"Attempt":0,"Launch Time":1427397542563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2511,"Index":508,"Attempt":0,"Launch Time":1427397542548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2519,"Index":516,"Attempt":0,"Launch Time":1427397542564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2513,"Index":510,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2520,"Index":517,"Attempt":0,"Launch Time":1427397542568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2517,"Index":514,"Attempt":0,"Launch Time":1427397542561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2521,"Index":518,"Attempt":0,"Launch Time":1427397542570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2518,"Index":515,"Attempt":0,"Launch Time":1427397542563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2522,"Index":519,"Attempt":0,"Launch Time":1427397542572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2516,"Index":513,"Attempt":0,"Launch Time":1427397542559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2523,"Index":520,"Attempt":0,"Launch Time":1427397542574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2519,"Index":516,"Attempt":0,"Launch Time":1427397542564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2524,"Index":521,"Attempt":0,"Launch Time":1427397542577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2520,"Index":517,"Attempt":0,"Launch Time":1427397542568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":371122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2525,"Index":522,"Attempt":0,"Launch Time":1427397542579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2521,"Index":518,"Attempt":0,"Launch Time":1427397542570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542579,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2526,"Index":523,"Attempt":0,"Launch Time":1427397542582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2522,"Index":519,"Attempt":0,"Launch Time":1427397542572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":453075,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2527,"Index":524,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2512,"Index":509,"Attempt":0,"Launch Time":1427397542549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":430800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2528,"Index":525,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2523,"Index":520,"Attempt":0,"Launch Time":1427397542574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":373774,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2529,"Index":526,"Attempt":0,"Launch Time":1427397542586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2524,"Index":521,"Attempt":0,"Launch Time":1427397542577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2530,"Index":527,"Attempt":0,"Launch Time":1427397542589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2526,"Index":523,"Attempt":0,"Launch Time":1427397542582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2531,"Index":528,"Attempt":0,"Launch Time":1427397542591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542598,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2509,"Index":506,"Attempt":0,"Launch Time":1427397542546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2532,"Index":529,"Attempt":0,"Launch Time":1427397542593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542602,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2475,"Index":472,"Attempt":0,"Launch Time":1427397542458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":135,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2533,"Index":530,"Attempt":0,"Launch Time":1427397542598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2531,"Index":528,"Attempt":0,"Launch Time":1427397542591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342602,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2534,"Index":531,"Attempt":0,"Launch Time":1427397542599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2527,"Index":524,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":435053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2535,"Index":532,"Attempt":0,"Launch Time":1427397542602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2532,"Index":529,"Attempt":0,"Launch Time":1427397542593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339748,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2536,"Index":533,"Attempt":0,"Launch Time":1427397542609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2514,"Index":511,"Attempt":0,"Launch Time":1427397542553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":427739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2537,"Index":534,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2538,"Index":535,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2535,"Index":532,"Attempt":0,"Launch Time":1427397542602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352160,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2534,"Index":531,"Attempt":0,"Launch Time":1427397542599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":556996,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2533,"Index":530,"Attempt":0,"Launch Time":1427397542598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416658,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2539,"Index":536,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2540,"Index":537,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2525,"Index":522,"Attempt":0,"Launch Time":1427397542579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2541,"Index":538,"Attempt":0,"Launch Time":1427397542620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2540,"Index":537,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2542,"Index":539,"Attempt":0,"Launch Time":1427397542631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2538,"Index":535,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2543,"Index":540,"Attempt":0,"Launch Time":1427397542632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2537,"Index":534,"Attempt":0,"Launch Time":1427397542610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2544,"Index":541,"Attempt":0,"Launch Time":1427397542635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2530,"Index":527,"Attempt":0,"Launch Time":1427397542589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334280,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2545,"Index":542,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2541,"Index":538,"Attempt":0,"Launch Time":1427397542620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2546,"Index":543,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2539,"Index":536,"Attempt":0,"Launch Time":1427397542613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2547,"Index":544,"Attempt":0,"Launch Time":1427397542640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2543,"Index":540,"Attempt":0,"Launch Time":1427397542632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":401032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2548,"Index":545,"Attempt":0,"Launch Time":1427397542641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2542,"Index":539,"Attempt":0,"Launch Time":1427397542631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2137446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2549,"Index":546,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2544,"Index":541,"Attempt":0,"Launch Time":1427397542635,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2550,"Index":547,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2546,"Index":543,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2551,"Index":548,"Attempt":0,"Launch Time":1427397542647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2547,"Index":544,"Attempt":0,"Launch Time":1427397542640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2552,"Index":549,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2553,"Index":550,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2548,"Index":545,"Attempt":0,"Launch Time":1427397542641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":672859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2545,"Index":542,"Attempt":0,"Launch Time":1427397542636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":807166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2554,"Index":551,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2555,"Index":552,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2549,"Index":546,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2550,"Index":547,"Attempt":0,"Launch Time":1427397542643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2556,"Index":553,"Attempt":0,"Launch Time":1427397542654,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2551,"Index":548,"Attempt":0,"Launch Time":1427397542647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2557,"Index":554,"Attempt":0,"Launch Time":1427397542657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2552,"Index":549,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2558,"Index":555,"Attempt":0,"Launch Time":1427397542658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2555,"Index":552,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332762,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2559,"Index":556,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2553,"Index":550,"Attempt":0,"Launch Time":1427397542650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2560,"Index":557,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2554,"Index":551,"Attempt":0,"Launch Time":1427397542651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":353880,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2561,"Index":558,"Attempt":0,"Launch Time":1427397542663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2557,"Index":554,"Attempt":0,"Launch Time":1427397542657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2562,"Index":559,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2563,"Index":560,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2559,"Index":556,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303541,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2560,"Index":557,"Attempt":0,"Launch Time":1427397542659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":433393,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2564,"Index":561,"Attempt":0,"Launch Time":1427397542668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2556,"Index":553,"Attempt":0,"Launch Time":1427397542654,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2565,"Index":562,"Attempt":0,"Launch Time":1427397542670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2561,"Index":558,"Attempt":0,"Launch Time":1427397542663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2566,"Index":563,"Attempt":0,"Launch Time":1427397542675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2562,"Index":559,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":480584,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2567,"Index":564,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2564,"Index":561,"Attempt":0,"Launch Time":1427397542668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":481129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2568,"Index":565,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2558,"Index":555,"Attempt":0,"Launch Time":1427397542658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2569,"Index":566,"Attempt":0,"Launch Time":1427397542678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2565,"Index":562,"Attempt":0,"Launch Time":1427397542670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":390115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2570,"Index":567,"Attempt":0,"Launch Time":1427397542680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2563,"Index":560,"Attempt":0,"Launch Time":1427397542667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":700911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2571,"Index":568,"Attempt":0,"Launch Time":1427397542684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2566,"Index":563,"Attempt":0,"Launch Time":1427397542675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":402443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2572,"Index":569,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2567,"Index":564,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2573,"Index":570,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2568,"Index":565,"Attempt":0,"Launch Time":1427397542677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":442654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2574,"Index":571,"Attempt":0,"Launch Time":1427397542687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2569,"Index":566,"Attempt":0,"Launch Time":1427397542678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2575,"Index":572,"Attempt":0,"Launch Time":1427397542692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2571,"Index":568,"Attempt":0,"Launch Time":1427397542684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":581749,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2576,"Index":573,"Attempt":0,"Launch Time":1427397542694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2577,"Index":574,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2570,"Index":567,"Attempt":0,"Launch Time":1427397542680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2574,"Index":571,"Attempt":0,"Launch Time":1427397542687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":386982,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2573,"Index":570,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":454222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2578,"Index":575,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2579,"Index":576,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2580,"Index":577,"Attempt":0,"Launch Time":1427397542696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2572,"Index":569,"Attempt":0,"Launch Time":1427397542686,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399381,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2536,"Index":533,"Attempt":0,"Launch Time":1427397542609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":49,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2581,"Index":578,"Attempt":0,"Launch Time":1427397542700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2575,"Index":572,"Attempt":0,"Launch Time":1427397542692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":388251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2582,"Index":579,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2583,"Index":580,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2578,"Index":575,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2580,"Index":577,"Attempt":0,"Launch Time":1427397542696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2584,"Index":581,"Attempt":0,"Launch Time":1427397542705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2579,"Index":576,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2585,"Index":582,"Attempt":0,"Launch Time":1427397542707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2581,"Index":578,"Attempt":0,"Launch Time":1427397542700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2586,"Index":583,"Attempt":0,"Launch Time":1427397542708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2577,"Index":574,"Attempt":0,"Launch Time":1427397542695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310991,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2587,"Index":584,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2588,"Index":585,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2582,"Index":579,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329215,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2583,"Index":580,"Attempt":0,"Launch Time":1427397542704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":312850,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2589,"Index":586,"Attempt":0,"Launch Time":1427397542712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2584,"Index":581,"Attempt":0,"Launch Time":1427397542705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2590,"Index":587,"Attempt":0,"Launch Time":1427397542722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2528,"Index":525,"Attempt":0,"Launch Time":1427397542583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":40517475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2591,"Index":588,"Attempt":0,"Launch Time":1427397542729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2529,"Index":526,"Attempt":0,"Launch Time":1427397542586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":70,"Executor Run Time":49,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2592,"Index":589,"Attempt":0,"Launch Time":1427397542732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2585,"Index":582,"Attempt":0,"Launch Time":1427397542707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2593,"Index":590,"Attempt":0,"Launch Time":1427397542734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2590,"Index":587,"Attempt":0,"Launch Time":1427397542722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":377828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2594,"Index":591,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2576,"Index":573,"Attempt":0,"Launch Time":1427397542694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1063619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2595,"Index":592,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2591,"Index":588,"Attempt":0,"Launch Time":1427397542729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":340638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2596,"Index":593,"Attempt":0,"Launch Time":1427397542744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2593,"Index":590,"Attempt":0,"Launch Time":1427397542734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":751198,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2597,"Index":594,"Attempt":0,"Launch Time":1427397542745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2595,"Index":592,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542745,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":698816,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2598,"Index":595,"Attempt":0,"Launch Time":1427397542746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2594,"Index":591,"Attempt":0,"Launch Time":1427397542736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2599,"Index":596,"Attempt":0,"Launch Time":1427397542748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2592,"Index":589,"Attempt":0,"Launch Time":1427397542732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":600526,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2600,"Index":597,"Attempt":0,"Launch Time":1427397542752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2597,"Index":594,"Attempt":0,"Launch Time":1427397542745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2601,"Index":598,"Attempt":0,"Launch Time":1427397542753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2598,"Index":595,"Attempt":0,"Launch Time":1427397542746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":364206,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2602,"Index":599,"Attempt":0,"Launch Time":1427397542755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2596,"Index":593,"Attempt":0,"Launch Time":1427397542744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2603,"Index":600,"Attempt":0,"Launch Time":1427397542759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2600,"Index":597,"Attempt":0,"Launch Time":1427397542752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":357617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2604,"Index":601,"Attempt":0,"Launch Time":1427397542761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2599,"Index":596,"Attempt":0,"Launch Time":1427397542748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2605,"Index":602,"Attempt":0,"Launch Time":1427397542762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2601,"Index":598,"Attempt":0,"Launch Time":1427397542753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":378242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2606,"Index":603,"Attempt":0,"Launch Time":1427397542763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2602,"Index":599,"Attempt":0,"Launch Time":1427397542755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2607,"Index":604,"Attempt":0,"Launch Time":1427397542768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2603,"Index":600,"Attempt":0,"Launch Time":1427397542759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2608,"Index":605,"Attempt":0,"Launch Time":1427397542769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2604,"Index":601,"Attempt":0,"Launch Time":1427397542761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353787,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2609,"Index":606,"Attempt":0,"Launch Time":1427397542771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2606,"Index":603,"Attempt":0,"Launch Time":1427397542763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2610,"Index":607,"Attempt":0,"Launch Time":1427397542773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2605,"Index":602,"Attempt":0,"Launch Time":1427397542762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":655953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2611,"Index":608,"Attempt":0,"Launch Time":1427397542776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2607,"Index":604,"Attempt":0,"Launch Time":1427397542768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409292,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2612,"Index":609,"Attempt":0,"Launch Time":1427397542778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2608,"Index":605,"Attempt":0,"Launch Time":1427397542769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392973,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2613,"Index":610,"Attempt":0,"Launch Time":1427397542779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2609,"Index":606,"Attempt":0,"Launch Time":1427397542771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":522248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2614,"Index":611,"Attempt":0,"Launch Time":1427397542781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2610,"Index":607,"Attempt":0,"Launch Time":1427397542773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2615,"Index":612,"Attempt":0,"Launch Time":1427397542784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2611,"Index":608,"Attempt":0,"Launch Time":1427397542776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":385253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2616,"Index":613,"Attempt":0,"Launch Time":1427397542787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2612,"Index":609,"Attempt":0,"Launch Time":1427397542778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":523803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2617,"Index":614,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2618,"Index":615,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2613,"Index":610,"Attempt":0,"Launch Time":1427397542779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":526643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2614,"Index":611,"Attempt":0,"Launch Time":1427397542781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2619,"Index":616,"Attempt":0,"Launch Time":1427397542791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2615,"Index":612,"Attempt":0,"Launch Time":1427397542784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2620,"Index":617,"Attempt":0,"Launch Time":1427397542794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2616,"Index":613,"Attempt":0,"Launch Time":1427397542787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2621,"Index":618,"Attempt":0,"Launch Time":1427397542798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2618,"Index":615,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2622,"Index":619,"Attempt":0,"Launch Time":1427397542799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2619,"Index":616,"Attempt":0,"Launch Time":1427397542791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2623,"Index":620,"Attempt":0,"Launch Time":1427397542800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2617,"Index":614,"Attempt":0,"Launch Time":1427397542789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2624,"Index":621,"Attempt":0,"Launch Time":1427397542801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2620,"Index":617,"Attempt":0,"Launch Time":1427397542794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346097,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2625,"Index":622,"Attempt":0,"Launch Time":1427397542803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2589,"Index":586,"Attempt":0,"Launch Time":1427397542712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":79,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1202931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2626,"Index":623,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2621,"Index":618,"Attempt":0,"Launch Time":1427397542798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2627,"Index":624,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2622,"Index":619,"Attempt":0,"Launch Time":1427397542799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2628,"Index":625,"Attempt":0,"Launch Time":1427397542825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2623,"Index":620,"Attempt":0,"Launch Time":1427397542800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370648,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2629,"Index":626,"Attempt":0,"Launch Time":1427397542826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2587,"Index":584,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":111,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":579883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2630,"Index":627,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2631,"Index":628,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2588,"Index":585,"Attempt":0,"Launch Time":1427397542711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2627,"Index":624,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327972,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2586,"Index":583,"Attempt":0,"Launch Time":1427397542708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":109,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":484121,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2632,"Index":629,"Attempt":0,"Launch Time":1427397542835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2633,"Index":630,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2626,"Index":623,"Attempt":0,"Launch Time":1427397542806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329889,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2634,"Index":631,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2625,"Index":622,"Attempt":0,"Launch Time":1427397542803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2635,"Index":632,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2624,"Index":621,"Attempt":0,"Launch Time":1427397542801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1116695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2636,"Index":633,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2629,"Index":626,"Attempt":0,"Launch Time":1427397542826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305168,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2630,"Index":627,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2637,"Index":634,"Attempt":0,"Launch Time":1427397542844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2638,"Index":635,"Attempt":0,"Launch Time":1427397542850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2628,"Index":625,"Attempt":0,"Launch Time":1427397542825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2639,"Index":636,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2633,"Index":630,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2640,"Index":637,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2636,"Index":633,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338975,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2641,"Index":638,"Attempt":0,"Launch Time":1427397542854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2642,"Index":639,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2643,"Index":640,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2639,"Index":636,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2637,"Index":634,"Attempt":0,"Launch Time":1427397542844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":471411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2638,"Index":635,"Attempt":0,"Launch Time":1427397542850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2640,"Index":637,"Attempt":0,"Launch Time":1427397542851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2644,"Index":641,"Attempt":0,"Launch Time":1427397542861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2645,"Index":642,"Attempt":0,"Launch Time":1427397542867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2631,"Index":628,"Attempt":0,"Launch Time":1427397542828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":19043003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2646,"Index":643,"Attempt":0,"Launch Time":1427397542868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2642,"Index":639,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2647,"Index":644,"Attempt":0,"Launch Time":1427397542870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2644,"Index":641,"Attempt":0,"Launch Time":1427397542861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358107,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2648,"Index":645,"Attempt":0,"Launch Time":1427397542871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2641,"Index":638,"Attempt":0,"Launch Time":1427397542854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2649,"Index":646,"Attempt":0,"Launch Time":1427397542875,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2643,"Index":640,"Attempt":0,"Launch Time":1427397542860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542875,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1248717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2650,"Index":647,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2646,"Index":643,"Attempt":0,"Launch Time":1427397542868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2651,"Index":648,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2647,"Index":644,"Attempt":0,"Launch Time":1427397542870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":625271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2652,"Index":649,"Attempt":0,"Launch Time":1427397542880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2648,"Index":645,"Attempt":0,"Launch Time":1427397542871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":641811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2653,"Index":650,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2654,"Index":651,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2655,"Index":652,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2651,"Index":648,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":547142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2649,"Index":646,"Attempt":0,"Launch Time":1427397542875,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":664834,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2650,"Index":647,"Attempt":0,"Launch Time":1427397542878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2656,"Index":653,"Attempt":0,"Launch Time":1427397542890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2632,"Index":629,"Attempt":0,"Launch Time":1427397542835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388731,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2657,"Index":654,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2652,"Index":649,"Attempt":0,"Launch Time":1427397542880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344434,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2658,"Index":655,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2654,"Index":651,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299160,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2659,"Index":656,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2655,"Index":652,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2660,"Index":657,"Attempt":0,"Launch Time":1427397542894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2653,"Index":650,"Attempt":0,"Launch Time":1427397542886,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325935,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2661,"Index":658,"Attempt":0,"Launch Time":1427397542899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2657,"Index":654,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307370,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2662,"Index":659,"Attempt":0,"Launch Time":1427397542900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2658,"Index":655,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308555,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2663,"Index":660,"Attempt":0,"Launch Time":1427397542902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2660,"Index":657,"Attempt":0,"Launch Time":1427397542894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":508307,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2664,"Index":661,"Attempt":0,"Launch Time":1427397542906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2661,"Index":658,"Attempt":0,"Launch Time":1427397542899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":313634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2665,"Index":662,"Attempt":0,"Launch Time":1427397542908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2663,"Index":660,"Attempt":0,"Launch Time":1427397542902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308580,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2666,"Index":663,"Attempt":0,"Launch Time":1427397542911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2659,"Index":656,"Attempt":0,"Launch Time":1427397542893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2667,"Index":664,"Attempt":0,"Launch Time":1427397542912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2664,"Index":661,"Attempt":0,"Launch Time":1427397542906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":292517,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2668,"Index":665,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2662,"Index":659,"Attempt":0,"Launch Time":1427397542900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344633,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2669,"Index":666,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2665,"Index":662,"Attempt":0,"Launch Time":1427397542908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2670,"Index":667,"Attempt":0,"Launch Time":1427397542919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2667,"Index":664,"Attempt":0,"Launch Time":1427397542912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2671,"Index":668,"Attempt":0,"Launch Time":1427397542920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2666,"Index":663,"Attempt":0,"Launch Time":1427397542911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2672,"Index":669,"Attempt":0,"Launch Time":1427397542926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2669,"Index":666,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":379100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2673,"Index":670,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2674,"Index":671,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2668,"Index":665,"Attempt":0,"Launch Time":1427397542917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":710441,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2670,"Index":667,"Attempt":0,"Launch Time":1427397542919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":394617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2675,"Index":672,"Attempt":0,"Launch Time":1427397542929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2671,"Index":668,"Attempt":0,"Launch Time":1427397542920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":461799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2676,"Index":673,"Attempt":0,"Launch Time":1427397542934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2672,"Index":669,"Attempt":0,"Launch Time":1427397542926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542935,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2677,"Index":674,"Attempt":0,"Launch Time":1427397542935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2674,"Index":671,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356693,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2678,"Index":675,"Attempt":0,"Launch Time":1427397542936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2673,"Index":670,"Attempt":0,"Launch Time":1427397542927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2679,"Index":676,"Attempt":0,"Launch Time":1427397542937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2675,"Index":672,"Attempt":0,"Launch Time":1427397542929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2680,"Index":677,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2676,"Index":673,"Attempt":0,"Launch Time":1427397542934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2681,"Index":678,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2682,"Index":679,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2656,"Index":653,"Attempt":0,"Launch Time":1427397542890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2677,"Index":674,"Attempt":0,"Launch Time":1427397542935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":359015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2683,"Index":680,"Attempt":0,"Launch Time":1427397542960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2679,"Index":676,"Attempt":0,"Launch Time":1427397542937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542961,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":11,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":444455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2684,"Index":681,"Attempt":0,"Launch Time":1427397542968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2634,"Index":631,"Attempt":0,"Launch Time":1427397542841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356384,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2685,"Index":682,"Attempt":0,"Launch Time":1427397542969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2645,"Index":642,"Attempt":0,"Launch Time":1427397542867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2686,"Index":683,"Attempt":0,"Launch Time":1427397542971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2635,"Index":632,"Attempt":0,"Launch Time":1427397542842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":389717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2687,"Index":684,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2685,"Index":682,"Attempt":0,"Launch Time":1427397542969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2688,"Index":685,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2684,"Index":681,"Attempt":0,"Launch Time":1427397542968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":403015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2689,"Index":686,"Attempt":0,"Launch Time":1427397542979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2686,"Index":683,"Attempt":0,"Launch Time":1427397542971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406198,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2690,"Index":687,"Attempt":0,"Launch Time":1427397542981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2681,"Index":678,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":831211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2691,"Index":688,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2687,"Index":684,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2692,"Index":689,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2688,"Index":685,"Attempt":0,"Launch Time":1427397542977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":364672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2693,"Index":690,"Attempt":0,"Launch Time":1427397542986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2689,"Index":686,"Attempt":0,"Launch Time":1427397542979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332041,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2694,"Index":691,"Attempt":0,"Launch Time":1427397542988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2690,"Index":687,"Attempt":0,"Launch Time":1427397542981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2691,"Index":688,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2695,"Index":692,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2696,"Index":693,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2692,"Index":689,"Attempt":0,"Launch Time":1427397542985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348531,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2697,"Index":694,"Attempt":0,"Launch Time":1427397542993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2693,"Index":690,"Attempt":0,"Launch Time":1427397542986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397542993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":341172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2698,"Index":695,"Attempt":0,"Launch Time":1427397543000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2695,"Index":692,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367496,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2699,"Index":696,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2700,"Index":697,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2696,"Index":693,"Attempt":0,"Launch Time":1427397542992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":412478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2697,"Index":694,"Attempt":0,"Launch Time":1427397542993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":425714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2701,"Index":698,"Attempt":0,"Launch Time":1427397543007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2698,"Index":695,"Attempt":0,"Launch Time":1427397543000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543007,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2702,"Index":699,"Attempt":0,"Launch Time":1427397543009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2699,"Index":696,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329732,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2703,"Index":700,"Attempt":0,"Launch Time":1427397543010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2700,"Index":697,"Attempt":0,"Launch Time":1427397543001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":591151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2704,"Index":701,"Attempt":0,"Launch Time":1427397543013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2701,"Index":698,"Attempt":0,"Launch Time":1427397543007,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2705,"Index":702,"Attempt":0,"Launch Time":1427397543017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2703,"Index":700,"Attempt":0,"Launch Time":1427397543010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333647,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2706,"Index":703,"Attempt":0,"Launch Time":1427397543020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2694,"Index":691,"Attempt":0,"Launch Time":1427397542988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":413483,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2707,"Index":704,"Attempt":0,"Launch Time":1427397543021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2704,"Index":701,"Attempt":0,"Launch Time":1427397543013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2708,"Index":705,"Attempt":0,"Launch Time":1427397543025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2702,"Index":699,"Attempt":0,"Launch Time":1427397543009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":651728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2709,"Index":706,"Attempt":0,"Launch Time":1427397543026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2705,"Index":702,"Attempt":0,"Launch Time":1427397543017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":695161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2710,"Index":707,"Attempt":0,"Launch Time":1427397543031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2707,"Index":704,"Attempt":0,"Launch Time":1427397543021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2711,"Index":708,"Attempt":0,"Launch Time":1427397543033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2708,"Index":705,"Attempt":0,"Launch Time":1427397543025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":418149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2712,"Index":709,"Attempt":0,"Launch Time":1427397543035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2709,"Index":706,"Attempt":0,"Launch Time":1427397543026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393559,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2713,"Index":710,"Attempt":0,"Launch Time":1427397543036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2706,"Index":703,"Attempt":0,"Launch Time":1427397543020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2020827,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2714,"Index":711,"Attempt":0,"Launch Time":1427397543039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2710,"Index":707,"Attempt":0,"Launch Time":1427397543031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":382803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2715,"Index":712,"Attempt":0,"Launch Time":1427397543041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2711,"Index":708,"Attempt":0,"Launch Time":1427397543033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2716,"Index":713,"Attempt":0,"Launch Time":1427397543043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2712,"Index":709,"Attempt":0,"Launch Time":1427397543035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2717,"Index":714,"Attempt":0,"Launch Time":1427397543048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2714,"Index":711,"Attempt":0,"Launch Time":1427397543039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2718,"Index":715,"Attempt":0,"Launch Time":1427397543049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2715,"Index":712,"Attempt":0,"Launch Time":1427397543041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357978,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2719,"Index":716,"Attempt":0,"Launch Time":1427397543050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2716,"Index":713,"Attempt":0,"Launch Time":1427397543043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":341089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2720,"Index":717,"Attempt":0,"Launch Time":1427397543056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2717,"Index":714,"Attempt":0,"Launch Time":1427397543048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2721,"Index":718,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2718,"Index":715,"Attempt":0,"Launch Time":1427397543049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345892,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2722,"Index":719,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2719,"Index":716,"Attempt":0,"Launch Time":1427397543050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327708,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2723,"Index":720,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2722,"Index":719,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371059,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2724,"Index":721,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2720,"Index":717,"Attempt":0,"Launch Time":1427397543056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2725,"Index":722,"Attempt":0,"Launch Time":1427397543086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2713,"Index":710,"Attempt":0,"Launch Time":1427397543036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":669486,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2726,"Index":723,"Attempt":0,"Launch Time":1427397543092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2724,"Index":721,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322890,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2727,"Index":724,"Attempt":0,"Launch Time":1427397543093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2725,"Index":722,"Attempt":0,"Launch Time":1427397543086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2728,"Index":725,"Attempt":0,"Launch Time":1427397543096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2683,"Index":680,"Attempt":0,"Launch Time":1427397542960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":391439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2729,"Index":726,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2680,"Index":677,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":149,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":437672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2730,"Index":727,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2728,"Index":725,"Attempt":0,"Launch Time":1427397543096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":343220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2731,"Index":728,"Attempt":0,"Launch Time":1427397543108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2682,"Index":679,"Attempt":0,"Launch Time":1427397542954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":147,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2732,"Index":729,"Attempt":0,"Launch Time":1427397543110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2729,"Index":726,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2733,"Index":730,"Attempt":0,"Launch Time":1427397543112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2730,"Index":727,"Attempt":0,"Launch Time":1427397543103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2734,"Index":731,"Attempt":0,"Launch Time":1427397543118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2732,"Index":729,"Attempt":0,"Launch Time":1427397543110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336057,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2735,"Index":732,"Attempt":0,"Launch Time":1427397543120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2736,"Index":733,"Attempt":0,"Launch Time":1427397543121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2733,"Index":730,"Attempt":0,"Launch Time":1427397543112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2731,"Index":728,"Attempt":0,"Launch Time":1427397543108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":446819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2737,"Index":734,"Attempt":0,"Launch Time":1427397543125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2734,"Index":731,"Attempt":0,"Launch Time":1427397543118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":417926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2738,"Index":735,"Attempt":0,"Launch Time":1427397543128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2736,"Index":733,"Attempt":0,"Launch Time":1427397543121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2739,"Index":736,"Attempt":0,"Launch Time":1427397543129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2735,"Index":732,"Attempt":0,"Launch Time":1427397543120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":434321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2740,"Index":737,"Attempt":0,"Launch Time":1427397543132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2737,"Index":734,"Attempt":0,"Launch Time":1427397543125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543133,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2741,"Index":738,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2742,"Index":739,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2738,"Index":735,"Attempt":0,"Launch Time":1427397543128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2727,"Index":724,"Attempt":0,"Launch Time":1427397543093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":444817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2743,"Index":740,"Attempt":0,"Launch Time":1427397543138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2739,"Index":736,"Attempt":0,"Launch Time":1427397543129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2744,"Index":741,"Attempt":0,"Launch Time":1427397543139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2740,"Index":737,"Attempt":0,"Launch Time":1427397543132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2745,"Index":742,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2741,"Index":738,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332691,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2746,"Index":743,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2742,"Index":739,"Attempt":0,"Launch Time":1427397543136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2747,"Index":744,"Attempt":0,"Launch Time":1427397543146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2743,"Index":740,"Attempt":0,"Launch Time":1427397543138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2748,"Index":745,"Attempt":0,"Launch Time":1427397543147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2744,"Index":741,"Attempt":0,"Launch Time":1427397543139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362624,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2749,"Index":746,"Attempt":0,"Launch Time":1427397543153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2746,"Index":743,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2750,"Index":747,"Attempt":0,"Launch Time":1427397543155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2747,"Index":744,"Attempt":0,"Launch Time":1427397543146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360229,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2751,"Index":748,"Attempt":0,"Launch Time":1427397543156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2748,"Index":745,"Attempt":0,"Launch Time":1427397543147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2752,"Index":749,"Attempt":0,"Launch Time":1427397543159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2745,"Index":742,"Attempt":0,"Launch Time":1427397543143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2753,"Index":750,"Attempt":0,"Launch Time":1427397543161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2749,"Index":746,"Attempt":0,"Launch Time":1427397543153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2754,"Index":751,"Attempt":0,"Launch Time":1427397543162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2750,"Index":747,"Attempt":0,"Launch Time":1427397543155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335041,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2755,"Index":752,"Attempt":0,"Launch Time":1427397543163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2751,"Index":748,"Attempt":0,"Launch Time":1427397543156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330655,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2756,"Index":753,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2753,"Index":750,"Attempt":0,"Launch Time":1427397543161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":623863,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2757,"Index":754,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2752,"Index":749,"Attempt":0,"Launch Time":1427397543159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2758,"Index":755,"Attempt":0,"Launch Time":1427397543170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2755,"Index":752,"Attempt":0,"Launch Time":1427397543163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543170,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2759,"Index":756,"Attempt":0,"Launch Time":1427397543173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2754,"Index":751,"Attempt":0,"Launch Time":1427397543162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2760,"Index":757,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2756,"Index":753,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2761,"Index":758,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2757,"Index":754,"Attempt":0,"Launch Time":1427397543169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354349,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2762,"Index":759,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2758,"Index":755,"Attempt":0,"Launch Time":1427397543170,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354808,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2763,"Index":760,"Attempt":0,"Launch Time":1427397543181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2759,"Index":756,"Attempt":0,"Launch Time":1427397543173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":313403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2764,"Index":761,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2765,"Index":762,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2762,"Index":759,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327180,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2760,"Index":757,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2766,"Index":763,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2761,"Index":758,"Attempt":0,"Launch Time":1427397543178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":451128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2767,"Index":764,"Attempt":0,"Launch Time":1427397543189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2763,"Index":760,"Attempt":0,"Launch Time":1427397543181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2768,"Index":765,"Attempt":0,"Launch Time":1427397543205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2766,"Index":763,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2769,"Index":766,"Attempt":0,"Launch Time":1427397543206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2764,"Index":761,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356389,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2770,"Index":767,"Attempt":0,"Launch Time":1427397543208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2765,"Index":762,"Attempt":0,"Launch Time":1427397543186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2771,"Index":768,"Attempt":0,"Launch Time":1427397543210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2726,"Index":723,"Attempt":0,"Launch Time":1427397543092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2772,"Index":769,"Attempt":0,"Launch Time":1427397543214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2768,"Index":765,"Attempt":0,"Launch Time":1427397543205,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543214,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2773,"Index":770,"Attempt":0,"Launch Time":1427397543217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2678,"Index":675,"Attempt":0,"Launch Time":1427397542936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":280,"Result Size":930,"JVM GC Time":23,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":419616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2774,"Index":771,"Attempt":0,"Launch Time":1427397543218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2723,"Index":720,"Attempt":0,"Launch Time":1427397543085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":635551,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2775,"Index":772,"Attempt":0,"Launch Time":1427397543221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2721,"Index":718,"Attempt":0,"Launch Time":1427397543058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":163,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2776,"Index":773,"Attempt":0,"Launch Time":1427397543224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2773,"Index":770,"Attempt":0,"Launch Time":1427397543217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":446862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2777,"Index":774,"Attempt":0,"Launch Time":1427397543226,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2774,"Index":771,"Attempt":0,"Launch Time":1427397543218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392666,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2778,"Index":775,"Attempt":0,"Launch Time":1427397543229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2775,"Index":772,"Attempt":0,"Launch Time":1427397543221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2779,"Index":776,"Attempt":0,"Launch Time":1427397543232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2776,"Index":773,"Attempt":0,"Launch Time":1427397543224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":390960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2780,"Index":777,"Attempt":0,"Launch Time":1427397543234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2771,"Index":768,"Attempt":0,"Launch Time":1427397543210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2781,"Index":778,"Attempt":0,"Launch Time":1427397543236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2777,"Index":774,"Attempt":0,"Launch Time":1427397543226,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467336,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2782,"Index":779,"Attempt":0,"Launch Time":1427397543238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2778,"Index":775,"Attempt":0,"Launch Time":1427397543229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2783,"Index":780,"Attempt":0,"Launch Time":1427397543240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2779,"Index":776,"Attempt":0,"Launch Time":1427397543232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":371000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2784,"Index":781,"Attempt":0,"Launch Time":1427397543242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2780,"Index":777,"Attempt":0,"Launch Time":1427397543234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2785,"Index":782,"Attempt":0,"Launch Time":1427397543248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2783,"Index":780,"Attempt":0,"Launch Time":1427397543240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2786,"Index":783,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2782,"Index":779,"Attempt":0,"Launch Time":1427397543238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2787,"Index":784,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2784,"Index":781,"Attempt":0,"Launch Time":1427397543242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364565,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2788,"Index":785,"Attempt":0,"Launch Time":1427397543251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2781,"Index":778,"Attempt":0,"Launch Time":1427397543236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":529162,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2789,"Index":786,"Attempt":0,"Launch Time":1427397543255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2785,"Index":782,"Attempt":0,"Launch Time":1427397543248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":305615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2790,"Index":787,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2791,"Index":788,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2786,"Index":783,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2787,"Index":784,"Attempt":0,"Launch Time":1427397543250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":372179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2792,"Index":789,"Attempt":0,"Launch Time":1427397543260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2788,"Index":785,"Attempt":0,"Launch Time":1427397543251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396147,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2793,"Index":790,"Attempt":0,"Launch Time":1427397543262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2789,"Index":786,"Attempt":0,"Launch Time":1427397543255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368901,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2794,"Index":791,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2791,"Index":788,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":353284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2795,"Index":792,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2770,"Index":767,"Attempt":0,"Launch Time":1427397543208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336829,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2796,"Index":793,"Attempt":0,"Launch Time":1427397543268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2792,"Index":789,"Attempt":0,"Launch Time":1427397543260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330263,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2797,"Index":794,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2798,"Index":795,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2790,"Index":787,"Attempt":0,"Launch Time":1427397543258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2793,"Index":790,"Attempt":0,"Launch Time":1427397543262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2799,"Index":796,"Attempt":0,"Launch Time":1427397543273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2794,"Index":791,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2800,"Index":797,"Attempt":0,"Launch Time":1427397543276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2796,"Index":793,"Attempt":0,"Launch Time":1427397543268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2801,"Index":798,"Attempt":0,"Launch Time":1427397543279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2798,"Index":795,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2802,"Index":799,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2797,"Index":794,"Attempt":0,"Launch Time":1427397543272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321135,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2803,"Index":800,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2795,"Index":792,"Attempt":0,"Launch Time":1427397543266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2855698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2804,"Index":801,"Attempt":0,"Launch Time":1427397543281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2799,"Index":796,"Attempt":0,"Launch Time":1427397543273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":382919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2805,"Index":802,"Attempt":0,"Launch Time":1427397543284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2800,"Index":797,"Attempt":0,"Launch Time":1427397543276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362422,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2806,"Index":803,"Attempt":0,"Launch Time":1427397543286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2807,"Index":804,"Attempt":0,"Launch Time":1427397543287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2801,"Index":798,"Attempt":0,"Launch Time":1427397543279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2802,"Index":799,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2808,"Index":805,"Attempt":0,"Launch Time":1427397543288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2803,"Index":800,"Attempt":0,"Launch Time":1427397543280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2809,"Index":806,"Attempt":0,"Launch Time":1427397543290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2805,"Index":802,"Attempt":0,"Launch Time":1427397543284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2810,"Index":807,"Attempt":0,"Launch Time":1427397543293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2804,"Index":801,"Attempt":0,"Launch Time":1427397543281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382608,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2811,"Index":808,"Attempt":0,"Launch Time":1427397543294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2806,"Index":803,"Attempt":0,"Launch Time":1427397543286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2812,"Index":809,"Attempt":0,"Launch Time":1427397543295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2807,"Index":804,"Attempt":0,"Launch Time":1427397543287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380653,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2813,"Index":810,"Attempt":0,"Launch Time":1427397543298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2808,"Index":805,"Attempt":0,"Launch Time":1427397543288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":349616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2814,"Index":811,"Attempt":0,"Launch Time":1427397543312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2811,"Index":808,"Attempt":0,"Launch Time":1427397543294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2815,"Index":812,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2809,"Index":806,"Attempt":0,"Launch Time":1427397543290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":689066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2816,"Index":813,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2812,"Index":809,"Attempt":0,"Launch Time":1427397543295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2817,"Index":814,"Attempt":0,"Launch Time":1427397543319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2814,"Index":811,"Attempt":0,"Launch Time":1427397543312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2818,"Index":815,"Attempt":0,"Launch Time":1427397543320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2816,"Index":813,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":341965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2819,"Index":816,"Attempt":0,"Launch Time":1427397543321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2815,"Index":812,"Attempt":0,"Launch Time":1427397543313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320049,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2820,"Index":817,"Attempt":0,"Launch Time":1427397543326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2817,"Index":814,"Attempt":0,"Launch Time":1427397543319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2821,"Index":818,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2822,"Index":819,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2767,"Index":764,"Attempt":0,"Launch Time":1427397543189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":142,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":380909,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2823,"Index":820,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2824,"Index":821,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2825,"Index":822,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2769,"Index":766,"Attempt":0,"Launch Time":1427397543206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":122,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":560598,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2810,"Index":807,"Attempt":0,"Launch Time":1427397543293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2819,"Index":816,"Attempt":0,"Launch Time":1427397543321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2818,"Index":815,"Attempt":0,"Launch Time":1427397543320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2826,"Index":823,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2820,"Index":817,"Attempt":0,"Launch Time":1427397543326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":414763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2827,"Index":824,"Attempt":0,"Launch Time":1427397543341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2821,"Index":818,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348013,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2823,"Index":820,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355959,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2828,"Index":825,"Attempt":0,"Launch Time":1427397543342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2829,"Index":826,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2826,"Index":823,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":649904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2830,"Index":827,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2824,"Index":821,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":358518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2831,"Index":828,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2832,"Index":829,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2828,"Index":825,"Attempt":0,"Launch Time":1427397543342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2827,"Index":824,"Attempt":0,"Launch Time":1427397543341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1079638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2833,"Index":830,"Attempt":0,"Launch Time":1427397543353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2829,"Index":826,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2834,"Index":831,"Attempt":0,"Launch Time":1427397543356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2835,"Index":832,"Attempt":0,"Launch Time":1427397543357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2830,"Index":827,"Attempt":0,"Launch Time":1427397543346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2832,"Index":829,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352560,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2836,"Index":833,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2813,"Index":810,"Attempt":0,"Launch Time":1427397543298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2837,"Index":834,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2831,"Index":828,"Attempt":0,"Launch Time":1427397543349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2838,"Index":835,"Attempt":0,"Launch Time":1427397543360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2833,"Index":830,"Attempt":0,"Launch Time":1427397543353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368434,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2839,"Index":836,"Attempt":0,"Launch Time":1427397543365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2835,"Index":832,"Attempt":0,"Launch Time":1427397543357,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":441303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2840,"Index":837,"Attempt":0,"Launch Time":1427397543366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2834,"Index":831,"Attempt":0,"Launch Time":1427397543356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543366,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":448575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2841,"Index":838,"Attempt":0,"Launch Time":1427397543367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2837,"Index":834,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2842,"Index":839,"Attempt":0,"Launch Time":1427397543369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2838,"Index":835,"Attempt":0,"Launch Time":1427397543360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":397984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2843,"Index":840,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2839,"Index":836,"Attempt":0,"Launch Time":1427397543365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543373,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325516,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2844,"Index":841,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2840,"Index":837,"Attempt":0,"Launch Time":1427397543366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":439084,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2845,"Index":842,"Attempt":0,"Launch Time":1427397543374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2841,"Index":838,"Attempt":0,"Launch Time":1427397543367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2846,"Index":843,"Attempt":0,"Launch Time":1427397543377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2842,"Index":839,"Attempt":0,"Launch Time":1427397543369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301374,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2847,"Index":844,"Attempt":0,"Launch Time":1427397543380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2843,"Index":840,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2848,"Index":845,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2845,"Index":842,"Attempt":0,"Launch Time":1427397543374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2849,"Index":846,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2844,"Index":841,"Attempt":0,"Launch Time":1427397543373,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2850,"Index":847,"Attempt":0,"Launch Time":1427397543388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2847,"Index":844,"Attempt":0,"Launch Time":1427397543380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":427393,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2851,"Index":848,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2852,"Index":849,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2848,"Index":845,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2849,"Index":846,"Attempt":0,"Launch Time":1427397543381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543389,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423444,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2853,"Index":850,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2846,"Index":843,"Attempt":0,"Launch Time":1427397543377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2854,"Index":851,"Attempt":0,"Launch Time":1427397543397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2772,"Index":769,"Attempt":0,"Launch Time":1427397543214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":40,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2855,"Index":852,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2850,"Index":847,"Attempt":0,"Launch Time":1427397543388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2856,"Index":853,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2836,"Index":833,"Attempt":0,"Launch Time":1427397543358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":410705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2857,"Index":854,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2854,"Index":851,"Attempt":0,"Launch Time":1427397543397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2858,"Index":855,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2851,"Index":848,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":367832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2859,"Index":856,"Attempt":0,"Launch Time":1427397543406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2852,"Index":849,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2860,"Index":857,"Attempt":0,"Launch Time":1427397543408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2856,"Index":853,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":392767,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2861,"Index":858,"Attempt":0,"Launch Time":1427397543423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543430,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2858,"Index":855,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543424,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2862,"Index":859,"Attempt":0,"Launch Time":1427397543427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543434,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2855,"Index":852,"Attempt":0,"Launch Time":1427397543398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":364830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2863,"Index":860,"Attempt":0,"Launch Time":1427397543429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543436,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2853,"Index":850,"Attempt":0,"Launch Time":1427397543389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2864,"Index":861,"Attempt":0,"Launch Time":1427397543430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543437,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2861,"Index":858,"Attempt":0,"Launch Time":1427397543423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543430,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2865,"Index":862,"Attempt":0,"Launch Time":1427397543432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543438,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2822,"Index":819,"Attempt":0,"Launch Time":1427397543334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543432,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2866,"Index":863,"Attempt":0,"Launch Time":1427397543434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543446,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2862,"Index":859,"Attempt":0,"Launch Time":1427397543427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2867,"Index":864,"Attempt":0,"Launch Time":1427397543436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2863,"Index":860,"Attempt":0,"Launch Time":1427397543429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309915,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2868,"Index":865,"Attempt":0,"Launch Time":1427397543437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543444,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2864,"Index":861,"Attempt":0,"Launch Time":1427397543430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311270,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2869,"Index":866,"Attempt":0,"Launch Time":1427397543438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2865,"Index":862,"Attempt":0,"Launch Time":1427397543432,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2870,"Index":867,"Attempt":0,"Launch Time":1427397543444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543451,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2868,"Index":865,"Attempt":0,"Launch Time":1427397543437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2871,"Index":868,"Attempt":0,"Launch Time":1427397543446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543453,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2866,"Index":863,"Attempt":0,"Launch Time":1427397543434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2872,"Index":869,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2869,"Index":866,"Attempt":0,"Launch Time":1427397543438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2873,"Index":870,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543454,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2867,"Index":864,"Attempt":0,"Launch Time":1427397543436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":438391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2874,"Index":871,"Attempt":0,"Launch Time":1427397543450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2870,"Index":867,"Attempt":0,"Launch Time":1427397543444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":496793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2875,"Index":872,"Attempt":0,"Launch Time":1427397543453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2871,"Index":868,"Attempt":0,"Launch Time":1427397543446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2876,"Index":873,"Attempt":0,"Launch Time":1427397543454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2873,"Index":870,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354390,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2877,"Index":874,"Attempt":0,"Launch Time":1427397543460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2874,"Index":871,"Attempt":0,"Launch Time":1427397543450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":473061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2878,"Index":875,"Attempt":0,"Launch Time":1427397543462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2859,"Index":856,"Attempt":0,"Launch Time":1427397543406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":55,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":498948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2825,"Index":822,"Attempt":0,"Launch Time":1427397543335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":130,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":376945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2879,"Index":876,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2880,"Index":877,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2881,"Index":878,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2882,"Index":879,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2883,"Index":880,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2884,"Index":881,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2876,"Index":873,"Attempt":0,"Launch Time":1427397543454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":468133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2860,"Index":857,"Attempt":0,"Launch Time":1427397543408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":47,"Executor Run Time":40,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2857,"Index":854,"Attempt":0,"Launch Time":1427397543405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":64,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2875,"Index":872,"Attempt":0,"Launch Time":1427397543453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":414588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2872,"Index":869,"Attempt":0,"Launch Time":1427397543447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":400218,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2885,"Index":882,"Attempt":0,"Launch Time":1427397543501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2878,"Index":875,"Attempt":0,"Launch Time":1427397543462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352195,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2886,"Index":883,"Attempt":0,"Launch Time":1427397543504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2884,"Index":881,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2887,"Index":884,"Attempt":0,"Launch Time":1427397543506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2882,"Index":879,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2888,"Index":885,"Attempt":0,"Launch Time":1427397543508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2881,"Index":878,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327212,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2889,"Index":886,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2883,"Index":880,"Attempt":0,"Launch Time":1427397543496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2890,"Index":887,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2877,"Index":874,"Attempt":0,"Launch Time":1427397543460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543509,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":665763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2891,"Index":888,"Attempt":0,"Launch Time":1427397543515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2886,"Index":883,"Attempt":0,"Launch Time":1427397543504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":437322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2892,"Index":889,"Attempt":0,"Launch Time":1427397543517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2889,"Index":886,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":399308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2893,"Index":890,"Attempt":0,"Launch Time":1427397543518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2887,"Index":884,"Attempt":0,"Launch Time":1427397543506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":496498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2894,"Index":891,"Attempt":0,"Launch Time":1427397543523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2891,"Index":888,"Attempt":0,"Launch Time":1427397543515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2895,"Index":892,"Attempt":0,"Launch Time":1427397543524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2893,"Index":890,"Attempt":0,"Launch Time":1427397543518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543524,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2896,"Index":893,"Attempt":0,"Launch Time":1427397543528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2888,"Index":885,"Attempt":0,"Launch Time":1427397543508,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":420784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2897,"Index":894,"Attempt":0,"Launch Time":1427397543530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2894,"Index":891,"Attempt":0,"Launch Time":1427397543523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313552,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2898,"Index":895,"Attempt":0,"Launch Time":1427397543531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2892,"Index":889,"Attempt":0,"Launch Time":1427397543517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2980201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2899,"Index":896,"Attempt":0,"Launch Time":1427397543532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2895,"Index":892,"Attempt":0,"Launch Time":1427397543524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322500,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2900,"Index":897,"Attempt":0,"Launch Time":1427397543537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2897,"Index":894,"Attempt":0,"Launch Time":1427397543530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":407322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2901,"Index":898,"Attempt":0,"Launch Time":1427397543540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2898,"Index":895,"Attempt":0,"Launch Time":1427397543531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2902,"Index":899,"Attempt":0,"Launch Time":1427397543545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2900,"Index":897,"Attempt":0,"Launch Time":1427397543537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":372246,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2903,"Index":900,"Attempt":0,"Launch Time":1427397543547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2901,"Index":898,"Attempt":0,"Launch Time":1427397543540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2904,"Index":901,"Attempt":0,"Launch Time":1427397543549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2899,"Index":896,"Attempt":0,"Launch Time":1427397543532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2905,"Index":902,"Attempt":0,"Launch Time":1427397543550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2896,"Index":893,"Attempt":0,"Launch Time":1427397543528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":601961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2906,"Index":903,"Attempt":0,"Launch Time":1427397543552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2902,"Index":899,"Attempt":0,"Launch Time":1427397543545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2907,"Index":904,"Attempt":0,"Launch Time":1427397543572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2906,"Index":903,"Attempt":0,"Launch Time":1427397543552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":397809,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2908,"Index":905,"Attempt":0,"Launch Time":1427397543573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2905,"Index":902,"Attempt":0,"Launch Time":1427397543550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":20,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2909,"Index":906,"Attempt":0,"Launch Time":1427397543580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2907,"Index":904,"Attempt":0,"Launch Time":1427397543572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":434945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2910,"Index":907,"Attempt":0,"Launch Time":1427397543583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2908,"Index":905,"Attempt":0,"Launch Time":1427397543573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":436567,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2911,"Index":908,"Attempt":0,"Launch Time":1427397543588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2909,"Index":906,"Attempt":0,"Launch Time":1427397543580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2912,"Index":909,"Attempt":0,"Launch Time":1427397543589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2910,"Index":907,"Attempt":0,"Launch Time":1427397543583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318107,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2913,"Index":910,"Attempt":0,"Launch Time":1427397543596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2879,"Index":876,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2914,"Index":911,"Attempt":0,"Launch Time":1427397543597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2880,"Index":877,"Attempt":0,"Launch Time":1427397543495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2915,"Index":912,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2890,"Index":887,"Attempt":0,"Launch Time":1427397543509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319283,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2916,"Index":913,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2885,"Index":882,"Attempt":0,"Launch Time":1427397543501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":876044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2917,"Index":914,"Attempt":0,"Launch Time":1427397543602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2913,"Index":910,"Attempt":0,"Launch Time":1427397543596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2918,"Index":915,"Attempt":0,"Launch Time":1427397543604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2914,"Index":911,"Attempt":0,"Launch Time":1427397543597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2919,"Index":916,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2920,"Index":917,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2916,"Index":913,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2915,"Index":912,"Attempt":0,"Launch Time":1427397543599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2917,"Index":914,"Attempt":0,"Launch Time":1427397543602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335840,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2921,"Index":918,"Attempt":0,"Launch Time":1427397543610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2922,"Index":919,"Attempt":0,"Launch Time":1427397543613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2918,"Index":915,"Attempt":0,"Launch Time":1427397543604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2923,"Index":920,"Attempt":0,"Launch Time":1427397543616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2919,"Index":916,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2924,"Index":921,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2920,"Index":917,"Attempt":0,"Launch Time":1427397543607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":704507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2925,"Index":922,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2921,"Index":918,"Attempt":0,"Launch Time":1427397543610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2926,"Index":923,"Attempt":0,"Launch Time":1427397543621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2922,"Index":919,"Attempt":0,"Launch Time":1427397543613,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2927,"Index":924,"Attempt":0,"Launch Time":1427397543623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2923,"Index":920,"Attempt":0,"Launch Time":1427397543616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2928,"Index":925,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2925,"Index":922,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2929,"Index":926,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2924,"Index":921,"Attempt":0,"Launch Time":1427397543617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325576,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2930,"Index":927,"Attempt":0,"Launch Time":1427397543630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2926,"Index":923,"Attempt":0,"Launch Time":1427397543621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543630,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":412153,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2931,"Index":928,"Attempt":0,"Launch Time":1427397543632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2929,"Index":926,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2932,"Index":929,"Attempt":0,"Launch Time":1427397543633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2927,"Index":924,"Attempt":0,"Launch Time":1427397543623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":919364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2933,"Index":930,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2930,"Index":927,"Attempt":0,"Launch Time":1427397543630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335546,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2934,"Index":931,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2928,"Index":925,"Attempt":0,"Launch Time":1427397543625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2935,"Index":932,"Attempt":0,"Launch Time":1427397543639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2931,"Index":928,"Attempt":0,"Launch Time":1427397543632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372101,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2936,"Index":933,"Attempt":0,"Launch Time":1427397543642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2932,"Index":929,"Attempt":0,"Launch Time":1427397543633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2937,"Index":934,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2933,"Index":930,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":358501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2938,"Index":935,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2934,"Index":931,"Attempt":0,"Launch Time":1427397543637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":466896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2939,"Index":936,"Attempt":0,"Launch Time":1427397543646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2935,"Index":932,"Attempt":0,"Launch Time":1427397543639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2940,"Index":937,"Attempt":0,"Launch Time":1427397543649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2936,"Index":933,"Attempt":0,"Launch Time":1427397543642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2941,"Index":938,"Attempt":0,"Launch Time":1427397543652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2938,"Index":935,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2942,"Index":939,"Attempt":0,"Launch Time":1427397543653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2939,"Index":936,"Attempt":0,"Launch Time":1427397543646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2943,"Index":940,"Attempt":0,"Launch Time":1427397543656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2937,"Index":934,"Attempt":0,"Launch Time":1427397543645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":817315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2944,"Index":941,"Attempt":0,"Launch Time":1427397543657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2940,"Index":937,"Attempt":0,"Launch Time":1427397543649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2945,"Index":942,"Attempt":0,"Launch Time":1427397543658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2904,"Index":901,"Attempt":0,"Launch Time":1427397543549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":107,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4345747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2946,"Index":943,"Attempt":0,"Launch Time":1427397543660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2947,"Index":944,"Attempt":0,"Launch Time":1427397543661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2942,"Index":939,"Attempt":0,"Launch Time":1427397543653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":410686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2941,"Index":938,"Attempt":0,"Launch Time":1427397543652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2948,"Index":945,"Attempt":0,"Launch Time":1427397543663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2943,"Index":940,"Attempt":0,"Launch Time":1427397543656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2949,"Index":946,"Attempt":0,"Launch Time":1427397543666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2944,"Index":941,"Attempt":0,"Launch Time":1427397543657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":457216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2950,"Index":947,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2947,"Index":944,"Attempt":0,"Launch Time":1427397543661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2951,"Index":948,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2946,"Index":943,"Attempt":0,"Launch Time":1427397543660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304208,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2952,"Index":949,"Attempt":0,"Launch Time":1427397543682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2949,"Index":946,"Attempt":0,"Launch Time":1427397543666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2953,"Index":950,"Attempt":0,"Launch Time":1427397543685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2951,"Index":948,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1444822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2954,"Index":951,"Attempt":0,"Launch Time":1427397543691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2952,"Index":949,"Attempt":0,"Launch Time":1427397543682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":332714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2955,"Index":952,"Attempt":0,"Launch Time":1427397543692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2953,"Index":950,"Attempt":0,"Launch Time":1427397543685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332463,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2956,"Index":953,"Attempt":0,"Launch Time":1427397543694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2903,"Index":900,"Attempt":0,"Launch Time":1427397543547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":146,"Result Size":930,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373709,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2957,"Index":954,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2955,"Index":952,"Attempt":0,"Launch Time":1427397543692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2958,"Index":955,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2950,"Index":947,"Attempt":0,"Launch Time":1427397543668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543701,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2959,"Index":956,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543712,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2954,"Index":951,"Attempt":0,"Launch Time":1427397543691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2960,"Index":957,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543711,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2948,"Index":945,"Attempt":0,"Launch Time":1427397543663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2961,"Index":958,"Attempt":0,"Launch Time":1427397543703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2912,"Index":909,"Attempt":0,"Launch Time":1427397543589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":366516,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2962,"Index":959,"Attempt":0,"Launch Time":1427397543709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2958,"Index":955,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2963,"Index":960,"Attempt":0,"Launch Time":1427397543711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2960,"Index":957,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2964,"Index":961,"Attempt":0,"Launch Time":1427397543712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2959,"Index":956,"Attempt":0,"Launch Time":1427397543702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":358696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2965,"Index":962,"Attempt":0,"Launch Time":1427397543715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2956,"Index":953,"Attempt":0,"Launch Time":1427397543694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2966,"Index":963,"Attempt":0,"Launch Time":1427397543728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2962,"Index":959,"Attempt":0,"Launch Time":1427397543709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2967,"Index":964,"Attempt":0,"Launch Time":1427397543729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2963,"Index":960,"Attempt":0,"Launch Time":1427397543711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2911,"Index":908,"Attempt":0,"Launch Time":1427397543588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324219,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2968,"Index":965,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2969,"Index":966,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2965,"Index":962,"Attempt":0,"Launch Time":1427397543715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341296,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2970,"Index":967,"Attempt":0,"Launch Time":1427397543732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2945,"Index":942,"Attempt":0,"Launch Time":1427397543658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":6,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362655,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2961,"Index":958,"Attempt":0,"Launch Time":1427397543703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2971,"Index":968,"Attempt":0,"Launch Time":1427397543733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2972,"Index":969,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2967,"Index":964,"Attempt":0,"Launch Time":1427397543729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2973,"Index":970,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2968,"Index":965,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340884,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2974,"Index":971,"Attempt":0,"Launch Time":1427397543740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2969,"Index":966,"Attempt":0,"Launch Time":1427397543731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2975,"Index":972,"Attempt":0,"Launch Time":1427397543747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2966,"Index":963,"Attempt":0,"Launch Time":1427397543728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2976,"Index":973,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2972,"Index":969,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339999,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2977,"Index":974,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2973,"Index":970,"Attempt":0,"Launch Time":1427397543739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2978,"Index":975,"Attempt":0,"Launch Time":1427397543754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2977,"Index":974,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":262907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2979,"Index":976,"Attempt":0,"Launch Time":1427397543755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2974,"Index":971,"Attempt":0,"Launch Time":1427397543740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":393681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2980,"Index":977,"Attempt":0,"Launch Time":1427397543758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2976,"Index":973,"Attempt":0,"Launch Time":1427397543748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2981,"Index":978,"Attempt":0,"Launch Time":1427397543761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2978,"Index":975,"Attempt":0,"Launch Time":1427397543754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2982,"Index":979,"Attempt":0,"Launch Time":1427397543763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2979,"Index":976,"Attempt":0,"Launch Time":1427397543755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":386648,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2983,"Index":980,"Attempt":0,"Launch Time":1427397543767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2980,"Index":977,"Attempt":0,"Launch Time":1427397543758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":371622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2984,"Index":981,"Attempt":0,"Launch Time":1427397543768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2981,"Index":978,"Attempt":0,"Launch Time":1427397543761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2985,"Index":982,"Attempt":0,"Launch Time":1427397543770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2975,"Index":972,"Attempt":0,"Launch Time":1427397543747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2986,"Index":983,"Attempt":0,"Launch Time":1427397543771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2982,"Index":979,"Attempt":0,"Launch Time":1427397543763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2987,"Index":984,"Attempt":0,"Launch Time":1427397543774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2983,"Index":980,"Attempt":0,"Launch Time":1427397543767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2988,"Index":985,"Attempt":0,"Launch Time":1427397543776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2984,"Index":981,"Attempt":0,"Launch Time":1427397543768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":414503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2989,"Index":986,"Attempt":0,"Launch Time":1427397543782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2985,"Index":982,"Attempt":0,"Launch Time":1427397543770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2990,"Index":987,"Attempt":0,"Launch Time":1427397543783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2987,"Index":984,"Attempt":0,"Launch Time":1427397543774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":430686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2991,"Index":988,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2988,"Index":985,"Attempt":0,"Launch Time":1427397543776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":473711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2992,"Index":989,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2986,"Index":983,"Attempt":0,"Launch Time":1427397543771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":497203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2993,"Index":990,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2964,"Index":961,"Attempt":0,"Launch Time":1427397543712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":77,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":34027291,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2994,"Index":991,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2995,"Index":992,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2989,"Index":986,"Attempt":0,"Launch Time":1427397543782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2990,"Index":987,"Attempt":0,"Launch Time":1427397543783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345766,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2996,"Index":993,"Attempt":0,"Launch Time":1427397543802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2992,"Index":989,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2997,"Index":994,"Attempt":0,"Launch Time":1427397543808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2996,"Index":993,"Attempt":0,"Launch Time":1427397543802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2998,"Index":995,"Attempt":0,"Launch Time":1427397543813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2991,"Index":988,"Attempt":0,"Launch Time":1427397543785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":491685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":2999,"Index":996,"Attempt":0,"Launch Time":1427397543814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2993,"Index":990,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":405241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3000,"Index":997,"Attempt":0,"Launch Time":1427397543816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2994,"Index":991,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3001,"Index":998,"Attempt":0,"Launch Time":1427397543817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2995,"Index":992,"Attempt":0,"Launch Time":1427397543790,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":3002,"Index":999,"Attempt":0,"Launch Time":1427397543818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2997,"Index":994,"Attempt":0,"Launch Time":1427397543808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2998,"Index":995,"Attempt":0,"Launch Time":1427397543813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320246,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2999,"Index":996,"Attempt":0,"Launch Time":1427397543814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319459,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3000,"Index":997,"Attempt":0,"Launch Time":1427397543816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354077,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3001,"Index":998,"Attempt":0,"Launch Time":1427397543817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":668882,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2970,"Index":967,"Attempt":0,"Launch Time":1427397543732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":2935869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3002,"Index":999,"Attempt":0,"Launch Time":1427397543818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2957,"Index":954,"Attempt":0,"Launch Time":1427397543701,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":120,"Executor Run Time":9,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":449183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2971,"Index":968,"Attempt":0,"Launch Time":1427397543733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397541495,"Completion Time":1427397543832,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":3003,"Index":0,"Attempt":0,"Launch Time":1427397543837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3003,"Index":0,"Attempt":0,"Launch Time":1427397543837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397543951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":113,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192030,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line26.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line26.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line26.$read$$iwC$$iwC$$iwC.(:39)\n$line26.$read$$iwC$$iwC.(:41)\n$line26.$read$$iwC.(:43)\n$line26.$read.(:45)\n$line26.$read$.(:49)\n$line26.$read$.()\n$line26.$eval$.(:7)\n$line26.$eval$.()\n$line26.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397543837,"Completion Time":1427397543951,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1427397543952,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1427397546619,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[9,8]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":3004,"Index":0,"Attempt":0,"Launch Time":1427397546623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3004,"Index":0,"Attempt":0,"Launch Time":1427397546623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397546709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":86,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192019,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line28.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line28.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line28.$read$$iwC$$iwC$$iwC.(:39)\n$line28.$read$$iwC$$iwC.(:41)\n$line28.$read$$iwC.(:43)\n$line28.$read.(:45)\n$line28.$read$.(:49)\n$line28.$read$.()\n$line28.$eval$.(:7)\n$line28.$eval$.()\n$line28.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397546623,"Completion Time":1427397546710,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1427397546710,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1427397548267,"Stage Infos":[{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[10,11]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":3005,"Index":0,"Attempt":0,"Launch Time":1427397548270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3005,"Index":0,"Attempt":0,"Launch Time":1427397548270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397548375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":104,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192019,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line30.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line30.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line30.$read$$iwC$$iwC$$iwC.(:39)\n$line30.$read$$iwC$$iwC.(:41)\n$line30.$read$$iwC.(:43)\n$line30.$read.(:45)\n$line30.$read$.(:49)\n$line30.$read$.()\n$line30.$eval$.(:7)\n$line30.$eval$.()\n$line30.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397548270,"Completion Time":1427397548375,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1427397548375,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1427397549487,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[12,13]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3006,"Index":0,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3007,"Index":1,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3008,"Index":2,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3009,"Index":3,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3010,"Index":4,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3011,"Index":5,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3012,"Index":6,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3013,"Index":7,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3014,"Index":8,"Attempt":0,"Launch Time":1427397549504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3011,"Index":5,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3008,"Index":2,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302702,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3015,"Index":9,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3016,"Index":10,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3017,"Index":11,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3009,"Index":3,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":276274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3006,"Index":0,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3018,"Index":12,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3019,"Index":13,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3010,"Index":4,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3012,"Index":6,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549517,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3020,"Index":14,"Attempt":0,"Launch Time":1427397549520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3007,"Index":1,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305324,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3021,"Index":15,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3022,"Index":16,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3023,"Index":17,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3018,"Index":12,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3016,"Index":10,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374792,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3024,"Index":18,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3025,"Index":19,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3013,"Index":7,"Attempt":0,"Launch Time":1427397549496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1447021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3015,"Index":9,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":633706,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3014,"Index":8,"Attempt":0,"Launch Time":1427397549504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3026,"Index":20,"Attempt":0,"Launch Time":1427397549528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3027,"Index":21,"Attempt":0,"Launch Time":1427397549529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3019,"Index":13,"Attempt":0,"Launch Time":1427397549517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3028,"Index":22,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3029,"Index":23,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3020,"Index":14,"Attempt":0,"Launch Time":1427397549520,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":266687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3022,"Index":16,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3017,"Index":11,"Attempt":0,"Launch Time":1427397549515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1418988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3030,"Index":24,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3031,"Index":25,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3032,"Index":26,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3033,"Index":27,"Attempt":0,"Launch Time":1427397549537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3034,"Index":28,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3035,"Index":29,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3025,"Index":19,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":357151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3023,"Index":17,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312482,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3026,"Index":20,"Attempt":0,"Launch Time":1427397549528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3021,"Index":15,"Attempt":0,"Launch Time":1427397549524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3024,"Index":18,"Attempt":0,"Launch Time":1427397549526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":387862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3036,"Index":30,"Attempt":0,"Launch Time":1427397549545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3037,"Index":31,"Attempt":0,"Launch Time":1427397549546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3028,"Index":22,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3029,"Index":23,"Attempt":0,"Launch Time":1427397549534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3027,"Index":21,"Attempt":0,"Launch Time":1427397549529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":288479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3038,"Index":32,"Attempt":0,"Launch Time":1427397549547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3039,"Index":33,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3040,"Index":34,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3041,"Index":35,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3042,"Index":36,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3032,"Index":26,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":398608,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3034,"Index":28,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405084,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3043,"Index":37,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3044,"Index":38,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3033,"Index":27,"Attempt":0,"Launch Time":1427397549537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411363,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3035,"Index":29,"Attempt":0,"Launch Time":1427397549538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3036,"Index":30,"Attempt":0,"Launch Time":1427397549545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3031,"Index":25,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":431403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3045,"Index":39,"Attempt":0,"Launch Time":1427397549557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3046,"Index":40,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3047,"Index":41,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3048,"Index":42,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3049,"Index":43,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3050,"Index":44,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3051,"Index":45,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3052,"Index":46,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3030,"Index":24,"Attempt":0,"Launch Time":1427397549536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":462571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3042,"Index":36,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307346,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3041,"Index":35,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297041,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3039,"Index":33,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3043,"Index":37,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":304146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3044,"Index":38,"Attempt":0,"Launch Time":1427397549552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3040,"Index":34,"Attempt":0,"Launch Time":1427397549548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3038,"Index":32,"Attempt":0,"Launch Time":1427397549547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3037,"Index":31,"Attempt":0,"Launch Time":1427397549546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":271847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3053,"Index":47,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3054,"Index":48,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3047,"Index":41,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3045,"Index":39,"Attempt":0,"Launch Time":1427397549557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3055,"Index":49,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3048,"Index":42,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3056,"Index":50,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3057,"Index":51,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3058,"Index":52,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3059,"Index":53,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3046,"Index":40,"Attempt":0,"Launch Time":1427397549558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3049,"Index":43,"Attempt":0,"Launch Time":1427397549559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3050,"Index":44,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317180,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3052,"Index":46,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":312931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3060,"Index":54,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3061,"Index":55,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3062,"Index":56,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3063,"Index":57,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3051,"Index":45,"Attempt":0,"Launch Time":1427397549562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3053,"Index":47,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3055,"Index":49,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3054,"Index":48,"Attempt":0,"Launch Time":1427397549568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3064,"Index":58,"Attempt":0,"Launch Time":1427397549601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3065,"Index":59,"Attempt":0,"Launch Time":1427397549602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3066,"Index":60,"Attempt":0,"Launch Time":1427397549603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3067,"Index":61,"Attempt":0,"Launch Time":1427397549605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3068,"Index":62,"Attempt":0,"Launch Time":1427397549606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3069,"Index":63,"Attempt":0,"Launch Time":1427397549612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3070,"Index":64,"Attempt":0,"Launch Time":1427397549614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3071,"Index":65,"Attempt":0,"Launch Time":1427397549615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3072,"Index":66,"Attempt":0,"Launch Time":1427397549621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3073,"Index":67,"Attempt":0,"Launch Time":1427397549622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3074,"Index":68,"Attempt":0,"Launch Time":1427397549627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3075,"Index":69,"Attempt":0,"Launch Time":1427397549628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3076,"Index":70,"Attempt":0,"Launch Time":1427397549630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3077,"Index":71,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3078,"Index":72,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3079,"Index":73,"Attempt":0,"Launch Time":1427397549638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3080,"Index":74,"Attempt":0,"Launch Time":1427397549639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3057,"Index":51,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3061,"Index":55,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306237,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3058,"Index":52,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3062,"Index":56,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3067,"Index":61,"Attempt":0,"Launch Time":1427397549605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3070,"Index":64,"Attempt":0,"Launch Time":1427397549614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316217,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3063,"Index":57,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":266148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3069,"Index":63,"Attempt":0,"Launch Time":1427397549612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":446110,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3065,"Index":59,"Attempt":0,"Launch Time":1427397549602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":2040919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3071,"Index":65,"Attempt":0,"Launch Time":1427397549615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":360161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3068,"Index":62,"Attempt":0,"Launch Time":1427397549606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3073,"Index":67,"Attempt":0,"Launch Time":1427397549622,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":396314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3066,"Index":60,"Attempt":0,"Launch Time":1427397549603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3076,"Index":70,"Attempt":0,"Launch Time":1427397549630,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329537,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3074,"Index":68,"Attempt":0,"Launch Time":1427397549627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":332201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3081,"Index":75,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3082,"Index":76,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3072,"Index":66,"Attempt":0,"Launch Time":1427397549621,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3077,"Index":71,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3064,"Index":58,"Attempt":0,"Launch Time":1427397549601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":461624,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3075,"Index":69,"Attempt":0,"Launch Time":1427397549628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":316157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3083,"Index":77,"Attempt":0,"Launch Time":1427397549647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3079,"Index":73,"Attempt":0,"Launch Time":1427397549638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549647,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394675,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3084,"Index":78,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3059,"Index":53,"Attempt":0,"Launch Time":1427397549575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":474463,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3085,"Index":79,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3060,"Index":54,"Attempt":0,"Launch Time":1427397549577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":72,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328460,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3086,"Index":80,"Attempt":0,"Launch Time":1427397549651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3087,"Index":81,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3081,"Index":75,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3082,"Index":76,"Attempt":0,"Launch Time":1427397549642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3088,"Index":82,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3078,"Index":72,"Attempt":0,"Launch Time":1427397549634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4972058,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3089,"Index":83,"Attempt":0,"Launch Time":1427397549653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3056,"Index":50,"Attempt":0,"Launch Time":1427397549573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":77,"Result Size":930,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3090,"Index":84,"Attempt":0,"Launch Time":1427397549655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3083,"Index":77,"Attempt":0,"Launch Time":1427397549647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549655,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3091,"Index":85,"Attempt":0,"Launch Time":1427397549656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3080,"Index":74,"Attempt":0,"Launch Time":1427397549639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549656,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":628401,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3092,"Index":86,"Attempt":0,"Launch Time":1427397549659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3088,"Index":82,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3093,"Index":87,"Attempt":0,"Launch Time":1427397549661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3089,"Index":83,"Attempt":0,"Launch Time":1427397549653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3094,"Index":88,"Attempt":0,"Launch Time":1427397549662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3090,"Index":84,"Attempt":0,"Launch Time":1427397549655,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3095,"Index":89,"Attempt":0,"Launch Time":1427397549664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3086,"Index":80,"Attempt":0,"Launch Time":1427397549651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3096,"Index":90,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549673,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3091,"Index":85,"Attempt":0,"Launch Time":1427397549656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313649,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3097,"Index":91,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3092,"Index":86,"Attempt":0,"Launch Time":1427397549659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3098,"Index":92,"Attempt":0,"Launch Time":1427397549669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549677,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3093,"Index":87,"Attempt":0,"Launch Time":1427397549661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3099,"Index":93,"Attempt":0,"Launch Time":1427397549670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549678,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3094,"Index":88,"Attempt":0,"Launch Time":1427397549662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3100,"Index":94,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3095,"Index":89,"Attempt":0,"Launch Time":1427397549664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549672,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3101,"Index":95,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3096,"Index":90,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3102,"Index":96,"Attempt":0,"Launch Time":1427397549677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3098,"Index":92,"Attempt":0,"Launch Time":1427397549669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":309600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3103,"Index":97,"Attempt":0,"Launch Time":1427397549678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549696,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3099,"Index":93,"Attempt":0,"Launch Time":1427397549670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":363292,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3104,"Index":98,"Attempt":0,"Launch Time":1427397549679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549705,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3085,"Index":79,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424375,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3105,"Index":99,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549695,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3106,"Index":100,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3100,"Index":94,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3101,"Index":95,"Attempt":0,"Launch Time":1427397549672,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371110,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3107,"Index":101,"Attempt":0,"Launch Time":1427397549693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3087,"Index":81,"Attempt":0,"Launch Time":1427397549652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3108,"Index":102,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3105,"Index":99,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3109,"Index":103,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3103,"Index":97,"Attempt":0,"Launch Time":1427397549678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3110,"Index":104,"Attempt":0,"Launch Time":1427397549698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3111,"Index":105,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3084,"Index":78,"Attempt":0,"Launch Time":1427397549649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3097,"Index":91,"Attempt":0,"Launch Time":1427397549666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3112,"Index":106,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549706,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3102,"Index":96,"Attempt":0,"Launch Time":1427397549677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3113,"Index":107,"Attempt":0,"Launch Time":1427397549702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549709,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3109,"Index":103,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3114,"Index":108,"Attempt":0,"Launch Time":1427397549703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549710,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3108,"Index":102,"Attempt":0,"Launch Time":1427397549695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3115,"Index":109,"Attempt":0,"Launch Time":1427397549705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549714,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3104,"Index":98,"Attempt":0,"Launch Time":1427397549679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2719222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3112,"Index":106,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3116,"Index":110,"Attempt":0,"Launch Time":1427397549706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549713,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3117,"Index":111,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549728,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3111,"Index":105,"Attempt":0,"Launch Time":1427397549699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3118,"Index":112,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3110,"Index":104,"Attempt":0,"Launch Time":1427397549698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3119,"Index":113,"Attempt":0,"Launch Time":1427397549709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549716,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3113,"Index":107,"Attempt":0,"Launch Time":1427397549702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354368,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3120,"Index":114,"Attempt":0,"Launch Time":1427397549710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3114,"Index":108,"Attempt":0,"Launch Time":1427397549703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3121,"Index":115,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549726,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3116,"Index":110,"Attempt":0,"Launch Time":1427397549706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":267819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3122,"Index":116,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549720,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3115,"Index":109,"Attempt":0,"Launch Time":1427397549705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3123,"Index":117,"Attempt":0,"Launch Time":1427397549716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549723,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3119,"Index":113,"Attempt":0,"Launch Time":1427397549709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549716,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272712,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3124,"Index":118,"Attempt":0,"Launch Time":1427397549720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3122,"Index":116,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":272628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3125,"Index":119,"Attempt":0,"Launch Time":1427397549723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3123,"Index":117,"Attempt":0,"Launch Time":1427397549716,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3126,"Index":120,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549732,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3106,"Index":100,"Attempt":0,"Launch Time":1427397549680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336601,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3127,"Index":121,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3120,"Index":114,"Attempt":0,"Launch Time":1427397549710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549725,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3128,"Index":122,"Attempt":0,"Launch Time":1427397549726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3121,"Index":115,"Attempt":0,"Launch Time":1427397549713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":5256232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3129,"Index":123,"Attempt":0,"Launch Time":1427397549728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3117,"Index":111,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":941933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3130,"Index":124,"Attempt":0,"Launch Time":1427397549732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3126,"Index":120,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3131,"Index":125,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3118,"Index":112,"Attempt":0,"Launch Time":1427397549707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3132,"Index":126,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3127,"Index":121,"Attempt":0,"Launch Time":1427397549725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313749,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3133,"Index":127,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3129,"Index":123,"Attempt":0,"Launch Time":1427397549728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325048,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3134,"Index":128,"Attempt":0,"Launch Time":1427397549738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3125,"Index":119,"Attempt":0,"Launch Time":1427397549723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331932,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3135,"Index":129,"Attempt":0,"Launch Time":1427397549739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3107,"Index":101,"Attempt":0,"Launch Time":1427397549693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":35,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":447854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3136,"Index":130,"Attempt":0,"Launch Time":1427397549745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3132,"Index":126,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":346323,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3137,"Index":131,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3135,"Index":129,"Attempt":0,"Launch Time":1427397549739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3138,"Index":132,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3133,"Index":127,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3139,"Index":133,"Attempt":0,"Launch Time":1427397549753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3136,"Index":130,"Attempt":0,"Launch Time":1427397549745,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3140,"Index":134,"Attempt":0,"Launch Time":1427397549754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3137,"Index":131,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3141,"Index":135,"Attempt":0,"Launch Time":1427397549755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3138,"Index":132,"Attempt":0,"Launch Time":1427397549748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3142,"Index":136,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3139,"Index":133,"Attempt":0,"Launch Time":1427397549753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3143,"Index":137,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3140,"Index":134,"Attempt":0,"Launch Time":1427397549754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3144,"Index":138,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3141,"Index":135,"Attempt":0,"Launch Time":1427397549755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3145,"Index":139,"Attempt":0,"Launch Time":1427397549765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3131,"Index":125,"Attempt":0,"Launch Time":1427397549737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549765,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3146,"Index":140,"Attempt":0,"Launch Time":1427397549771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3144,"Index":138,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":428495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3147,"Index":141,"Attempt":0,"Launch Time":1427397549772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3143,"Index":137,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388162,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3148,"Index":142,"Attempt":0,"Launch Time":1427397549773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3142,"Index":136,"Attempt":0,"Launch Time":1427397549763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3149,"Index":143,"Attempt":0,"Launch Time":1427397549774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3145,"Index":139,"Attempt":0,"Launch Time":1427397549765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3150,"Index":144,"Attempt":0,"Launch Time":1427397549778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3128,"Index":122,"Attempt":0,"Launch Time":1427397549726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3151,"Index":145,"Attempt":0,"Launch Time":1427397549786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3146,"Index":140,"Attempt":0,"Launch Time":1427397549771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":531674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3152,"Index":146,"Attempt":0,"Launch Time":1427397549788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3148,"Index":142,"Attempt":0,"Launch Time":1427397549773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3153,"Index":147,"Attempt":0,"Launch Time":1427397549794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3151,"Index":145,"Attempt":0,"Launch Time":1427397549786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":324859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3152,"Index":146,"Attempt":0,"Launch Time":1427397549788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332480,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3154,"Index":148,"Attempt":0,"Launch Time":1427397549795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3155,"Index":149,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3156,"Index":150,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3147,"Index":141,"Attempt":0,"Launch Time":1427397549772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":21818146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3153,"Index":147,"Attempt":0,"Launch Time":1427397549794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3157,"Index":151,"Attempt":0,"Launch Time":1427397549803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3154,"Index":148,"Attempt":0,"Launch Time":1427397549795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3158,"Index":152,"Attempt":0,"Launch Time":1427397549806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3149,"Index":143,"Attempt":0,"Launch Time":1427397549774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401792,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3159,"Index":153,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3156,"Index":150,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3160,"Index":154,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3155,"Index":149,"Attempt":0,"Launch Time":1427397549802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":311636,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3161,"Index":155,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3157,"Index":151,"Attempt":0,"Launch Time":1427397549803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":358117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3162,"Index":156,"Attempt":0,"Launch Time":1427397549814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3134,"Index":128,"Attempt":0,"Launch Time":1427397549738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":29710089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3163,"Index":157,"Attempt":0,"Launch Time":1427397549817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3150,"Index":144,"Attempt":0,"Launch Time":1427397549778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3164,"Index":158,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3160,"Index":154,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312659,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3165,"Index":159,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3158,"Index":152,"Attempt":0,"Launch Time":1427397549806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3166,"Index":160,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3161,"Index":155,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3167,"Index":161,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3159,"Index":153,"Attempt":0,"Launch Time":1427397549810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3168,"Index":162,"Attempt":0,"Launch Time":1427397549824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3162,"Index":156,"Attempt":0,"Launch Time":1427397549814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3169,"Index":163,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3164,"Index":158,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3170,"Index":164,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3167,"Index":161,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3171,"Index":165,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3172,"Index":166,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3165,"Index":159,"Attempt":0,"Launch Time":1427397549819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3166,"Index":160,"Attempt":0,"Launch Time":1427397549820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1305606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3173,"Index":167,"Attempt":0,"Launch Time":1427397549833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3170,"Index":164,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3174,"Index":168,"Attempt":0,"Launch Time":1427397549834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3169,"Index":163,"Attempt":0,"Launch Time":1427397549826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":379741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3175,"Index":169,"Attempt":0,"Launch Time":1427397549836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3171,"Index":165,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":314950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3176,"Index":170,"Attempt":0,"Launch Time":1427397549837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3172,"Index":166,"Attempt":0,"Launch Time":1427397549828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3177,"Index":171,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3178,"Index":172,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3173,"Index":167,"Attempt":0,"Launch Time":1427397549833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3174,"Index":168,"Attempt":0,"Launch Time":1427397549834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3179,"Index":173,"Attempt":0,"Launch Time":1427397549843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3175,"Index":169,"Attempt":0,"Launch Time":1427397549836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3180,"Index":174,"Attempt":0,"Launch Time":1427397549844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3168,"Index":162,"Attempt":0,"Launch Time":1427397549824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1085204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3181,"Index":175,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3182,"Index":176,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3178,"Index":172,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3177,"Index":171,"Attempt":0,"Launch Time":1427397549841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3183,"Index":177,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3180,"Index":174,"Attempt":0,"Launch Time":1427397549844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307954,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3184,"Index":178,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3179,"Index":173,"Attempt":0,"Launch Time":1427397549843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":897796,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3185,"Index":179,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3176,"Index":170,"Attempt":0,"Launch Time":1427397549837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3186,"Index":180,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3163,"Index":157,"Attempt":0,"Launch Time":1427397549817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352920,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3187,"Index":181,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3181,"Index":175,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3188,"Index":182,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3182,"Index":176,"Attempt":0,"Launch Time":1427397549848,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3189,"Index":183,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3183,"Index":177,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311205,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3190,"Index":184,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549866,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3185,"Index":179,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3191,"Index":185,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3186,"Index":180,"Attempt":0,"Launch Time":1427397549852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320641,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3192,"Index":186,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3124,"Index":118,"Attempt":0,"Launch Time":1427397549720,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":139,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":430162,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3193,"Index":187,"Attempt":0,"Launch Time":1427397549863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3130,"Index":124,"Attempt":0,"Launch Time":1427397549732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3194,"Index":188,"Attempt":0,"Launch Time":1427397549865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3190,"Index":184,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3195,"Index":189,"Attempt":0,"Launch Time":1427397549868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3191,"Index":185,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":407518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3196,"Index":190,"Attempt":0,"Launch Time":1427397549872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3192,"Index":186,"Attempt":0,"Launch Time":1427397549859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348024,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3197,"Index":191,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3198,"Index":192,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3193,"Index":187,"Attempt":0,"Launch Time":1427397549863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":536036,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3194,"Index":188,"Attempt":0,"Launch Time":1427397549865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327410,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3199,"Index":193,"Attempt":0,"Launch Time":1427397549888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3197,"Index":191,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549888,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3200,"Index":194,"Attempt":0,"Launch Time":1427397549889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3195,"Index":189,"Attempt":0,"Launch Time":1427397549868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3201,"Index":195,"Attempt":0,"Launch Time":1427397549895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3199,"Index":193,"Attempt":0,"Launch Time":1427397549888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":330404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3202,"Index":196,"Attempt":0,"Launch Time":1427397549896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3200,"Index":194,"Attempt":0,"Launch Time":1427397549889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3203,"Index":197,"Attempt":0,"Launch Time":1427397549911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3196,"Index":190,"Attempt":0,"Launch Time":1427397549872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3204,"Index":198,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3198,"Index":192,"Attempt":0,"Launch Time":1427397549873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346766,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3205,"Index":199,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3184,"Index":178,"Attempt":0,"Launch Time":1427397549851,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":56,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3206,"Index":200,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3207,"Index":201,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3202,"Index":196,"Attempt":0,"Launch Time":1427397549896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277639,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3201,"Index":195,"Attempt":0,"Launch Time":1427397549895,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3208,"Index":202,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3204,"Index":198,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393547,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3209,"Index":203,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3207,"Index":201,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322197,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3210,"Index":204,"Attempt":0,"Launch Time":1427397549924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3203,"Index":197,"Attempt":0,"Launch Time":1427397549911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":293232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3211,"Index":205,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3206,"Index":200,"Attempt":0,"Launch Time":1427397549914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3205,"Index":199,"Attempt":0,"Launch Time":1427397549913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3212,"Index":206,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3213,"Index":207,"Attempt":0,"Launch Time":1427397549939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3211,"Index":205,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":475869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3210,"Index":204,"Attempt":0,"Launch Time":1427397549924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3214,"Index":208,"Attempt":0,"Launch Time":1427397549940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3215,"Index":209,"Attempt":0,"Launch Time":1427397549942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3208,"Index":202,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":435687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3216,"Index":210,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3209,"Index":203,"Attempt":0,"Launch Time":1427397549923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3217,"Index":211,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3212,"Index":206,"Attempt":0,"Launch Time":1427397549931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":351817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3218,"Index":212,"Attempt":0,"Launch Time":1427397549946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3213,"Index":207,"Attempt":0,"Launch Time":1427397549939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326374,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3219,"Index":213,"Attempt":0,"Launch Time":1427397549948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3214,"Index":208,"Attempt":0,"Launch Time":1427397549940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3220,"Index":214,"Attempt":0,"Launch Time":1427397549949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3215,"Index":209,"Attempt":0,"Launch Time":1427397549942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3221,"Index":215,"Attempt":0,"Launch Time":1427397549955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3217,"Index":211,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":417789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3222,"Index":216,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3218,"Index":212,"Attempt":0,"Launch Time":1427397549946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381636,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3223,"Index":217,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3224,"Index":218,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3225,"Index":219,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3219,"Index":213,"Attempt":0,"Launch Time":1427397549948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":354763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3216,"Index":210,"Attempt":0,"Launch Time":1427397549943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":411060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3189,"Index":183,"Attempt":0,"Launch Time":1427397549858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":414384,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3226,"Index":220,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3220,"Index":214,"Attempt":0,"Launch Time":1427397549949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3227,"Index":221,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3222,"Index":216,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549964,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":741697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3228,"Index":222,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3224,"Index":218,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3229,"Index":223,"Attempt":0,"Launch Time":1427397549965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3225,"Index":219,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":742622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3230,"Index":224,"Attempt":0,"Launch Time":1427397549967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3223,"Index":217,"Attempt":0,"Launch Time":1427397549956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549967,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1645111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3231,"Index":225,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3228,"Index":222,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3232,"Index":226,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3229,"Index":223,"Attempt":0,"Launch Time":1427397549965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322680,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3233,"Index":227,"Attempt":0,"Launch Time":1427397549973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3227,"Index":221,"Attempt":0,"Launch Time":1427397549964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":587326,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3234,"Index":228,"Attempt":0,"Launch Time":1427397549974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3226,"Index":220,"Attempt":0,"Launch Time":1427397549957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":3312988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3235,"Index":229,"Attempt":0,"Launch Time":1427397549975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3230,"Index":224,"Attempt":0,"Launch Time":1427397549967,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316513,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3236,"Index":230,"Attempt":0,"Launch Time":1427397549977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3221,"Index":215,"Attempt":0,"Launch Time":1427397549955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":726902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3237,"Index":231,"Attempt":0,"Launch Time":1427397549979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3231,"Index":225,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3238,"Index":232,"Attempt":0,"Launch Time":1427397549981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3232,"Index":226,"Attempt":0,"Launch Time":1427397549972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":466523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3239,"Index":233,"Attempt":0,"Launch Time":1427397549983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3234,"Index":228,"Attempt":0,"Launch Time":1427397549974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":400249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3240,"Index":234,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3236,"Index":230,"Attempt":0,"Launch Time":1427397549977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3241,"Index":235,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3188,"Index":182,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397549985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":96,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3242,"Index":236,"Attempt":0,"Launch Time":1427397550004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3187,"Index":181,"Attempt":0,"Launch Time":1427397549856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3243,"Index":237,"Attempt":0,"Launch Time":1427397550015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3240,"Index":234,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":683417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3244,"Index":238,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3238,"Index":232,"Attempt":0,"Launch Time":1427397549981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":427300,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3245,"Index":239,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3246,"Index":240,"Attempt":0,"Launch Time":1427397550018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3242,"Index":236,"Attempt":0,"Launch Time":1427397550004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":474973,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3239,"Index":233,"Attempt":0,"Launch Time":1427397549983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1436403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3247,"Index":241,"Attempt":0,"Launch Time":1427397550021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3243,"Index":237,"Attempt":0,"Launch Time":1427397550015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3248,"Index":242,"Attempt":0,"Launch Time":1427397550023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3244,"Index":238,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3249,"Index":243,"Attempt":0,"Launch Time":1427397550025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3245,"Index":239,"Attempt":0,"Launch Time":1427397550017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3250,"Index":244,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3247,"Index":241,"Attempt":0,"Launch Time":1427397550021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3251,"Index":245,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3246,"Index":240,"Attempt":0,"Launch Time":1427397550018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3252,"Index":246,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3248,"Index":242,"Attempt":0,"Launch Time":1427397550023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3253,"Index":247,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3249,"Index":243,"Attempt":0,"Launch Time":1427397550025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3254,"Index":248,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3255,"Index":249,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3250,"Index":244,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318347,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3251,"Index":245,"Attempt":0,"Launch Time":1427397550029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550036,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":291954,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3256,"Index":250,"Attempt":0,"Launch Time":1427397550039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3252,"Index":246,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":404002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3257,"Index":251,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3258,"Index":252,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3254,"Index":248,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347800,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3253,"Index":247,"Attempt":0,"Launch Time":1427397550031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1004819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3259,"Index":253,"Attempt":0,"Launch Time":1427397550045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3255,"Index":249,"Attempt":0,"Launch Time":1427397550036,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":362739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3260,"Index":254,"Attempt":0,"Launch Time":1427397550048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3241,"Index":235,"Attempt":0,"Launch Time":1427397549985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":39,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3548090,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3261,"Index":255,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3258,"Index":252,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3262,"Index":256,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3259,"Index":253,"Attempt":0,"Launch Time":1427397550045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361049,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3263,"Index":257,"Attempt":0,"Launch Time":1427397550055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3256,"Index":250,"Attempt":0,"Launch Time":1427397550039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":497293,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3264,"Index":258,"Attempt":0,"Launch Time":1427397550056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3257,"Index":251,"Attempt":0,"Launch Time":1427397550044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1120524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3265,"Index":259,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3262,"Index":256,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3266,"Index":260,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3261,"Index":255,"Attempt":0,"Launch Time":1427397550053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315574,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3267,"Index":261,"Attempt":0,"Launch Time":1427397550062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3263,"Index":257,"Attempt":0,"Launch Time":1427397550055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":322093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3268,"Index":262,"Attempt":0,"Launch Time":1427397550066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3264,"Index":258,"Attempt":0,"Launch Time":1427397550056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550066,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317587,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3269,"Index":263,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3265,"Index":259,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3270,"Index":264,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3266,"Index":260,"Attempt":0,"Launch Time":1427397550061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3271,"Index":265,"Attempt":0,"Launch Time":1427397550069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3235,"Index":229,"Attempt":0,"Launch Time":1427397549975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":365642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3272,"Index":266,"Attempt":0,"Launch Time":1427397550070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3267,"Index":261,"Attempt":0,"Launch Time":1427397550062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336375,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3273,"Index":267,"Attempt":0,"Launch Time":1427397550073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3260,"Index":254,"Attempt":0,"Launch Time":1427397550048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350737,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3274,"Index":268,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3269,"Index":263,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336010,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3275,"Index":269,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3268,"Index":262,"Attempt":0,"Launch Time":1427397550066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3276,"Index":270,"Attempt":0,"Launch Time":1427397550076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3237,"Index":231,"Attempt":0,"Launch Time":1427397549979,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":97,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":574497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3277,"Index":271,"Attempt":0,"Launch Time":1427397550077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3272,"Index":266,"Attempt":0,"Launch Time":1427397550070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3278,"Index":272,"Attempt":0,"Launch Time":1427397550081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3273,"Index":267,"Attempt":0,"Launch Time":1427397550073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":384061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3279,"Index":273,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3274,"Index":268,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396011,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3280,"Index":274,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3270,"Index":264,"Attempt":0,"Launch Time":1427397550068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5326852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3281,"Index":275,"Attempt":0,"Launch Time":1427397550084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3275,"Index":269,"Attempt":0,"Launch Time":1427397550075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550084,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388499,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3282,"Index":276,"Attempt":0,"Launch Time":1427397550086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3277,"Index":271,"Attempt":0,"Launch Time":1427397550077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3283,"Index":277,"Attempt":0,"Launch Time":1427397550088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3271,"Index":265,"Attempt":0,"Launch Time":1427397550069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":333904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3284,"Index":278,"Attempt":0,"Launch Time":1427397550089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3278,"Index":272,"Attempt":0,"Launch Time":1427397550081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550089,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3285,"Index":279,"Attempt":0,"Launch Time":1427397550090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3279,"Index":273,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":329420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3286,"Index":280,"Attempt":0,"Launch Time":1427397550091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3280,"Index":274,"Attempt":0,"Launch Time":1427397550083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3287,"Index":281,"Attempt":0,"Launch Time":1427397550105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3281,"Index":275,"Attempt":0,"Launch Time":1427397550084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3288,"Index":282,"Attempt":0,"Launch Time":1427397550106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3233,"Index":227,"Attempt":0,"Launch Time":1427397549973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":131,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":101411330,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3289,"Index":283,"Attempt":0,"Launch Time":1427397550107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3276,"Index":270,"Attempt":0,"Launch Time":1427397550076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":633627,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3290,"Index":284,"Attempt":0,"Launch Time":1427397550109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3286,"Index":280,"Attempt":0,"Launch Time":1427397550091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":266678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3291,"Index":285,"Attempt":0,"Launch Time":1427397550112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3283,"Index":277,"Attempt":0,"Launch Time":1427397550088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":316164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3292,"Index":286,"Attempt":0,"Launch Time":1427397550113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3288,"Index":282,"Attempt":0,"Launch Time":1427397550106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3293,"Index":287,"Attempt":0,"Launch Time":1427397550116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3289,"Index":283,"Attempt":0,"Launch Time":1427397550107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3294,"Index":288,"Attempt":0,"Launch Time":1427397550121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3292,"Index":286,"Attempt":0,"Launch Time":1427397550113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3295,"Index":289,"Attempt":0,"Launch Time":1427397550122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3291,"Index":285,"Attempt":0,"Launch Time":1427397550112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3296,"Index":290,"Attempt":0,"Launch Time":1427397550124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3285,"Index":279,"Attempt":0,"Launch Time":1427397550090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3297,"Index":291,"Attempt":0,"Launch Time":1427397550127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3293,"Index":287,"Attempt":0,"Launch Time":1427397550116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3298,"Index":292,"Attempt":0,"Launch Time":1427397550130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3294,"Index":288,"Attempt":0,"Launch Time":1427397550121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1258984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3299,"Index":293,"Attempt":0,"Launch Time":1427397550132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3295,"Index":289,"Attempt":0,"Launch Time":1427397550122,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2267281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3300,"Index":294,"Attempt":0,"Launch Time":1427397550134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3297,"Index":291,"Attempt":0,"Launch Time":1427397550127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3301,"Index":295,"Attempt":0,"Launch Time":1427397550136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3296,"Index":290,"Attempt":0,"Launch Time":1427397550124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":294689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3302,"Index":296,"Attempt":0,"Launch Time":1427397550137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3298,"Index":292,"Attempt":0,"Launch Time":1427397550130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3303,"Index":297,"Attempt":0,"Launch Time":1427397550138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3284,"Index":278,"Attempt":0,"Launch Time":1427397550089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550138,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":48,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3304,"Index":298,"Attempt":0,"Launch Time":1427397550139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3299,"Index":293,"Attempt":0,"Launch Time":1427397550132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":321300,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3305,"Index":299,"Attempt":0,"Launch Time":1427397550141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3300,"Index":294,"Attempt":0,"Launch Time":1427397550134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3306,"Index":300,"Attempt":0,"Launch Time":1427397550144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3302,"Index":296,"Attempt":0,"Launch Time":1427397550137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318419,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3307,"Index":301,"Attempt":0,"Launch Time":1427397550145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3304,"Index":298,"Attempt":0,"Launch Time":1427397550139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3308,"Index":302,"Attempt":0,"Launch Time":1427397550147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3301,"Index":295,"Attempt":0,"Launch Time":1427397550136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325189,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3309,"Index":303,"Attempt":0,"Launch Time":1427397550149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3305,"Index":299,"Attempt":0,"Launch Time":1427397550141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550150,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":419631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3310,"Index":304,"Attempt":0,"Launch Time":1427397550151,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3306,"Index":300,"Attempt":0,"Launch Time":1427397550144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550151,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3311,"Index":305,"Attempt":0,"Launch Time":1427397550153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3307,"Index":301,"Attempt":0,"Launch Time":1427397550145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361918,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3312,"Index":306,"Attempt":0,"Launch Time":1427397550154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3308,"Index":302,"Attempt":0,"Launch Time":1427397550147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3313,"Index":307,"Attempt":0,"Launch Time":1427397550156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3309,"Index":303,"Attempt":0,"Launch Time":1427397550149,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274199,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3314,"Index":308,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3303,"Index":297,"Attempt":0,"Launch Time":1427397550138,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550158,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":716078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3315,"Index":309,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3310,"Index":304,"Attempt":0,"Launch Time":1427397550151,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3316,"Index":310,"Attempt":0,"Launch Time":1427397550160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3311,"Index":305,"Attempt":0,"Launch Time":1427397550153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550160,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3317,"Index":311,"Attempt":0,"Launch Time":1427397550162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3312,"Index":306,"Attempt":0,"Launch Time":1427397550154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3318,"Index":312,"Attempt":0,"Launch Time":1427397550167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3315,"Index":309,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1270519,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3319,"Index":313,"Attempt":0,"Launch Time":1427397550171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3313,"Index":307,"Attempt":0,"Launch Time":1427397550156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550171,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":402941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3320,"Index":314,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3317,"Index":311,"Attempt":0,"Launch Time":1427397550162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":354674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3321,"Index":315,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3316,"Index":310,"Attempt":0,"Launch Time":1427397550160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3322,"Index":316,"Attempt":0,"Launch Time":1427397550174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3318,"Index":312,"Attempt":0,"Launch Time":1427397550167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3323,"Index":317,"Attempt":0,"Launch Time":1427397550178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3319,"Index":313,"Attempt":0,"Launch Time":1427397550171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344289,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3324,"Index":318,"Attempt":0,"Launch Time":1427397550181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3322,"Index":316,"Attempt":0,"Launch Time":1427397550174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3325,"Index":319,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3320,"Index":314,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3326,"Index":320,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3321,"Index":315,"Attempt":0,"Launch Time":1427397550173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":428159,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3327,"Index":321,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3314,"Index":308,"Attempt":0,"Launch Time":1427397550158,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337624,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3328,"Index":322,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3323,"Index":317,"Attempt":0,"Launch Time":1427397550178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3329,"Index":323,"Attempt":0,"Launch Time":1427397550191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3324,"Index":318,"Attempt":0,"Launch Time":1427397550181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":402708,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3330,"Index":324,"Attempt":0,"Launch Time":1427397550192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3326,"Index":320,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":493205,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3331,"Index":325,"Attempt":0,"Launch Time":1427397550201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3325,"Index":319,"Attempt":0,"Launch Time":1427397550182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550201,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":471053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3332,"Index":326,"Attempt":0,"Launch Time":1427397550208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3331,"Index":325,"Attempt":0,"Launch Time":1427397550201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":326431,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3333,"Index":327,"Attempt":0,"Launch Time":1427397550216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3332,"Index":326,"Attempt":0,"Launch Time":1427397550208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3334,"Index":328,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3328,"Index":322,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326474,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3335,"Index":329,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3327,"Index":321,"Attempt":0,"Launch Time":1427397550185,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3336,"Index":330,"Attempt":0,"Launch Time":1427397550224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3337,"Index":331,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3338,"Index":332,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3329,"Index":323,"Attempt":0,"Launch Time":1427397550191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3333,"Index":327,"Attempt":0,"Launch Time":1427397550216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":374930,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3330,"Index":324,"Attempt":0,"Launch Time":1427397550192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3339,"Index":333,"Attempt":0,"Launch Time":1427397550228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3334,"Index":328,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3335,"Index":329,"Attempt":0,"Launch Time":1427397550220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348756,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3340,"Index":334,"Attempt":0,"Launch Time":1427397550229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3341,"Index":335,"Attempt":0,"Launch Time":1427397550232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3337,"Index":331,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":454506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3342,"Index":336,"Attempt":0,"Launch Time":1427397550233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3336,"Index":330,"Attempt":0,"Launch Time":1427397550224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":385922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3343,"Index":337,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3344,"Index":338,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3287,"Index":281,"Attempt":0,"Launch Time":1427397550105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3282,"Index":276,"Attempt":0,"Launch Time":1427397550086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":148,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":976102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3345,"Index":339,"Attempt":0,"Launch Time":1427397550245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3290,"Index":284,"Attempt":0,"Launch Time":1427397550109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298273,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3346,"Index":340,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3344,"Index":338,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":320858,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3347,"Index":341,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3343,"Index":337,"Attempt":0,"Launch Time":1427397550244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":333484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3348,"Index":342,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3346,"Index":340,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":331943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3349,"Index":343,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3347,"Index":341,"Attempt":0,"Launch Time":1427397550251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":356093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3350,"Index":344,"Attempt":0,"Launch Time":1427397550259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3345,"Index":339,"Attempt":0,"Launch Time":1427397550245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":573406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3351,"Index":345,"Attempt":0,"Launch Time":1427397550265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3349,"Index":343,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3352,"Index":346,"Attempt":0,"Launch Time":1427397550266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3348,"Index":342,"Attempt":0,"Launch Time":1427397550258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327827,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3353,"Index":347,"Attempt":0,"Launch Time":1427397550267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3350,"Index":344,"Attempt":0,"Launch Time":1427397550259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3354,"Index":348,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3340,"Index":334,"Attempt":0,"Launch Time":1427397550229,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":346703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3355,"Index":349,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3351,"Index":345,"Attempt":0,"Launch Time":1427397550265,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550273,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3356,"Index":350,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3352,"Index":346,"Attempt":0,"Launch Time":1427397550266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332368,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3357,"Index":351,"Attempt":0,"Launch Time":1427397550278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3353,"Index":347,"Attempt":0,"Launch Time":1427397550267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550278,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":591338,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3358,"Index":352,"Attempt":0,"Launch Time":1427397550282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3355,"Index":349,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":468108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3359,"Index":353,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3360,"Index":354,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3356,"Index":350,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":471297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3354,"Index":348,"Attempt":0,"Launch Time":1427397550273,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1061149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3361,"Index":355,"Attempt":0,"Launch Time":1427397550284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3342,"Index":336,"Attempt":0,"Launch Time":1427397550233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":422100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3362,"Index":356,"Attempt":0,"Launch Time":1427397550286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3357,"Index":351,"Attempt":0,"Launch Time":1427397550278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298004,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3363,"Index":357,"Attempt":0,"Launch Time":1427397550290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3360,"Index":354,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":321176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3364,"Index":358,"Attempt":0,"Launch Time":1427397550291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3358,"Index":352,"Attempt":0,"Launch Time":1427397550282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336461,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3365,"Index":359,"Attempt":0,"Launch Time":1427397550293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3361,"Index":355,"Attempt":0,"Launch Time":1427397550284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333781,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3366,"Index":360,"Attempt":0,"Launch Time":1427397550295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3362,"Index":356,"Attempt":0,"Launch Time":1427397550286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":467129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3367,"Index":361,"Attempt":0,"Launch Time":1427397550298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3359,"Index":353,"Attempt":0,"Launch Time":1427397550283,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550298,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3368,"Index":362,"Attempt":0,"Launch Time":1427397550300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3364,"Index":358,"Attempt":0,"Launch Time":1427397550291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":787625,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3369,"Index":363,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3365,"Index":359,"Attempt":0,"Launch Time":1427397550293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3370,"Index":364,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3363,"Index":357,"Attempt":0,"Launch Time":1427397550290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":386484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3371,"Index":365,"Attempt":0,"Launch Time":1427397550307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3366,"Index":360,"Attempt":0,"Launch Time":1427397550295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3372,"Index":366,"Attempt":0,"Launch Time":1427397550308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3368,"Index":362,"Attempt":0,"Launch Time":1427397550300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339205,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3373,"Index":367,"Attempt":0,"Launch Time":1427397550309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3370,"Index":364,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3374,"Index":368,"Attempt":0,"Launch Time":1427397550310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3369,"Index":363,"Attempt":0,"Launch Time":1427397550301,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3375,"Index":369,"Attempt":0,"Launch Time":1427397550312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3367,"Index":361,"Attempt":0,"Launch Time":1427397550298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3376,"Index":370,"Attempt":0,"Launch Time":1427397550320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3338,"Index":332,"Attempt":0,"Launch Time":1427397550225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":91,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":397788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3377,"Index":371,"Attempt":0,"Launch Time":1427397550323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3372,"Index":366,"Attempt":0,"Launch Time":1427397550308,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342054,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3378,"Index":372,"Attempt":0,"Launch Time":1427397550325,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3375,"Index":369,"Attempt":0,"Launch Time":1427397550312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550325,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3379,"Index":373,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3373,"Index":367,"Attempt":0,"Launch Time":1427397550309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287637,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3380,"Index":374,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3374,"Index":368,"Attempt":0,"Launch Time":1427397550310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3381,"Index":375,"Attempt":0,"Launch Time":1427397550330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3371,"Index":365,"Attempt":0,"Launch Time":1427397550307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550331,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":8857312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3382,"Index":376,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3377,"Index":371,"Attempt":0,"Launch Time":1427397550323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3383,"Index":377,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3379,"Index":373,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3384,"Index":378,"Attempt":0,"Launch Time":1427397550334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3380,"Index":374,"Attempt":0,"Launch Time":1427397550326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":385951,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3385,"Index":379,"Attempt":0,"Launch Time":1427397550336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3378,"Index":372,"Attempt":0,"Launch Time":1427397550325,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3386,"Index":380,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3339,"Index":333,"Attempt":0,"Launch Time":1427397550228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333469,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3387,"Index":381,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3381,"Index":375,"Attempt":0,"Launch Time":1427397550330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3388,"Index":382,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3389,"Index":383,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3390,"Index":384,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3382,"Index":376,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3383,"Index":377,"Attempt":0,"Launch Time":1427397550333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322022,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3384,"Index":378,"Attempt":0,"Launch Time":1427397550334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340512,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3391,"Index":385,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3386,"Index":380,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3392,"Index":386,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3385,"Index":379,"Attempt":0,"Launch Time":1427397550336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3393,"Index":387,"Attempt":0,"Launch Time":1427397550349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3388,"Index":382,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":392539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3394,"Index":388,"Attempt":0,"Launch Time":1427397550350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3390,"Index":384,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":389745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3395,"Index":389,"Attempt":0,"Launch Time":1427397550351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3389,"Index":383,"Attempt":0,"Launch Time":1427397550341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3396,"Index":390,"Attempt":0,"Launch Time":1427397550353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3391,"Index":385,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":291786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3397,"Index":391,"Attempt":0,"Launch Time":1427397550356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3393,"Index":387,"Attempt":0,"Launch Time":1427397550349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3398,"Index":392,"Attempt":0,"Launch Time":1427397550358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3395,"Index":389,"Attempt":0,"Launch Time":1427397550351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":292691,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3399,"Index":393,"Attempt":0,"Launch Time":1427397550361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3387,"Index":381,"Attempt":0,"Launch Time":1427397550338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275604,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3400,"Index":394,"Attempt":0,"Launch Time":1427397550368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3376,"Index":370,"Attempt":0,"Launch Time":1427397550320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":941138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3401,"Index":395,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3399,"Index":393,"Attempt":0,"Launch Time":1427397550361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3402,"Index":396,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3341,"Index":335,"Attempt":0,"Launch Time":1427397550232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":332089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3403,"Index":397,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3404,"Index":398,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3401,"Index":395,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3400,"Index":394,"Attempt":0,"Launch Time":1427397550368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":334696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3405,"Index":399,"Attempt":0,"Launch Time":1427397550379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3402,"Index":396,"Attempt":0,"Launch Time":1427397550369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3406,"Index":400,"Attempt":0,"Launch Time":1427397550383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3403,"Index":397,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550383,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":365980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3407,"Index":401,"Attempt":0,"Launch Time":1427397550384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3404,"Index":398,"Attempt":0,"Launch Time":1427397550376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3408,"Index":402,"Attempt":0,"Launch Time":1427397550388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3405,"Index":399,"Attempt":0,"Launch Time":1427397550379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306424,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3409,"Index":403,"Attempt":0,"Launch Time":1427397550390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3406,"Index":400,"Attempt":0,"Launch Time":1427397550383,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296309,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3410,"Index":404,"Attempt":0,"Launch Time":1427397550392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3407,"Index":401,"Attempt":0,"Launch Time":1427397550384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3411,"Index":405,"Attempt":0,"Launch Time":1427397550395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3408,"Index":402,"Attempt":0,"Launch Time":1427397550388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349027,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3412,"Index":406,"Attempt":0,"Launch Time":1427397550397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3409,"Index":403,"Attempt":0,"Launch Time":1427397550390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550397,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3413,"Index":407,"Attempt":0,"Launch Time":1427397550399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3410,"Index":404,"Attempt":0,"Launch Time":1427397550392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3414,"Index":408,"Attempt":0,"Launch Time":1427397550404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3412,"Index":406,"Attempt":0,"Launch Time":1427397550397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3415,"Index":409,"Attempt":0,"Launch Time":1427397550405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3411,"Index":405,"Attempt":0,"Launch Time":1427397550395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":360399,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3416,"Index":410,"Attempt":0,"Launch Time":1427397550406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3413,"Index":407,"Attempt":0,"Launch Time":1427397550399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3417,"Index":411,"Attempt":0,"Launch Time":1427397550411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3414,"Index":408,"Attempt":0,"Launch Time":1427397550404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3418,"Index":412,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3415,"Index":409,"Attempt":0,"Launch Time":1427397550405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3419,"Index":413,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3416,"Index":410,"Attempt":0,"Launch Time":1427397550406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3420,"Index":414,"Attempt":0,"Launch Time":1427397550425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3421,"Index":415,"Attempt":0,"Launch Time":1427397550426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3418,"Index":412,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3417,"Index":411,"Attempt":0,"Launch Time":1427397550411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3422,"Index":416,"Attempt":0,"Launch Time":1427397550428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3419,"Index":413,"Attempt":0,"Launch Time":1427397550413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550428,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3423,"Index":417,"Attempt":0,"Launch Time":1427397550430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3392,"Index":386,"Attempt":0,"Launch Time":1427397550346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":781620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3424,"Index":418,"Attempt":0,"Launch Time":1427397550434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3420,"Index":414,"Attempt":0,"Launch Time":1427397550425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":420273,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3425,"Index":419,"Attempt":0,"Launch Time":1427397550435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3421,"Index":415,"Attempt":0,"Launch Time":1427397550426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3426,"Index":420,"Attempt":0,"Launch Time":1427397550436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3422,"Index":416,"Attempt":0,"Launch Time":1427397550428,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":369531,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3427,"Index":421,"Attempt":0,"Launch Time":1427397550440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3423,"Index":417,"Attempt":0,"Launch Time":1427397550430,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":403698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3428,"Index":422,"Attempt":0,"Launch Time":1427397550442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3425,"Index":419,"Attempt":0,"Launch Time":1427397550435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3424,"Index":418,"Attempt":0,"Launch Time":1427397550434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313304,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3429,"Index":423,"Attempt":0,"Launch Time":1427397550443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3430,"Index":424,"Attempt":0,"Launch Time":1427397550445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3426,"Index":420,"Attempt":0,"Launch Time":1427397550436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":506662,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3431,"Index":425,"Attempt":0,"Launch Time":1427397550446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3427,"Index":421,"Attempt":0,"Launch Time":1427397550440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3432,"Index":426,"Attempt":0,"Launch Time":1427397550449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3428,"Index":422,"Attempt":0,"Launch Time":1427397550442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3433,"Index":427,"Attempt":0,"Launch Time":1427397550450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3429,"Index":423,"Attempt":0,"Launch Time":1427397550443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352772,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3434,"Index":428,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3430,"Index":424,"Attempt":0,"Launch Time":1427397550445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3435,"Index":429,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3431,"Index":425,"Attempt":0,"Launch Time":1427397550446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":259006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3436,"Index":430,"Attempt":0,"Launch Time":1427397550457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3433,"Index":427,"Attempt":0,"Launch Time":1427397550450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":314581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3437,"Index":431,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3434,"Index":428,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3438,"Index":432,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3432,"Index":426,"Attempt":0,"Launch Time":1427397550449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3439,"Index":433,"Attempt":0,"Launch Time":1427397550459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3435,"Index":429,"Attempt":0,"Launch Time":1427397550452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319441,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3440,"Index":434,"Attempt":0,"Launch Time":1427397550463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3436,"Index":430,"Attempt":0,"Launch Time":1427397550457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3441,"Index":435,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3438,"Index":432,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328956,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3442,"Index":436,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3437,"Index":431,"Attempt":0,"Launch Time":1427397550458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3443,"Index":437,"Attempt":0,"Launch Time":1427397550470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3439,"Index":433,"Attempt":0,"Launch Time":1427397550459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3444,"Index":438,"Attempt":0,"Launch Time":1427397550471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3440,"Index":434,"Attempt":0,"Launch Time":1427397550463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3445,"Index":439,"Attempt":0,"Launch Time":1427397550472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3398,"Index":392,"Attempt":0,"Launch Time":1427397550358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1675806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3446,"Index":440,"Attempt":0,"Launch Time":1427397550473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3441,"Index":435,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550473,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3447,"Index":441,"Attempt":0,"Launch Time":1427397550474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3442,"Index":436,"Attempt":0,"Launch Time":1427397550466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":430470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3448,"Index":442,"Attempt":0,"Launch Time":1427397550478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3443,"Index":437,"Attempt":0,"Launch Time":1427397550470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":406632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3449,"Index":443,"Attempt":0,"Launch Time":1427397550479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3444,"Index":438,"Attempt":0,"Launch Time":1427397550471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":407969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3450,"Index":444,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3451,"Index":445,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3396,"Index":390,"Attempt":0,"Launch Time":1427397550353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":84,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":443340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3446,"Index":440,"Attempt":0,"Launch Time":1427397550473,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":455765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3452,"Index":446,"Attempt":0,"Launch Time":1427397550483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3447,"Index":441,"Attempt":0,"Launch Time":1427397550474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":418697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3453,"Index":447,"Attempt":0,"Launch Time":1427397550488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3454,"Index":448,"Attempt":0,"Launch Time":1427397550489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3394,"Index":388,"Attempt":0,"Launch Time":1427397550350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":138,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":60231827,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3445,"Index":439,"Attempt":0,"Launch Time":1427397550472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":390154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3455,"Index":449,"Attempt":0,"Launch Time":1427397550490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3450,"Index":444,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550490,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3456,"Index":450,"Attempt":0,"Launch Time":1427397550495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3453,"Index":447,"Attempt":0,"Launch Time":1427397550488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3457,"Index":451,"Attempt":0,"Launch Time":1427397550496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3454,"Index":448,"Attempt":0,"Launch Time":1427397550489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314109,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3458,"Index":452,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3397,"Index":391,"Attempt":0,"Launch Time":1427397550356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":61,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3459,"Index":453,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3455,"Index":449,"Attempt":0,"Launch Time":1427397550490,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349915,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3460,"Index":454,"Attempt":0,"Launch Time":1427397550502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3456,"Index":450,"Attempt":0,"Launch Time":1427397550495,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":323477,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3461,"Index":455,"Attempt":0,"Launch Time":1427397550503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3459,"Index":453,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3462,"Index":456,"Attempt":0,"Launch Time":1427397550505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3457,"Index":451,"Attempt":0,"Launch Time":1427397550496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3463,"Index":457,"Attempt":0,"Launch Time":1427397550509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3460,"Index":454,"Attempt":0,"Launch Time":1427397550502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340127,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3464,"Index":458,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3461,"Index":455,"Attempt":0,"Launch Time":1427397550503,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3465,"Index":459,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3458,"Index":452,"Attempt":0,"Launch Time":1427397550497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3466,"Index":460,"Attempt":0,"Launch Time":1427397550513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3462,"Index":456,"Attempt":0,"Launch Time":1427397550505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326111,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3467,"Index":461,"Attempt":0,"Launch Time":1427397550525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3464,"Index":458,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3468,"Index":462,"Attempt":0,"Launch Time":1427397550526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3465,"Index":459,"Attempt":0,"Launch Time":1427397550511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362293,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3469,"Index":463,"Attempt":0,"Launch Time":1427397550528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3466,"Index":460,"Attempt":0,"Launch Time":1427397550513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550528,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3470,"Index":464,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3448,"Index":442,"Attempt":0,"Launch Time":1427397550478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":52,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3471,"Index":465,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3467,"Index":461,"Attempt":0,"Launch Time":1427397550525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550533,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":366910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3472,"Index":466,"Attempt":0,"Launch Time":1427397550535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3468,"Index":462,"Attempt":0,"Launch Time":1427397550526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344424,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3473,"Index":467,"Attempt":0,"Launch Time":1427397550536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3469,"Index":463,"Attempt":0,"Launch Time":1427397550528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":299905,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3474,"Index":468,"Attempt":0,"Launch Time":1427397550541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3471,"Index":465,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3475,"Index":469,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3470,"Index":464,"Attempt":0,"Launch Time":1427397550532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3476,"Index":470,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3472,"Index":466,"Attempt":0,"Launch Time":1427397550535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":561749,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3477,"Index":471,"Attempt":0,"Launch Time":1427397550545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3473,"Index":467,"Attempt":0,"Launch Time":1427397550536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409328,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3478,"Index":472,"Attempt":0,"Launch Time":1427397550550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3474,"Index":468,"Attempt":0,"Launch Time":1427397550541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306166,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3479,"Index":473,"Attempt":0,"Launch Time":1427397550552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3475,"Index":469,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304543,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3480,"Index":474,"Attempt":0,"Launch Time":1427397550554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3477,"Index":471,"Attempt":0,"Launch Time":1427397550545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":305615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3481,"Index":475,"Attempt":0,"Launch Time":1427397550557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3478,"Index":472,"Attempt":0,"Launch Time":1427397550550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":405334,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3482,"Index":476,"Attempt":0,"Launch Time":1427397550559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3479,"Index":473,"Attempt":0,"Launch Time":1427397550552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3483,"Index":477,"Attempt":0,"Launch Time":1427397550560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3476,"Index":470,"Attempt":0,"Launch Time":1427397550543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3484,"Index":478,"Attempt":0,"Launch Time":1427397550561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3480,"Index":474,"Attempt":0,"Launch Time":1427397550554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3485,"Index":479,"Attempt":0,"Launch Time":1427397550565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3481,"Index":475,"Attempt":0,"Launch Time":1427397550557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3486,"Index":480,"Attempt":0,"Launch Time":1427397550567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3482,"Index":476,"Attempt":0,"Launch Time":1427397550559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":395186,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3487,"Index":481,"Attempt":0,"Launch Time":1427397550569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3484,"Index":478,"Attempt":0,"Launch Time":1427397550561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3488,"Index":482,"Attempt":0,"Launch Time":1427397550571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3452,"Index":446,"Attempt":0,"Launch Time":1427397550483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550571,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347897,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3489,"Index":483,"Attempt":0,"Launch Time":1427397550574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3485,"Index":479,"Attempt":0,"Launch Time":1427397550565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363386,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3490,"Index":484,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3491,"Index":485,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3483,"Index":477,"Attempt":0,"Launch Time":1427397550560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1051371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3486,"Index":480,"Attempt":0,"Launch Time":1427397550567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":377867,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3492,"Index":486,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3487,"Index":481,"Attempt":0,"Launch Time":1427397550569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":444427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3493,"Index":487,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3463,"Index":457,"Attempt":0,"Launch Time":1427397550509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":68,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":425290,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3494,"Index":488,"Attempt":0,"Launch Time":1427397550581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3489,"Index":483,"Attempt":0,"Launch Time":1427397550574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3495,"Index":489,"Attempt":0,"Launch Time":1427397550584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3491,"Index":485,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":324874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3496,"Index":490,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3493,"Index":487,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333952,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3497,"Index":491,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3492,"Index":486,"Attempt":0,"Launch Time":1427397550578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3498,"Index":492,"Attempt":0,"Launch Time":1427397550587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3490,"Index":484,"Attempt":0,"Launch Time":1427397550576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3499,"Index":493,"Attempt":0,"Launch Time":1427397550588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3449,"Index":443,"Attempt":0,"Launch Time":1427397550479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3500,"Index":494,"Attempt":0,"Launch Time":1427397550590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3494,"Index":488,"Attempt":0,"Launch Time":1427397550581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3501,"Index":495,"Attempt":0,"Launch Time":1427397550592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3495,"Index":489,"Attempt":0,"Launch Time":1427397550584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3502,"Index":496,"Attempt":0,"Launch Time":1427397550593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3496,"Index":490,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3503,"Index":497,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3497,"Index":491,"Attempt":0,"Launch Time":1427397550586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336397,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3504,"Index":498,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3498,"Index":492,"Attempt":0,"Launch Time":1427397550587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312525,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3505,"Index":499,"Attempt":0,"Launch Time":1427397550597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3500,"Index":494,"Attempt":0,"Launch Time":1427397550590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":324745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3506,"Index":500,"Attempt":0,"Launch Time":1427397550600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3499,"Index":493,"Attempt":0,"Launch Time":1427397550588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3507,"Index":501,"Attempt":0,"Launch Time":1427397550601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3502,"Index":496,"Attempt":0,"Launch Time":1427397550593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":381192,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3508,"Index":502,"Attempt":0,"Launch Time":1427397550602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3503,"Index":497,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3509,"Index":503,"Attempt":0,"Launch Time":1427397550604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3505,"Index":499,"Attempt":0,"Launch Time":1427397550597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3510,"Index":504,"Attempt":0,"Launch Time":1427397550609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3501,"Index":495,"Attempt":0,"Launch Time":1427397550592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3511,"Index":505,"Attempt":0,"Launch Time":1427397550610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3506,"Index":500,"Attempt":0,"Launch Time":1427397550600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2619212,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3512,"Index":506,"Attempt":0,"Launch Time":1427397550624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3451,"Index":445,"Attempt":0,"Launch Time":1427397550481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":332404,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3513,"Index":507,"Attempt":0,"Launch Time":1427397550625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3511,"Index":505,"Attempt":0,"Launch Time":1427397550610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3514,"Index":508,"Attempt":0,"Launch Time":1427397550627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3488,"Index":482,"Attempt":0,"Launch Time":1427397550571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":421653,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3515,"Index":509,"Attempt":0,"Launch Time":1427397550628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3507,"Index":501,"Attempt":0,"Launch Time":1427397550601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":366611,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3516,"Index":510,"Attempt":0,"Launch Time":1427397550634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3512,"Index":506,"Attempt":0,"Launch Time":1427397550624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":393862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3517,"Index":511,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3518,"Index":512,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3513,"Index":507,"Attempt":0,"Launch Time":1427397550625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":430528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3514,"Index":508,"Attempt":0,"Launch Time":1427397550627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3519,"Index":513,"Attempt":0,"Launch Time":1427397550637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3508,"Index":502,"Attempt":0,"Launch Time":1427397550602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324040,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3520,"Index":514,"Attempt":0,"Launch Time":1427397550639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3515,"Index":509,"Attempt":0,"Launch Time":1427397550628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393007,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3521,"Index":515,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3517,"Index":511,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3522,"Index":516,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3518,"Index":512,"Attempt":0,"Launch Time":1427397550636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363549,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3523,"Index":517,"Attempt":0,"Launch Time":1427397550644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3516,"Index":510,"Attempt":0,"Launch Time":1427397550634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3524,"Index":518,"Attempt":0,"Launch Time":1427397550648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3520,"Index":514,"Attempt":0,"Launch Time":1427397550639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3525,"Index":519,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3523,"Index":517,"Attempt":0,"Launch Time":1427397550644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":340929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3526,"Index":520,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3521,"Index":515,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3527,"Index":521,"Attempt":0,"Launch Time":1427397550659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3522,"Index":516,"Attempt":0,"Launch Time":1427397550643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3528,"Index":522,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3525,"Index":519,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":371697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3529,"Index":523,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3526,"Index":520,"Attempt":0,"Launch Time":1427397550652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3530,"Index":524,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3529,"Index":523,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":400243,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3531,"Index":525,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3528,"Index":522,"Attempt":0,"Launch Time":1427397550660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":624123,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3532,"Index":526,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3524,"Index":518,"Attempt":0,"Launch Time":1427397550648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":669491,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3533,"Index":527,"Attempt":0,"Launch Time":1427397550671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3527,"Index":521,"Attempt":0,"Launch Time":1427397550659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550671,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330376,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3534,"Index":528,"Attempt":0,"Launch Time":1427397550673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3519,"Index":513,"Attempt":0,"Launch Time":1427397550637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3535,"Index":529,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3530,"Index":524,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3536,"Index":530,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3531,"Index":525,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3537,"Index":531,"Attempt":0,"Launch Time":1427397550678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3532,"Index":526,"Attempt":0,"Launch Time":1427397550670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550678,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3538,"Index":532,"Attempt":0,"Launch Time":1427397550682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3534,"Index":528,"Attempt":0,"Launch Time":1427397550673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315328,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3539,"Index":533,"Attempt":0,"Launch Time":1427397550683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3504,"Index":498,"Attempt":0,"Launch Time":1427397550594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331034,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3540,"Index":534,"Attempt":0,"Launch Time":1427397550685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3535,"Index":529,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3541,"Index":535,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3542,"Index":536,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3537,"Index":531,"Attempt":0,"Launch Time":1427397550678,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":364116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3536,"Index":530,"Attempt":0,"Launch Time":1427397550677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":729269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3543,"Index":537,"Attempt":0,"Launch Time":1427397550694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3540,"Index":534,"Attempt":0,"Launch Time":1427397550685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":368745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3544,"Index":538,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3541,"Index":535,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327048,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3545,"Index":539,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3542,"Index":536,"Attempt":0,"Launch Time":1427397550688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3546,"Index":540,"Attempt":0,"Launch Time":1427397550704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3547,"Index":541,"Attempt":0,"Launch Time":1427397550705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3544,"Index":538,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354029,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3545,"Index":539,"Attempt":0,"Launch Time":1427397550696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319906,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3548,"Index":542,"Attempt":0,"Launch Time":1427397550706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3538,"Index":532,"Attempt":0,"Launch Time":1427397550682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":723600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3549,"Index":543,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3550,"Index":544,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3551,"Index":545,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3510,"Index":504,"Attempt":0,"Launch Time":1427397550609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":328793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3533,"Index":527,"Attempt":0,"Launch Time":1427397550671,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":923303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3543,"Index":537,"Attempt":0,"Launch Time":1427397550694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334192,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3552,"Index":546,"Attempt":0,"Launch Time":1427397550712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3547,"Index":541,"Attempt":0,"Launch Time":1427397550705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3553,"Index":547,"Attempt":0,"Launch Time":1427397550714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3546,"Index":540,"Attempt":0,"Launch Time":1427397550704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":845136,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3554,"Index":548,"Attempt":0,"Launch Time":1427397550717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3551,"Index":545,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":356025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3555,"Index":549,"Attempt":0,"Launch Time":1427397550718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3549,"Index":543,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3556,"Index":550,"Attempt":0,"Launch Time":1427397550734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3552,"Index":546,"Attempt":0,"Launch Time":1427397550712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3557,"Index":551,"Attempt":0,"Launch Time":1427397550735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3548,"Index":542,"Attempt":0,"Launch Time":1427397550706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1946356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3558,"Index":552,"Attempt":0,"Launch Time":1427397550738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3555,"Index":549,"Attempt":0,"Launch Time":1427397550718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":341079,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3559,"Index":553,"Attempt":0,"Launch Time":1427397550739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3550,"Index":544,"Attempt":0,"Launch Time":1427397550709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3560,"Index":554,"Attempt":0,"Launch Time":1427397550747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3557,"Index":551,"Attempt":0,"Launch Time":1427397550735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3561,"Index":555,"Attempt":0,"Launch Time":1427397550748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3558,"Index":552,"Attempt":0,"Launch Time":1427397550738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3559,"Index":553,"Attempt":0,"Launch Time":1427397550739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3562,"Index":556,"Attempt":0,"Launch Time":1427397550749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3563,"Index":557,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3539,"Index":533,"Attempt":0,"Launch Time":1427397550683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342873,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3564,"Index":558,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3560,"Index":554,"Attempt":0,"Launch Time":1427397550747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":350622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3565,"Index":559,"Attempt":0,"Launch Time":1427397550755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3561,"Index":555,"Attempt":0,"Launch Time":1427397550748,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340525,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3566,"Index":560,"Attempt":0,"Launch Time":1427397550761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3509,"Index":503,"Attempt":0,"Launch Time":1427397550604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":143,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":357352,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3567,"Index":561,"Attempt":0,"Launch Time":1427397550762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3563,"Index":557,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362684,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3568,"Index":562,"Attempt":0,"Launch Time":1427397550763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3565,"Index":559,"Attempt":0,"Launch Time":1427397550755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":451346,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3569,"Index":563,"Attempt":0,"Launch Time":1427397550766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3562,"Index":556,"Attempt":0,"Launch Time":1427397550749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3570,"Index":564,"Attempt":0,"Launch Time":1427397550769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3567,"Index":561,"Attempt":0,"Launch Time":1427397550762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3571,"Index":565,"Attempt":0,"Launch Time":1427397550771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3568,"Index":562,"Attempt":0,"Launch Time":1427397550763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":396378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3572,"Index":566,"Attempt":0,"Launch Time":1427397550773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3564,"Index":558,"Attempt":0,"Launch Time":1427397550754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3573,"Index":567,"Attempt":0,"Launch Time":1427397550777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3570,"Index":564,"Attempt":0,"Launch Time":1427397550769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370212,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3574,"Index":568,"Attempt":0,"Launch Time":1427397550778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3571,"Index":565,"Attempt":0,"Launch Time":1427397550771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":324474,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3575,"Index":569,"Attempt":0,"Launch Time":1427397550782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3566,"Index":560,"Attempt":0,"Launch Time":1427397550761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":399018,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3576,"Index":570,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3577,"Index":571,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3574,"Index":568,"Attempt":0,"Launch Time":1427397550778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338337,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3572,"Index":566,"Attempt":0,"Launch Time":1427397550773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3578,"Index":572,"Attempt":0,"Launch Time":1427397550788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3573,"Index":567,"Attempt":0,"Launch Time":1427397550777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335726,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3579,"Index":573,"Attempt":0,"Launch Time":1427397550792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3569,"Index":563,"Attempt":0,"Launch Time":1427397550766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3580,"Index":574,"Attempt":0,"Launch Time":1427397550795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3581,"Index":575,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3577,"Index":571,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":455439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3576,"Index":570,"Attempt":0,"Launch Time":1427397550787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3582,"Index":576,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3575,"Index":569,"Attempt":0,"Launch Time":1427397550782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3583,"Index":577,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3578,"Index":572,"Attempt":0,"Launch Time":1427397550788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":322642,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3579,"Index":573,"Attempt":0,"Launch Time":1427397550792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":328898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3584,"Index":578,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3585,"Index":579,"Attempt":0,"Launch Time":1427397550803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3581,"Index":575,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":339956,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3586,"Index":580,"Attempt":0,"Launch Time":1427397550805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3580,"Index":574,"Attempt":0,"Launch Time":1427397550795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3587,"Index":581,"Attempt":0,"Launch Time":1427397550808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3582,"Index":576,"Attempt":0,"Launch Time":1427397550796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":308529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3588,"Index":582,"Attempt":0,"Launch Time":1427397550810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3585,"Index":579,"Attempt":0,"Launch Time":1427397550803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351263,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3589,"Index":583,"Attempt":0,"Launch Time":1427397550813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3584,"Index":578,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3590,"Index":584,"Attempt":0,"Launch Time":1427397550815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3586,"Index":580,"Attempt":0,"Launch Time":1427397550805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":406400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3591,"Index":585,"Attempt":0,"Launch Time":1427397550818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3588,"Index":582,"Attempt":0,"Launch Time":1427397550810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354655,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3592,"Index":586,"Attempt":0,"Launch Time":1427397550819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3583,"Index":577,"Attempt":0,"Launch Time":1427397550801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1105697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3593,"Index":587,"Attempt":0,"Launch Time":1427397550822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3589,"Index":583,"Attempt":0,"Launch Time":1427397550813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3594,"Index":588,"Attempt":0,"Launch Time":1427397550824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3587,"Index":581,"Attempt":0,"Launch Time":1427397550808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589127,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3595,"Index":589,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3590,"Index":584,"Attempt":0,"Launch Time":1427397550815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3596,"Index":590,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3591,"Index":585,"Attempt":0,"Launch Time":1427397550818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3597,"Index":591,"Attempt":0,"Launch Time":1427397550826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3592,"Index":586,"Attempt":0,"Launch Time":1427397550819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3598,"Index":592,"Attempt":0,"Launch Time":1427397550830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3593,"Index":587,"Attempt":0,"Launch Time":1427397550822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348872,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3599,"Index":593,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3595,"Index":589,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":395875,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3600,"Index":594,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3596,"Index":590,"Attempt":0,"Launch Time":1427397550825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":452620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3601,"Index":595,"Attempt":0,"Launch Time":1427397550842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3553,"Index":547,"Attempt":0,"Launch Time":1427397550714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":432694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3602,"Index":596,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3598,"Index":592,"Attempt":0,"Launch Time":1427397550830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3603,"Index":597,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3600,"Index":594,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3604,"Index":598,"Attempt":0,"Launch Time":1427397550847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3597,"Index":591,"Attempt":0,"Launch Time":1427397550826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3605,"Index":599,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3594,"Index":588,"Attempt":0,"Launch Time":1427397550824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":617334,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3606,"Index":600,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3602,"Index":596,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323574,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3607,"Index":601,"Attempt":0,"Launch Time":1427397550854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3603,"Index":597,"Attempt":0,"Launch Time":1427397550846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":310985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3608,"Index":602,"Attempt":0,"Launch Time":1427397550855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3604,"Index":598,"Attempt":0,"Launch Time":1427397550847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3609,"Index":603,"Attempt":0,"Launch Time":1427397550856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3599,"Index":593,"Attempt":0,"Launch Time":1427397550833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344369,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3610,"Index":604,"Attempt":0,"Launch Time":1427397550859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3554,"Index":548,"Attempt":0,"Launch Time":1427397550717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":140,"Result Size":930,"JVM GC Time":15,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":110325665,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3611,"Index":605,"Attempt":0,"Launch Time":1427397550861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3556,"Index":550,"Attempt":0,"Launch Time":1427397550734,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3612,"Index":606,"Attempt":0,"Launch Time":1427397550863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3601,"Index":595,"Attempt":0,"Launch Time":1427397550842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330264,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3613,"Index":607,"Attempt":0,"Launch Time":1427397550867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3610,"Index":604,"Attempt":0,"Launch Time":1427397550859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":392555,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3614,"Index":608,"Attempt":0,"Launch Time":1427397550869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3611,"Index":605,"Attempt":0,"Launch Time":1427397550861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3615,"Index":609,"Attempt":0,"Launch Time":1427397550871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3612,"Index":606,"Attempt":0,"Launch Time":1427397550863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3616,"Index":610,"Attempt":0,"Launch Time":1427397550877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3613,"Index":607,"Attempt":0,"Launch Time":1427397550867,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3617,"Index":611,"Attempt":0,"Launch Time":1427397550878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3607,"Index":601,"Attempt":0,"Launch Time":1427397550854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3618,"Index":612,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3614,"Index":608,"Attempt":0,"Launch Time":1427397550869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1915385,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3619,"Index":613,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3615,"Index":609,"Attempt":0,"Launch Time":1427397550871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":394987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3620,"Index":614,"Attempt":0,"Launch Time":1427397550885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3616,"Index":610,"Attempt":0,"Launch Time":1427397550877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3621,"Index":615,"Attempt":0,"Launch Time":1427397550888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3622,"Index":616,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3618,"Index":612,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3605,"Index":599,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378009,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3623,"Index":617,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3619,"Index":613,"Attempt":0,"Launch Time":1427397550879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550889,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":357994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3624,"Index":618,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3625,"Index":619,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3620,"Index":614,"Attempt":0,"Launch Time":1427397550885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294489,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3617,"Index":611,"Attempt":0,"Launch Time":1427397550878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3626,"Index":620,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3627,"Index":621,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3622,"Index":616,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3621,"Index":615,"Attempt":0,"Launch Time":1427397550888,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3628,"Index":622,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3623,"Index":617,"Attempt":0,"Launch Time":1427397550889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550897,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":279345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3629,"Index":623,"Attempt":0,"Launch Time":1427397550901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3625,"Index":619,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3630,"Index":624,"Attempt":0,"Launch Time":1427397550903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3624,"Index":618,"Attempt":0,"Launch Time":1427397550894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":417954,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3631,"Index":625,"Attempt":0,"Launch Time":1427397550906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3627,"Index":621,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3632,"Index":626,"Attempt":0,"Launch Time":1427397550909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3629,"Index":623,"Attempt":0,"Launch Time":1427397550901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550909,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3633,"Index":627,"Attempt":0,"Launch Time":1427397550910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3626,"Index":620,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":360712,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3634,"Index":628,"Attempt":0,"Launch Time":1427397550911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3630,"Index":624,"Attempt":0,"Launch Time":1427397550903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3635,"Index":629,"Attempt":0,"Launch Time":1427397550914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3631,"Index":625,"Attempt":0,"Launch Time":1427397550906,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3636,"Index":630,"Attempt":0,"Launch Time":1427397550917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3628,"Index":622,"Attempt":0,"Launch Time":1427397550897,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3637,"Index":631,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3632,"Index":626,"Attempt":0,"Launch Time":1427397550909,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":455218,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3638,"Index":632,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3608,"Index":602,"Attempt":0,"Launch Time":1427397550855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550918,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":381036,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3639,"Index":633,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3634,"Index":628,"Attempt":0,"Launch Time":1427397550911,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":434108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3640,"Index":634,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3633,"Index":627,"Attempt":0,"Launch Time":1427397550910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":523592,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3641,"Index":635,"Attempt":0,"Launch Time":1427397550923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3635,"Index":629,"Attempt":0,"Launch Time":1427397550914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327563,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3642,"Index":636,"Attempt":0,"Launch Time":1427397550925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3637,"Index":631,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":403142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3643,"Index":637,"Attempt":0,"Launch Time":1427397550927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3639,"Index":633,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3644,"Index":638,"Attempt":0,"Launch Time":1427397550928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3640,"Index":634,"Attempt":0,"Launch Time":1427397550920,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3645,"Index":639,"Attempt":0,"Launch Time":1427397550931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3641,"Index":635,"Attempt":0,"Launch Time":1427397550923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299097,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3646,"Index":640,"Attempt":0,"Launch Time":1427397550940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3643,"Index":637,"Attempt":0,"Launch Time":1427397550927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":433144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3647,"Index":641,"Attempt":0,"Launch Time":1427397550944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3642,"Index":636,"Attempt":0,"Launch Time":1427397550925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":11277309,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3648,"Index":642,"Attempt":0,"Launch Time":1427397550945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3645,"Index":639,"Attempt":0,"Launch Time":1427397550931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331458,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3649,"Index":643,"Attempt":0,"Launch Time":1427397550947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3646,"Index":640,"Attempt":0,"Launch Time":1427397550940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3650,"Index":644,"Attempt":0,"Launch Time":1427397550949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3644,"Index":638,"Attempt":0,"Launch Time":1427397550928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306280,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3651,"Index":645,"Attempt":0,"Launch Time":1427397550951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3647,"Index":641,"Attempt":0,"Launch Time":1427397550944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3652,"Index":646,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3636,"Index":630,"Attempt":0,"Launch Time":1427397550917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362593,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3653,"Index":647,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3606,"Index":600,"Attempt":0,"Launch Time":1427397550853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":101,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3654,"Index":648,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3649,"Index":643,"Attempt":0,"Launch Time":1427397550947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3655,"Index":649,"Attempt":0,"Launch Time":1427397550959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3651,"Index":645,"Attempt":0,"Launch Time":1427397550951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":361288,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3656,"Index":650,"Attempt":0,"Launch Time":1427397550960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3648,"Index":642,"Attempt":0,"Launch Time":1427397550945,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550960,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3657,"Index":651,"Attempt":0,"Launch Time":1427397550961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3650,"Index":644,"Attempt":0,"Launch Time":1427397550949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550961,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3658,"Index":652,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3653,"Index":647,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372464,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3659,"Index":653,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3652,"Index":646,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3660,"Index":654,"Attempt":0,"Launch Time":1427397550969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3654,"Index":648,"Attempt":0,"Launch Time":1427397550955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550969,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":395470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3661,"Index":655,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3656,"Index":650,"Attempt":0,"Launch Time":1427397550960,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3662,"Index":656,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3655,"Index":649,"Attempt":0,"Launch Time":1427397550959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317085,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3663,"Index":657,"Attempt":0,"Launch Time":1427397550971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3659,"Index":653,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3664,"Index":658,"Attempt":0,"Launch Time":1427397550975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3658,"Index":652,"Attempt":0,"Launch Time":1427397550963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":635443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3665,"Index":659,"Attempt":0,"Launch Time":1427397550977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3662,"Index":656,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327254,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3666,"Index":660,"Attempt":0,"Launch Time":1427397550978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3663,"Index":657,"Attempt":0,"Launch Time":1427397550971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550978,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3667,"Index":661,"Attempt":0,"Launch Time":1427397550980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3609,"Index":603,"Attempt":0,"Launch Time":1427397550856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327957,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3668,"Index":662,"Attempt":0,"Launch Time":1427397550982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3638,"Index":632,"Attempt":0,"Launch Time":1427397550918,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":58,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":841939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3669,"Index":663,"Attempt":0,"Launch Time":1427397550984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3665,"Index":659,"Attempt":0,"Launch Time":1427397550977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3670,"Index":664,"Attempt":0,"Launch Time":1427397550985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3660,"Index":654,"Attempt":0,"Launch Time":1427397550969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324106,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3671,"Index":665,"Attempt":0,"Launch Time":1427397550986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3667,"Index":661,"Attempt":0,"Launch Time":1427397550980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3672,"Index":666,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3666,"Index":660,"Attempt":0,"Launch Time":1427397550978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3673,"Index":667,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3668,"Index":662,"Attempt":0,"Launch Time":1427397550982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550990,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313189,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3674,"Index":668,"Attempt":0,"Launch Time":1427397550992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3669,"Index":663,"Attempt":0,"Launch Time":1427397550984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311584,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3675,"Index":669,"Attempt":0,"Launch Time":1427397550993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3671,"Index":665,"Attempt":0,"Launch Time":1427397550986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":285728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3676,"Index":670,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3673,"Index":667,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550996,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3677,"Index":671,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3672,"Index":666,"Attempt":0,"Launch Time":1427397550989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397550997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3678,"Index":672,"Attempt":0,"Launch Time":1427397550999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3675,"Index":669,"Attempt":0,"Launch Time":1427397550993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263372,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3679,"Index":673,"Attempt":0,"Launch Time":1427397551001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3674,"Index":668,"Attempt":0,"Launch Time":1427397550992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3680,"Index":674,"Attempt":0,"Launch Time":1427397551003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3676,"Index":670,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331373,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3681,"Index":675,"Attempt":0,"Launch Time":1427397551004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3677,"Index":671,"Attempt":0,"Launch Time":1427397550996,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358844,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3682,"Index":676,"Attempt":0,"Launch Time":1427397551006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3678,"Index":672,"Attempt":0,"Launch Time":1427397550999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3683,"Index":677,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3684,"Index":678,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3680,"Index":674,"Attempt":0,"Launch Time":1427397551003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":349227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3679,"Index":673,"Attempt":0,"Launch Time":1427397551001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393602,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3685,"Index":679,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3686,"Index":680,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3682,"Index":676,"Attempt":0,"Launch Time":1427397551006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":358627,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3681,"Index":675,"Attempt":0,"Launch Time":1427397551004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1407392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3687,"Index":681,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3688,"Index":682,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3664,"Index":658,"Attempt":0,"Launch Time":1427397550975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":328489,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3683,"Index":677,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3689,"Index":683,"Attempt":0,"Launch Time":1427397551021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3684,"Index":678,"Attempt":0,"Launch Time":1427397551010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306442,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3690,"Index":684,"Attempt":0,"Launch Time":1427397551027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3685,"Index":679,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319840,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3691,"Index":685,"Attempt":0,"Launch Time":1427397551028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3686,"Index":680,"Attempt":0,"Launch Time":1427397551013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350880,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3692,"Index":686,"Attempt":0,"Launch Time":1427397551029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3670,"Index":664,"Attempt":0,"Launch Time":1427397550985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":17,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308194,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3693,"Index":687,"Attempt":0,"Launch Time":1427397551032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3688,"Index":682,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286104,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3694,"Index":688,"Attempt":0,"Launch Time":1427397551034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3690,"Index":684,"Attempt":0,"Launch Time":1427397551027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3695,"Index":689,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3689,"Index":683,"Attempt":0,"Launch Time":1427397551021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3696,"Index":690,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3691,"Index":685,"Attempt":0,"Launch Time":1427397551028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":349609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3697,"Index":691,"Attempt":0,"Launch Time":1427397551038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3657,"Index":651,"Attempt":0,"Launch Time":1427397550961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551038,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":75,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":50093659,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3698,"Index":692,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3692,"Index":686,"Attempt":0,"Launch Time":1427397551029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":321257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3699,"Index":693,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3694,"Index":688,"Attempt":0,"Launch Time":1427397551034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333823,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3700,"Index":694,"Attempt":0,"Launch Time":1427397551042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3696,"Index":690,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":312412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3701,"Index":695,"Attempt":0,"Launch Time":1427397551043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3693,"Index":687,"Attempt":0,"Launch Time":1427397551032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551043,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305639,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3702,"Index":696,"Attempt":0,"Launch Time":1427397551045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3697,"Index":691,"Attempt":0,"Launch Time":1427397551038,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":335432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3703,"Index":697,"Attempt":0,"Launch Time":1427397551047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3695,"Index":689,"Attempt":0,"Launch Time":1427397551035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3704,"Index":698,"Attempt":0,"Launch Time":1427397551048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3698,"Index":692,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3705,"Index":699,"Attempt":0,"Launch Time":1427397551050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3699,"Index":693,"Attempt":0,"Launch Time":1427397551041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551051,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":328832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3706,"Index":700,"Attempt":0,"Launch Time":1427397551053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3702,"Index":696,"Attempt":0,"Launch Time":1427397551045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309215,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3707,"Index":701,"Attempt":0,"Launch Time":1427397551055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3704,"Index":698,"Attempt":0,"Launch Time":1427397551048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551055,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315621,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3708,"Index":702,"Attempt":0,"Launch Time":1427397551056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3703,"Index":697,"Attempt":0,"Launch Time":1427397551047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":308638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3709,"Index":703,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3710,"Index":704,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3700,"Index":694,"Attempt":0,"Launch Time":1427397551042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317341,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3705,"Index":699,"Attempt":0,"Launch Time":1427397551050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3711,"Index":705,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3706,"Index":700,"Attempt":0,"Launch Time":1427397551053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3712,"Index":706,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3707,"Index":701,"Attempt":0,"Launch Time":1427397551055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322147,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3713,"Index":707,"Attempt":0,"Launch Time":1427397551064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3714,"Index":708,"Attempt":0,"Launch Time":1427397551065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3710,"Index":704,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3709,"Index":703,"Attempt":0,"Launch Time":1427397551058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3715,"Index":709,"Attempt":0,"Launch Time":1427397551066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3701,"Index":695,"Attempt":0,"Launch Time":1427397551043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551066,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324557,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3716,"Index":710,"Attempt":0,"Launch Time":1427397551067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3711,"Index":705,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":324246,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3717,"Index":711,"Attempt":0,"Launch Time":1427397551068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3712,"Index":706,"Attempt":0,"Launch Time":1427397551061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301451,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3718,"Index":712,"Attempt":0,"Launch Time":1427397551071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3719,"Index":713,"Attempt":0,"Launch Time":1427397551072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3714,"Index":708,"Attempt":0,"Launch Time":1427397551065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331440,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3713,"Index":707,"Attempt":0,"Launch Time":1427397551064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342598,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3720,"Index":714,"Attempt":0,"Launch Time":1427397551073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3715,"Index":709,"Attempt":0,"Launch Time":1427397551066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3721,"Index":715,"Attempt":0,"Launch Time":1427397551074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3708,"Index":702,"Attempt":0,"Launch Time":1427397551056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3722,"Index":716,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3723,"Index":717,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3717,"Index":711,"Attempt":0,"Launch Time":1427397551068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3716,"Index":710,"Attempt":0,"Launch Time":1427397551067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3724,"Index":718,"Attempt":0,"Launch Time":1427397551078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3725,"Index":719,"Attempt":0,"Launch Time":1427397551079,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3719,"Index":713,"Attempt":0,"Launch Time":1427397551072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551079,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":349748,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3718,"Index":712,"Attempt":0,"Launch Time":1427397551071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551079,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3726,"Index":720,"Attempt":0,"Launch Time":1427397551080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3720,"Index":714,"Attempt":0,"Launch Time":1427397551073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551080,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3727,"Index":721,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3722,"Index":716,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3728,"Index":722,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3723,"Index":717,"Attempt":0,"Launch Time":1427397551075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3729,"Index":723,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3730,"Index":724,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3724,"Index":718,"Attempt":0,"Launch Time":1427397551078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3725,"Index":719,"Attempt":0,"Launch Time":1427397551079,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3731,"Index":725,"Attempt":0,"Launch Time":1427397551087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3726,"Index":720,"Attempt":0,"Launch Time":1427397551080,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3732,"Index":726,"Attempt":0,"Launch Time":1427397551088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3721,"Index":715,"Attempt":0,"Launch Time":1427397551074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551088,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":606916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3733,"Index":727,"Attempt":0,"Launch Time":1427397551089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3727,"Index":721,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551089,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3734,"Index":728,"Attempt":0,"Launch Time":1427397551090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3728,"Index":722,"Attempt":0,"Launch Time":1427397551082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3735,"Index":729,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3736,"Index":730,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3729,"Index":723,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3661,"Index":655,"Attempt":0,"Launch Time":1427397550970,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":120,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3737,"Index":731,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3730,"Index":724,"Attempt":0,"Launch Time":1427397551086,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":360087,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3738,"Index":732,"Attempt":0,"Launch Time":1427397551101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3732,"Index":726,"Attempt":0,"Launch Time":1427397551088,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3739,"Index":733,"Attempt":0,"Launch Time":1427397551103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3734,"Index":728,"Attempt":0,"Launch Time":1427397551090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3740,"Index":734,"Attempt":0,"Launch Time":1427397551105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3735,"Index":729,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3741,"Index":735,"Attempt":0,"Launch Time":1427397551106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3736,"Index":730,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3742,"Index":736,"Attempt":0,"Launch Time":1427397551108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3738,"Index":732,"Attempt":0,"Launch Time":1427397551101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319112,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3743,"Index":737,"Attempt":0,"Launch Time":1427397551112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3687,"Index":681,"Attempt":0,"Launch Time":1427397551018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":82,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3744,"Index":738,"Attempt":0,"Launch Time":1427397551119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3743,"Index":737,"Attempt":0,"Launch Time":1427397551112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551119,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295464,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3745,"Index":739,"Attempt":0,"Launch Time":1427397551126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3744,"Index":738,"Attempt":0,"Launch Time":1427397551119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3746,"Index":740,"Attempt":0,"Launch Time":1427397551128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3741,"Index":735,"Attempt":0,"Launch Time":1427397551106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3747,"Index":741,"Attempt":0,"Launch Time":1427397551131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3742,"Index":736,"Attempt":0,"Launch Time":1427397551108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3748,"Index":742,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3737,"Index":731,"Attempt":0,"Launch Time":1427397551099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346677,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3749,"Index":743,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3745,"Index":739,"Attempt":0,"Launch Time":1427397551126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":333078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3750,"Index":744,"Attempt":0,"Launch Time":1427397551136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3746,"Index":740,"Attempt":0,"Launch Time":1427397551128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3751,"Index":745,"Attempt":0,"Launch Time":1427397551139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3747,"Index":741,"Attempt":0,"Launch Time":1427397551131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3752,"Index":746,"Attempt":0,"Launch Time":1427397551143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3749,"Index":743,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":370422,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3753,"Index":747,"Attempt":0,"Launch Time":1427397551144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3750,"Index":744,"Attempt":0,"Launch Time":1427397551136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3754,"Index":748,"Attempt":0,"Launch Time":1427397551147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3751,"Index":745,"Attempt":0,"Launch Time":1427397551139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343442,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3755,"Index":749,"Attempt":0,"Launch Time":1427397551153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3753,"Index":747,"Attempt":0,"Launch Time":1427397551144,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342903,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3756,"Index":750,"Attempt":0,"Launch Time":1427397551155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3748,"Index":742,"Attempt":0,"Launch Time":1427397551135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345750,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3757,"Index":751,"Attempt":0,"Launch Time":1427397551156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3752,"Index":746,"Attempt":0,"Launch Time":1427397551143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":378697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3758,"Index":752,"Attempt":0,"Launch Time":1427397551161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3755,"Index":749,"Attempt":0,"Launch Time":1427397551153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358213,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3759,"Index":753,"Attempt":0,"Launch Time":1427397551162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3756,"Index":750,"Attempt":0,"Launch Time":1427397551155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3760,"Index":754,"Attempt":0,"Launch Time":1427397551164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3757,"Index":751,"Attempt":0,"Launch Time":1427397551156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":330076,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3761,"Index":755,"Attempt":0,"Launch Time":1427397551169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3754,"Index":748,"Attempt":0,"Launch Time":1427397551147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345264,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3762,"Index":756,"Attempt":0,"Launch Time":1427397551171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3760,"Index":754,"Attempt":0,"Launch Time":1427397551164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551171,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":393299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3763,"Index":757,"Attempt":0,"Launch Time":1427397551172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3758,"Index":752,"Attempt":0,"Launch Time":1427397551161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3764,"Index":758,"Attempt":0,"Launch Time":1427397551175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3731,"Index":725,"Attempt":0,"Launch Time":1427397551087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":87,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":474164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3765,"Index":759,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3762,"Index":756,"Attempt":0,"Launch Time":1427397551171,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":390028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3766,"Index":760,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3761,"Index":755,"Attempt":0,"Launch Time":1427397551169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551179,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":409115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3767,"Index":761,"Attempt":0,"Launch Time":1427397551180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3763,"Index":757,"Attempt":0,"Launch Time":1427397551172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3768,"Index":762,"Attempt":0,"Launch Time":1427397551182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3759,"Index":753,"Attempt":0,"Launch Time":1427397551162,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551182,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":539119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3769,"Index":763,"Attempt":0,"Launch Time":1427397551183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3764,"Index":758,"Attempt":0,"Launch Time":1427397551175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551183,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3770,"Index":764,"Attempt":0,"Launch Time":1427397551186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3766,"Index":760,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309213,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3771,"Index":765,"Attempt":0,"Launch Time":1427397551187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3767,"Index":761,"Attempt":0,"Launch Time":1427397551180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3772,"Index":766,"Attempt":0,"Launch Time":1427397551188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3765,"Index":759,"Attempt":0,"Launch Time":1427397551179,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3773,"Index":767,"Attempt":0,"Launch Time":1427397551189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3768,"Index":762,"Attempt":0,"Launch Time":1427397551182,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3774,"Index":768,"Attempt":0,"Launch Time":1427397551193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3770,"Index":764,"Attempt":0,"Launch Time":1427397551186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322991,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3775,"Index":769,"Attempt":0,"Launch Time":1427397551194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3771,"Index":765,"Attempt":0,"Launch Time":1427397551187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3776,"Index":770,"Attempt":0,"Launch Time":1427397551195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3769,"Index":763,"Attempt":0,"Launch Time":1427397551183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3777,"Index":771,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3773,"Index":767,"Attempt":0,"Launch Time":1427397551189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3778,"Index":772,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3772,"Index":766,"Attempt":0,"Launch Time":1427397551188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":325763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3779,"Index":773,"Attempt":0,"Launch Time":1427397551199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3740,"Index":734,"Attempt":0,"Launch Time":1427397551105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3780,"Index":774,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3775,"Index":769,"Attempt":0,"Launch Time":1427397551194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":442769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3781,"Index":775,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3774,"Index":768,"Attempt":0,"Launch Time":1427397551193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":422319,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3782,"Index":776,"Attempt":0,"Launch Time":1427397551212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3778,"Index":772,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340328,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3783,"Index":777,"Attempt":0,"Launch Time":1427397551213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3779,"Index":773,"Attempt":0,"Launch Time":1427397551199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551213,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":333833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3784,"Index":778,"Attempt":0,"Launch Time":1427397551215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3781,"Index":775,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329555,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3785,"Index":779,"Attempt":0,"Launch Time":1427397551216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3780,"Index":774,"Attempt":0,"Launch Time":1427397551208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3786,"Index":780,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3782,"Index":776,"Attempt":0,"Launch Time":1427397551212,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":334965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3787,"Index":781,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3783,"Index":777,"Attempt":0,"Launch Time":1427397551213,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3788,"Index":782,"Attempt":0,"Launch Time":1427397551222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3784,"Index":778,"Attempt":0,"Launch Time":1427397551215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3789,"Index":783,"Attempt":0,"Launch Time":1427397551225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3777,"Index":771,"Attempt":0,"Launch Time":1427397551196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338062,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3790,"Index":784,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3791,"Index":785,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3786,"Index":780,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3785,"Index":779,"Attempt":0,"Launch Time":1427397551216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551229,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362552,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3792,"Index":786,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3787,"Index":781,"Attempt":0,"Launch Time":1427397551220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3793,"Index":787,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3776,"Index":770,"Attempt":0,"Launch Time":1427397551195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551232,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":410900,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3794,"Index":788,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3733,"Index":727,"Attempt":0,"Launch Time":1427397551089,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":145,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":375105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3795,"Index":789,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3739,"Index":733,"Attempt":0,"Launch Time":1427397551103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":441092,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3796,"Index":790,"Attempt":0,"Launch Time":1427397551236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3791,"Index":785,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":384088,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3797,"Index":791,"Attempt":0,"Launch Time":1427397551241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3794,"Index":788,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3798,"Index":792,"Attempt":0,"Launch Time":1427397551243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3795,"Index":789,"Attempt":0,"Launch Time":1427397551234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3799,"Index":793,"Attempt":0,"Launch Time":1427397551244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3796,"Index":790,"Attempt":0,"Launch Time":1427397551236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3800,"Index":794,"Attempt":0,"Launch Time":1427397551248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3797,"Index":791,"Attempt":0,"Launch Time":1427397551241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":348194,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3801,"Index":795,"Attempt":0,"Launch Time":1427397551252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3799,"Index":793,"Attempt":0,"Launch Time":1427397551244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3802,"Index":796,"Attempt":0,"Launch Time":1427397551255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3800,"Index":794,"Attempt":0,"Launch Time":1427397551248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324721,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3803,"Index":797,"Attempt":0,"Launch Time":1427397551258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3792,"Index":786,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2142651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3804,"Index":798,"Attempt":0,"Launch Time":1427397551260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3801,"Index":795,"Attempt":0,"Launch Time":1427397551252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551260,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3805,"Index":799,"Attempt":0,"Launch Time":1427397551263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3802,"Index":796,"Attempt":0,"Launch Time":1427397551255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3806,"Index":800,"Attempt":0,"Launch Time":1427397551267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3804,"Index":798,"Attempt":0,"Launch Time":1427397551260,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346243,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3807,"Index":801,"Attempt":0,"Launch Time":1427397551271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3805,"Index":799,"Attempt":0,"Launch Time":1427397551263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":406899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3808,"Index":802,"Attempt":0,"Launch Time":1427397551275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3806,"Index":800,"Attempt":0,"Launch Time":1427397551267,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":383383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3809,"Index":803,"Attempt":0,"Launch Time":1427397551278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3803,"Index":797,"Attempt":0,"Launch Time":1427397551258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398112,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3810,"Index":804,"Attempt":0,"Launch Time":1427397551282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3807,"Index":801,"Attempt":0,"Launch Time":1427397551271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":788941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3811,"Index":805,"Attempt":0,"Launch Time":1427397551284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3808,"Index":802,"Attempt":0,"Launch Time":1427397551275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3812,"Index":806,"Attempt":0,"Launch Time":1427397551286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3809,"Index":803,"Attempt":0,"Launch Time":1427397551278,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3813,"Index":807,"Attempt":0,"Launch Time":1427397551288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3789,"Index":783,"Attempt":0,"Launch Time":1427397551225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1963714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3814,"Index":808,"Attempt":0,"Launch Time":1427397551291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3810,"Index":804,"Attempt":0,"Launch Time":1427397551282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551291,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3815,"Index":809,"Attempt":0,"Launch Time":1427397551295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3812,"Index":806,"Attempt":0,"Launch Time":1427397551286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1302904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3816,"Index":810,"Attempt":0,"Launch Time":1427397551298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3793,"Index":787,"Attempt":0,"Launch Time":1427397551232,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1693262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3817,"Index":811,"Attempt":0,"Launch Time":1427397551300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3814,"Index":808,"Attempt":0,"Launch Time":1427397551291,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338570,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3818,"Index":812,"Attempt":0,"Launch Time":1427397551304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3813,"Index":807,"Attempt":0,"Launch Time":1427397551288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":918032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3819,"Index":813,"Attempt":0,"Launch Time":1427397551306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3815,"Index":809,"Attempt":0,"Launch Time":1427397551295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":535287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3820,"Index":814,"Attempt":0,"Launch Time":1427397551307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3817,"Index":811,"Attempt":0,"Launch Time":1427397551300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3821,"Index":815,"Attempt":0,"Launch Time":1427397551311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3818,"Index":812,"Attempt":0,"Launch Time":1427397551304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3822,"Index":816,"Attempt":0,"Launch Time":1427397551312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3819,"Index":813,"Attempt":0,"Launch Time":1427397551306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551312,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":290276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3823,"Index":817,"Attempt":0,"Launch Time":1427397551316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3820,"Index":814,"Attempt":0,"Launch Time":1427397551307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282563,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3824,"Index":818,"Attempt":0,"Launch Time":1427397551318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3821,"Index":815,"Attempt":0,"Launch Time":1427397551311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3825,"Index":819,"Attempt":0,"Launch Time":1427397551319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3822,"Index":816,"Attempt":0,"Launch Time":1427397551312,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3826,"Index":820,"Attempt":0,"Launch Time":1427397551322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3823,"Index":817,"Attempt":0,"Launch Time":1427397551316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3827,"Index":821,"Attempt":0,"Launch Time":1427397551333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3788,"Index":782,"Attempt":0,"Launch Time":1427397551222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3828,"Index":822,"Attempt":0,"Launch Time":1427397551335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3816,"Index":810,"Attempt":0,"Launch Time":1427397551298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3829,"Index":823,"Attempt":0,"Launch Time":1427397551336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3824,"Index":818,"Attempt":0,"Launch Time":1427397551318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3830,"Index":824,"Attempt":0,"Launch Time":1427397551337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3825,"Index":819,"Attempt":0,"Launch Time":1427397551319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":758265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3831,"Index":825,"Attempt":0,"Launch Time":1427397551344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3790,"Index":784,"Attempt":0,"Launch Time":1427397551227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":115,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3832,"Index":826,"Attempt":0,"Launch Time":1427397551345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3829,"Index":823,"Attempt":0,"Launch Time":1427397551336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551345,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3833,"Index":827,"Attempt":0,"Launch Time":1427397551346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3826,"Index":820,"Attempt":0,"Launch Time":1427397551322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334041,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3834,"Index":828,"Attempt":0,"Launch Time":1427397551350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3811,"Index":805,"Attempt":0,"Launch Time":1427397551284,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3835,"Index":829,"Attempt":0,"Launch Time":1427397551352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3798,"Index":792,"Attempt":0,"Launch Time":1427397551243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":107,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3836,"Index":830,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3832,"Index":826,"Attempt":0,"Launch Time":1427397551345,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3837,"Index":831,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3833,"Index":827,"Attempt":0,"Launch Time":1427397551346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3838,"Index":832,"Attempt":0,"Launch Time":1427397551358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3834,"Index":828,"Attempt":0,"Launch Time":1427397551350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":411820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3839,"Index":833,"Attempt":0,"Launch Time":1427397551359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3835,"Index":829,"Attempt":0,"Launch Time":1427397551352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3840,"Index":834,"Attempt":0,"Launch Time":1427397551360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3837,"Index":831,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3841,"Index":835,"Attempt":0,"Launch Time":1427397551361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3836,"Index":830,"Attempt":0,"Launch Time":1427397551353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3842,"Index":836,"Attempt":0,"Launch Time":1427397551363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3828,"Index":822,"Attempt":0,"Launch Time":1427397551335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":319257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3843,"Index":837,"Attempt":0,"Launch Time":1427397551369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3841,"Index":835,"Attempt":0,"Launch Time":1427397551361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328016,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3844,"Index":838,"Attempt":0,"Launch Time":1427397551370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3842,"Index":836,"Attempt":0,"Launch Time":1427397551363,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":412650,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3845,"Index":839,"Attempt":0,"Launch Time":1427397551376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3831,"Index":825,"Attempt":0,"Launch Time":1427397551344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3846,"Index":840,"Attempt":0,"Launch Time":1427397551377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3843,"Index":837,"Attempt":0,"Launch Time":1427397551369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":327924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3847,"Index":841,"Attempt":0,"Launch Time":1427397551379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3827,"Index":821,"Attempt":0,"Launch Time":1427397551333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":439456,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3848,"Index":842,"Attempt":0,"Launch Time":1427397551380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3844,"Index":838,"Attempt":0,"Launch Time":1427397551370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360337,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3849,"Index":843,"Attempt":0,"Launch Time":1427397551384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3845,"Index":839,"Attempt":0,"Launch Time":1427397551376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352309,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3850,"Index":844,"Attempt":0,"Launch Time":1427397551387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3846,"Index":840,"Attempt":0,"Launch Time":1427397551377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3851,"Index":845,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3847,"Index":841,"Attempt":0,"Launch Time":1427397551379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":336248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3852,"Index":846,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3848,"Index":842,"Attempt":0,"Launch Time":1427397551380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":547859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3853,"Index":847,"Attempt":0,"Launch Time":1427397551394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3849,"Index":843,"Attempt":0,"Launch Time":1427397551384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3854,"Index":848,"Attempt":0,"Launch Time":1427397551396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3851,"Index":845,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551396,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3855,"Index":849,"Attempt":0,"Launch Time":1427397551398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3850,"Index":844,"Attempt":0,"Launch Time":1427397551387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551398,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":575956,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3856,"Index":850,"Attempt":0,"Launch Time":1427397551399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3838,"Index":832,"Attempt":0,"Launch Time":1427397551358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340774,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3857,"Index":851,"Attempt":0,"Launch Time":1427397551401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3853,"Index":847,"Attempt":0,"Launch Time":1427397551394,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551401,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3858,"Index":852,"Attempt":0,"Launch Time":1427397551404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3854,"Index":848,"Attempt":0,"Launch Time":1427397551396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3859,"Index":853,"Attempt":0,"Launch Time":1427397551406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3855,"Index":849,"Attempt":0,"Launch Time":1427397551398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551406,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360590,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3860,"Index":854,"Attempt":0,"Launch Time":1427397551407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3856,"Index":850,"Attempt":0,"Launch Time":1427397551399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330366,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3861,"Index":855,"Attempt":0,"Launch Time":1427397551408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3857,"Index":851,"Attempt":0,"Launch Time":1427397551401,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551408,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3862,"Index":856,"Attempt":0,"Launch Time":1427397551410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3852,"Index":846,"Attempt":0,"Launch Time":1427397551388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3863,"Index":857,"Attempt":0,"Launch Time":1427397551411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3858,"Index":852,"Attempt":0,"Launch Time":1427397551404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3864,"Index":858,"Attempt":0,"Launch Time":1427397551413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3860,"Index":854,"Attempt":0,"Launch Time":1427397551407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551414,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3865,"Index":859,"Attempt":0,"Launch Time":1427397551415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3861,"Index":855,"Attempt":0,"Launch Time":1427397551408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":312298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3866,"Index":860,"Attempt":0,"Launch Time":1427397551418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3863,"Index":857,"Attempt":0,"Launch Time":1427397551411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3867,"Index":861,"Attempt":0,"Launch Time":1427397551421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3859,"Index":853,"Attempt":0,"Launch Time":1427397551406,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":366161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3868,"Index":862,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3869,"Index":863,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3862,"Index":856,"Attempt":0,"Launch Time":1427397551410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":595053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3865,"Index":859,"Attempt":0,"Launch Time":1427397551415,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551424,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3870,"Index":864,"Attempt":0,"Launch Time":1427397551424,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3864,"Index":858,"Attempt":0,"Launch Time":1427397551413,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":388546,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3871,"Index":865,"Attempt":0,"Launch Time":1427397551425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3866,"Index":860,"Attempt":0,"Launch Time":1427397551418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551425,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3872,"Index":866,"Attempt":0,"Launch Time":1427397551437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3868,"Index":862,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":418006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3873,"Index":867,"Attempt":0,"Launch Time":1427397551439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3867,"Index":861,"Attempt":0,"Launch Time":1427397551421,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366325,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3874,"Index":868,"Attempt":0,"Launch Time":1427397551440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3871,"Index":865,"Attempt":0,"Launch Time":1427397551425,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349582,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3875,"Index":869,"Attempt":0,"Launch Time":1427397551443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3870,"Index":864,"Attempt":0,"Launch Time":1427397551424,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":6,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3876,"Index":870,"Attempt":0,"Launch Time":1427397551445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3872,"Index":866,"Attempt":0,"Launch Time":1427397551437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3877,"Index":871,"Attempt":0,"Launch Time":1427397551446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3873,"Index":867,"Attempt":0,"Launch Time":1427397551439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346653,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3878,"Index":872,"Attempt":0,"Launch Time":1427397551447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3874,"Index":868,"Attempt":0,"Launch Time":1427397551440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3879,"Index":873,"Attempt":0,"Launch Time":1427397551450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3875,"Index":869,"Attempt":0,"Launch Time":1427397551443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551450,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":327866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3880,"Index":874,"Attempt":0,"Launch Time":1427397551452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3876,"Index":870,"Attempt":0,"Launch Time":1427397551445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3881,"Index":875,"Attempt":0,"Launch Time":1427397551454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3877,"Index":871,"Attempt":0,"Launch Time":1427397551446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311018,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3882,"Index":876,"Attempt":0,"Launch Time":1427397551456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3878,"Index":872,"Attempt":0,"Launch Time":1427397551447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":339783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3883,"Index":877,"Attempt":0,"Launch Time":1427397551460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3880,"Index":874,"Attempt":0,"Launch Time":1427397551452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3884,"Index":878,"Attempt":0,"Launch Time":1427397551463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3881,"Index":875,"Attempt":0,"Launch Time":1427397551454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3885,"Index":879,"Attempt":0,"Launch Time":1427397551464,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3882,"Index":876,"Attempt":0,"Launch Time":1427397551456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551464,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3886,"Index":880,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3879,"Index":873,"Attempt":0,"Launch Time":1427397551450,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1914941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3887,"Index":881,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3883,"Index":877,"Attempt":0,"Launch Time":1427397551460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342630,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3888,"Index":882,"Attempt":0,"Launch Time":1427397551471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3884,"Index":878,"Attempt":0,"Launch Time":1427397551463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319636,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3889,"Index":883,"Attempt":0,"Launch Time":1427397551472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3885,"Index":879,"Attempt":0,"Launch Time":1427397551464,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3890,"Index":884,"Attempt":0,"Launch Time":1427397551479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3891,"Index":885,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3887,"Index":881,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":395064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3869,"Index":863,"Attempt":0,"Launch Time":1427397551423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":381395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3892,"Index":886,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3888,"Index":882,"Attempt":0,"Launch Time":1427397551471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":421383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3893,"Index":887,"Attempt":0,"Launch Time":1427397551486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3830,"Index":824,"Attempt":0,"Launch Time":1427397551337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":148,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":450141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3894,"Index":888,"Attempt":0,"Launch Time":1427397551487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3839,"Index":833,"Attempt":0,"Launch Time":1427397551359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306500,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3895,"Index":889,"Attempt":0,"Launch Time":1427397551492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3893,"Index":887,"Attempt":0,"Launch Time":1427397551486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284206,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3896,"Index":890,"Attempt":0,"Launch Time":1427397551494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3894,"Index":888,"Attempt":0,"Launch Time":1427397551487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315872,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3897,"Index":891,"Attempt":0,"Launch Time":1427397551497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3892,"Index":886,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3898,"Index":892,"Attempt":0,"Launch Time":1427397551499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3895,"Index":889,"Attempt":0,"Launch Time":1427397551492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293580,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3899,"Index":893,"Attempt":0,"Launch Time":1427397551502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3896,"Index":890,"Attempt":0,"Launch Time":1427397551494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355629,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3900,"Index":894,"Attempt":0,"Launch Time":1427397551506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3898,"Index":892,"Attempt":0,"Launch Time":1427397551499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":379362,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3901,"Index":895,"Attempt":0,"Launch Time":1427397551510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3899,"Index":893,"Attempt":0,"Launch Time":1427397551502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3902,"Index":896,"Attempt":0,"Launch Time":1427397551513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3897,"Index":891,"Attempt":0,"Launch Time":1427397551497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551513,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334387,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3903,"Index":897,"Attempt":0,"Launch Time":1427397551514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3900,"Index":894,"Attempt":0,"Launch Time":1427397551506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337590,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3904,"Index":898,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3840,"Index":834,"Attempt":0,"Launch Time":1427397551360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3905,"Index":899,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3901,"Index":895,"Attempt":0,"Launch Time":1427397551510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":390905,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3906,"Index":900,"Attempt":0,"Launch Time":1427397551521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3903,"Index":897,"Attempt":0,"Launch Time":1427397551514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3907,"Index":901,"Attempt":0,"Launch Time":1427397551522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3902,"Index":896,"Attempt":0,"Launch Time":1427397551513,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3908,"Index":902,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3904,"Index":898,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3909,"Index":903,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3905,"Index":899,"Attempt":0,"Launch Time":1427397551518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551526,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348629,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3910,"Index":904,"Attempt":0,"Launch Time":1427397551529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3906,"Index":900,"Attempt":0,"Launch Time":1427397551521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":411220,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3911,"Index":905,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3908,"Index":902,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3912,"Index":906,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3907,"Index":901,"Attempt":0,"Launch Time":1427397551522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551534,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2521253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3913,"Index":907,"Attempt":0,"Launch Time":1427397551535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3909,"Index":903,"Attempt":0,"Launch Time":1427397551526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":459258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3914,"Index":908,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3886,"Index":880,"Attempt":0,"Launch Time":1427397551469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4180501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3910,"Index":904,"Attempt":0,"Launch Time":1427397551529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355708,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3915,"Index":909,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3916,"Index":910,"Attempt":0,"Launch Time":1427397551547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3889,"Index":883,"Attempt":0,"Launch Time":1427397551472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551547,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":533167,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3917,"Index":911,"Attempt":0,"Launch Time":1427397551549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3912,"Index":906,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3918,"Index":912,"Attempt":0,"Launch Time":1427397551550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3913,"Index":907,"Attempt":0,"Launch Time":1427397551535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551550,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3919,"Index":913,"Attempt":0,"Launch Time":1427397551551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3911,"Index":905,"Attempt":0,"Launch Time":1427397551534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3920,"Index":914,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3917,"Index":911,"Attempt":0,"Launch Time":1427397551549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340754,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3921,"Index":915,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3918,"Index":912,"Attempt":0,"Launch Time":1427397551550,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":357416,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3922,"Index":916,"Attempt":0,"Launch Time":1427397551558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3919,"Index":913,"Attempt":0,"Launch Time":1427397551551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":343356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3923,"Index":917,"Attempt":0,"Launch Time":1427397551564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3920,"Index":914,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3924,"Index":918,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3921,"Index":915,"Attempt":0,"Launch Time":1427397551557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3925,"Index":919,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3922,"Index":916,"Attempt":0,"Launch Time":1427397551558,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3926,"Index":920,"Attempt":0,"Launch Time":1427397551568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3891,"Index":885,"Attempt":0,"Launch Time":1427397551480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":369699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3927,"Index":921,"Attempt":0,"Launch Time":1427397551572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3924,"Index":918,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3928,"Index":922,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3923,"Index":917,"Attempt":0,"Launch Time":1427397551564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":670768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3929,"Index":923,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3925,"Index":919,"Attempt":0,"Launch Time":1427397551565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3930,"Index":924,"Attempt":0,"Launch Time":1427397551575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3916,"Index":910,"Attempt":0,"Launch Time":1427397551547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325226,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3931,"Index":925,"Attempt":0,"Launch Time":1427397551579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3915,"Index":909,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":334823,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3932,"Index":926,"Attempt":0,"Launch Time":1427397551580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3927,"Index":921,"Attempt":0,"Launch Time":1427397551572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3933,"Index":927,"Attempt":0,"Launch Time":1427397551581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3928,"Index":922,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327491,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3934,"Index":928,"Attempt":0,"Launch Time":1427397551584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3929,"Index":923,"Attempt":0,"Launch Time":1427397551574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334077,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3935,"Index":929,"Attempt":0,"Launch Time":1427397551587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3932,"Index":926,"Attempt":0,"Launch Time":1427397551580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3936,"Index":930,"Attempt":0,"Launch Time":1427397551588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3933,"Index":927,"Attempt":0,"Launch Time":1427397551581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345525,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3937,"Index":931,"Attempt":0,"Launch Time":1427397551591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3934,"Index":928,"Attempt":0,"Launch Time":1427397551584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3938,"Index":932,"Attempt":0,"Launch Time":1427397551593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3914,"Index":908,"Attempt":0,"Launch Time":1427397551537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328123,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3939,"Index":933,"Attempt":0,"Launch Time":1427397551601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3935,"Index":929,"Attempt":0,"Launch Time":1427397551587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3940,"Index":934,"Attempt":0,"Launch Time":1427397551610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3938,"Index":932,"Attempt":0,"Launch Time":1427397551593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":356053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3941,"Index":935,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3942,"Index":936,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3943,"Index":937,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3937,"Index":931,"Attempt":0,"Launch Time":1427397551591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3944,"Index":938,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3926,"Index":920,"Attempt":0,"Launch Time":1427397551568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3890,"Index":884,"Attempt":0,"Launch Time":1427397551479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":65,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1488130,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3936,"Index":930,"Attempt":0,"Launch Time":1427397551588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3945,"Index":939,"Attempt":0,"Launch Time":1427397551620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3931,"Index":925,"Attempt":0,"Launch Time":1427397551579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353637,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3941,"Index":935,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3946,"Index":940,"Attempt":0,"Launch Time":1427397551623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3947,"Index":941,"Attempt":0,"Launch Time":1427397551626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3945,"Index":939,"Attempt":0,"Launch Time":1427397551620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3948,"Index":942,"Attempt":0,"Launch Time":1427397551627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3944,"Index":938,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3949,"Index":943,"Attempt":0,"Launch Time":1427397551628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3940,"Index":934,"Attempt":0,"Launch Time":1427397551610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298923,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3950,"Index":944,"Attempt":0,"Launch Time":1427397551629,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3930,"Index":924,"Attempt":0,"Launch Time":1427397551575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551629,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3951,"Index":945,"Attempt":0,"Launch Time":1427397551634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3947,"Index":941,"Attempt":0,"Launch Time":1427397551626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300067,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3952,"Index":946,"Attempt":0,"Launch Time":1427397551636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3946,"Index":940,"Attempt":0,"Launch Time":1427397551623,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326017,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3953,"Index":947,"Attempt":0,"Launch Time":1427397551638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3949,"Index":943,"Attempt":0,"Launch Time":1427397551628,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551638,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3954,"Index":948,"Attempt":0,"Launch Time":1427397551642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3951,"Index":945,"Attempt":0,"Launch Time":1427397551634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339198,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3955,"Index":949,"Attempt":0,"Launch Time":1427397551645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3950,"Index":944,"Attempt":0,"Launch Time":1427397551629,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3956,"Index":950,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3952,"Index":946,"Attempt":0,"Launch Time":1427397551636,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364780,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3957,"Index":951,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3954,"Index":948,"Attempt":0,"Launch Time":1427397551642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3958,"Index":952,"Attempt":0,"Launch Time":1427397551650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3953,"Index":947,"Attempt":0,"Launch Time":1427397551638,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551650,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":2462476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3959,"Index":953,"Attempt":0,"Launch Time":1427397551653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3948,"Index":942,"Attempt":0,"Launch Time":1427397551627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1047171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3960,"Index":954,"Attempt":0,"Launch Time":1427397551662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3958,"Index":952,"Attempt":0,"Launch Time":1427397551650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551662,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":359646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3961,"Index":955,"Attempt":0,"Launch Time":1427397551663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3957,"Index":951,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551663,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":385485,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3962,"Index":956,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3963,"Index":957,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3955,"Index":949,"Attempt":0,"Launch Time":1427397551645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":370681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3956,"Index":950,"Attempt":0,"Launch Time":1427397551649,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551665,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319858,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3964,"Index":958,"Attempt":0,"Launch Time":1427397551674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3939,"Index":933,"Attempt":0,"Launch Time":1427397551601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":504765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3965,"Index":959,"Attempt":0,"Launch Time":1427397551680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3961,"Index":955,"Attempt":0,"Launch Time":1427397551663,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3966,"Index":960,"Attempt":0,"Launch Time":1427397551682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3959,"Index":953,"Attempt":0,"Launch Time":1427397551653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3967,"Index":961,"Attempt":0,"Launch Time":1427397551689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3942,"Index":936,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1502164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3968,"Index":962,"Attempt":0,"Launch Time":1427397551691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3966,"Index":960,"Attempt":0,"Launch Time":1427397551682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3969,"Index":963,"Attempt":0,"Launch Time":1427397551696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3967,"Index":961,"Attempt":0,"Launch Time":1427397551689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":352255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3970,"Index":964,"Attempt":0,"Launch Time":1427397551697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3968,"Index":962,"Attempt":0,"Launch Time":1427397551691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326615,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3971,"Index":965,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3972,"Index":966,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3960,"Index":954,"Attempt":0,"Launch Time":1427397551662,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":535917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3962,"Index":956,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":361531,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3973,"Index":967,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3970,"Index":964,"Attempt":0,"Launch Time":1427397551697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":380742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3974,"Index":968,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3963,"Index":957,"Attempt":0,"Launch Time":1427397551665,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400756,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3975,"Index":969,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3969,"Index":963,"Attempt":0,"Launch Time":1427397551696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":418545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3976,"Index":970,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3964,"Index":958,"Attempt":0,"Launch Time":1427397551674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":830920,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3977,"Index":971,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3943,"Index":937,"Attempt":0,"Launch Time":1427397551611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":30,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3978,"Index":972,"Attempt":0,"Launch Time":1427397551713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3973,"Index":967,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378855,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3979,"Index":973,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3972,"Index":966,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":373817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3980,"Index":974,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3971,"Index":965,"Attempt":0,"Launch Time":1427397551704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":445079,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3981,"Index":975,"Attempt":0,"Launch Time":1427397551715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3965,"Index":959,"Attempt":0,"Launch Time":1427397551680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3982,"Index":976,"Attempt":0,"Launch Time":1427397551717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3974,"Index":968,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":541572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3983,"Index":977,"Attempt":0,"Launch Time":1427397551722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3978,"Index":972,"Attempt":0,"Launch Time":1427397551713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3984,"Index":978,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3980,"Index":974,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322703,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3985,"Index":979,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3986,"Index":980,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3981,"Index":975,"Attempt":0,"Launch Time":1427397551715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3979,"Index":973,"Attempt":0,"Launch Time":1427397551714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":378392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3987,"Index":981,"Attempt":0,"Launch Time":1427397551726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3976,"Index":970,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3346030,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3988,"Index":982,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3989,"Index":983,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3986,"Index":980,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3985,"Index":979,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551731,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":553017,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3990,"Index":984,"Attempt":0,"Launch Time":1427397551736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3987,"Index":981,"Attempt":0,"Launch Time":1427397551726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3991,"Index":985,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3988,"Index":982,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3992,"Index":986,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3989,"Index":983,"Attempt":0,"Launch Time":1427397551731,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":351861,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3993,"Index":987,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3990,"Index":984,"Attempt":0,"Launch Time":1427397551736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":290684,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3994,"Index":988,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3975,"Index":969,"Attempt":0,"Launch Time":1427397551705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316807,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3995,"Index":989,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3984,"Index":978,"Attempt":0,"Launch Time":1427397551723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":380037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3996,"Index":990,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3977,"Index":971,"Attempt":0,"Launch Time":1427397551709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":359340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3997,"Index":991,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3983,"Index":977,"Attempt":0,"Launch Time":1427397551722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362230,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3998,"Index":992,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3993,"Index":987,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3994,"Index":988,"Attempt":0,"Launch Time":1427397551744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":3999,"Index":993,"Attempt":0,"Launch Time":1427397551752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4000,"Index":994,"Attempt":0,"Launch Time":1427397551757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3995,"Index":989,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4001,"Index":995,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3998,"Index":992,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349489,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4002,"Index":996,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3999,"Index":993,"Attempt":0,"Launch Time":1427397551752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4003,"Index":997,"Attempt":0,"Launch Time":1427397551760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3996,"Index":990,"Attempt":0,"Launch Time":1427397551750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":359503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4004,"Index":998,"Attempt":0,"Launch Time":1427397551766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4000,"Index":994,"Attempt":0,"Launch Time":1427397551757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318496,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":4005,"Index":999,"Attempt":0,"Launch Time":1427397551768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4002,"Index":996,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":381403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4001,"Index":995,"Attempt":0,"Launch Time":1427397551759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4003,"Index":997,"Attempt":0,"Launch Time":1427397551760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":375553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3997,"Index":991,"Attempt":0,"Launch Time":1427397551751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":415810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3991,"Index":985,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3982,"Index":976,"Attempt":0,"Launch Time":1427397551717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":63,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4005,"Index":999,"Attempt":0,"Launch Time":1427397551768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3992,"Index":986,"Attempt":0,"Launch Time":1427397551738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":426007,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4004,"Index":998,"Attempt":0,"Launch Time":1427397551766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397549495,"Completion Time":1427397551784,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":4006,"Index":0,"Attempt":0,"Launch Time":1427397551788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4006,"Index":0,"Attempt":0,"Launch Time":1427397551788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397551931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":142,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192090,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line32.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line32.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line32.$read$$iwC$$iwC$$iwC.(:39)\n$line32.$read$$iwC$$iwC.(:41)\n$line32.$read$$iwC.(:43)\n$line32.$read.(:45)\n$line32.$read$.(:49)\n$line32.$read$.()\n$line32.$eval$.(:7)\n$line32.$eval$.()\n$line32.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397551788,"Completion Time":1427397551931,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1427397551931,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1427397553570,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[15,14]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":4007,"Index":0,"Attempt":0,"Launch Time":1427397553573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4007,"Index":0,"Attempt":0,"Launch Time":1427397553573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397553687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":112,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":1,"Remote Bytes Read":0,"Local Bytes Read":192017,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line34.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line34.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line34.$read$$iwC$$iwC$$iwC.(:39)\n$line34.$read$$iwC$$iwC.(:41)\n$line34.$read$$iwC.(:43)\n$line34.$read.(:45)\n$line34.$read$.(:49)\n$line34.$read$.()\n$line34.$eval$.(:7)\n$line34.$eval$.()\n$line34.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397553573,"Completion Time":1427397553687,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1427397553687,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":8,"Submission Time":1427397593688,"Stage Infos":[{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[16,17]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4008,"Index":0,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4009,"Index":1,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4010,"Index":2,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4011,"Index":3,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4012,"Index":4,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4013,"Index":5,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4014,"Index":6,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4015,"Index":7,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4016,"Index":8,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4017,"Index":9,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4012,"Index":4,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4018,"Index":10,"Attempt":0,"Launch Time":1427397593704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4019,"Index":11,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4020,"Index":12,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4008,"Index":0,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":239517,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4021,"Index":13,"Attempt":0,"Launch Time":1427397593709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4009,"Index":1,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4010,"Index":2,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":569524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4014,"Index":6,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260818,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4013,"Index":5,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4022,"Index":14,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4023,"Index":15,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4024,"Index":16,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4011,"Index":3,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":268311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4015,"Index":7,"Attempt":0,"Launch Time":1427397593695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267486,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4016,"Index":8,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593710,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":261740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4025,"Index":17,"Attempt":0,"Launch Time":1427397593712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4017,"Index":9,"Attempt":0,"Launch Time":1427397593702,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593712,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4026,"Index":18,"Attempt":0,"Launch Time":1427397593713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4020,"Index":12,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270373,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4027,"Index":19,"Attempt":0,"Launch Time":1427397593714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4018,"Index":10,"Attempt":0,"Launch Time":1427397593704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4028,"Index":20,"Attempt":0,"Launch Time":1427397593717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4019,"Index":11,"Attempt":0,"Launch Time":1427397593705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593717,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":896763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4029,"Index":21,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4030,"Index":22,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4024,"Index":16,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":259127,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4021,"Index":13,"Attempt":0,"Launch Time":1427397593709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":528453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4031,"Index":23,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4032,"Index":24,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4033,"Index":25,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4022,"Index":14,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593720,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":491624,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4023,"Index":15,"Attempt":0,"Launch Time":1427397593710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":587498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4034,"Index":26,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4035,"Index":27,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4025,"Index":17,"Attempt":0,"Launch Time":1427397593712,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":491089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4027,"Index":19,"Attempt":0,"Launch Time":1427397593714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4026,"Index":18,"Attempt":0,"Launch Time":1427397593713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310954,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4036,"Index":28,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4028,"Index":20,"Attempt":0,"Launch Time":1427397593717,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299714,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4037,"Index":29,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4038,"Index":30,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4030,"Index":22,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4029,"Index":21,"Attempt":0,"Launch Time":1427397593718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4039,"Index":31,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4040,"Index":32,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4041,"Index":33,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4036,"Index":28,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":268786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4037,"Index":29,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":262819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4042,"Index":34,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4043,"Index":35,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4038,"Index":30,"Attempt":0,"Launch Time":1427397593740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":240790,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4035,"Index":27,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290601,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4034,"Index":26,"Attempt":0,"Launch Time":1427397593722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593748,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":296886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4044,"Index":36,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4045,"Index":37,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4046,"Index":38,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4039,"Index":31,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":359948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4040,"Index":32,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271270,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4047,"Index":39,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4048,"Index":40,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4042,"Index":34,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":571799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4043,"Index":35,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4041,"Index":33,"Attempt":0,"Launch Time":1427397593747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356384,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4049,"Index":41,"Attempt":0,"Launch Time":1427397593760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4044,"Index":36,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4050,"Index":42,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4051,"Index":43,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4046,"Index":38,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264052,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4045,"Index":37,"Attempt":0,"Launch Time":1427397593754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":271026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4052,"Index":44,"Attempt":0,"Launch Time":1427397593762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4048,"Index":40,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4053,"Index":45,"Attempt":0,"Launch Time":1427397593763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4047,"Index":39,"Attempt":0,"Launch Time":1427397593755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286825,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4054,"Index":46,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4049,"Index":41,"Attempt":0,"Launch Time":1427397593760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4055,"Index":47,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4050,"Index":42,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4056,"Index":48,"Attempt":0,"Launch Time":1427397593768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4051,"Index":43,"Attempt":0,"Launch Time":1427397593761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307576,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4057,"Index":49,"Attempt":0,"Launch Time":1427397593769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4053,"Index":45,"Attempt":0,"Launch Time":1427397593763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":289841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4058,"Index":50,"Attempt":0,"Launch Time":1427397593775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4052,"Index":44,"Attempt":0,"Launch Time":1427397593762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4059,"Index":51,"Attempt":0,"Launch Time":1427397593776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4057,"Index":49,"Attempt":0,"Launch Time":1427397593769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":253573,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4060,"Index":52,"Attempt":0,"Launch Time":1427397593778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4056,"Index":48,"Attempt":0,"Launch Time":1427397593768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4061,"Index":53,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4058,"Index":50,"Attempt":0,"Launch Time":1427397593775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":303835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4062,"Index":54,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4059,"Index":51,"Attempt":0,"Launch Time":1427397593776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4063,"Index":55,"Attempt":0,"Launch Time":1427397593783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4054,"Index":46,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":311122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4064,"Index":56,"Attempt":0,"Launch Time":1427397593789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4061,"Index":53,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":330279,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4065,"Index":57,"Attempt":0,"Launch Time":1427397593791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4055,"Index":47,"Attempt":0,"Launch Time":1427397593767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593791,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":611625,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4066,"Index":58,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4062,"Index":54,"Attempt":0,"Launch Time":1427397593782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":641167,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4067,"Index":59,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4063,"Index":55,"Attempt":0,"Launch Time":1427397593783,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275692,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4068,"Index":60,"Attempt":0,"Launch Time":1427397593794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4060,"Index":52,"Attempt":0,"Launch Time":1427397593778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1067821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4069,"Index":61,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4064,"Index":56,"Attempt":0,"Launch Time":1427397593789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373084,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4070,"Index":62,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4065,"Index":57,"Attempt":0,"Launch Time":1427397593791,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4071,"Index":63,"Attempt":0,"Launch Time":1427397593799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4066,"Index":58,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593799,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":252889,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4072,"Index":64,"Attempt":0,"Launch Time":1427397593800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4067,"Index":59,"Attempt":0,"Launch Time":1427397593792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":272564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4073,"Index":65,"Attempt":0,"Launch Time":1427397593804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4068,"Index":60,"Attempt":0,"Launch Time":1427397593794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":495597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4074,"Index":66,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4071,"Index":63,"Attempt":0,"Launch Time":1427397593799,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4075,"Index":67,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4069,"Index":61,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":256885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4076,"Index":68,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4072,"Index":64,"Attempt":0,"Launch Time":1427397593800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4077,"Index":69,"Attempt":0,"Launch Time":1427397593807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4031,"Index":23,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4078,"Index":70,"Attempt":0,"Launch Time":1427397593808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4032,"Index":24,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":353467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4079,"Index":71,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4070,"Index":62,"Attempt":0,"Launch Time":1427397593798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4080,"Index":72,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4033,"Index":25,"Attempt":0,"Launch Time":1427397593719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":89,"Result Size":930,"JVM GC Time":13,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326875,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4081,"Index":73,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4075,"Index":67,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593820,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":266058,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4082,"Index":74,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4073,"Index":65,"Attempt":0,"Launch Time":1427397593804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1547586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4083,"Index":75,"Attempt":0,"Launch Time":1427397593821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4078,"Index":70,"Attempt":0,"Launch Time":1427397593808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288286,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4084,"Index":76,"Attempt":0,"Launch Time":1427397593826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4080,"Index":72,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4085,"Index":77,"Attempt":0,"Launch Time":1427397593829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4079,"Index":71,"Attempt":0,"Launch Time":1427397593809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":422406,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4086,"Index":78,"Attempt":0,"Launch Time":1427397593830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4083,"Index":75,"Attempt":0,"Launch Time":1427397593821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":400339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4087,"Index":79,"Attempt":0,"Launch Time":1427397593831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4074,"Index":66,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":362316,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4088,"Index":80,"Attempt":0,"Launch Time":1427397593832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4077,"Index":69,"Attempt":0,"Launch Time":1427397593807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":305782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4089,"Index":81,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4084,"Index":76,"Attempt":0,"Launch Time":1427397593826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":651502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4090,"Index":82,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4082,"Index":74,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593834,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313038,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4091,"Index":83,"Attempt":0,"Launch Time":1427397593835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4085,"Index":77,"Attempt":0,"Launch Time":1427397593829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4092,"Index":84,"Attempt":0,"Launch Time":1427397593837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4086,"Index":78,"Attempt":0,"Launch Time":1427397593830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":311004,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4093,"Index":85,"Attempt":0,"Launch Time":1427397593839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4088,"Index":80,"Attempt":0,"Launch Time":1427397593832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4094,"Index":86,"Attempt":0,"Launch Time":1427397593841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4090,"Index":82,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4095,"Index":87,"Attempt":0,"Launch Time":1427397593842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4089,"Index":81,"Attempt":0,"Launch Time":1427397593834,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593842,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4096,"Index":88,"Attempt":0,"Launch Time":1427397593843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4092,"Index":84,"Attempt":0,"Launch Time":1427397593837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593843,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274823,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4097,"Index":89,"Attempt":0,"Launch Time":1427397593844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4091,"Index":83,"Attempt":0,"Launch Time":1427397593835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272098,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4098,"Index":90,"Attempt":0,"Launch Time":1427397593846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4093,"Index":85,"Attempt":0,"Launch Time":1427397593839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4099,"Index":91,"Attempt":0,"Launch Time":1427397593847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4094,"Index":86,"Attempt":0,"Launch Time":1427397593841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":263265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4100,"Index":92,"Attempt":0,"Launch Time":1427397593849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4096,"Index":88,"Attempt":0,"Launch Time":1427397593843,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593849,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":261761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4101,"Index":93,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4097,"Index":89,"Attempt":0,"Launch Time":1427397593844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4102,"Index":94,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4095,"Index":87,"Attempt":0,"Launch Time":1427397593842,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593850,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4103,"Index":95,"Attempt":0,"Launch Time":1427397593852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4076,"Index":68,"Attempt":0,"Launch Time":1427397593806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":518506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4104,"Index":96,"Attempt":0,"Launch Time":1427397593853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4081,"Index":73,"Attempt":0,"Launch Time":1427397593820,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1316831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4105,"Index":97,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4098,"Index":90,"Attempt":0,"Launch Time":1427397593846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4106,"Index":98,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4099,"Index":91,"Attempt":0,"Launch Time":1427397593847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4107,"Index":99,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4102,"Index":94,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317818,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4108,"Index":100,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4100,"Index":92,"Attempt":0,"Launch Time":1427397593849,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593857,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362768,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4109,"Index":101,"Attempt":0,"Launch Time":1427397593860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4106,"Index":98,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":250758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4110,"Index":102,"Attempt":0,"Launch Time":1427397593861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4103,"Index":95,"Attempt":0,"Launch Time":1427397593852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282720,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4111,"Index":103,"Attempt":0,"Launch Time":1427397593863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4108,"Index":100,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4112,"Index":104,"Attempt":0,"Launch Time":1427397593864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4101,"Index":93,"Attempt":0,"Launch Time":1427397593850,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":4773208,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4113,"Index":105,"Attempt":0,"Launch Time":1427397593865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4109,"Index":101,"Attempt":0,"Launch Time":1427397593860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":222257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4114,"Index":106,"Attempt":0,"Launch Time":1427397593868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4105,"Index":97,"Attempt":0,"Launch Time":1427397593854,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":220351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4115,"Index":107,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4111,"Index":103,"Attempt":0,"Launch Time":1427397593863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4116,"Index":108,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4112,"Index":104,"Attempt":0,"Launch Time":1427397593864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4117,"Index":109,"Attempt":0,"Launch Time":1427397593871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4113,"Index":105,"Attempt":0,"Launch Time":1427397593865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":261141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4118,"Index":110,"Attempt":0,"Launch Time":1427397593872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4107,"Index":99,"Attempt":0,"Launch Time":1427397593857,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4119,"Index":111,"Attempt":0,"Launch Time":1427397593874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4114,"Index":106,"Attempt":0,"Launch Time":1427397593868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593874,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4120,"Index":112,"Attempt":0,"Launch Time":1427397593876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4115,"Index":107,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273288,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4121,"Index":113,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4116,"Index":108,"Attempt":0,"Launch Time":1427397593870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4122,"Index":114,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4117,"Index":109,"Attempt":0,"Launch Time":1427397593871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":267537,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4123,"Index":115,"Attempt":0,"Launch Time":1427397593882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4119,"Index":111,"Attempt":0,"Launch Time":1427397593874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1426520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4124,"Index":116,"Attempt":0,"Launch Time":1427397593890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4087,"Index":79,"Attempt":0,"Launch Time":1427397593831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4125,"Index":117,"Attempt":0,"Launch Time":1427397593893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4121,"Index":113,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4126,"Index":118,"Attempt":0,"Launch Time":1427397593894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4104,"Index":96,"Attempt":0,"Launch Time":1427397593853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":18,"Executor Run Time":19,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":434417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4127,"Index":119,"Attempt":0,"Launch Time":1427397593896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4118,"Index":110,"Attempt":0,"Launch Time":1427397593872,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":279988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4128,"Index":120,"Attempt":0,"Launch Time":1427397593898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4124,"Index":116,"Attempt":0,"Launch Time":1427397593890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593899,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4129,"Index":121,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4126,"Index":118,"Attempt":0,"Launch Time":1427397593894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":254380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4130,"Index":122,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4127,"Index":119,"Attempt":0,"Launch Time":1427397593896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":267506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4131,"Index":123,"Attempt":0,"Launch Time":1427397593907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4125,"Index":117,"Attempt":0,"Launch Time":1427397593893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":517869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4132,"Index":124,"Attempt":0,"Launch Time":1427397593908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4130,"Index":122,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291277,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4133,"Index":125,"Attempt":0,"Launch Time":1427397593910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4128,"Index":120,"Attempt":0,"Launch Time":1427397593898,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283559,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4134,"Index":126,"Attempt":0,"Launch Time":1427397593914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4132,"Index":124,"Attempt":0,"Launch Time":1427397593908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4135,"Index":127,"Attempt":0,"Launch Time":1427397593916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4131,"Index":123,"Attempt":0,"Launch Time":1427397593907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4136,"Index":128,"Attempt":0,"Launch Time":1427397593917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4133,"Index":125,"Attempt":0,"Launch Time":1427397593910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593917,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4137,"Index":129,"Attempt":0,"Launch Time":1427397593921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4134,"Index":126,"Attempt":0,"Launch Time":1427397593914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":238008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4138,"Index":130,"Attempt":0,"Launch Time":1427397593923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4136,"Index":128,"Attempt":0,"Launch Time":1427397593917,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264420,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4139,"Index":131,"Attempt":0,"Launch Time":1427397593924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4135,"Index":127,"Attempt":0,"Launch Time":1427397593916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":269186,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4140,"Index":132,"Attempt":0,"Launch Time":1427397593927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4137,"Index":129,"Attempt":0,"Launch Time":1427397593921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":245361,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4141,"Index":133,"Attempt":0,"Launch Time":1427397593930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4139,"Index":131,"Attempt":0,"Launch Time":1427397593924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4142,"Index":134,"Attempt":0,"Launch Time":1427397593931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4138,"Index":130,"Attempt":0,"Launch Time":1427397593923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":255791,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4143,"Index":135,"Attempt":0,"Launch Time":1427397593932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4122,"Index":114,"Attempt":0,"Launch Time":1427397593877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4144,"Index":136,"Attempt":0,"Launch Time":1427397593933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4140,"Index":132,"Attempt":0,"Launch Time":1427397593927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4145,"Index":137,"Attempt":0,"Launch Time":1427397593935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4120,"Index":112,"Attempt":0,"Launch Time":1427397593876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4146,"Index":138,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4123,"Index":115,"Attempt":0,"Launch Time":1427397593882,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346710,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4147,"Index":139,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4142,"Index":134,"Attempt":0,"Launch Time":1427397593931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4148,"Index":140,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4141,"Index":133,"Attempt":0,"Launch Time":1427397593930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4149,"Index":141,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4144,"Index":136,"Attempt":0,"Launch Time":1427397593933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593940,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314267,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4150,"Index":142,"Attempt":0,"Launch Time":1427397593942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4110,"Index":102,"Attempt":0,"Launch Time":1427397593861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593942,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":60,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":431493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4151,"Index":143,"Attempt":0,"Launch Time":1427397593944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4146,"Index":138,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4152,"Index":144,"Attempt":0,"Launch Time":1427397593946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4148,"Index":140,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":256320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4153,"Index":145,"Attempt":0,"Launch Time":1427397593948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4150,"Index":142,"Attempt":0,"Launch Time":1427397593942,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593948,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297662,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4154,"Index":146,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4151,"Index":143,"Attempt":0,"Launch Time":1427397593944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4155,"Index":147,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4147,"Index":139,"Attempt":0,"Launch Time":1427397593937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4156,"Index":148,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4157,"Index":149,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4143,"Index":135,"Attempt":0,"Launch Time":1427397593932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304088,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4149,"Index":141,"Attempt":0,"Launch Time":1427397593940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593952,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":292971,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4158,"Index":150,"Attempt":0,"Launch Time":1427397593955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4154,"Index":146,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":284204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4159,"Index":151,"Attempt":0,"Launch Time":1427397593956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4152,"Index":144,"Attempt":0,"Launch Time":1427397593946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4160,"Index":152,"Attempt":0,"Launch Time":1427397593957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4153,"Index":145,"Attempt":0,"Launch Time":1427397593948,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593957,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":259864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4161,"Index":153,"Attempt":0,"Launch Time":1427397593959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4157,"Index":149,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4162,"Index":154,"Attempt":0,"Launch Time":1427397593962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4145,"Index":137,"Attempt":0,"Launch Time":1427397593935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":242789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4158,"Index":150,"Attempt":0,"Launch Time":1427397593955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268649,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4163,"Index":155,"Attempt":0,"Launch Time":1427397593963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4164,"Index":156,"Attempt":0,"Launch Time":1427397593965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4159,"Index":151,"Attempt":0,"Launch Time":1427397593956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":282955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4165,"Index":157,"Attempt":0,"Launch Time":1427397593966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4161,"Index":153,"Attempt":0,"Launch Time":1427397593959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593966,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282858,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4166,"Index":158,"Attempt":0,"Launch Time":1427397593968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4162,"Index":154,"Attempt":0,"Launch Time":1427397593962,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280732,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4167,"Index":159,"Attempt":0,"Launch Time":1427397593969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4160,"Index":152,"Attempt":0,"Launch Time":1427397593957,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593970,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4168,"Index":160,"Attempt":0,"Launch Time":1427397593971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4155,"Index":147,"Attempt":0,"Launch Time":1427397593950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3026501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4169,"Index":161,"Attempt":0,"Launch Time":1427397593973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4165,"Index":157,"Attempt":0,"Launch Time":1427397593966,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":392185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4170,"Index":162,"Attempt":0,"Launch Time":1427397593980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4166,"Index":158,"Attempt":0,"Launch Time":1427397593968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5917945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4171,"Index":163,"Attempt":0,"Launch Time":1427397593981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4156,"Index":148,"Attempt":0,"Launch Time":1427397593952,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4172,"Index":164,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4164,"Index":156,"Attempt":0,"Launch Time":1427397593965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4173,"Index":165,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4167,"Index":159,"Attempt":0,"Launch Time":1427397593969,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":276415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4174,"Index":166,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4175,"Index":167,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4170,"Index":162,"Attempt":0,"Launch Time":1427397593980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":254114,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4169,"Index":161,"Attempt":0,"Launch Time":1427397593973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4176,"Index":168,"Attempt":0,"Launch Time":1427397593988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4168,"Index":160,"Attempt":0,"Launch Time":1427397593971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":451476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4177,"Index":169,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4173,"Index":165,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":274036,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4178,"Index":170,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4171,"Index":163,"Attempt":0,"Launch Time":1427397593981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":275731,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4179,"Index":171,"Attempt":0,"Launch Time":1427397593991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4180,"Index":172,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4181,"Index":173,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4172,"Index":164,"Attempt":0,"Launch Time":1427397593983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288918,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4174,"Index":166,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":238619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4175,"Index":167,"Attempt":0,"Launch Time":1427397593986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593992,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4182,"Index":174,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4176,"Index":168,"Attempt":0,"Launch Time":1427397593988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":370589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4183,"Index":175,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4163,"Index":155,"Attempt":0,"Launch Time":1427397593963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281915,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4184,"Index":176,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4179,"Index":171,"Attempt":0,"Launch Time":1427397593991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593997,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4185,"Index":177,"Attempt":0,"Launch Time":1427397593998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4180,"Index":172,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4186,"Index":178,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4187,"Index":179,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4177,"Index":169,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":569415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4181,"Index":173,"Attempt":0,"Launch Time":1427397593992,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397593999,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4188,"Index":180,"Attempt":0,"Launch Time":1427397594003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4184,"Index":176,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":257720,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4189,"Index":181,"Attempt":0,"Launch Time":1427397594004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4185,"Index":177,"Attempt":0,"Launch Time":1427397593998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":254218,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4190,"Index":182,"Attempt":0,"Launch Time":1427397594006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4186,"Index":178,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":251967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4191,"Index":183,"Attempt":0,"Launch Time":1427397594008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4188,"Index":180,"Attempt":0,"Launch Time":1427397594003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4192,"Index":184,"Attempt":0,"Launch Time":1427397594009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4189,"Index":181,"Attempt":0,"Launch Time":1427397594004,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4193,"Index":185,"Attempt":0,"Launch Time":1427397594011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4187,"Index":179,"Attempt":0,"Launch Time":1427397593999,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":232146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4194,"Index":186,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4190,"Index":182,"Attempt":0,"Launch Time":1427397594006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265430,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4195,"Index":187,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4182,"Index":174,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4196,"Index":188,"Attempt":0,"Launch Time":1427397594015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4178,"Index":170,"Attempt":0,"Launch Time":1427397593989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594015,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4197,"Index":189,"Attempt":0,"Launch Time":1427397594017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4192,"Index":184,"Attempt":0,"Launch Time":1427397594009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1530912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4198,"Index":190,"Attempt":0,"Launch Time":1427397594018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4193,"Index":185,"Attempt":0,"Launch Time":1427397594011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4199,"Index":191,"Attempt":0,"Launch Time":1427397594019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4129,"Index":121,"Attempt":0,"Launch Time":1427397593902,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":116,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4062514,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4200,"Index":192,"Attempt":0,"Launch Time":1427397594020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4183,"Index":175,"Attempt":0,"Launch Time":1427397593997,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4201,"Index":193,"Attempt":0,"Launch Time":1427397594022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4196,"Index":188,"Attempt":0,"Launch Time":1427397594015,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":354798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4202,"Index":194,"Attempt":0,"Launch Time":1427397594025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4197,"Index":189,"Attempt":0,"Launch Time":1427397594017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594025,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329059,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4203,"Index":195,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4204,"Index":196,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4198,"Index":190,"Attempt":0,"Launch Time":1427397594018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261897,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4199,"Index":191,"Attempt":0,"Launch Time":1427397594019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4205,"Index":197,"Attempt":0,"Launch Time":1427397594028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4201,"Index":193,"Attempt":0,"Launch Time":1427397594022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594028,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4206,"Index":198,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4200,"Index":192,"Attempt":0,"Launch Time":1427397594020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4202,"Index":194,"Attempt":0,"Launch Time":1427397594025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4207,"Index":199,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4208,"Index":200,"Attempt":0,"Launch Time":1427397594034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4204,"Index":196,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":451595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4209,"Index":201,"Attempt":0,"Launch Time":1427397594035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4205,"Index":197,"Attempt":0,"Launch Time":1427397594028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296983,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4210,"Index":202,"Attempt":0,"Launch Time":1427397594039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4207,"Index":199,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4211,"Index":203,"Attempt":0,"Launch Time":1427397594040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4208,"Index":200,"Attempt":0,"Launch Time":1427397594034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274641,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4212,"Index":204,"Attempt":0,"Launch Time":1427397594041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4209,"Index":201,"Attempt":0,"Launch Time":1427397594035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594041,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262055,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4213,"Index":205,"Attempt":0,"Launch Time":1427397594044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4203,"Index":195,"Attempt":0,"Launch Time":1427397594027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":539141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4214,"Index":206,"Attempt":0,"Launch Time":1427397594046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4211,"Index":203,"Attempt":0,"Launch Time":1427397594040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4215,"Index":207,"Attempt":0,"Launch Time":1427397594053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4210,"Index":202,"Attempt":0,"Launch Time":1427397594039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320607,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4216,"Index":208,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4206,"Index":198,"Attempt":0,"Launch Time":1427397594033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":559993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4217,"Index":209,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4213,"Index":205,"Attempt":0,"Launch Time":1427397594044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4218,"Index":210,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4212,"Index":204,"Attempt":0,"Launch Time":1427397594041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594057,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4219,"Index":211,"Attempt":0,"Launch Time":1427397594059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4215,"Index":207,"Attempt":0,"Launch Time":1427397594053,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594059,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":238848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4220,"Index":212,"Attempt":0,"Launch Time":1427397594061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4214,"Index":206,"Attempt":0,"Launch Time":1427397594046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594061,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4221,"Index":213,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4218,"Index":210,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4222,"Index":214,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4219,"Index":211,"Attempt":0,"Launch Time":1427397594059,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":277776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4223,"Index":215,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4222,"Index":214,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324732,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4224,"Index":216,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4220,"Index":212,"Attempt":0,"Launch Time":1427397594061,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":368112,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4225,"Index":217,"Attempt":0,"Launch Time":1427397594073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4216,"Index":208,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":366276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4226,"Index":218,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4217,"Index":209,"Attempt":0,"Launch Time":1427397594057,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4227,"Index":219,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4221,"Index":213,"Attempt":0,"Launch Time":1427397594064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309587,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4228,"Index":220,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4224,"Index":216,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4229,"Index":221,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4223,"Index":215,"Attempt":0,"Launch Time":1427397594072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":283494,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4230,"Index":222,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4225,"Index":217,"Attempt":0,"Launch Time":1427397594073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":273191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4231,"Index":223,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4195,"Index":187,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":21,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310401,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4232,"Index":224,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4233,"Index":225,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4230,"Index":222,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4229,"Index":221,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4234,"Index":226,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4228,"Index":220,"Attempt":0,"Launch Time":1427397594078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4235,"Index":227,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4227,"Index":219,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4236,"Index":228,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4231,"Index":223,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":256734,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4237,"Index":229,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4232,"Index":224,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4238,"Index":230,"Attempt":0,"Launch Time":1427397594092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4234,"Index":226,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":265995,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4239,"Index":231,"Attempt":0,"Launch Time":1427397594093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4235,"Index":227,"Attempt":0,"Launch Time":1427397594087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":253134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4240,"Index":232,"Attempt":0,"Launch Time":1427397594096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4236,"Index":228,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":250411,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4241,"Index":233,"Attempt":0,"Launch Time":1427397594099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4237,"Index":229,"Attempt":0,"Launch Time":1427397594091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":351852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4242,"Index":234,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4238,"Index":230,"Attempt":0,"Launch Time":1427397594092,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4243,"Index":235,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4240,"Index":232,"Attempt":0,"Launch Time":1427397594096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":243302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4244,"Index":236,"Attempt":0,"Launch Time":1427397594106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4241,"Index":233,"Attempt":0,"Launch Time":1427397594099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":282740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4245,"Index":237,"Attempt":0,"Launch Time":1427397594107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4242,"Index":234,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273630,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4246,"Index":238,"Attempt":0,"Launch Time":1427397594108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4243,"Index":235,"Attempt":0,"Launch Time":1427397594102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4247,"Index":239,"Attempt":0,"Launch Time":1427397594112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4244,"Index":236,"Attempt":0,"Launch Time":1427397594106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322557,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4248,"Index":240,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4239,"Index":231,"Attempt":0,"Launch Time":1427397594093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1976562,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4249,"Index":241,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4246,"Index":238,"Attempt":0,"Launch Time":1427397594108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299755,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4250,"Index":242,"Attempt":0,"Launch Time":1427397594116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4245,"Index":237,"Attempt":0,"Launch Time":1427397594107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":509637,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4251,"Index":243,"Attempt":0,"Launch Time":1427397594119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4247,"Index":239,"Attempt":0,"Launch Time":1427397594112,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4252,"Index":244,"Attempt":0,"Launch Time":1427397594120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4249,"Index":241,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4253,"Index":245,"Attempt":0,"Launch Time":1427397594126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4250,"Index":242,"Attempt":0,"Launch Time":1427397594116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4254,"Index":246,"Attempt":0,"Launch Time":1427397594127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4251,"Index":243,"Attempt":0,"Launch Time":1427397594119,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4255,"Index":247,"Attempt":0,"Launch Time":1427397594128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4252,"Index":244,"Attempt":0,"Launch Time":1427397594120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4256,"Index":248,"Attempt":0,"Launch Time":1427397594129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4226,"Index":218,"Attempt":0,"Launch Time":1427397594076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":442113,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4257,"Index":249,"Attempt":0,"Launch Time":1427397594131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4248,"Index":240,"Attempt":0,"Launch Time":1427397594114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594131,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":498765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4258,"Index":250,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4253,"Index":245,"Attempt":0,"Launch Time":1427397594126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":273227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4259,"Index":251,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4254,"Index":246,"Attempt":0,"Launch Time":1427397594127,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4260,"Index":252,"Attempt":0,"Launch Time":1427397594134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4255,"Index":247,"Attempt":0,"Launch Time":1427397594128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265565,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4261,"Index":253,"Attempt":0,"Launch Time":1427397594145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594152,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4191,"Index":183,"Attempt":0,"Launch Time":1427397594008,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594145,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":136,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":283942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4262,"Index":254,"Attempt":0,"Launch Time":1427397594146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594153,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4233,"Index":225,"Attempt":0,"Launch Time":1427397594085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301188,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4263,"Index":255,"Attempt":0,"Launch Time":1427397594148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594155,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4194,"Index":186,"Attempt":0,"Launch Time":1427397594012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":88,"Executor Run Time":6,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273456,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4264,"Index":256,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4261,"Index":253,"Attempt":0,"Launch Time":1427397594145,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594152,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4265,"Index":257,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4262,"Index":254,"Attempt":0,"Launch Time":1427397594146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4266,"Index":258,"Attempt":0,"Launch Time":1427397594155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594161,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4263,"Index":255,"Attempt":0,"Launch Time":1427397594148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594155,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4267,"Index":259,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594165,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4264,"Index":256,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289776,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4268,"Index":260,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594166,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4265,"Index":257,"Attempt":0,"Launch Time":1427397594152,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594159,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":302377,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4269,"Index":261,"Attempt":0,"Launch Time":1427397594161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594167,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4266,"Index":258,"Attempt":0,"Launch Time":1427397594155,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":626399,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4270,"Index":262,"Attempt":0,"Launch Time":1427397594165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594173,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4267,"Index":259,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594165,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283515,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4271,"Index":263,"Attempt":0,"Launch Time":1427397594166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4268,"Index":260,"Attempt":0,"Launch Time":1427397594159,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594166,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4272,"Index":264,"Attempt":0,"Launch Time":1427397594167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594174,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4269,"Index":261,"Attempt":0,"Launch Time":1427397594161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4273,"Index":265,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4257,"Index":249,"Attempt":0,"Launch Time":1427397594131,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":304628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4274,"Index":266,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4271,"Index":263,"Attempt":0,"Launch Time":1427397594166,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4275,"Index":267,"Attempt":0,"Launch Time":1427397594173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4270,"Index":262,"Attempt":0,"Launch Time":1427397594165,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594173,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":470938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4276,"Index":268,"Attempt":0,"Launch Time":1427397594174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4272,"Index":264,"Attempt":0,"Launch Time":1427397594167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":854969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4277,"Index":269,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4273,"Index":265,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":248967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4278,"Index":270,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594184,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4274,"Index":266,"Attempt":0,"Launch Time":1427397594172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272225,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4279,"Index":271,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594186,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4280,"Index":272,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4276,"Index":268,"Attempt":0,"Launch Time":1427397594174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263445,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4275,"Index":267,"Attempt":0,"Launch Time":1427397594173,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594180,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":269248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4281,"Index":273,"Attempt":0,"Launch Time":1427397594183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594189,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4282,"Index":274,"Attempt":0,"Launch Time":1427397594184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594190,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4278,"Index":270,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":286626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4283,"Index":275,"Attempt":0,"Launch Time":1427397594186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594192,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4279,"Index":271,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594186,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4284,"Index":276,"Attempt":0,"Launch Time":1427397594189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594195,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4281,"Index":273,"Attempt":0,"Launch Time":1427397594183,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":259293,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4285,"Index":277,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594196,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4282,"Index":274,"Attempt":0,"Launch Time":1427397594184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594190,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":264635,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4286,"Index":278,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594197,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4280,"Index":272,"Attempt":0,"Launch Time":1427397594180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4277,"Index":269,"Attempt":0,"Launch Time":1427397594178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":250216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4287,"Index":279,"Attempt":0,"Launch Time":1427397594192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594198,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4283,"Index":275,"Attempt":0,"Launch Time":1427397594186,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594192,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":268695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4288,"Index":280,"Attempt":0,"Launch Time":1427397594195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4284,"Index":276,"Attempt":0,"Launch Time":1427397594189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4289,"Index":281,"Attempt":0,"Launch Time":1427397594196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4285,"Index":277,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":280310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4290,"Index":282,"Attempt":0,"Launch Time":1427397594197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594206,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4286,"Index":278,"Attempt":0,"Launch Time":1427397594190,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594197,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4291,"Index":283,"Attempt":0,"Launch Time":1427397594198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594207,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4287,"Index":279,"Attempt":0,"Launch Time":1427397594192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290598,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4292,"Index":284,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4288,"Index":280,"Attempt":0,"Launch Time":1427397594195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4293,"Index":285,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4289,"Index":281,"Attempt":0,"Launch Time":1427397594196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594203,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341649,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4294,"Index":286,"Attempt":0,"Launch Time":1427397594206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594216,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4290,"Index":282,"Attempt":0,"Launch Time":1427397594197,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":394189,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4295,"Index":287,"Attempt":0,"Launch Time":1427397594207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594214,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4291,"Index":283,"Attempt":0,"Launch Time":1427397594198,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4296,"Index":288,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4292,"Index":284,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4297,"Index":289,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4293,"Index":285,"Attempt":0,"Launch Time":1427397594203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4298,"Index":290,"Attempt":0,"Launch Time":1427397594214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4295,"Index":287,"Attempt":0,"Launch Time":1427397594207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594214,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":260288,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4299,"Index":291,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4296,"Index":288,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287251,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4300,"Index":292,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4297,"Index":289,"Attempt":0,"Launch Time":1427397594209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4301,"Index":293,"Attempt":0,"Launch Time":1427397594216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4294,"Index":286,"Attempt":0,"Launch Time":1427397594206,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594216,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294790,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4302,"Index":294,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4303,"Index":295,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4304,"Index":296,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4300,"Index":292,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4299,"Index":291,"Attempt":0,"Launch Time":1427397594215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272038,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4298,"Index":290,"Attempt":0,"Launch Time":1427397594214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282546,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4305,"Index":297,"Attempt":0,"Launch Time":1427397594222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594233,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4301,"Index":293,"Attempt":0,"Launch Time":1427397594216,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":291819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4306,"Index":298,"Attempt":0,"Launch Time":1427397594233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4305,"Index":297,"Attempt":0,"Launch Time":1427397594222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594233,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4307,"Index":299,"Attempt":0,"Launch Time":1427397594236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4256,"Index":248,"Attempt":0,"Launch Time":1427397594129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594236,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":106,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4308,"Index":300,"Attempt":0,"Launch Time":1427397594239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4306,"Index":298,"Attempt":0,"Launch Time":1427397594233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279735,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4309,"Index":301,"Attempt":0,"Launch Time":1427397594242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4307,"Index":299,"Attempt":0,"Launch Time":1427397594236,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300366,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4310,"Index":302,"Attempt":0,"Launch Time":1427397594245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4308,"Index":300,"Attempt":0,"Launch Time":1427397594239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273268,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4311,"Index":303,"Attempt":0,"Launch Time":1427397594246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4303,"Index":295,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":405418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4312,"Index":304,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4304,"Index":296,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290552,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4313,"Index":305,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4302,"Index":294,"Attempt":0,"Launch Time":1427397594221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4314,"Index":306,"Attempt":0,"Launch Time":1427397594249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4309,"Index":301,"Attempt":0,"Launch Time":1427397594242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4315,"Index":307,"Attempt":0,"Launch Time":1427397594254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4311,"Index":303,"Attempt":0,"Launch Time":1427397594246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594254,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":354308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4316,"Index":308,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4317,"Index":309,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4310,"Index":302,"Attempt":0,"Launch Time":1427397594245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318426,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4313,"Index":305,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309779,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4318,"Index":310,"Attempt":0,"Launch Time":1427397594257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4312,"Index":304,"Attempt":0,"Launch Time":1427397594248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4319,"Index":311,"Attempt":0,"Launch Time":1427397594261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4315,"Index":307,"Attempt":0,"Launch Time":1427397594254,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594261,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280814,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4320,"Index":312,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4314,"Index":306,"Attempt":0,"Launch Time":1427397594249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":344720,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4321,"Index":313,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4317,"Index":309,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594262,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":295462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4322,"Index":314,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4318,"Index":310,"Attempt":0,"Launch Time":1427397594257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284250,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4323,"Index":315,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4324,"Index":316,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4259,"Index":251,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2000241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4258,"Index":250,"Attempt":0,"Launch Time":1427397594132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":270452,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4325,"Index":317,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4326,"Index":318,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4320,"Index":312,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":297380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4260,"Index":252,"Attempt":0,"Launch Time":1427397594134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":50,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":816869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4327,"Index":319,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4321,"Index":313,"Attempt":0,"Launch Time":1427397594262,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288200,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4328,"Index":320,"Attempt":0,"Launch Time":1427397594271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4316,"Index":308,"Attempt":0,"Launch Time":1427397594256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":275349,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4329,"Index":321,"Attempt":0,"Launch Time":1427397594277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4328,"Index":320,"Attempt":0,"Launch Time":1427397594271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594277,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289377,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4330,"Index":322,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4324,"Index":316,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4331,"Index":323,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4325,"Index":317,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4332,"Index":324,"Attempt":0,"Launch Time":1427397594281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4326,"Index":318,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4333,"Index":325,"Attempt":0,"Launch Time":1427397594282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4329,"Index":321,"Attempt":0,"Launch Time":1427397594277,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":264391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4334,"Index":326,"Attempt":0,"Launch Time":1427397594286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4330,"Index":322,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":263311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4335,"Index":327,"Attempt":0,"Launch Time":1427397594287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4331,"Index":323,"Attempt":0,"Launch Time":1427397594279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298025,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4336,"Index":328,"Attempt":0,"Launch Time":1427397594288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4333,"Index":325,"Attempt":0,"Launch Time":1427397594282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":233522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4337,"Index":329,"Attempt":0,"Launch Time":1427397594292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4327,"Index":319,"Attempt":0,"Launch Time":1427397594270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4338,"Index":330,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4335,"Index":327,"Attempt":0,"Launch Time":1427397594287,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":263660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4339,"Index":331,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4332,"Index":324,"Attempt":0,"Launch Time":1427397594281,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594294,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4340,"Index":332,"Attempt":0,"Launch Time":1427397594294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4341,"Index":333,"Attempt":0,"Launch Time":1427397594295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4336,"Index":328,"Attempt":0,"Launch Time":1427397594288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4334,"Index":326,"Attempt":0,"Launch Time":1427397594286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293194,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4338,"Index":330,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594299,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4342,"Index":334,"Attempt":0,"Launch Time":1427397594300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4343,"Index":335,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4340,"Index":332,"Attempt":0,"Launch Time":1427397594294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594302,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4344,"Index":336,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4341,"Index":333,"Attempt":0,"Launch Time":1427397594295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4345,"Index":337,"Attempt":0,"Launch Time":1427397594303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4339,"Index":331,"Attempt":0,"Launch Time":1427397594293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":704728,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4346,"Index":338,"Attempt":0,"Launch Time":1427397594304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4337,"Index":329,"Attempt":0,"Launch Time":1427397594292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":421856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4347,"Index":339,"Attempt":0,"Launch Time":1427397594307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4342,"Index":334,"Attempt":0,"Launch Time":1427397594300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594307,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":383361,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4348,"Index":340,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4349,"Index":341,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4344,"Index":336,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":262787,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4343,"Index":335,"Attempt":0,"Launch Time":1427397594302,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":279478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4350,"Index":342,"Attempt":0,"Launch Time":1427397594311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4346,"Index":338,"Attempt":0,"Launch Time":1427397594304,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":267661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4351,"Index":343,"Attempt":0,"Launch Time":1427397594321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4345,"Index":337,"Attempt":0,"Launch Time":1427397594303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":264449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4352,"Index":344,"Attempt":0,"Launch Time":1427397594323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4348,"Index":340,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":615195,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4353,"Index":345,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4349,"Index":341,"Attempt":0,"Launch Time":1427397594309,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594324,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":467378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4354,"Index":346,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4350,"Index":342,"Attempt":0,"Launch Time":1427397594311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594325,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":264201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4355,"Index":347,"Attempt":0,"Launch Time":1427397594328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4351,"Index":343,"Attempt":0,"Launch Time":1427397594321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594328,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":266498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4356,"Index":348,"Attempt":0,"Launch Time":1427397594329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4352,"Index":344,"Attempt":0,"Launch Time":1427397594323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594329,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274157,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4357,"Index":349,"Attempt":0,"Launch Time":1427397594330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4354,"Index":346,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258544,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4358,"Index":350,"Attempt":0,"Launch Time":1427397594332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4353,"Index":345,"Attempt":0,"Launch Time":1427397594324,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284593,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4359,"Index":351,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4355,"Index":347,"Attempt":0,"Launch Time":1427397594328,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4360,"Index":352,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4356,"Index":348,"Attempt":0,"Launch Time":1427397594329,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":271228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4361,"Index":353,"Attempt":0,"Launch Time":1427397594337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4357,"Index":349,"Attempt":0,"Launch Time":1427397594330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":267539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4362,"Index":354,"Attempt":0,"Launch Time":1427397594338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4358,"Index":350,"Attempt":0,"Launch Time":1427397594332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":280237,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4363,"Index":355,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4364,"Index":356,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4360,"Index":352,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4359,"Index":351,"Attempt":0,"Launch Time":1427397594336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4365,"Index":357,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4361,"Index":353,"Attempt":0,"Launch Time":1427397594337,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":264746,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4366,"Index":358,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4362,"Index":354,"Attempt":0,"Launch Time":1427397594338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":271008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4367,"Index":359,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4363,"Index":355,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4368,"Index":360,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4364,"Index":356,"Attempt":0,"Launch Time":1427397594343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4369,"Index":361,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4322,"Index":314,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":40,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4370,"Index":362,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4365,"Index":357,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377010,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4371,"Index":363,"Attempt":0,"Launch Time":1427397594354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4366,"Index":358,"Attempt":0,"Launch Time":1427397594344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":479847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4372,"Index":364,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4347,"Index":339,"Attempt":0,"Launch Time":1427397594307,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":50,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":278845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4373,"Index":365,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4367,"Index":359,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4368,"Index":360,"Attempt":0,"Launch Time":1427397594350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":346840,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4374,"Index":366,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4375,"Index":367,"Attempt":0,"Launch Time":1427397594360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4370,"Index":362,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285451,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4376,"Index":368,"Attempt":0,"Launch Time":1427397594361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4371,"Index":363,"Attempt":0,"Launch Time":1427397594354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4377,"Index":369,"Attempt":0,"Launch Time":1427397594364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4372,"Index":364,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4378,"Index":370,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4379,"Index":371,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4374,"Index":366,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4319,"Index":311,"Attempt":0,"Launch Time":1427397594261,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":103,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":562891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4376,"Index":368,"Attempt":0,"Launch Time":1427397594361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":256403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4380,"Index":372,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4381,"Index":373,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4382,"Index":374,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4373,"Index":365,"Attempt":0,"Launch Time":1427397594358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4369,"Index":361,"Attempt":0,"Launch Time":1427397594352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":526869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4383,"Index":375,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4375,"Index":367,"Attempt":0,"Launch Time":1427397594360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4384,"Index":376,"Attempt":0,"Launch Time":1427397594372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4378,"Index":370,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":269364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4385,"Index":377,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4381,"Index":373,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267475,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4386,"Index":378,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4383,"Index":375,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":263811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4387,"Index":379,"Attempt":0,"Launch Time":1427397594376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4380,"Index":372,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4388,"Index":380,"Attempt":0,"Launch Time":1427397594378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4384,"Index":376,"Attempt":0,"Launch Time":1427397594372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4389,"Index":381,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4382,"Index":374,"Attempt":0,"Launch Time":1427397594368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":290355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4390,"Index":382,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4386,"Index":378,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296618,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4391,"Index":383,"Attempt":0,"Launch Time":1427397594381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4385,"Index":377,"Attempt":0,"Launch Time":1427397594374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594381,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4392,"Index":384,"Attempt":0,"Launch Time":1427397594384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4387,"Index":379,"Attempt":0,"Launch Time":1427397594376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":260443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4393,"Index":385,"Attempt":0,"Launch Time":1427397594385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4388,"Index":380,"Attempt":0,"Launch Time":1427397594378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4394,"Index":386,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4390,"Index":382,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":282791,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4391,"Index":383,"Attempt":0,"Launch Time":1427397594381,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4395,"Index":387,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4396,"Index":388,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4389,"Index":381,"Attempt":0,"Launch Time":1427397594380,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":319738,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4397,"Index":389,"Attempt":0,"Launch Time":1427397594395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4323,"Index":315,"Attempt":0,"Launch Time":1427397594269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":38,"Executor Run Time":45,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":390014,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4398,"Index":390,"Attempt":0,"Launch Time":1427397594399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4379,"Index":371,"Attempt":0,"Launch Time":1427397594367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594399,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4399,"Index":391,"Attempt":0,"Launch Time":1427397594402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4377,"Index":369,"Attempt":0,"Launch Time":1427397594364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1446908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4400,"Index":392,"Attempt":0,"Launch Time":1427397594403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4395,"Index":387,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":8,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":314006,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4401,"Index":393,"Attempt":0,"Launch Time":1427397594404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4397,"Index":389,"Attempt":0,"Launch Time":1427397594395,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":350497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4402,"Index":394,"Attempt":0,"Launch Time":1427397594405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4396,"Index":388,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":960054,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4403,"Index":395,"Attempt":0,"Launch Time":1427397594407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4398,"Index":390,"Attempt":0,"Launch Time":1427397594399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":297221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4404,"Index":396,"Attempt":0,"Launch Time":1427397594408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4399,"Index":391,"Attempt":0,"Launch Time":1427397594402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4405,"Index":397,"Attempt":0,"Launch Time":1427397594409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4400,"Index":392,"Attempt":0,"Launch Time":1427397594403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594409,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":249654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4406,"Index":398,"Attempt":0,"Launch Time":1427397594410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4401,"Index":393,"Attempt":0,"Launch Time":1427397594404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277643,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4407,"Index":399,"Attempt":0,"Launch Time":1427397594411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4402,"Index":394,"Attempt":0,"Launch Time":1427397594405,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":245163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4408,"Index":400,"Attempt":0,"Launch Time":1427397594414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4403,"Index":395,"Attempt":0,"Launch Time":1427397594407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594414,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4409,"Index":401,"Attempt":0,"Launch Time":1427397594416,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4405,"Index":397,"Attempt":0,"Launch Time":1427397594409,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594416,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":274623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4410,"Index":402,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4411,"Index":403,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4407,"Index":399,"Attempt":0,"Launch Time":1427397594411,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269665,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4406,"Index":398,"Attempt":0,"Launch Time":1427397594410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299962,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4412,"Index":404,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4409,"Index":401,"Attempt":0,"Launch Time":1427397594416,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":280342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4413,"Index":405,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4404,"Index":396,"Attempt":0,"Launch Time":1427397594408,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594422,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280597,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4414,"Index":406,"Attempt":0,"Launch Time":1427397594423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4411,"Index":403,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594423,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274170,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4415,"Index":407,"Attempt":0,"Launch Time":1427397594427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4408,"Index":400,"Attempt":0,"Launch Time":1427397594414,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4416,"Index":408,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4414,"Index":406,"Attempt":0,"Launch Time":1427397594423,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4417,"Index":409,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4412,"Index":404,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594429,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4418,"Index":410,"Attempt":0,"Launch Time":1427397594431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4413,"Index":405,"Attempt":0,"Launch Time":1427397594422,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594431,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":295894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4419,"Index":411,"Attempt":0,"Launch Time":1427397594434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4415,"Index":407,"Attempt":0,"Launch Time":1427397594427,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":355320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4420,"Index":412,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4421,"Index":413,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4410,"Index":402,"Attempt":0,"Launch Time":1427397594417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":369718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4393,"Index":385,"Attempt":0,"Launch Time":1427397594385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":671473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4422,"Index":414,"Attempt":0,"Launch Time":1427397594438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4418,"Index":410,"Attempt":0,"Launch Time":1427397594431,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349078,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4423,"Index":415,"Attempt":0,"Launch Time":1427397594439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4417,"Index":409,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":435782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4424,"Index":416,"Attempt":0,"Launch Time":1427397594442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4420,"Index":412,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300098,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4425,"Index":417,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4421,"Index":413,"Attempt":0,"Launch Time":1427397594435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4426,"Index":418,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4416,"Index":408,"Attempt":0,"Launch Time":1427397594429,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262755,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4427,"Index":419,"Attempt":0,"Launch Time":1427397594444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4419,"Index":411,"Attempt":0,"Launch Time":1427397594434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338366,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4428,"Index":420,"Attempt":0,"Launch Time":1427397594445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4423,"Index":415,"Attempt":0,"Launch Time":1427397594439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":370241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4429,"Index":421,"Attempt":0,"Launch Time":1427397594448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4422,"Index":414,"Attempt":0,"Launch Time":1427397594438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594448,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258932,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4430,"Index":422,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4425,"Index":417,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":260708,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4431,"Index":423,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4426,"Index":418,"Attempt":0,"Launch Time":1427397594443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":256452,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4432,"Index":424,"Attempt":0,"Launch Time":1427397594451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4428,"Index":420,"Attempt":0,"Launch Time":1427397594445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":268913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4433,"Index":425,"Attempt":0,"Launch Time":1427397594452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4424,"Index":416,"Attempt":0,"Launch Time":1427397594442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276654,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4434,"Index":426,"Attempt":0,"Launch Time":1427397594453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4427,"Index":419,"Attempt":0,"Launch Time":1427397594444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4435,"Index":427,"Attempt":0,"Launch Time":1427397594455,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4430,"Index":422,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594455,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4436,"Index":428,"Attempt":0,"Launch Time":1427397594456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4431,"Index":423,"Attempt":0,"Launch Time":1427397594449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":294053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4437,"Index":429,"Attempt":0,"Launch Time":1427397594457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4429,"Index":421,"Attempt":0,"Launch Time":1427397594448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":261479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4438,"Index":430,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4439,"Index":431,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4434,"Index":426,"Attempt":0,"Launch Time":1427397594453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4433,"Index":425,"Attempt":0,"Launch Time":1427397594452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4440,"Index":432,"Attempt":0,"Launch Time":1427397594459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4432,"Index":424,"Attempt":0,"Launch Time":1427397594451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4441,"Index":433,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4437,"Index":429,"Attempt":0,"Launch Time":1427397594457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":338486,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4442,"Index":434,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4439,"Index":431,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":375314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4443,"Index":435,"Attempt":0,"Launch Time":1427397594474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4438,"Index":430,"Attempt":0,"Launch Time":1427397594458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4444,"Index":436,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4436,"Index":428,"Attempt":0,"Launch Time":1427397594456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338130,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4445,"Index":437,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4435,"Index":427,"Attempt":0,"Launch Time":1427397594455,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":414144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4446,"Index":438,"Attempt":0,"Launch Time":1427397594480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4441,"Index":433,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333383,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4447,"Index":439,"Attempt":0,"Launch Time":1427397594483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4442,"Index":434,"Attempt":0,"Launch Time":1427397594472,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":674596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4448,"Index":440,"Attempt":0,"Launch Time":1427397594484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4443,"Index":435,"Attempt":0,"Launch Time":1427397594474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321046,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4449,"Index":441,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4450,"Index":442,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4444,"Index":436,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":337944,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4445,"Index":437,"Attempt":0,"Launch Time":1427397594478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":394149,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4451,"Index":443,"Attempt":0,"Launch Time":1427397594488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4446,"Index":438,"Attempt":0,"Launch Time":1427397594480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4452,"Index":444,"Attempt":0,"Launch Time":1427397594492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4450,"Index":442,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4453,"Index":445,"Attempt":0,"Launch Time":1427397594494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4440,"Index":432,"Attempt":0,"Launch Time":1427397594459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":572631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4454,"Index":446,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4448,"Index":440,"Attempt":0,"Launch Time":1427397594484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":328403,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4455,"Index":447,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4451,"Index":443,"Attempt":0,"Launch Time":1427397594488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4456,"Index":448,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4449,"Index":441,"Attempt":0,"Launch Time":1427397594486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1197457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4457,"Index":449,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4447,"Index":439,"Attempt":0,"Launch Time":1427397594483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":481396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4458,"Index":450,"Attempt":0,"Launch Time":1427397594498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4452,"Index":444,"Attempt":0,"Launch Time":1427397594492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594498,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290887,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4459,"Index":451,"Attempt":0,"Launch Time":1427397594501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4392,"Index":384,"Attempt":0,"Launch Time":1427397594384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":116,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4460,"Index":452,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4456,"Index":448,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4461,"Index":453,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4454,"Index":446,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261040,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4462,"Index":454,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4458,"Index":450,"Attempt":0,"Launch Time":1427397594498,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594504,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":279548,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4463,"Index":455,"Attempt":0,"Launch Time":1427397594506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4457,"Index":449,"Attempt":0,"Launch Time":1427397594497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4464,"Index":456,"Attempt":0,"Launch Time":1427397594507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4455,"Index":447,"Attempt":0,"Launch Time":1427397594496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594507,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":308241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4465,"Index":457,"Attempt":0,"Launch Time":1427397594510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4462,"Index":454,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4466,"Index":458,"Attempt":0,"Launch Time":1427397594511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4460,"Index":452,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4467,"Index":459,"Attempt":0,"Launch Time":1427397594515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4394,"Index":386,"Attempt":0,"Launch Time":1427397594387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":87,"Executor Run Time":40,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309244,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4468,"Index":460,"Attempt":0,"Launch Time":1427397594516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4465,"Index":457,"Attempt":0,"Launch Time":1427397594510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288438,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4469,"Index":461,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4467,"Index":459,"Attempt":0,"Launch Time":1427397594515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304384,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4470,"Index":462,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4459,"Index":451,"Attempt":0,"Launch Time":1427397594501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594522,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4471,"Index":463,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4468,"Index":460,"Attempt":0,"Launch Time":1427397594516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307892,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4472,"Index":464,"Attempt":0,"Launch Time":1427397594525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4453,"Index":445,"Attempt":0,"Launch Time":1427397594494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594525,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":319462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4473,"Index":465,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4474,"Index":466,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4475,"Index":467,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4470,"Index":462,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4471,"Index":463,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":328838,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4469,"Index":461,"Attempt":0,"Launch Time":1427397594522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":340359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4476,"Index":468,"Attempt":0,"Launch Time":1427397594534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4472,"Index":464,"Attempt":0,"Launch Time":1427397594525,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4477,"Index":469,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4474,"Index":466,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594537,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4478,"Index":470,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4473,"Index":465,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":343847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4479,"Index":471,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4480,"Index":472,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4478,"Index":470,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":277966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4477,"Index":469,"Attempt":0,"Launch Time":1427397594537,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4481,"Index":473,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4464,"Index":456,"Attempt":0,"Launch Time":1427397594507,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594544,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1185560,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4482,"Index":474,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4475,"Index":467,"Attempt":0,"Launch Time":1427397594530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":449100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4483,"Index":475,"Attempt":0,"Launch Time":1427397594547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4476,"Index":468,"Attempt":0,"Launch Time":1427397594534,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4484,"Index":476,"Attempt":0,"Launch Time":1427397594555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4479,"Index":471,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":317631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4485,"Index":477,"Attempt":0,"Launch Time":1427397594556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4482,"Index":474,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594556,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4486,"Index":478,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4481,"Index":473,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4487,"Index":479,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4480,"Index":472,"Attempt":0,"Launch Time":1427397594544,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":7114742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4488,"Index":480,"Attempt":0,"Launch Time":1427397594562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4484,"Index":476,"Attempt":0,"Launch Time":1427397594555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594562,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276598,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4489,"Index":481,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4483,"Index":475,"Attempt":0,"Launch Time":1427397594547,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":8,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":237463,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4490,"Index":482,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4491,"Index":483,"Attempt":0,"Launch Time":1427397594564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4487,"Index":479,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318394,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4485,"Index":477,"Attempt":0,"Launch Time":1427397594556,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":453428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4492,"Index":484,"Attempt":0,"Launch Time":1427397594568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4488,"Index":480,"Attempt":0,"Launch Time":1427397594562,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272582,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4493,"Index":485,"Attempt":0,"Launch Time":1427397594569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4494,"Index":486,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4491,"Index":483,"Attempt":0,"Launch Time":1427397594564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":242881,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4490,"Index":482,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":367542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4495,"Index":487,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4489,"Index":481,"Attempt":0,"Launch Time":1427397594563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283454,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4496,"Index":488,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4463,"Index":455,"Attempt":0,"Launch Time":1427397594506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":62,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290822,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4497,"Index":489,"Attempt":0,"Launch Time":1427397594573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4486,"Index":478,"Attempt":0,"Launch Time":1427397594557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4498,"Index":490,"Attempt":0,"Launch Time":1427397594574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4492,"Index":484,"Attempt":0,"Launch Time":1427397594568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594574,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4499,"Index":491,"Attempt":0,"Launch Time":1427397594575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4500,"Index":492,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4493,"Index":485,"Attempt":0,"Launch Time":1427397594569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":246401,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4495,"Index":487,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594576,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":244495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4501,"Index":493,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4496,"Index":488,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280975,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4502,"Index":494,"Attempt":0,"Launch Time":1427397594578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4494,"Index":486,"Attempt":0,"Launch Time":1427397594570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4503,"Index":495,"Attempt":0,"Launch Time":1427397594582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4498,"Index":490,"Attempt":0,"Launch Time":1427397594574,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4504,"Index":496,"Attempt":0,"Launch Time":1427397594583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4500,"Index":492,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4505,"Index":497,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4499,"Index":491,"Attempt":0,"Launch Time":1427397594575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475796,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4506,"Index":498,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4502,"Index":494,"Attempt":0,"Launch Time":1427397594578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":313747,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4507,"Index":499,"Attempt":0,"Launch Time":1427397594589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4497,"Index":489,"Attempt":0,"Launch Time":1427397594573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4508,"Index":500,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4509,"Index":501,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4503,"Index":495,"Attempt":0,"Launch Time":1427397594582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4504,"Index":496,"Attempt":0,"Launch Time":1427397594583,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594590,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4510,"Index":502,"Attempt":0,"Launch Time":1427397594591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4501,"Index":493,"Attempt":0,"Launch Time":1427397594576,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2261520,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4511,"Index":503,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4506,"Index":498,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277692,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4512,"Index":504,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4505,"Index":497,"Attempt":0,"Launch Time":1427397594585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268462,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4513,"Index":505,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4509,"Index":501,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4514,"Index":506,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4507,"Index":499,"Attempt":0,"Launch Time":1427397594589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":250443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4515,"Index":507,"Attempt":0,"Launch Time":1427397594598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4511,"Index":503,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":275318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4516,"Index":508,"Attempt":0,"Launch Time":1427397594599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4512,"Index":504,"Attempt":0,"Launch Time":1427397594592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594599,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":271864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4517,"Index":509,"Attempt":0,"Launch Time":1427397594600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4510,"Index":502,"Attempt":0,"Launch Time":1427397594591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280716,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4518,"Index":510,"Attempt":0,"Launch Time":1427397594601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4508,"Index":500,"Attempt":0,"Launch Time":1427397594590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":607026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4519,"Index":511,"Attempt":0,"Launch Time":1427397594602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4513,"Index":505,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292029,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4520,"Index":512,"Attempt":0,"Launch Time":1427397594604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4515,"Index":507,"Attempt":0,"Launch Time":1427397594598,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4521,"Index":513,"Attempt":0,"Launch Time":1427397594606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4517,"Index":509,"Attempt":0,"Launch Time":1427397594600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4522,"Index":514,"Attempt":0,"Launch Time":1427397594607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4516,"Index":508,"Attempt":0,"Launch Time":1427397594599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594607,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290435,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4523,"Index":515,"Attempt":0,"Launch Time":1427397594608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4519,"Index":511,"Attempt":0,"Launch Time":1427397594602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":259250,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4524,"Index":516,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4525,"Index":517,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4521,"Index":513,"Attempt":0,"Launch Time":1427397594606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4520,"Index":512,"Attempt":0,"Launch Time":1427397594604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4526,"Index":518,"Attempt":0,"Launch Time":1427397594614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4523,"Index":515,"Attempt":0,"Launch Time":1427397594608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4527,"Index":519,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4518,"Index":510,"Attempt":0,"Launch Time":1427397594601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4528,"Index":520,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4522,"Index":514,"Attempt":0,"Launch Time":1427397594607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":5035572,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4529,"Index":521,"Attempt":0,"Launch Time":1427397594619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4525,"Index":517,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594619,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":289288,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4530,"Index":522,"Attempt":0,"Launch Time":1427397594620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4524,"Index":516,"Attempt":0,"Launch Time":1427397594612,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281465,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4531,"Index":523,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4532,"Index":524,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4526,"Index":518,"Attempt":0,"Launch Time":1427397594614,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":288610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4514,"Index":506,"Attempt":0,"Launch Time":1427397594596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4533,"Index":525,"Attempt":0,"Launch Time":1427397594626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4461,"Index":453,"Attempt":0,"Launch Time":1427397594504,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":121,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":340571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4534,"Index":526,"Attempt":0,"Launch Time":1427397594631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4535,"Index":527,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4536,"Index":528,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4527,"Index":519,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4528,"Index":520,"Attempt":0,"Launch Time":1427397594618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4531,"Index":523,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311274,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4537,"Index":529,"Attempt":0,"Launch Time":1427397594640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4534,"Index":526,"Attempt":0,"Launch Time":1427397594631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594640,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4538,"Index":530,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4539,"Index":531,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4540,"Index":532,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4535,"Index":527,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4536,"Index":528,"Attempt":0,"Launch Time":1427397594632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":266744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4466,"Index":458,"Attempt":0,"Launch Time":1427397594511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":127,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":595143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4541,"Index":533,"Attempt":0,"Launch Time":1427397594644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4530,"Index":522,"Attempt":0,"Launch Time":1427397594620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":7,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293972,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4542,"Index":534,"Attempt":0,"Launch Time":1427397594645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4533,"Index":525,"Attempt":0,"Launch Time":1427397594626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4543,"Index":535,"Attempt":0,"Launch Time":1427397594648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4532,"Index":524,"Attempt":0,"Launch Time":1427397594625,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335031,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4544,"Index":536,"Attempt":0,"Launch Time":1427397594651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4541,"Index":533,"Attempt":0,"Launch Time":1427397594644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4545,"Index":537,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4537,"Index":529,"Attempt":0,"Launch Time":1427397594640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349792,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4546,"Index":538,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4542,"Index":534,"Attempt":0,"Launch Time":1427397594645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594652,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4547,"Index":539,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4539,"Index":531,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288838,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4548,"Index":540,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4543,"Index":535,"Attempt":0,"Launch Time":1427397594648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4549,"Index":541,"Attempt":0,"Launch Time":1427397594669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4540,"Index":532,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4550,"Index":542,"Attempt":0,"Launch Time":1427397594670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4548,"Index":540,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291213,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4551,"Index":543,"Attempt":0,"Launch Time":1427397594675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4545,"Index":537,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304236,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4552,"Index":544,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4544,"Index":536,"Attempt":0,"Launch Time":1427397594651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4553,"Index":545,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4550,"Index":542,"Attempt":0,"Launch Time":1427397594670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4554,"Index":546,"Attempt":0,"Launch Time":1427397594681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4546,"Index":538,"Attempt":0,"Launch Time":1427397594652,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":27,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":841769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4555,"Index":547,"Attempt":0,"Launch Time":1427397594682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4551,"Index":543,"Attempt":0,"Launch Time":1427397594675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594682,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":244258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4556,"Index":548,"Attempt":0,"Launch Time":1427397594683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4547,"Index":539,"Attempt":0,"Launch Time":1427397594664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594683,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":260161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4557,"Index":549,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4553,"Index":545,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":265224,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4558,"Index":550,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4552,"Index":544,"Attempt":0,"Launch Time":1427397594677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4559,"Index":551,"Attempt":0,"Launch Time":1427397594688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4554,"Index":546,"Attempt":0,"Launch Time":1427397594681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594688,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4560,"Index":552,"Attempt":0,"Launch Time":1427397594689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4529,"Index":521,"Attempt":0,"Launch Time":1427397594619,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4561,"Index":553,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4557,"Index":549,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4562,"Index":554,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4556,"Index":548,"Attempt":0,"Launch Time":1427397594683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":277293,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4563,"Index":555,"Attempt":0,"Launch Time":1427397594692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4555,"Index":547,"Attempt":0,"Launch Time":1427397594682,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4564,"Index":556,"Attempt":0,"Launch Time":1427397594693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4558,"Index":550,"Attempt":0,"Launch Time":1427397594684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":303606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4565,"Index":557,"Attempt":0,"Launch Time":1427397594697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4538,"Index":530,"Attempt":0,"Launch Time":1427397594641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4566,"Index":558,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4549,"Index":541,"Attempt":0,"Launch Time":1427397594669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4567,"Index":559,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4568,"Index":560,"Attempt":0,"Launch Time":1427397594699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4561,"Index":553,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335482,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4559,"Index":551,"Attempt":0,"Launch Time":1427397594688,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":323623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4569,"Index":561,"Attempt":0,"Launch Time":1427397594703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4564,"Index":556,"Attempt":0,"Launch Time":1427397594693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265132,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4570,"Index":562,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4566,"Index":558,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":228605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4571,"Index":563,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4568,"Index":560,"Attempt":0,"Launch Time":1427397594699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":233577,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4572,"Index":564,"Attempt":0,"Launch Time":1427397594707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4567,"Index":559,"Attempt":0,"Launch Time":1427397594698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295950,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4573,"Index":565,"Attempt":0,"Launch Time":1427397594709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4562,"Index":554,"Attempt":0,"Launch Time":1427397594691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594709,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":257751,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4574,"Index":566,"Attempt":0,"Launch Time":1427397594710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4571,"Index":563,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":265407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4575,"Index":567,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4570,"Index":562,"Attempt":0,"Launch Time":1427397594704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274763,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4576,"Index":568,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4563,"Index":555,"Attempt":0,"Launch Time":1427397594692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":278164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4577,"Index":569,"Attempt":0,"Launch Time":1427397594718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4572,"Index":564,"Attempt":0,"Launch Time":1427397594707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4578,"Index":570,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4565,"Index":557,"Attempt":0,"Launch Time":1427397594697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313910,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4579,"Index":571,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4573,"Index":565,"Attempt":0,"Launch Time":1427397594709,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594722,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":426213,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4580,"Index":572,"Attempt":0,"Launch Time":1427397594723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4576,"Index":568,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594723,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":7,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4581,"Index":573,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4577,"Index":569,"Attempt":0,"Launch Time":1427397594718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":267243,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4582,"Index":574,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4583,"Index":575,"Attempt":0,"Launch Time":1427397594727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4575,"Index":567,"Attempt":0,"Launch Time":1427397594711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":241318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4574,"Index":566,"Attempt":0,"Launch Time":1427397594710,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":264682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4584,"Index":576,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4578,"Index":570,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4585,"Index":577,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4580,"Index":572,"Attempt":0,"Launch Time":1427397594723,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594730,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":331989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4586,"Index":578,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4569,"Index":561,"Attempt":0,"Launch Time":1427397594703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4587,"Index":579,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4581,"Index":573,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":278184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4588,"Index":580,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4560,"Index":552,"Attempt":0,"Launch Time":1427397594689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594734,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":6535811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4589,"Index":581,"Attempt":0,"Launch Time":1427397594736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4585,"Index":577,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294135,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4590,"Index":582,"Attempt":0,"Launch Time":1427397594737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4584,"Index":576,"Attempt":0,"Launch Time":1427397594730,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":295089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4591,"Index":583,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4582,"Index":574,"Attempt":0,"Launch Time":1427397594726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":770061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4592,"Index":584,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4587,"Index":579,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339235,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4593,"Index":585,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4583,"Index":575,"Attempt":0,"Launch Time":1427397594727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4594,"Index":586,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4588,"Index":580,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594741,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":310864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4595,"Index":587,"Attempt":0,"Launch Time":1427397594742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4589,"Index":581,"Attempt":0,"Launch Time":1427397594736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284200,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4596,"Index":588,"Attempt":0,"Launch Time":1427397594744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4590,"Index":582,"Attempt":0,"Launch Time":1427397594737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":289178,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4597,"Index":589,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4593,"Index":585,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4598,"Index":590,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4599,"Index":591,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4595,"Index":587,"Attempt":0,"Launch Time":1427397594742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":441864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4592,"Index":584,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4600,"Index":592,"Attempt":0,"Launch Time":1427397594751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4591,"Index":583,"Attempt":0,"Launch Time":1427397594740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594751,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4601,"Index":593,"Attempt":0,"Launch Time":1427397594754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4596,"Index":588,"Attempt":0,"Launch Time":1427397594744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":270958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4602,"Index":594,"Attempt":0,"Launch Time":1427397594755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4594,"Index":586,"Attempt":0,"Launch Time":1427397594741,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":235924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4603,"Index":595,"Attempt":0,"Launch Time":1427397594756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4598,"Index":590,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594756,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279801,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4604,"Index":596,"Attempt":0,"Launch Time":1427397594757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4599,"Index":591,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4605,"Index":597,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4601,"Index":593,"Attempt":0,"Launch Time":1427397594754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594760,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280546,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4606,"Index":598,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4597,"Index":589,"Attempt":0,"Launch Time":1427397594749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4607,"Index":599,"Attempt":0,"Launch Time":1427397594763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4604,"Index":596,"Attempt":0,"Launch Time":1427397594757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4608,"Index":600,"Attempt":0,"Launch Time":1427397594764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4586,"Index":578,"Attempt":0,"Launch Time":1427397594733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594764,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4609,"Index":601,"Attempt":0,"Launch Time":1427397594767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4605,"Index":597,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":300322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4610,"Index":602,"Attempt":0,"Launch Time":1427397594768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4579,"Index":571,"Attempt":0,"Launch Time":1427397594722,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4611,"Index":603,"Attempt":0,"Launch Time":1427397594770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4602,"Index":594,"Attempt":0,"Launch Time":1427397594755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4612,"Index":604,"Attempt":0,"Launch Time":1427397594771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4608,"Index":600,"Attempt":0,"Launch Time":1427397594764,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4613,"Index":605,"Attempt":0,"Launch Time":1427397594774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4610,"Index":602,"Attempt":0,"Launch Time":1427397594768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":267994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4614,"Index":606,"Attempt":0,"Launch Time":1427397594777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4615,"Index":607,"Attempt":0,"Launch Time":1427397594778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4612,"Index":604,"Attempt":0,"Launch Time":1427397594771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307837,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4611,"Index":603,"Attempt":0,"Launch Time":1427397594770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594778,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4616,"Index":608,"Attempt":0,"Launch Time":1427397594779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4603,"Index":595,"Attempt":0,"Launch Time":1427397594756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594779,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":342585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4617,"Index":609,"Attempt":0,"Launch Time":1427397594781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4613,"Index":605,"Attempt":0,"Launch Time":1427397594774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594781,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322401,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4618,"Index":610,"Attempt":0,"Launch Time":1427397594785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4615,"Index":607,"Attempt":0,"Launch Time":1427397594778,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":329836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4619,"Index":611,"Attempt":0,"Launch Time":1427397594787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4614,"Index":606,"Attempt":0,"Launch Time":1427397594777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4620,"Index":612,"Attempt":0,"Launch Time":1427397594788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4616,"Index":608,"Attempt":0,"Launch Time":1427397594779,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":301921,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4621,"Index":613,"Attempt":0,"Launch Time":1427397594789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4617,"Index":609,"Attempt":0,"Launch Time":1427397594781,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":389606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4622,"Index":614,"Attempt":0,"Launch Time":1427397594797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4618,"Index":610,"Attempt":0,"Launch Time":1427397594785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292132,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4623,"Index":615,"Attempt":0,"Launch Time":1427397594800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4619,"Index":611,"Attempt":0,"Launch Time":1427397594787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594800,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4624,"Index":616,"Attempt":0,"Launch Time":1427397594801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4621,"Index":613,"Attempt":0,"Launch Time":1427397594789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":299795,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4625,"Index":617,"Attempt":0,"Launch Time":1427397594804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4620,"Index":612,"Attempt":0,"Launch Time":1427397594788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272277,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4626,"Index":618,"Attempt":0,"Launch Time":1427397594806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4622,"Index":614,"Attempt":0,"Launch Time":1427397594797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346752,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4627,"Index":619,"Attempt":0,"Launch Time":1427397594807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4623,"Index":615,"Attempt":0,"Launch Time":1427397594800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594807,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281762,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4628,"Index":620,"Attempt":0,"Launch Time":1427397594808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4624,"Index":616,"Attempt":0,"Launch Time":1427397594801,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594808,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276282,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4629,"Index":621,"Attempt":0,"Launch Time":1427397594810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4625,"Index":617,"Attempt":0,"Launch Time":1427397594804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":277247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4630,"Index":622,"Attempt":0,"Launch Time":1427397594814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4626,"Index":618,"Attempt":0,"Launch Time":1427397594806,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4631,"Index":623,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4627,"Index":619,"Attempt":0,"Launch Time":1427397594807,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4632,"Index":624,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4628,"Index":620,"Attempt":0,"Launch Time":1427397594808,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293250,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4633,"Index":625,"Attempt":0,"Launch Time":1427397594816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4629,"Index":621,"Attempt":0,"Launch Time":1427397594810,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306135,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4634,"Index":626,"Attempt":0,"Launch Time":1427397594818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4609,"Index":601,"Attempt":0,"Launch Time":1427397594767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":28,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4635,"Index":627,"Attempt":0,"Launch Time":1427397594819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4600,"Index":592,"Attempt":0,"Launch Time":1427397594751,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594819,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":39,"Result Size":930,"JVM GC Time":3,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3423963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4636,"Index":628,"Attempt":0,"Launch Time":1427397594822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4630,"Index":622,"Attempt":0,"Launch Time":1427397594814,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370562,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4637,"Index":629,"Attempt":0,"Launch Time":1427397594824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4631,"Index":623,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":364008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4638,"Index":630,"Attempt":0,"Launch Time":1427397594825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4632,"Index":624,"Attempt":0,"Launch Time":1427397594815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325347,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4639,"Index":631,"Attempt":0,"Launch Time":1427397594826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4634,"Index":626,"Attempt":0,"Launch Time":1427397594818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4640,"Index":632,"Attempt":0,"Launch Time":1427397594827,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4633,"Index":625,"Attempt":0,"Launch Time":1427397594816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594827,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":800522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4641,"Index":633,"Attempt":0,"Launch Time":1427397594829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4636,"Index":628,"Attempt":0,"Launch Time":1427397594822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4642,"Index":634,"Attempt":0,"Launch Time":1427397594831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4638,"Index":630,"Attempt":0,"Launch Time":1427397594825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594831,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4643,"Index":635,"Attempt":0,"Launch Time":1427397594832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4637,"Index":629,"Attempt":0,"Launch Time":1427397594824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4644,"Index":636,"Attempt":0,"Launch Time":1427397594833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4639,"Index":631,"Attempt":0,"Launch Time":1427397594826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":285940,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4641,"Index":633,"Attempt":0,"Launch Time":1427397594829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594838,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4645,"Index":637,"Attempt":0,"Launch Time":1427397594838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4646,"Index":638,"Attempt":0,"Launch Time":1427397594839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4644,"Index":636,"Attempt":0,"Launch Time":1427397594833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594839,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4647,"Index":639,"Attempt":0,"Launch Time":1427397594840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4643,"Index":635,"Attempt":0,"Launch Time":1427397594832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":298614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4648,"Index":640,"Attempt":0,"Launch Time":1427397594841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4642,"Index":634,"Attempt":0,"Launch Time":1427397594831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4649,"Index":641,"Attempt":0,"Launch Time":1427397594846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4646,"Index":638,"Attempt":0,"Launch Time":1427397594839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305945,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4650,"Index":642,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4645,"Index":637,"Attempt":0,"Launch Time":1427397594838,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":399734,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4651,"Index":643,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4647,"Index":639,"Attempt":0,"Launch Time":1427397594840,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594847,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4652,"Index":644,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4648,"Index":640,"Attempt":0,"Launch Time":1427397594841,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594848,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4653,"Index":645,"Attempt":0,"Launch Time":1427397594853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4649,"Index":641,"Attempt":0,"Launch Time":1427397594846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310199,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4654,"Index":646,"Attempt":0,"Launch Time":1427397594855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4651,"Index":643,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594855,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":281493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4650,"Index":642,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4655,"Index":647,"Attempt":0,"Launch Time":1427397594856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4656,"Index":648,"Attempt":0,"Launch Time":1427397594859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4640,"Index":632,"Attempt":0,"Launch Time":1427397594827,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594859,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":410372,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4657,"Index":649,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4653,"Index":645,"Attempt":0,"Launch Time":1427397594853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":320466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4658,"Index":650,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4659,"Index":651,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4654,"Index":646,"Attempt":0,"Launch Time":1427397594855,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":268623,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4652,"Index":644,"Attempt":0,"Launch Time":1427397594847,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":288399,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4660,"Index":652,"Attempt":0,"Launch Time":1427397594864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4655,"Index":647,"Attempt":0,"Launch Time":1427397594856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4661,"Index":653,"Attempt":0,"Launch Time":1427397594868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4659,"Index":651,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":341361,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4662,"Index":654,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4657,"Index":649,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4663,"Index":655,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4658,"Index":650,"Attempt":0,"Launch Time":1427397594861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594869,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4660,"Index":652,"Attempt":0,"Launch Time":1427397594864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594872,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":304089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4664,"Index":656,"Attempt":0,"Launch Time":1427397594873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4665,"Index":657,"Attempt":0,"Launch Time":1427397594880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4656,"Index":648,"Attempt":0,"Launch Time":1427397594859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4666,"Index":658,"Attempt":0,"Launch Time":1427397594881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4662,"Index":654,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4667,"Index":659,"Attempt":0,"Launch Time":1427397594883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4663,"Index":655,"Attempt":0,"Launch Time":1427397594869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295388,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4668,"Index":660,"Attempt":0,"Launch Time":1427397594884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4664,"Index":656,"Attempt":0,"Launch Time":1427397594873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272387,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4669,"Index":661,"Attempt":0,"Launch Time":1427397594887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4666,"Index":658,"Attempt":0,"Launch Time":1427397594881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594887,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":272143,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4670,"Index":662,"Attempt":0,"Launch Time":1427397594894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4607,"Index":599,"Attempt":0,"Launch Time":1427397594763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594894,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":99,"Result Size":930,"JVM GC Time":8,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4671,"Index":663,"Attempt":0,"Launch Time":1427397594896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4606,"Index":598,"Attempt":0,"Launch Time":1427397594760,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594896,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":102,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4672,"Index":664,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4670,"Index":662,"Attempt":0,"Launch Time":1427397594894,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594901,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":415632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4673,"Index":665,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4635,"Index":627,"Attempt":0,"Launch Time":1427397594819,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594902,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":67,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":379788,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4674,"Index":666,"Attempt":0,"Launch Time":1427397594907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4672,"Index":664,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594907,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320996,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4675,"Index":667,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4676,"Index":668,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4673,"Index":665,"Attempt":0,"Launch Time":1427397594901,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":298044,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4671,"Index":663,"Attempt":0,"Launch Time":1427397594896,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":298146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4677,"Index":669,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4667,"Index":659,"Attempt":0,"Launch Time":1427397594883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4678,"Index":670,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4679,"Index":671,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4676,"Index":668,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289049,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4674,"Index":666,"Attempt":0,"Launch Time":1427397594907,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4680,"Index":672,"Attempt":0,"Launch Time":1427397594915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4675,"Index":667,"Attempt":0,"Launch Time":1427397594908,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594915,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":302874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4681,"Index":673,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4682,"Index":674,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4679,"Index":671,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4678,"Index":670,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4683,"Index":675,"Attempt":0,"Launch Time":1427397594922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4680,"Index":672,"Attempt":0,"Launch Time":1427397594915,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594922,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4684,"Index":676,"Attempt":0,"Launch Time":1427397594923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4677,"Index":669,"Attempt":0,"Launch Time":1427397594914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":390836,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4685,"Index":677,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4668,"Index":660,"Attempt":0,"Launch Time":1427397594884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":271074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4686,"Index":678,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4669,"Index":661,"Attempt":0,"Launch Time":1427397594887,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309194,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4687,"Index":679,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4682,"Index":674,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4688,"Index":680,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4681,"Index":673,"Attempt":0,"Launch Time":1427397594921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4689,"Index":681,"Attempt":0,"Launch Time":1427397594930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4684,"Index":676,"Attempt":0,"Launch Time":1427397594923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4690,"Index":682,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4683,"Index":675,"Attempt":0,"Launch Time":1427397594922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":505448,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4691,"Index":683,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4665,"Index":657,"Attempt":0,"Launch Time":1427397594880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594933,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280946,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4692,"Index":684,"Attempt":0,"Launch Time":1427397594933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4687,"Index":679,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":253877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4693,"Index":685,"Attempt":0,"Launch Time":1427397594934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4688,"Index":680,"Attempt":0,"Launch Time":1427397594928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276890,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4694,"Index":686,"Attempt":0,"Launch Time":1427397594936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4689,"Index":681,"Attempt":0,"Launch Time":1427397594930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271754,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4695,"Index":687,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4690,"Index":682,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4696,"Index":688,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4691,"Index":683,"Attempt":0,"Launch Time":1427397594932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4697,"Index":689,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4692,"Index":684,"Attempt":0,"Launch Time":1427397594933,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337316,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4698,"Index":690,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4694,"Index":686,"Attempt":0,"Launch Time":1427397594936,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336430,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4699,"Index":691,"Attempt":0,"Launch Time":1427397594944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4693,"Index":685,"Attempt":0,"Launch Time":1427397594934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594945,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":381769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4700,"Index":692,"Attempt":0,"Launch Time":1427397594946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4696,"Index":688,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4701,"Index":693,"Attempt":0,"Launch Time":1427397594949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4697,"Index":689,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594949,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292961,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4702,"Index":694,"Attempt":0,"Launch Time":1427397594950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4698,"Index":690,"Attempt":0,"Launch Time":1427397594943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":260322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4703,"Index":695,"Attempt":0,"Launch Time":1427397594953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4700,"Index":692,"Attempt":0,"Launch Time":1427397594946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":281395,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4704,"Index":696,"Attempt":0,"Launch Time":1427397594954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4661,"Index":653,"Attempt":0,"Launch Time":1427397594868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":86,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4705,"Index":697,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4706,"Index":698,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4701,"Index":693,"Attempt":0,"Launch Time":1427397594949,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":284584,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4699,"Index":691,"Attempt":0,"Launch Time":1427397594944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":347340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4707,"Index":699,"Attempt":0,"Launch Time":1427397594956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4702,"Index":694,"Attempt":0,"Launch Time":1427397594950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594956,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295337,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4708,"Index":700,"Attempt":0,"Launch Time":1427397594958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4695,"Index":687,"Attempt":0,"Launch Time":1427397594939,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1018066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4709,"Index":701,"Attempt":0,"Launch Time":1427397594959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4703,"Index":695,"Attempt":0,"Launch Time":1427397594953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594959,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":274940,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4710,"Index":702,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4705,"Index":697,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":297831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4711,"Index":703,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4686,"Index":678,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":43,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":9089528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4712,"Index":704,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4713,"Index":705,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4706,"Index":698,"Attempt":0,"Launch Time":1427397594955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10414784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4707,"Index":699,"Attempt":0,"Launch Time":1427397594956,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10426024,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4714,"Index":706,"Attempt":0,"Launch Time":1427397594974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4710,"Index":702,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328629,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4715,"Index":707,"Attempt":0,"Launch Time":1427397594975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4708,"Index":700,"Attempt":0,"Launch Time":1427397594958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301068,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4716,"Index":708,"Attempt":0,"Launch Time":1427397594976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4709,"Index":701,"Attempt":0,"Launch Time":1427397594959,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":268140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4717,"Index":709,"Attempt":0,"Launch Time":1427397594977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4711,"Index":703,"Attempt":0,"Launch Time":1427397594968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594977,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291868,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4718,"Index":710,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4719,"Index":711,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4712,"Index":704,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4713,"Index":705,"Attempt":0,"Launch Time":1427397594973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":291390,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4720,"Index":712,"Attempt":0,"Launch Time":1427397594981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4714,"Index":706,"Attempt":0,"Launch Time":1427397594974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4721,"Index":713,"Attempt":0,"Launch Time":1427397594984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4716,"Index":708,"Attempt":0,"Launch Time":1427397594976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594984,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":335105,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4722,"Index":714,"Attempt":0,"Launch Time":1427397594986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4718,"Index":710,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4723,"Index":715,"Attempt":0,"Launch Time":1427397594987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4720,"Index":712,"Attempt":0,"Launch Time":1427397594981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594987,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":251203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4724,"Index":716,"Attempt":0,"Launch Time":1427397594988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4719,"Index":711,"Attempt":0,"Launch Time":1427397594980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292869,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4725,"Index":717,"Attempt":0,"Launch Time":1427397594990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4721,"Index":713,"Attempt":0,"Launch Time":1427397594984,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":299388,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4726,"Index":718,"Attempt":0,"Launch Time":1427397594994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4722,"Index":714,"Attempt":0,"Launch Time":1427397594986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397594994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1293355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4727,"Index":719,"Attempt":0,"Launch Time":1427397595005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4724,"Index":716,"Attempt":0,"Launch Time":1427397594988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335275,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4728,"Index":720,"Attempt":0,"Launch Time":1427397595017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4729,"Index":721,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4730,"Index":722,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4731,"Index":723,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4704,"Index":696,"Attempt":0,"Launch Time":1427397594954,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4723,"Index":715,"Attempt":0,"Launch Time":1427397594987,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":351228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4732,"Index":724,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4727,"Index":719,"Attempt":0,"Launch Time":1427397595005,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4685,"Index":677,"Attempt":0,"Launch Time":1427397594925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":582616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4726,"Index":718,"Attempt":0,"Launch Time":1427397594994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4725,"Index":717,"Attempt":0,"Launch Time":1427397594990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348577,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4733,"Index":725,"Attempt":0,"Launch Time":1427397595019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4734,"Index":726,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4735,"Index":727,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4715,"Index":707,"Attempt":0,"Launch Time":1427397594975,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":274948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4717,"Index":709,"Attempt":0,"Launch Time":1427397594977,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595024,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":330099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4736,"Index":728,"Attempt":0,"Launch Time":1427397595027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4730,"Index":722,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279983,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4737,"Index":729,"Attempt":0,"Launch Time":1427397595031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4734,"Index":726,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271176,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4738,"Index":730,"Attempt":0,"Launch Time":1427397595034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4736,"Index":728,"Attempt":0,"Launch Time":1427397595027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317389,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4739,"Index":731,"Attempt":0,"Launch Time":1427397595035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4732,"Index":724,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595035,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":428120,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4740,"Index":732,"Attempt":0,"Launch Time":1427397595040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4737,"Index":729,"Attempt":0,"Launch Time":1427397595031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320574,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4741,"Index":733,"Attempt":0,"Launch Time":1427397595041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4738,"Index":730,"Attempt":0,"Launch Time":1427397595034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4742,"Index":734,"Attempt":0,"Launch Time":1427397595043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4735,"Index":727,"Attempt":0,"Launch Time":1427397595024,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":282370,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4743,"Index":735,"Attempt":0,"Launch Time":1427397595045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4740,"Index":732,"Attempt":0,"Launch Time":1427397595040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276662,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4744,"Index":736,"Attempt":0,"Launch Time":1427397595046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4739,"Index":731,"Attempt":0,"Launch Time":1427397595035,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4745,"Index":737,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4741,"Index":733,"Attempt":0,"Launch Time":1427397595041,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4746,"Index":738,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4744,"Index":736,"Attempt":0,"Launch Time":1427397595046,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4747,"Index":739,"Attempt":0,"Launch Time":1427397595054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4743,"Index":735,"Attempt":0,"Launch Time":1427397595045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595054,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4748,"Index":740,"Attempt":0,"Launch Time":1427397595055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4742,"Index":734,"Attempt":0,"Launch Time":1427397595043,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4749,"Index":741,"Attempt":0,"Launch Time":1427397595058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4745,"Index":737,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":263187,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4750,"Index":742,"Attempt":0,"Launch Time":1427397595060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4746,"Index":738,"Attempt":0,"Launch Time":1427397595052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":493093,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4751,"Index":743,"Attempt":0,"Launch Time":1427397595062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4747,"Index":739,"Attempt":0,"Launch Time":1427397595054,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595063,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":265015,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4752,"Index":744,"Attempt":0,"Launch Time":1427397595064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4749,"Index":741,"Attempt":0,"Launch Time":1427397595058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":251216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4753,"Index":745,"Attempt":0,"Launch Time":1427397595067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4750,"Index":742,"Attempt":0,"Launch Time":1427397595060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4754,"Index":746,"Attempt":0,"Launch Time":1427397595069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4748,"Index":740,"Attempt":0,"Launch Time":1427397595055,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595069,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1538385,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4755,"Index":747,"Attempt":0,"Launch Time":1427397595072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4751,"Index":743,"Attempt":0,"Launch Time":1427397595062,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595072,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4756,"Index":748,"Attempt":0,"Launch Time":1427397595073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4752,"Index":744,"Attempt":0,"Launch Time":1427397595064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595073,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4757,"Index":749,"Attempt":0,"Launch Time":1427397595074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4753,"Index":745,"Attempt":0,"Launch Time":1427397595067,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595085,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":379690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4758,"Index":750,"Attempt":0,"Launch Time":1427397595085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4754,"Index":746,"Attempt":0,"Launch Time":1427397595069,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595086,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4759,"Index":751,"Attempt":0,"Launch Time":1427397595087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4731,"Index":723,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595087,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337114,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4760,"Index":752,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4761,"Index":753,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4733,"Index":725,"Attempt":0,"Launch Time":1427397595019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":289673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4755,"Index":747,"Attempt":0,"Launch Time":1427397595072,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":263126,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4762,"Index":754,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4758,"Index":750,"Attempt":0,"Launch Time":1427397595085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595092,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":266885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4763,"Index":755,"Attempt":0,"Launch Time":1427397595093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4756,"Index":748,"Attempt":0,"Launch Time":1427397595073,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595093,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1006719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4764,"Index":756,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4757,"Index":749,"Attempt":0,"Launch Time":1427397595074,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289715,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4765,"Index":757,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4728,"Index":720,"Attempt":0,"Launch Time":1427397595017,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595094,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4033611,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4766,"Index":758,"Attempt":0,"Launch Time":1427397595095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4759,"Index":751,"Attempt":0,"Launch Time":1427397595087,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285611,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4767,"Index":759,"Attempt":0,"Launch Time":1427397595097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4762,"Index":754,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4768,"Index":760,"Attempt":0,"Launch Time":1427397595098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4760,"Index":752,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4769,"Index":761,"Attempt":0,"Launch Time":1427397595100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4770,"Index":762,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4763,"Index":755,"Attempt":0,"Launch Time":1427397595093,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291495,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4765,"Index":757,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295193,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4771,"Index":763,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4729,"Index":721,"Attempt":0,"Launch Time":1427397595018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":490152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4772,"Index":764,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4768,"Index":760,"Attempt":0,"Launch Time":1427397595098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4773,"Index":765,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4767,"Index":759,"Attempt":0,"Launch Time":1427397595097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595104,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308214,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4774,"Index":766,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4775,"Index":767,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4771,"Index":763,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278757,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4769,"Index":761,"Attempt":0,"Launch Time":1427397595100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274476,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4776,"Index":768,"Attempt":0,"Launch Time":1427397595108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4770,"Index":762,"Attempt":0,"Launch Time":1427397595101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":240805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4777,"Index":769,"Attempt":0,"Launch Time":1427397595109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4772,"Index":764,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4778,"Index":770,"Attempt":0,"Launch Time":1427397595113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4774,"Index":766,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273188,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4779,"Index":771,"Attempt":0,"Launch Time":1427397595114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4775,"Index":767,"Attempt":0,"Launch Time":1427397595107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4780,"Index":772,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4776,"Index":768,"Attempt":0,"Launch Time":1427397595108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273824,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4781,"Index":773,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4777,"Index":769,"Attempt":0,"Launch Time":1427397595109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":240871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4782,"Index":774,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4778,"Index":770,"Attempt":0,"Launch Time":1427397595113,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":334314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4783,"Index":775,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4779,"Index":771,"Attempt":0,"Launch Time":1427397595114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4784,"Index":776,"Attempt":0,"Launch Time":1427397595121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4781,"Index":773,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595121,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":321540,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4785,"Index":777,"Attempt":0,"Launch Time":1427397595125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4773,"Index":765,"Attempt":0,"Launch Time":1427397595104,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595125,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":299744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4786,"Index":778,"Attempt":0,"Launch Time":1427397595126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4780,"Index":772,"Attempt":0,"Launch Time":1427397595115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595126,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317123,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4787,"Index":779,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4788,"Index":780,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4783,"Index":775,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4766,"Index":758,"Attempt":0,"Launch Time":1427397595095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":412446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4789,"Index":781,"Attempt":0,"Launch Time":1427397595129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4784,"Index":776,"Attempt":0,"Launch Time":1427397595121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":308969,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4790,"Index":782,"Attempt":0,"Launch Time":1427397595132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4785,"Index":777,"Attempt":0,"Launch Time":1427397595125,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313009,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4791,"Index":783,"Attempt":0,"Launch Time":1427397595134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4786,"Index":778,"Attempt":0,"Launch Time":1427397595126,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4792,"Index":784,"Attempt":0,"Launch Time":1427397595135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4782,"Index":774,"Attempt":0,"Launch Time":1427397595120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2954636,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4793,"Index":785,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4788,"Index":780,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":382005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4794,"Index":786,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4789,"Index":781,"Attempt":0,"Launch Time":1427397595129,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595144,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4795,"Index":787,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4790,"Index":782,"Attempt":0,"Launch Time":1427397595132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4761,"Index":753,"Attempt":0,"Launch Time":1427397595091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299843,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4796,"Index":788,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4797,"Index":789,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4798,"Index":790,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4787,"Index":779,"Attempt":0,"Launch Time":1427397595128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":300202,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4764,"Index":756,"Attempt":0,"Launch Time":1427397595094,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4799,"Index":791,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4800,"Index":792,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4792,"Index":784,"Attempt":0,"Launch Time":1427397595135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":237939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4791,"Index":783,"Attempt":0,"Launch Time":1427397595134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4801,"Index":793,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4798,"Index":790,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4802,"Index":794,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4803,"Index":795,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4799,"Index":791,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4797,"Index":789,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":311758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4804,"Index":796,"Attempt":0,"Launch Time":1427397595164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4800,"Index":792,"Attempt":0,"Launch Time":1427397595147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305238,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4805,"Index":797,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4803,"Index":795,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":415798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4806,"Index":798,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4802,"Index":794,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276793,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4807,"Index":799,"Attempt":0,"Launch Time":1427397595176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4801,"Index":793,"Attempt":0,"Launch Time":1427397595163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4808,"Index":800,"Attempt":0,"Launch Time":1427397595177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4804,"Index":796,"Attempt":0,"Launch Time":1427397595164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595177,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298639,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4809,"Index":801,"Attempt":0,"Launch Time":1427397595178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4796,"Index":788,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595178,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289907,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4810,"Index":802,"Attempt":0,"Launch Time":1427397595180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4805,"Index":797,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302370,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4811,"Index":803,"Attempt":0,"Launch Time":1427397595184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4808,"Index":800,"Attempt":0,"Launch Time":1427397595177,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595185,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308557,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4812,"Index":804,"Attempt":0,"Launch Time":1427397595187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4806,"Index":798,"Attempt":0,"Launch Time":1427397595174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595187,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4813,"Index":805,"Attempt":0,"Launch Time":1427397595188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4810,"Index":802,"Attempt":0,"Launch Time":1427397595180,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316760,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4814,"Index":806,"Attempt":0,"Launch Time":1427397595189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4807,"Index":799,"Attempt":0,"Launch Time":1427397595176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4815,"Index":807,"Attempt":0,"Launch Time":1427397595191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4811,"Index":803,"Attempt":0,"Launch Time":1427397595184,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595191,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":307338,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4816,"Index":808,"Attempt":0,"Launch Time":1427397595193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4809,"Index":801,"Attempt":0,"Launch Time":1427397595178,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595193,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":259878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4817,"Index":809,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4818,"Index":810,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4812,"Index":804,"Attempt":0,"Launch Time":1427397595187,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":288158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4813,"Index":805,"Attempt":0,"Launch Time":1427397595188,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4819,"Index":811,"Attempt":0,"Launch Time":1427397595195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4814,"Index":806,"Attempt":0,"Launch Time":1427397595189,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595195,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4820,"Index":812,"Attempt":0,"Launch Time":1427397595196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4815,"Index":807,"Attempt":0,"Launch Time":1427397595191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":278782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4821,"Index":813,"Attempt":0,"Launch Time":1427397595200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4818,"Index":810,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271556,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4822,"Index":814,"Attempt":0,"Launch Time":1427397595201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4823,"Index":815,"Attempt":0,"Launch Time":1427397595203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4824,"Index":816,"Attempt":0,"Launch Time":1427397595204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4819,"Index":811,"Attempt":0,"Launch Time":1427397595195,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4820,"Index":812,"Attempt":0,"Launch Time":1427397595196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":587311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4817,"Index":809,"Attempt":0,"Launch Time":1427397595194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595206,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":407980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4825,"Index":817,"Attempt":0,"Launch Time":1427397595207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4821,"Index":813,"Attempt":0,"Launch Time":1427397595200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":317554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4826,"Index":818,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4822,"Index":814,"Attempt":0,"Launch Time":1427397595201,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306731,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4827,"Index":819,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4828,"Index":820,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4824,"Index":816,"Attempt":0,"Launch Time":1427397595204,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":296470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4823,"Index":815,"Attempt":0,"Launch Time":1427397595203,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595212,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4829,"Index":821,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4830,"Index":822,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4831,"Index":823,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4827,"Index":819,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595217,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4828,"Index":820,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4826,"Index":818,"Attempt":0,"Launch Time":1427397595211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":247770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4832,"Index":824,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4831,"Index":823,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":284595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4833,"Index":825,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4830,"Index":822,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":320532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4834,"Index":826,"Attempt":0,"Launch Time":1427397595228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4825,"Index":817,"Attempt":0,"Launch Time":1427397595207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595228,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":11285036,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4835,"Index":827,"Attempt":0,"Launch Time":1427397595230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4832,"Index":824,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595230,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4836,"Index":828,"Attempt":0,"Launch Time":1427397595231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4833,"Index":825,"Attempt":0,"Launch Time":1427397595224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4837,"Index":829,"Attempt":0,"Launch Time":1427397595239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4835,"Index":827,"Attempt":0,"Launch Time":1427397595230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595239,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":391326,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4838,"Index":830,"Attempt":0,"Launch Time":1427397595240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4834,"Index":826,"Attempt":0,"Launch Time":1427397595228,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":397240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4839,"Index":831,"Attempt":0,"Launch Time":1427397595245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4836,"Index":828,"Attempt":0,"Launch Time":1427397595231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2485150,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4840,"Index":832,"Attempt":0,"Launch Time":1427397595250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4838,"Index":830,"Attempt":0,"Launch Time":1427397595240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350861,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4841,"Index":833,"Attempt":0,"Launch Time":1427397595252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4837,"Index":829,"Attempt":0,"Launch Time":1427397595239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4842,"Index":834,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4843,"Index":835,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4840,"Index":832,"Attempt":0,"Launch Time":1427397595250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4839,"Index":831,"Attempt":0,"Launch Time":1427397595245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":721284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4844,"Index":836,"Attempt":0,"Launch Time":1427397595263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4842,"Index":834,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595263,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4845,"Index":837,"Attempt":0,"Launch Time":1427397595268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4795,"Index":787,"Attempt":0,"Launch Time":1427397595146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595268,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":104,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1057437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4846,"Index":838,"Attempt":0,"Launch Time":1427397595274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4794,"Index":786,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":113,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298757,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4847,"Index":839,"Attempt":0,"Launch Time":1427397595275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4829,"Index":821,"Attempt":0,"Launch Time":1427397595217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":273545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4848,"Index":840,"Attempt":0,"Launch Time":1427397595276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4793,"Index":785,"Attempt":0,"Launch Time":1427397595143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595292,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":113,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1121142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4849,"Index":841,"Attempt":0,"Launch Time":1427397595292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4846,"Index":838,"Attempt":0,"Launch Time":1427397595274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595293,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":464691,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4850,"Index":842,"Attempt":0,"Launch Time":1427397595293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4816,"Index":808,"Attempt":0,"Launch Time":1427397595193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595309,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4851,"Index":843,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4852,"Index":844,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4845,"Index":837,"Attempt":0,"Launch Time":1427397595268,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":24,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":501477,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4853,"Index":845,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4854,"Index":846,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4855,"Index":847,"Attempt":0,"Launch Time":1427397595311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4844,"Index":836,"Attempt":0,"Launch Time":1427397595263,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345970,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4849,"Index":841,"Attempt":0,"Launch Time":1427397595292,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405145,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4841,"Index":833,"Attempt":0,"Launch Time":1427397595252,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":413061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4848,"Index":840,"Attempt":0,"Launch Time":1427397595276,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595311,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":900368,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4856,"Index":848,"Attempt":0,"Launch Time":1427397595318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4843,"Index":835,"Attempt":0,"Launch Time":1427397595257,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":60,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1903116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4857,"Index":849,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4854,"Index":846,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4858,"Index":850,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4852,"Index":844,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":300702,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4859,"Index":851,"Attempt":0,"Launch Time":1427397595322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4853,"Index":845,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4860,"Index":852,"Attempt":0,"Launch Time":1427397595326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4858,"Index":850,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4857,"Index":849,"Attempt":0,"Launch Time":1427397595319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":512553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4861,"Index":853,"Attempt":0,"Launch Time":1427397595330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4862,"Index":854,"Attempt":0,"Launch Time":1427397595332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4855,"Index":847,"Attempt":0,"Launch Time":1427397595311,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595332,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4863,"Index":855,"Attempt":0,"Launch Time":1427397595340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4860,"Index":852,"Attempt":0,"Launch Time":1427397595326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":272303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4864,"Index":856,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4865,"Index":857,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4866,"Index":858,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4867,"Index":859,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4861,"Index":853,"Attempt":0,"Launch Time":1427397595330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":442445,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4859,"Index":851,"Attempt":0,"Launch Time":1427397595322,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":546805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4868,"Index":860,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4862,"Index":854,"Attempt":0,"Launch Time":1427397595332,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300619,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4847,"Index":839,"Attempt":0,"Launch Time":1427397595275,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":26,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":388881,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4850,"Index":842,"Attempt":0,"Launch Time":1427397595293,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4869,"Index":861,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4868,"Index":860,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":244655,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4870,"Index":862,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4865,"Index":857,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":590745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4871,"Index":863,"Attempt":0,"Launch Time":1427397595348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4863,"Index":855,"Attempt":0,"Launch Time":1427397595340,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283124,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4872,"Index":864,"Attempt":0,"Launch Time":1427397595350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4867,"Index":859,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595350,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":615228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4873,"Index":865,"Attempt":0,"Launch Time":1427397595354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4874,"Index":866,"Attempt":0,"Launch Time":1427397595356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4869,"Index":861,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":326694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4870,"Index":862,"Attempt":0,"Launch Time":1427397595347,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595357,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318204,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4875,"Index":867,"Attempt":0,"Launch Time":1427397595359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4872,"Index":864,"Attempt":0,"Launch Time":1427397595350,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":282982,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4876,"Index":868,"Attempt":0,"Launch Time":1427397595360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4871,"Index":863,"Attempt":0,"Launch Time":1427397595348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4877,"Index":869,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4873,"Index":865,"Attempt":0,"Launch Time":1427397595354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":3434867,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4878,"Index":870,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4874,"Index":866,"Attempt":0,"Launch Time":1427397595356,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595365,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":261569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4879,"Index":871,"Attempt":0,"Launch Time":1427397595367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4876,"Index":868,"Attempt":0,"Launch Time":1427397595360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595367,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4880,"Index":872,"Attempt":0,"Launch Time":1427397595369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4875,"Index":867,"Attempt":0,"Launch Time":1427397595359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":262835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4881,"Index":873,"Attempt":0,"Launch Time":1427397595374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4879,"Index":871,"Attempt":0,"Launch Time":1427397595367,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4882,"Index":874,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4878,"Index":870,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":277408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4883,"Index":875,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4880,"Index":872,"Attempt":0,"Launch Time":1427397595369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":289794,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4884,"Index":876,"Attempt":0,"Launch Time":1427397595377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4877,"Index":869,"Attempt":0,"Launch Time":1427397595364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":580046,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4885,"Index":877,"Attempt":0,"Launch Time":1427397595378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4864,"Index":856,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":241883,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4886,"Index":878,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4881,"Index":873,"Attempt":0,"Launch Time":1427397595374,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268538,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4887,"Index":879,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4866,"Index":858,"Attempt":0,"Launch Time":1427397595341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595382,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4888,"Index":880,"Attempt":0,"Launch Time":1427397595385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4882,"Index":874,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":280418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4889,"Index":881,"Attempt":0,"Launch Time":1427397595386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4883,"Index":875,"Attempt":0,"Launch Time":1427397595376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4890,"Index":882,"Attempt":0,"Launch Time":1427397595388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4886,"Index":878,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595388,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324834,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4891,"Index":883,"Attempt":0,"Launch Time":1427397595390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595404,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4851,"Index":843,"Attempt":0,"Launch Time":1427397595310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298334,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4892,"Index":884,"Attempt":0,"Launch Time":1427397595393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4856,"Index":848,"Attempt":0,"Launch Time":1427397595318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595393,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":254064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4893,"Index":885,"Attempt":0,"Launch Time":1427397595402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4884,"Index":876,"Attempt":0,"Launch Time":1427397595377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595402,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4894,"Index":886,"Attempt":0,"Launch Time":1427397595403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595410,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4887,"Index":879,"Attempt":0,"Launch Time":1427397595382,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595403,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":350815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4895,"Index":887,"Attempt":0,"Launch Time":1427397595404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4891,"Index":883,"Attempt":0,"Launch Time":1427397595390,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595404,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":423857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4896,"Index":888,"Attempt":0,"Launch Time":1427397595410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595418,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4894,"Index":886,"Attempt":0,"Launch Time":1427397595403,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4897,"Index":889,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4898,"Index":890,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4885,"Index":877,"Attempt":0,"Launch Time":1427397595378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361470,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4892,"Index":884,"Attempt":0,"Launch Time":1427397595393,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595412,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392519,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4899,"Index":891,"Attempt":0,"Launch Time":1427397595418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4896,"Index":888,"Attempt":0,"Launch Time":1427397595410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324227,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4900,"Index":892,"Attempt":0,"Launch Time":1427397595434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4898,"Index":890,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595434,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10685409,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4901,"Index":893,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4902,"Index":894,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4893,"Index":885,"Attempt":0,"Launch Time":1427397595402,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":541919,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4897,"Index":889,"Attempt":0,"Launch Time":1427397595412,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324234,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4903,"Index":895,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4899,"Index":891,"Attempt":0,"Launch Time":1427397595418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595435,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":10763889,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4904,"Index":896,"Attempt":0,"Launch Time":1427397595440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4890,"Index":882,"Attempt":0,"Launch Time":1427397595388,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":11359990,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4905,"Index":897,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4895,"Index":887,"Attempt":0,"Launch Time":1427397595404,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":31,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4906,"Index":898,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4901,"Index":893,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4907,"Index":899,"Attempt":0,"Launch Time":1427397595442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4903,"Index":895,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279689,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4908,"Index":900,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4888,"Index":880,"Attempt":0,"Launch Time":1427397595385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":48,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269019,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4909,"Index":901,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4902,"Index":894,"Attempt":0,"Launch Time":1427397595435,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595444,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4910,"Index":902,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4900,"Index":892,"Attempt":0,"Launch Time":1427397595434,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1280351,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4911,"Index":903,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4905,"Index":897,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4912,"Index":904,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4913,"Index":905,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4907,"Index":899,"Attempt":0,"Launch Time":1427397595442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4906,"Index":898,"Attempt":0,"Launch Time":1427397595441,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4914,"Index":906,"Attempt":0,"Launch Time":1427397595451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4908,"Index":900,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595451,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4915,"Index":907,"Attempt":0,"Launch Time":1427397595454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4911,"Index":903,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":354831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4916,"Index":908,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4917,"Index":909,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4912,"Index":904,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":379803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4909,"Index":901,"Attempt":0,"Launch Time":1427397595444,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595457,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316990,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4918,"Index":910,"Attempt":0,"Launch Time":1427397595458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4913,"Index":905,"Attempt":0,"Launch Time":1427397595449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595458,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1321660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4919,"Index":911,"Attempt":0,"Launch Time":1427397595460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4914,"Index":906,"Attempt":0,"Launch Time":1427397595451,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":315089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4920,"Index":912,"Attempt":0,"Launch Time":1427397595461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4915,"Index":907,"Attempt":0,"Launch Time":1427397595454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276366,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4921,"Index":913,"Attempt":0,"Launch Time":1427397595463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4910,"Index":902,"Attempt":0,"Launch Time":1427397595447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302635,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4922,"Index":914,"Attempt":0,"Launch Time":1427397595466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4917,"Index":909,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":293116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4923,"Index":915,"Attempt":0,"Launch Time":1427397595467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4920,"Index":912,"Attempt":0,"Launch Time":1427397595461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4924,"Index":916,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4921,"Index":913,"Attempt":0,"Launch Time":1427397595463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595469,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":252484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4925,"Index":917,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4919,"Index":911,"Attempt":0,"Launch Time":1427397595460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4926,"Index":918,"Attempt":0,"Launch Time":1427397595471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4918,"Index":910,"Attempt":0,"Launch Time":1427397595458,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595472,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1728264,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4927,"Index":919,"Attempt":0,"Launch Time":1427397595474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4923,"Index":915,"Attempt":0,"Launch Time":1427397595467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4928,"Index":920,"Attempt":0,"Launch Time":1427397595475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4924,"Index":916,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310321,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4929,"Index":921,"Attempt":0,"Launch Time":1427397595476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4925,"Index":917,"Attempt":0,"Launch Time":1427397595469,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595476,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304147,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4930,"Index":922,"Attempt":0,"Launch Time":1427397595480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4927,"Index":919,"Attempt":0,"Launch Time":1427397595474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4931,"Index":923,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4928,"Index":920,"Attempt":0,"Launch Time":1427397595475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304796,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4932,"Index":924,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4926,"Index":918,"Attempt":0,"Launch Time":1427397595471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4933,"Index":925,"Attempt":0,"Launch Time":1427397595484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4929,"Index":921,"Attempt":0,"Launch Time":1427397595476,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285552,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4934,"Index":926,"Attempt":0,"Launch Time":1427397595485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4922,"Index":914,"Attempt":0,"Launch Time":1427397595466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4935,"Index":927,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4916,"Index":908,"Attempt":0,"Launch Time":1427397595457,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317529,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4936,"Index":928,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4930,"Index":922,"Attempt":0,"Launch Time":1427397595480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595487,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365451,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4937,"Index":929,"Attempt":0,"Launch Time":1427397595488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4931,"Index":923,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595488,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":266929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4938,"Index":930,"Attempt":0,"Launch Time":1427397595494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4889,"Index":881,"Attempt":0,"Launch Time":1427397595386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595495,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":29,"Executor Run Time":72,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":8656692,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4939,"Index":931,"Attempt":0,"Launch Time":1427397595497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4932,"Index":924,"Attempt":0,"Launch Time":1427397595482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595497,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":451659,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4940,"Index":932,"Attempt":0,"Launch Time":1427397595499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4936,"Index":928,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":273567,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4941,"Index":933,"Attempt":0,"Launch Time":1427397595500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4937,"Index":929,"Attempt":0,"Launch Time":1427397595488,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4942,"Index":934,"Attempt":0,"Launch Time":1427397595501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4943,"Index":935,"Attempt":0,"Launch Time":1427397595502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4933,"Index":925,"Attempt":0,"Launch Time":1427397595484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318912,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4934,"Index":926,"Attempt":0,"Launch Time":1427397595485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":12,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302838,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4944,"Index":936,"Attempt":0,"Launch Time":1427397595505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4940,"Index":932,"Attempt":0,"Launch Time":1427397595499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595505,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4945,"Index":937,"Attempt":0,"Launch Time":1427397595506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4939,"Index":931,"Attempt":0,"Launch Time":1427397595497,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595506,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":588088,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4946,"Index":938,"Attempt":0,"Launch Time":1427397595510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4943,"Index":935,"Attempt":0,"Launch Time":1427397595502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":984894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4947,"Index":939,"Attempt":0,"Launch Time":1427397595511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4942,"Index":934,"Attempt":0,"Launch Time":1427397595501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295492,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4948,"Index":940,"Attempt":0,"Launch Time":1427397595515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4938,"Index":930,"Attempt":0,"Launch Time":1427397595494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":313262,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4949,"Index":941,"Attempt":0,"Launch Time":1427397595516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4904,"Index":896,"Attempt":0,"Launch Time":1427397595440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4950,"Index":942,"Attempt":0,"Launch Time":1427397595519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4946,"Index":938,"Attempt":0,"Launch Time":1427397595510,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595519,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4951,"Index":943,"Attempt":0,"Launch Time":1427397595521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4935,"Index":927,"Attempt":0,"Launch Time":1427397595487,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":388904,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4952,"Index":944,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4948,"Index":940,"Attempt":0,"Launch Time":1427397595515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4953,"Index":945,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4949,"Index":941,"Attempt":0,"Launch Time":1427397595516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595523,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331196,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4954,"Index":946,"Attempt":0,"Launch Time":1427397595528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4951,"Index":943,"Attempt":0,"Launch Time":1427397595521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318937,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4955,"Index":947,"Attempt":0,"Launch Time":1427397595530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4952,"Index":944,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324905,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4956,"Index":948,"Attempt":0,"Launch Time":1427397595532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4953,"Index":945,"Attempt":0,"Launch Time":1427397595523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595532,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4957,"Index":949,"Attempt":0,"Launch Time":1427397595538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4944,"Index":936,"Attempt":0,"Launch Time":1427397595505,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1067240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4958,"Index":950,"Attempt":0,"Launch Time":1427397595540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4955,"Index":947,"Attempt":0,"Launch Time":1427397595530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":392630,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4959,"Index":951,"Attempt":0,"Launch Time":1427397595541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4956,"Index":948,"Attempt":0,"Launch Time":1427397595532,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4960,"Index":952,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4954,"Index":946,"Attempt":0,"Launch Time":1427397595528,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":766140,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4961,"Index":953,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4950,"Index":942,"Attempt":0,"Launch Time":1427397595519,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":720164,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4962,"Index":954,"Attempt":0,"Launch Time":1427397595546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4957,"Index":949,"Attempt":0,"Launch Time":1427397595538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":309977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4963,"Index":955,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4959,"Index":951,"Attempt":0,"Launch Time":1427397595541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":283008,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4964,"Index":956,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4960,"Index":952,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595548,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":280324,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4965,"Index":957,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4958,"Index":950,"Attempt":0,"Launch Time":1427397595540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267444,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4966,"Index":958,"Attempt":0,"Launch Time":1427397595549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4961,"Index":953,"Attempt":0,"Launch Time":1427397595542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":269989,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4967,"Index":959,"Attempt":0,"Launch Time":1427397595551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4945,"Index":937,"Attempt":0,"Launch Time":1427397595506,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595551,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294554,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4968,"Index":960,"Attempt":0,"Launch Time":1427397595552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4962,"Index":954,"Attempt":0,"Launch Time":1427397595546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286139,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4969,"Index":961,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4963,"Index":955,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":287846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4970,"Index":962,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4964,"Index":956,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4971,"Index":963,"Attempt":0,"Launch Time":1427397595555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4965,"Index":957,"Attempt":0,"Launch Time":1427397595548,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595555,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4972,"Index":964,"Attempt":0,"Launch Time":1427397595557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4966,"Index":958,"Attempt":0,"Launch Time":1427397595549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":255942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4973,"Index":965,"Attempt":0,"Launch Time":1427397595559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4968,"Index":960,"Attempt":0,"Launch Time":1427397595552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288473,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4974,"Index":966,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4969,"Index":961,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":252827,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4975,"Index":967,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4970,"Index":962,"Attempt":0,"Launch Time":1427397595554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":295502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4976,"Index":968,"Attempt":0,"Launch Time":1427397595563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4971,"Index":963,"Attempt":0,"Launch Time":1427397595555,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595563,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4977,"Index":969,"Attempt":0,"Launch Time":1427397595565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4973,"Index":965,"Attempt":0,"Launch Time":1427397595559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":256558,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4978,"Index":970,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4979,"Index":971,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4972,"Index":964,"Attempt":0,"Launch Time":1427397595557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4974,"Index":966,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4980,"Index":972,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4975,"Index":967,"Attempt":0,"Launch Time":1427397595560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":270771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4981,"Index":973,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4976,"Index":968,"Attempt":0,"Launch Time":1427397595563,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":231183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4982,"Index":974,"Attempt":0,"Launch Time":1427397595569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4967,"Index":959,"Attempt":0,"Launch Time":1427397595551,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":270758,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4983,"Index":975,"Attempt":0,"Launch Time":1427397595578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4978,"Index":970,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4984,"Index":976,"Attempt":0,"Launch Time":1427397595580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4977,"Index":969,"Attempt":0,"Launch Time":1427397595565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":268832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4985,"Index":977,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4947,"Index":939,"Attempt":0,"Launch Time":1427397595511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":32,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4986,"Index":978,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4980,"Index":972,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595583,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4987,"Index":979,"Attempt":0,"Launch Time":1427397595584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4981,"Index":973,"Attempt":0,"Launch Time":1427397595568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1284696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4988,"Index":980,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4989,"Index":981,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4984,"Index":976,"Attempt":0,"Launch Time":1427397595580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4979,"Index":971,"Attempt":0,"Launch Time":1427397595566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":354911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4990,"Index":982,"Attempt":0,"Launch Time":1427397595588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4983,"Index":975,"Attempt":0,"Launch Time":1427397595578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4991,"Index":983,"Attempt":0,"Launch Time":1427397595590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4985,"Index":977,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":416744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4992,"Index":984,"Attempt":0,"Launch Time":1427397595592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4987,"Index":979,"Attempt":0,"Launch Time":1427397595584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595593,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":330947,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4993,"Index":985,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4994,"Index":986,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4982,"Index":974,"Attempt":0,"Launch Time":1427397595569,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":308913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4989,"Index":981,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":276534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4995,"Index":987,"Attempt":0,"Launch Time":1427397595600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4990,"Index":982,"Attempt":0,"Launch Time":1427397595588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4996,"Index":988,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4992,"Index":984,"Attempt":0,"Launch Time":1427397595592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":274530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4997,"Index":989,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4941,"Index":933,"Attempt":0,"Launch Time":1427397595500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":97,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":40068811,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4998,"Index":990,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":4999,"Index":991,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4994,"Index":986,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4988,"Index":980,"Attempt":0,"Launch Time":1427397595587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":264650,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5000,"Index":992,"Attempt":0,"Launch Time":1427397595602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4991,"Index":983,"Attempt":0,"Launch Time":1427397595590,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5001,"Index":993,"Attempt":0,"Launch Time":1427397595603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4986,"Index":978,"Attempt":0,"Launch Time":1427397595582,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5002,"Index":994,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4995,"Index":987,"Attempt":0,"Launch Time":1427397595600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299626,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5003,"Index":995,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4999,"Index":991,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":308770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5004,"Index":996,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4998,"Index":990,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1517002,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5005,"Index":997,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4996,"Index":988,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5006,"Index":998,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5000,"Index":992,"Attempt":0,"Launch Time":1427397595602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595611,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":5007,"Index":999,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4997,"Index":989,"Attempt":0,"Launch Time":1427397595601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319333,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5001,"Index":993,"Attempt":0,"Launch Time":1427397595603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":276786,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5003,"Index":995,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273428,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5004,"Index":996,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":273868,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5006,"Index":998,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":275973,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5007,"Index":999,"Attempt":0,"Launch Time":1427397595611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":263073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4993,"Index":985,"Attempt":0,"Launch Time":1427397595593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2842856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5005,"Index":997,"Attempt":0,"Launch Time":1427397595609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595621,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":287183,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5002,"Index":994,"Attempt":0,"Launch Time":1427397595608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595623,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":297691,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397593694,"Completion Time":1427397595623,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":17,"Stage Attempt ID":0,"Task Info":{"Task ID":5008,"Index":0,"Attempt":0,"Launch Time":1427397595627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":17,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5008,"Index":0,"Attempt":0,"Launch Time":1427397595627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397595702,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":74,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192025,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line36.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line36.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line36.$read$$iwC$$iwC$$iwC.(:39)\n$line36.$read$$iwC$$iwC.(:41)\n$line36.$read$$iwC.(:43)\n$line36.$read.(:45)\n$line36.$read$.(:49)\n$line36.$read$.()\n$line36.$eval$.(:7)\n$line36.$eval$.()\n$line36.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397595627,"Completion Time":1427397595702,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":8,"Completion Time":1427397595702,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":9,"Submission Time":1427397597114,"Stage Infos":[{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[19,18]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":19,"Stage Attempt ID":0,"Task Info":{"Task ID":5009,"Index":0,"Attempt":0,"Launch Time":1427397597116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":19,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5009,"Index":0,"Attempt":0,"Launch Time":1427397597116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397597205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":192022,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line38.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line38.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line38.$read$$iwC$$iwC$$iwC.(:39)\n$line38.$read$$iwC$$iwC.(:41)\n$line38.$read$$iwC.(:43)\n$line38.$read.(:45)\n$line38.$read$.(:49)\n$line38.$read$.()\n$line38.$eval$.(:7)\n$line38.$eval$.()\n$line38.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397597116,"Completion Time":1427397597205,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":9,"Completion Time":1427397597205,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":10,"Submission Time":1427397599546,"Stage Infos":[{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}],"Stage IDs":[20,21]} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5010,"Index":0,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5011,"Index":1,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5012,"Index":2,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5013,"Index":3,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5014,"Index":4,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5015,"Index":5,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5016,"Index":6,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5017,"Index":7,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5018,"Index":8,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5019,"Index":9,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5020,"Index":10,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5021,"Index":11,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5022,"Index":12,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5023,"Index":13,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5011,"Index":1,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5012,"Index":2,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":274899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5016,"Index":6,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5024,"Index":14,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5025,"Index":15,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5026,"Index":16,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5027,"Index":17,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5019,"Index":9,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5020,"Index":10,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306724,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5014,"Index":4,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":316561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5021,"Index":11,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":280729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5017,"Index":7,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":267412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5015,"Index":5,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599592,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":269443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5028,"Index":18,"Attempt":0,"Launch Time":1427397599592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5029,"Index":19,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5030,"Index":20,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5031,"Index":21,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5032,"Index":22,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5033,"Index":23,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5034,"Index":24,"Attempt":0,"Launch Time":1427397599594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5010,"Index":0,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318464,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5026,"Index":16,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247775,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5022,"Index":12,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":273837,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5027,"Index":17,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":289503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5024,"Index":14,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":266308,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5023,"Index":13,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282108,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5018,"Index":8,"Attempt":0,"Launch Time":1427397599579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599598,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310279,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5013,"Index":3,"Attempt":0,"Launch Time":1427397599571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":315224,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5035,"Index":25,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5036,"Index":26,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5037,"Index":27,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5038,"Index":28,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5039,"Index":29,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5040,"Index":30,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5032,"Index":22,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5033,"Index":23,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301382,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5034,"Index":24,"Attempt":0,"Launch Time":1427397599594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340547,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5025,"Index":15,"Attempt":0,"Launch Time":1427397599586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":330830,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5041,"Index":31,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5042,"Index":32,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5030,"Index":20,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":367295,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5031,"Index":21,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":493701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5043,"Index":33,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5044,"Index":34,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5035,"Index":25,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324374,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5036,"Index":26,"Attempt":0,"Launch Time":1427397599600,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599608,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":321083,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5045,"Index":35,"Attempt":0,"Launch Time":1427397599608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5046,"Index":36,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5047,"Index":37,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5048,"Index":38,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5029,"Index":19,"Attempt":0,"Launch Time":1427397599593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":339081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5037,"Index":27,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318662,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5039,"Index":29,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":287042,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5049,"Index":39,"Attempt":0,"Launch Time":1427397599611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5028,"Index":18,"Attempt":0,"Launch Time":1427397599592,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5042,"Index":32,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599612,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5038,"Index":28,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599613,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334376,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5040,"Index":30,"Attempt":0,"Launch Time":1427397599601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332938,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5050,"Index":40,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5051,"Index":41,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5052,"Index":42,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5053,"Index":43,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5041,"Index":31,"Attempt":0,"Launch Time":1427397599602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":255099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5044,"Index":34,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":279851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5043,"Index":33,"Attempt":0,"Launch Time":1427397599607,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265929,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5045,"Index":35,"Attempt":0,"Launch Time":1427397599608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":278965,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5054,"Index":44,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5055,"Index":45,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5056,"Index":46,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5046,"Index":36,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340518,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5048,"Index":38,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":265447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5047,"Index":37,"Attempt":0,"Launch Time":1427397599609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":402877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5057,"Index":47,"Attempt":0,"Launch Time":1427397599639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599646,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5049,"Index":39,"Attempt":0,"Launch Time":1427397599611,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":23,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":304745,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5058,"Index":48,"Attempt":0,"Launch Time":1427397599640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599691,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5059,"Index":49,"Attempt":0,"Launch Time":1427397599641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5050,"Index":40,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5052,"Index":42,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":962033,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5060,"Index":50,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599666,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5061,"Index":51,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5051,"Index":41,"Attempt":0,"Launch Time":1427397599615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":25,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296177,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5062,"Index":52,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599661,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5056,"Index":46,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":306911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5055,"Index":45,"Attempt":0,"Launch Time":1427397599617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307821,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5063,"Index":53,"Attempt":0,"Launch Time":1427397599643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5053,"Index":43,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":16,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5064,"Index":54,"Attempt":0,"Launch Time":1427397599646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5057,"Index":47,"Attempt":0,"Launch Time":1427397599639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":274191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5065,"Index":55,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5066,"Index":56,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599664,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5067,"Index":57,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5061,"Index":51,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":313090,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5059,"Index":49,"Attempt":0,"Launch Time":1427397599641,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300675,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5063,"Index":53,"Attempt":0,"Launch Time":1427397599643,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":312481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5068,"Index":58,"Attempt":0,"Launch Time":1427397599659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599667,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5065,"Index":55,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":670271,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5069,"Index":59,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599668,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5070,"Index":60,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599669,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5064,"Index":54,"Attempt":0,"Launch Time":1427397599646,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5067,"Index":57,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599660,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":678632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5071,"Index":61,"Attempt":0,"Launch Time":1427397599661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599674,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5062,"Index":52,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1572828,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5072,"Index":62,"Attempt":0,"Launch Time":1427397599664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599670,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5066,"Index":56,"Attempt":0,"Launch Time":1427397599650,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599664,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1240309,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5073,"Index":63,"Attempt":0,"Launch Time":1427397599666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599681,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5060,"Index":50,"Attempt":0,"Launch Time":1427397599642,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599666,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":812196,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5074,"Index":64,"Attempt":0,"Launch Time":1427397599667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599675,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5068,"Index":58,"Attempt":0,"Launch Time":1427397599659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":302888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5075,"Index":65,"Attempt":0,"Launch Time":1427397599668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599679,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5069,"Index":59,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":383153,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5076,"Index":66,"Attempt":0,"Launch Time":1427397599669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599687,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5070,"Index":60,"Attempt":0,"Launch Time":1427397599660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310480,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5077,"Index":67,"Attempt":0,"Launch Time":1427397599670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599677,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5072,"Index":62,"Attempt":0,"Launch Time":1427397599664,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599670,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":331337,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5078,"Index":68,"Attempt":0,"Launch Time":1427397599673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599686,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5071,"Index":61,"Attempt":0,"Launch Time":1427397599661,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5079,"Index":69,"Attempt":0,"Launch Time":1427397599675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599684,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5074,"Index":64,"Attempt":0,"Launch Time":1427397599667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599675,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340004,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5080,"Index":70,"Attempt":0,"Launch Time":1427397599677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5077,"Index":67,"Attempt":0,"Launch Time":1427397599670,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":259332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5081,"Index":71,"Attempt":0,"Launch Time":1427397599679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5075,"Index":65,"Attempt":0,"Launch Time":1427397599668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599679,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5082,"Index":72,"Attempt":0,"Launch Time":1427397599681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5073,"Index":63,"Attempt":0,"Launch Time":1427397599666,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5083,"Index":73,"Attempt":0,"Launch Time":1427397599683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599690,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5079,"Index":69,"Attempt":0,"Launch Time":1427397599675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":488890,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5084,"Index":74,"Attempt":0,"Launch Time":1427397599685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5078,"Index":68,"Attempt":0,"Launch Time":1427397599673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599686,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303759,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5085,"Index":75,"Attempt":0,"Launch Time":1427397599687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599694,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5076,"Index":66,"Attempt":0,"Launch Time":1427397599669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1052247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5086,"Index":76,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5083,"Index":73,"Attempt":0,"Launch Time":1427397599683,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5087,"Index":77,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599697,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5058,"Index":48,"Attempt":0,"Launch Time":1427397599640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":10442141,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5088,"Index":78,"Attempt":0,"Launch Time":1427397599694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5085,"Index":75,"Attempt":0,"Launch Time":1427397599687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5089,"Index":79,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599706,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5082,"Index":72,"Attempt":0,"Launch Time":1427397599681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5090,"Index":80,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599703,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5080,"Index":70,"Attempt":0,"Launch Time":1427397599677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":704043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5091,"Index":81,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5084,"Index":74,"Attempt":0,"Launch Time":1427397599685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406415,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5092,"Index":82,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599707,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5081,"Index":71,"Attempt":0,"Launch Time":1427397599679,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599696,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":440434,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5093,"Index":83,"Attempt":0,"Launch Time":1427397599697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599705,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5087,"Index":77,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5094,"Index":84,"Attempt":0,"Launch Time":1427397599703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5090,"Index":80,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5095,"Index":85,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5091,"Index":81,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314890,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5096,"Index":86,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5088,"Index":78,"Attempt":0,"Launch Time":1427397599694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599704,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":446729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5097,"Index":87,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599718,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5093,"Index":83,"Attempt":0,"Launch Time":1427397599697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":428985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5098,"Index":88,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5089,"Index":79,"Attempt":0,"Launch Time":1427397599695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":604707,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5099,"Index":89,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5092,"Index":82,"Attempt":0,"Launch Time":1427397599696,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":382178,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5100,"Index":90,"Attempt":0,"Launch Time":1427397599718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5097,"Index":87,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599718,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5101,"Index":91,"Attempt":0,"Launch Time":1427397599724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5096,"Index":86,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599724,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321469,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5102,"Index":92,"Attempt":0,"Launch Time":1427397599727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5054,"Index":44,"Attempt":0,"Launch Time":1427397599616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":110,"Result Size":930,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5103,"Index":93,"Attempt":0,"Launch Time":1427397599729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5100,"Index":90,"Attempt":0,"Launch Time":1427397599718,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307060,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5104,"Index":94,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5098,"Index":88,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":345687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5105,"Index":95,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5099,"Index":89,"Attempt":0,"Launch Time":1427397599706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599733,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":517460,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5106,"Index":96,"Attempt":0,"Launch Time":1427397599735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5094,"Index":84,"Attempt":0,"Launch Time":1427397599703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5107,"Index":97,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5102,"Index":92,"Attempt":0,"Launch Time":1427397599727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5108,"Index":98,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5103,"Index":93,"Attempt":0,"Launch Time":1427397599729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318561,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5109,"Index":99,"Attempt":0,"Launch Time":1427397599738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5101,"Index":91,"Attempt":0,"Launch Time":1427397599724,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1663815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5110,"Index":100,"Attempt":0,"Launch Time":1427397599739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5095,"Index":85,"Attempt":0,"Launch Time":1427397599704,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":574348,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5111,"Index":101,"Attempt":0,"Launch Time":1427397599740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5104,"Index":94,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599740,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327802,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5112,"Index":102,"Attempt":0,"Launch Time":1427397599743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5107,"Index":97,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333263,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5113,"Index":103,"Attempt":0,"Launch Time":1427397599744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5108,"Index":98,"Attempt":0,"Launch Time":1427397599736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361129,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5114,"Index":104,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5106,"Index":96,"Attempt":0,"Launch Time":1427397599735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322313,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5115,"Index":105,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5105,"Index":95,"Attempt":0,"Launch Time":1427397599733,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":711638,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5116,"Index":106,"Attempt":0,"Launch Time":1427397599747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5109,"Index":99,"Attempt":0,"Launch Time":1427397599738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599747,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346158,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5117,"Index":107,"Attempt":0,"Launch Time":1427397599749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5111,"Index":101,"Attempt":0,"Launch Time":1427397599740,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":492381,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5118,"Index":108,"Attempt":0,"Launch Time":1427397599752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5113,"Index":103,"Attempt":0,"Launch Time":1427397599744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":457505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5119,"Index":109,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5112,"Index":102,"Attempt":0,"Launch Time":1427397599743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":581901,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5120,"Index":110,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5110,"Index":100,"Attempt":0,"Launch Time":1427397599739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599753,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":848349,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5121,"Index":111,"Attempt":0,"Launch Time":1427397599755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5114,"Index":104,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599755,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357847,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5122,"Index":112,"Attempt":0,"Launch Time":1427397599756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5123,"Index":113,"Attempt":0,"Launch Time":1427397599757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5117,"Index":107,"Attempt":0,"Launch Time":1427397599749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":409267,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5115,"Index":105,"Attempt":0,"Launch Time":1427397599746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5124,"Index":114,"Attempt":0,"Launch Time":1427397599759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5116,"Index":106,"Attempt":0,"Launch Time":1427397599747,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320162,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5125,"Index":115,"Attempt":0,"Launch Time":1427397599762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5120,"Index":110,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318743,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5126,"Index":116,"Attempt":0,"Launch Time":1427397599763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5119,"Index":109,"Attempt":0,"Launch Time":1427397599753,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599763,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301181,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5127,"Index":117,"Attempt":0,"Launch Time":1427397599769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5118,"Index":108,"Attempt":0,"Launch Time":1427397599752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5128,"Index":118,"Attempt":0,"Launch Time":1427397599770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5125,"Index":115,"Attempt":0,"Launch Time":1427397599762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":505917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5129,"Index":119,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5086,"Index":76,"Attempt":0,"Launch Time":1427397599690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":75,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5130,"Index":120,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5121,"Index":111,"Attempt":0,"Launch Time":1427397599755,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5131,"Index":121,"Attempt":0,"Launch Time":1427397599774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5126,"Index":116,"Attempt":0,"Launch Time":1427397599763,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":575200,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5132,"Index":122,"Attempt":0,"Launch Time":1427397599775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5127,"Index":117,"Attempt":0,"Launch Time":1427397599769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599775,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286242,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5133,"Index":123,"Attempt":0,"Launch Time":1427397599777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5129,"Index":119,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299613,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5134,"Index":124,"Attempt":0,"Launch Time":1427397599782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5133,"Index":123,"Attempt":0,"Launch Time":1427397599777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599783,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":290953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5135,"Index":125,"Attempt":0,"Launch Time":1427397599786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5131,"Index":121,"Attempt":0,"Launch Time":1427397599774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599786,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":329705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5136,"Index":126,"Attempt":0,"Launch Time":1427397599788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5134,"Index":124,"Attempt":0,"Launch Time":1427397599782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279741,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5137,"Index":127,"Attempt":0,"Launch Time":1427397599792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5130,"Index":120,"Attempt":0,"Launch Time":1427397599771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":302567,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5138,"Index":128,"Attempt":0,"Launch Time":1427397599793,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5135,"Index":125,"Attempt":0,"Launch Time":1427397599786,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599793,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5139,"Index":129,"Attempt":0,"Launch Time":1427397599794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5132,"Index":122,"Attempt":0,"Launch Time":1427397599775,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303257,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5140,"Index":130,"Attempt":0,"Launch Time":1427397599795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5136,"Index":126,"Attempt":0,"Launch Time":1427397599788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":290375,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5141,"Index":131,"Attempt":0,"Launch Time":1427397599800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5138,"Index":128,"Attempt":0,"Launch Time":1427397599793,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599801,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5142,"Index":132,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5140,"Index":130,"Attempt":0,"Launch Time":1427397599795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332673,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5143,"Index":133,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5137,"Index":127,"Attempt":0,"Launch Time":1427397599792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":341526,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5144,"Index":134,"Attempt":0,"Launch Time":1427397599805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5139,"Index":129,"Attempt":0,"Launch Time":1427397599794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599806,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348599,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5145,"Index":135,"Attempt":0,"Launch Time":1427397599813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5122,"Index":112,"Attempt":0,"Launch Time":1427397599756,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599813,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5146,"Index":136,"Attempt":0,"Launch Time":1427397599815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5142,"Index":132,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599815,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5147,"Index":137,"Attempt":0,"Launch Time":1427397599816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5143,"Index":133,"Attempt":0,"Launch Time":1427397599803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323115,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5148,"Index":138,"Attempt":0,"Launch Time":1427397599817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5123,"Index":113,"Attempt":0,"Launch Time":1427397599757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":770436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5149,"Index":139,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5145,"Index":135,"Attempt":0,"Launch Time":1427397599813,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":416045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5150,"Index":140,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5146,"Index":136,"Attempt":0,"Launch Time":1427397599815,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599822,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335994,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5151,"Index":141,"Attempt":0,"Launch Time":1427397599823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5147,"Index":137,"Attempt":0,"Launch Time":1427397599816,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342243,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5152,"Index":142,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5149,"Index":139,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5153,"Index":143,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5150,"Index":140,"Attempt":0,"Launch Time":1427397599822,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9606,"Shuffle Write Time":289035,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5154,"Index":144,"Attempt":0,"Launch Time":1427397599835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5155,"Index":145,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5156,"Index":146,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5141,"Index":131,"Attempt":0,"Launch Time":1427397599800,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":22680854,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5148,"Index":138,"Attempt":0,"Launch Time":1427397599817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297531,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5157,"Index":147,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5158,"Index":148,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5152,"Index":142,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5151,"Index":141,"Attempt":0,"Launch Time":1427397599823,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":283701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5159,"Index":149,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5153,"Index":143,"Attempt":0,"Launch Time":1427397599828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599836,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5124,"Index":114,"Attempt":0,"Launch Time":1427397599759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":67,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5160,"Index":150,"Attempt":0,"Launch Time":1427397599837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5144,"Index":134,"Attempt":0,"Launch Time":1427397599805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599837,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":484934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5161,"Index":151,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5154,"Index":144,"Attempt":0,"Launch Time":1427397599835,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5162,"Index":152,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5159,"Index":149,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599844,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334249,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5163,"Index":153,"Attempt":0,"Launch Time":1427397599845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5158,"Index":148,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599845,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":346497,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5164,"Index":154,"Attempt":0,"Launch Time":1427397599846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5155,"Index":145,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":323539,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5165,"Index":155,"Attempt":0,"Launch Time":1427397599852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5163,"Index":153,"Attempt":0,"Launch Time":1427397599845,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599852,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5166,"Index":156,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5161,"Index":151,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386707,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5167,"Index":157,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5162,"Index":152,"Attempt":0,"Launch Time":1427397599844,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599853,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":391681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5168,"Index":158,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5169,"Index":159,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5164,"Index":154,"Attempt":0,"Launch Time":1427397599846,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":374866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5156,"Index":146,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599856,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":494644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5170,"Index":160,"Attempt":0,"Launch Time":1427397599858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5160,"Index":150,"Attempt":0,"Launch Time":1427397599837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599858,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":350706,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5171,"Index":161,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5166,"Index":156,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387185,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5172,"Index":162,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5167,"Index":157,"Attempt":0,"Launch Time":1427397599853,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5173,"Index":163,"Attempt":0,"Launch Time":1427397599861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5165,"Index":155,"Attempt":0,"Launch Time":1427397599852,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599861,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":442425,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5174,"Index":164,"Attempt":0,"Launch Time":1427397599862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5168,"Index":158,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5175,"Index":165,"Attempt":0,"Launch Time":1427397599864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5157,"Index":147,"Attempt":0,"Launch Time":1427397599836,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599864,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5176,"Index":166,"Attempt":0,"Launch Time":1427397599865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5169,"Index":159,"Attempt":0,"Launch Time":1427397599856,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599865,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":300784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5177,"Index":167,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5172,"Index":162,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599866,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5178,"Index":168,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5179,"Index":169,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5171,"Index":161,"Attempt":0,"Launch Time":1427397599860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5173,"Index":163,"Attempt":0,"Launch Time":1427397599861,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599867,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5180,"Index":170,"Attempt":0,"Launch Time":1427397599868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5174,"Index":164,"Attempt":0,"Launch Time":1427397599862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599868,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":290102,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5181,"Index":171,"Attempt":0,"Launch Time":1427397599869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5175,"Index":165,"Attempt":0,"Launch Time":1427397599864,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":257298,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5182,"Index":172,"Attempt":0,"Launch Time":1427397599873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5178,"Index":168,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599873,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347702,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5183,"Index":173,"Attempt":0,"Launch Time":1427397599874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5177,"Index":167,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599874,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":384686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5184,"Index":174,"Attempt":0,"Launch Time":1427397599876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5176,"Index":166,"Attempt":0,"Launch Time":1427397599865,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599876,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1597738,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5185,"Index":175,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5182,"Index":172,"Attempt":0,"Launch Time":1427397599873,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5186,"Index":176,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5183,"Index":173,"Attempt":0,"Launch Time":1427397599874,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599880,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":272591,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5187,"Index":177,"Attempt":0,"Launch Time":1427397599881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5181,"Index":171,"Attempt":0,"Launch Time":1427397599869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599881,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":248233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5188,"Index":178,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5189,"Index":179,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5184,"Index":174,"Attempt":0,"Launch Time":1427397599876,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":258902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5170,"Index":160,"Attempt":0,"Launch Time":1427397599858,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1337090,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5190,"Index":180,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5180,"Index":170,"Attempt":0,"Launch Time":1427397599868,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599883,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328975,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5191,"Index":181,"Attempt":0,"Launch Time":1427397599885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5185,"Index":175,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599886,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":285928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5192,"Index":182,"Attempt":0,"Launch Time":1427397599889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5189,"Index":179,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337649,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5193,"Index":183,"Attempt":0,"Launch Time":1427397599890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5187,"Index":177,"Attempt":0,"Launch Time":1427397599881,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599890,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352526,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5194,"Index":184,"Attempt":0,"Launch Time":1427397599903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5128,"Index":118,"Attempt":0,"Launch Time":1427397599770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":32,"Executor Run Time":58,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":405099,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5195,"Index":185,"Attempt":0,"Launch Time":1427397599904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5190,"Index":180,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599904,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365683,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5196,"Index":186,"Attempt":0,"Launch Time":1427397599905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5193,"Index":183,"Attempt":0,"Launch Time":1427397599890,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":342432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5197,"Index":187,"Attempt":0,"Launch Time":1427397599910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5194,"Index":184,"Attempt":0,"Launch Time":1427397599903,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5198,"Index":188,"Attempt":0,"Launch Time":1427397599912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5195,"Index":185,"Attempt":0,"Launch Time":1427397599904,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":318573,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5199,"Index":189,"Attempt":0,"Launch Time":1427397599914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5196,"Index":186,"Attempt":0,"Launch Time":1427397599905,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5200,"Index":190,"Attempt":0,"Launch Time":1427397599916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5197,"Index":187,"Attempt":0,"Launch Time":1427397599910,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262527,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5201,"Index":191,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5198,"Index":188,"Attempt":0,"Launch Time":1427397599912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599919,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":296255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5202,"Index":192,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5188,"Index":178,"Attempt":0,"Launch Time":1427397599883,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599920,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":437603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5203,"Index":193,"Attempt":0,"Launch Time":1427397599921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5199,"Index":189,"Attempt":0,"Launch Time":1427397599914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599921,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307266,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5204,"Index":194,"Attempt":0,"Launch Time":1427397599924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5200,"Index":190,"Attempt":0,"Launch Time":1427397599916,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599924,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307790,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5205,"Index":195,"Attempt":0,"Launch Time":1427397599925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5201,"Index":191,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599925,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":520614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5206,"Index":196,"Attempt":0,"Launch Time":1427397599928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5202,"Index":192,"Attempt":0,"Launch Time":1427397599919,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599928,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306660,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5207,"Index":197,"Attempt":0,"Launch Time":1427397599929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5203,"Index":193,"Attempt":0,"Launch Time":1427397599921,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599929,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":291512,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5208,"Index":198,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5204,"Index":194,"Attempt":0,"Launch Time":1427397599924,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290751,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5209,"Index":199,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5205,"Index":195,"Attempt":0,"Launch Time":1427397599925,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":291081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5210,"Index":200,"Attempt":0,"Launch Time":1427397599935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5207,"Index":197,"Attempt":0,"Launch Time":1427397599929,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599936,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":479914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5211,"Index":201,"Attempt":0,"Launch Time":1427397599937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5208,"Index":198,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":289331,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5212,"Index":202,"Attempt":0,"Launch Time":1427397599941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5209,"Index":199,"Attempt":0,"Launch Time":1427397599931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319155,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5213,"Index":203,"Attempt":0,"Launch Time":1427397599943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5210,"Index":200,"Attempt":0,"Launch Time":1427397599935,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599943,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":304125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5214,"Index":204,"Attempt":0,"Launch Time":1427397599944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5211,"Index":201,"Attempt":0,"Launch Time":1427397599937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5215,"Index":205,"Attempt":0,"Launch Time":1427397599947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5212,"Index":202,"Attempt":0,"Launch Time":1427397599941,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":363297,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5216,"Index":206,"Attempt":0,"Launch Time":1427397599950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5213,"Index":203,"Attempt":0,"Launch Time":1427397599943,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599950,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":339923,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5217,"Index":207,"Attempt":0,"Launch Time":1427397599951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5214,"Index":204,"Attempt":0,"Launch Time":1427397599944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599951,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5218,"Index":208,"Attempt":0,"Launch Time":1427397599955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5215,"Index":205,"Attempt":0,"Launch Time":1427397599947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599955,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5219,"Index":209,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5216,"Index":206,"Attempt":0,"Launch Time":1427397599950,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310317,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5220,"Index":210,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5217,"Index":207,"Attempt":0,"Launch Time":1427397599951,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599958,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":356003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5221,"Index":211,"Attempt":0,"Launch Time":1427397599963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5218,"Index":208,"Attempt":0,"Launch Time":1427397599955,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":280371,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5222,"Index":212,"Attempt":0,"Launch Time":1427397599964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5219,"Index":209,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323104,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5223,"Index":213,"Attempt":0,"Launch Time":1427397599965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5220,"Index":210,"Attempt":0,"Launch Time":1427397599958,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599965,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293165,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5224,"Index":214,"Attempt":0,"Launch Time":1427397599968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5192,"Index":182,"Attempt":0,"Launch Time":1427397599889,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599968,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":78,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321195,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5225,"Index":215,"Attempt":0,"Launch Time":1427397599971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5221,"Index":211,"Attempt":0,"Launch Time":1427397599963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369873,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5226,"Index":216,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5223,"Index":213,"Attempt":0,"Launch Time":1427397599965,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":308723,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5227,"Index":217,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5222,"Index":212,"Attempt":0,"Launch Time":1427397599964,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5228,"Index":218,"Attempt":0,"Launch Time":1427397599974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5224,"Index":214,"Attempt":0,"Launch Time":1427397599968,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599974,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5229,"Index":219,"Attempt":0,"Launch Time":1427397599976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5191,"Index":181,"Attempt":0,"Launch Time":1427397599885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599976,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":90,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":344199,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5230,"Index":220,"Attempt":0,"Launch Time":1427397599981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5179,"Index":169,"Attempt":0,"Launch Time":1427397599866,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599981,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":114,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":276876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5231,"Index":221,"Attempt":0,"Launch Time":1427397599983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5226,"Index":216,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":290879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5232,"Index":222,"Attempt":0,"Launch Time":1427397599985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5225,"Index":215,"Attempt":0,"Launch Time":1427397599971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":317210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5233,"Index":223,"Attempt":0,"Launch Time":1427397599986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5227,"Index":217,"Attempt":0,"Launch Time":1427397599972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":299488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5234,"Index":224,"Attempt":0,"Launch Time":1427397599989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5206,"Index":196,"Attempt":0,"Launch Time":1427397599928,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5235,"Index":225,"Attempt":0,"Launch Time":1427397599990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5231,"Index":221,"Attempt":0,"Launch Time":1427397599983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397599990,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311192,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5236,"Index":226,"Attempt":0,"Launch Time":1427397600003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5232,"Index":222,"Attempt":0,"Launch Time":1427397599985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":623303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5237,"Index":227,"Attempt":0,"Launch Time":1427397600006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5233,"Index":223,"Attempt":0,"Launch Time":1427397599986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1216413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5238,"Index":228,"Attempt":0,"Launch Time":1427397600010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5235,"Index":225,"Attempt":0,"Launch Time":1427397599990,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327138,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5239,"Index":229,"Attempt":0,"Launch Time":1427397600011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5234,"Index":224,"Attempt":0,"Launch Time":1427397599989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":4753634,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5240,"Index":230,"Attempt":0,"Launch Time":1427397600012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5236,"Index":226,"Attempt":0,"Launch Time":1427397600003,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":285032,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5241,"Index":231,"Attempt":0,"Launch Time":1427397600014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5237,"Index":227,"Attempt":0,"Launch Time":1427397600006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600014,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5242,"Index":232,"Attempt":0,"Launch Time":1427397600016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5230,"Index":220,"Attempt":0,"Launch Time":1427397599981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600017,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":452882,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5243,"Index":233,"Attempt":0,"Launch Time":1427397600018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5238,"Index":228,"Attempt":0,"Launch Time":1427397600010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":456877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5239,"Index":229,"Attempt":0,"Launch Time":1427397600011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":379190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5244,"Index":234,"Attempt":0,"Launch Time":1427397600020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5245,"Index":235,"Attempt":0,"Launch Time":1427397600021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5240,"Index":230,"Attempt":0,"Launch Time":1427397600012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5246,"Index":236,"Attempt":0,"Launch Time":1427397600022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5241,"Index":231,"Attempt":0,"Launch Time":1427397600014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600022,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5247,"Index":237,"Attempt":0,"Launch Time":1427397600023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5242,"Index":232,"Attempt":0,"Launch Time":1427397600016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600023,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301727,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5248,"Index":238,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5249,"Index":239,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5186,"Index":176,"Attempt":0,"Launch Time":1427397599880,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":147,"Result Size":930,"JVM GC Time":12,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317390,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5229,"Index":219,"Attempt":0,"Launch Time":1427397599976,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600027,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":258967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5250,"Index":240,"Attempt":0,"Launch Time":1427397600028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5228,"Index":218,"Attempt":0,"Launch Time":1427397599974,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600029,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":6449843,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5251,"Index":241,"Attempt":0,"Launch Time":1427397600033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5247,"Index":237,"Attempt":0,"Launch Time":1427397600023,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600033,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":333224,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5252,"Index":242,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5248,"Index":238,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340833,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5253,"Index":243,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5249,"Index":239,"Attempt":0,"Launch Time":1427397600027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600034,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":301221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5254,"Index":244,"Attempt":0,"Launch Time":1427397600039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5250,"Index":240,"Attempt":0,"Launch Time":1427397600028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600039,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":297767,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5255,"Index":245,"Attempt":0,"Launch Time":1427397600042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5245,"Index":235,"Attempt":0,"Launch Time":1427397600021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337330,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5256,"Index":246,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5253,"Index":243,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5257,"Index":247,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5244,"Index":234,"Attempt":0,"Launch Time":1427397600020,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600044,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1397633,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5258,"Index":248,"Attempt":0,"Launch Time":1427397600045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5252,"Index":242,"Attempt":0,"Launch Time":1427397600034,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5259,"Index":249,"Attempt":0,"Launch Time":1427397600047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5251,"Index":241,"Attempt":0,"Launch Time":1427397600033,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":310857,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5260,"Index":250,"Attempt":0,"Launch Time":1427397600048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5254,"Index":244,"Attempt":0,"Launch Time":1427397600039,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368038,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5261,"Index":251,"Attempt":0,"Launch Time":1427397600051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5257,"Index":247,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600052,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":367849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5262,"Index":252,"Attempt":0,"Launch Time":1427397600052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5256,"Index":246,"Attempt":0,"Launch Time":1427397600044,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600053,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361785,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5263,"Index":253,"Attempt":0,"Launch Time":1427397600056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5258,"Index":248,"Attempt":0,"Launch Time":1427397600045,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":523419,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5264,"Index":254,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5259,"Index":249,"Attempt":0,"Launch Time":1427397600047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":593172,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5265,"Index":255,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5260,"Index":250,"Attempt":0,"Launch Time":1427397600048,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600058,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":353479,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5266,"Index":256,"Attempt":0,"Launch Time":1427397600060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5261,"Index":251,"Attempt":0,"Launch Time":1427397600051,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":446753,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5267,"Index":257,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5263,"Index":253,"Attempt":0,"Launch Time":1427397600056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":362429,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5268,"Index":258,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5262,"Index":252,"Attempt":0,"Launch Time":1427397600052,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600064,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":308408,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5269,"Index":259,"Attempt":0,"Launch Time":1427397600065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5265,"Index":255,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600065,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5270,"Index":260,"Attempt":0,"Launch Time":1427397600066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5266,"Index":256,"Attempt":0,"Launch Time":1427397600060,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600067,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289866,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5271,"Index":261,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5267,"Index":257,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":277152,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5272,"Index":262,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5268,"Index":258,"Attempt":0,"Launch Time":1427397600064,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301405,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5273,"Index":263,"Attempt":0,"Launch Time":1427397600071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5269,"Index":259,"Attempt":0,"Launch Time":1427397600065,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600071,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":346844,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5274,"Index":264,"Attempt":0,"Launch Time":1427397600075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5270,"Index":260,"Attempt":0,"Launch Time":1427397600066,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":507880,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5275,"Index":265,"Attempt":0,"Launch Time":1427397600076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5271,"Index":261,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600076,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":260991,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5276,"Index":266,"Attempt":0,"Launch Time":1427397600077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5273,"Index":263,"Attempt":0,"Launch Time":1427397600071,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600077,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":297896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5277,"Index":267,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5272,"Index":262,"Attempt":0,"Launch Time":1427397600070,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301116,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5278,"Index":268,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5264,"Index":254,"Attempt":0,"Launch Time":1427397600058,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600078,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":8980526,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5279,"Index":269,"Attempt":0,"Launch Time":1427397600081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5274,"Index":264,"Attempt":0,"Launch Time":1427397600075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600081,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287842,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5280,"Index":270,"Attempt":0,"Launch Time":1427397600083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5275,"Index":265,"Attempt":0,"Launch Time":1427397600076,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600083,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":304906,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5281,"Index":271,"Attempt":0,"Launch Time":1427397600084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5276,"Index":266,"Attempt":0,"Launch Time":1427397600077,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5282,"Index":272,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5283,"Index":273,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5246,"Index":236,"Attempt":0,"Launch Time":1427397600022,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5279,"Index":269,"Attempt":0,"Launch Time":1427397600081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":284270,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5284,"Index":274,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5280,"Index":270,"Attempt":0,"Launch Time":1427397600083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600096,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312412,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5285,"Index":275,"Attempt":0,"Launch Time":1427397600099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5278,"Index":268,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600099,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":20,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":11663284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5286,"Index":276,"Attempt":0,"Launch Time":1427397600101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5277,"Index":267,"Attempt":0,"Launch Time":1427397600078,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":345466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5287,"Index":277,"Attempt":0,"Launch Time":1427397600102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5281,"Index":271,"Attempt":0,"Launch Time":1427397600084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600102,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344209,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5288,"Index":278,"Attempt":0,"Launch Time":1427397600103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5283,"Index":273,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600103,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":335820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5289,"Index":279,"Attempt":0,"Launch Time":1427397600105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5284,"Index":274,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600105,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":394971,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5290,"Index":280,"Attempt":0,"Launch Time":1427397600107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5285,"Index":275,"Attempt":0,"Launch Time":1427397600099,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":350171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5291,"Index":281,"Attempt":0,"Launch Time":1427397600109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5286,"Index":276,"Attempt":0,"Launch Time":1427397600101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343340,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5292,"Index":282,"Attempt":0,"Launch Time":1427397600110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5288,"Index":278,"Attempt":0,"Launch Time":1427397600103,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5293,"Index":283,"Attempt":0,"Launch Time":1427397600111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5287,"Index":277,"Attempt":0,"Launch Time":1427397600102,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":319817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5294,"Index":284,"Attempt":0,"Launch Time":1427397600117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5292,"Index":282,"Attempt":0,"Launch Time":1427397600110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":293170,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5295,"Index":285,"Attempt":0,"Launch Time":1427397600120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5290,"Index":280,"Attempt":0,"Launch Time":1427397600107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600120,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":328056,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5296,"Index":286,"Attempt":0,"Launch Time":1427397600121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5291,"Index":281,"Attempt":0,"Launch Time":1427397600109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600122,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334607,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5297,"Index":287,"Attempt":0,"Launch Time":1427397600123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5294,"Index":284,"Attempt":0,"Launch Time":1427397600117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318380,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5298,"Index":288,"Attempt":0,"Launch Time":1427397600124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5289,"Index":279,"Attempt":0,"Launch Time":1427397600105,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5299,"Index":289,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5300,"Index":290,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5295,"Index":285,"Attempt":0,"Launch Time":1427397600120,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331174,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5296,"Index":286,"Attempt":0,"Launch Time":1427397600121,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":307354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5301,"Index":291,"Attempt":0,"Launch Time":1427397600130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5297,"Index":287,"Attempt":0,"Launch Time":1427397600123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":292616,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5302,"Index":292,"Attempt":0,"Launch Time":1427397600134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5255,"Index":245,"Attempt":0,"Launch Time":1427397600042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":334466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5303,"Index":293,"Attempt":0,"Launch Time":1427397600135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5298,"Index":288,"Attempt":0,"Launch Time":1427397600124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321344,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5304,"Index":294,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5300,"Index":290,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":348925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5305,"Index":295,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5301,"Index":291,"Attempt":0,"Launch Time":1427397600130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600136,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":321799,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5306,"Index":296,"Attempt":0,"Launch Time":1427397600141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5293,"Index":283,"Attempt":0,"Launch Time":1427397600111,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":355778,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5307,"Index":297,"Attempt":0,"Launch Time":1427397600142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5304,"Index":294,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5308,"Index":298,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5305,"Index":295,"Attempt":0,"Launch Time":1427397600136,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293874,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5309,"Index":299,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5303,"Index":293,"Attempt":0,"Launch Time":1427397600135,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600143,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":313339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5310,"Index":300,"Attempt":0,"Launch Time":1427397600147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5306,"Index":296,"Attempt":0,"Launch Time":1427397600141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":294088,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5311,"Index":301,"Attempt":0,"Launch Time":1427397600148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5243,"Index":233,"Attempt":0,"Launch Time":1427397600018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":113,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":12003053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5312,"Index":302,"Attempt":0,"Launch Time":1427397600153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5282,"Index":272,"Attempt":0,"Launch Time":1427397600096,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600153,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":47,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5313,"Index":303,"Attempt":0,"Launch Time":1427397600154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5310,"Index":300,"Attempt":0,"Launch Time":1427397600147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":369387,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5314,"Index":304,"Attempt":0,"Launch Time":1427397600156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5311,"Index":301,"Attempt":0,"Launch Time":1427397600148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600156,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358154,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5315,"Index":305,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5313,"Index":303,"Attempt":0,"Launch Time":1427397600154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600161,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5316,"Index":306,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5312,"Index":302,"Attempt":0,"Launch Time":1427397600153,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600162,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5317,"Index":307,"Attempt":0,"Launch Time":1427397600163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5314,"Index":304,"Attempt":0,"Launch Time":1427397600156,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":322237,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5318,"Index":308,"Attempt":0,"Launch Time":1427397600167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5315,"Index":305,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5319,"Index":309,"Attempt":0,"Launch Time":1427397600169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5316,"Index":306,"Attempt":0,"Launch Time":1427397600161,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600169,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":375832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5320,"Index":310,"Attempt":0,"Launch Time":1427397600172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5317,"Index":307,"Attempt":0,"Launch Time":1427397600163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600172,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352523,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5321,"Index":311,"Attempt":0,"Launch Time":1427397600174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5318,"Index":308,"Attempt":0,"Launch Time":1427397600167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600174,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":351413,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5322,"Index":312,"Attempt":0,"Launch Time":1427397600175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5299,"Index":289,"Attempt":0,"Launch Time":1427397600128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600175,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":383606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5323,"Index":313,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5319,"Index":309,"Attempt":0,"Launch Time":1427397600169,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":383568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5324,"Index":314,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5307,"Index":297,"Attempt":0,"Launch Time":1427397600142,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600176,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347123,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5325,"Index":315,"Attempt":0,"Launch Time":1427397600181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5302,"Index":292,"Attempt":0,"Launch Time":1427397600134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5326,"Index":316,"Attempt":0,"Launch Time":1427397600193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5321,"Index":311,"Attempt":0,"Launch Time":1427397600174,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":19,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309686,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5327,"Index":317,"Attempt":0,"Launch Time":1427397600194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5308,"Index":298,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600194,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":51,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5328,"Index":318,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5326,"Index":316,"Attempt":0,"Launch Time":1427397600193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":449179,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5329,"Index":319,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5327,"Index":317,"Attempt":0,"Launch Time":1427397600194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600200,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310490,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5330,"Index":320,"Attempt":0,"Launch Time":1427397600207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5320,"Index":310,"Attempt":0,"Launch Time":1427397600172,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600207,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":409070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5331,"Index":321,"Attempt":0,"Launch Time":1427397600208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5329,"Index":319,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":417127,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5332,"Index":322,"Attempt":0,"Launch Time":1427397600209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5328,"Index":318,"Attempt":0,"Launch Time":1427397600200,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":339926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5333,"Index":323,"Attempt":0,"Launch Time":1427397600211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5324,"Index":314,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":4542501,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5334,"Index":324,"Attempt":0,"Launch Time":1427397600214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5331,"Index":321,"Attempt":0,"Launch Time":1427397600208,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313699,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5335,"Index":325,"Attempt":0,"Launch Time":1427397600215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5330,"Index":320,"Attempt":0,"Launch Time":1427397600207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":398043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5336,"Index":326,"Attempt":0,"Launch Time":1427397600218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5333,"Index":323,"Attempt":0,"Launch Time":1427397600211,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600219,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1186835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5337,"Index":327,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5334,"Index":324,"Attempt":0,"Launch Time":1427397600214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":298729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5338,"Index":328,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5325,"Index":315,"Attempt":0,"Launch Time":1427397600181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600221,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5339,"Index":329,"Attempt":0,"Launch Time":1427397600222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5335,"Index":325,"Attempt":0,"Launch Time":1427397600215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":314750,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5340,"Index":330,"Attempt":0,"Launch Time":1427397600225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5336,"Index":326,"Attempt":0,"Launch Time":1427397600218,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600226,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":286947,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5341,"Index":331,"Attempt":0,"Launch Time":1427397600227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5337,"Index":327,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600227,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":370885,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5342,"Index":332,"Attempt":0,"Launch Time":1427397600234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5341,"Index":331,"Attempt":0,"Launch Time":1427397600227,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5343,"Index":333,"Attempt":0,"Launch Time":1427397600241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5342,"Index":332,"Attempt":0,"Launch Time":1427397600234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600241,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326409,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5344,"Index":334,"Attempt":0,"Launch Time":1427397600242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5340,"Index":330,"Attempt":0,"Launch Time":1427397600225,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600242,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":480980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5345,"Index":335,"Attempt":0,"Launch Time":1427397600245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5339,"Index":329,"Attempt":0,"Launch Time":1427397600222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600245,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317991,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5346,"Index":336,"Attempt":0,"Launch Time":1427397600246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5343,"Index":333,"Attempt":0,"Launch Time":1427397600241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600247,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":260259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5347,"Index":337,"Attempt":0,"Launch Time":1427397600249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5344,"Index":334,"Attempt":0,"Launch Time":1427397600242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600249,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9666,"Shuffle Write Time":327161,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5348,"Index":338,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5346,"Index":336,"Attempt":0,"Launch Time":1427397600246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9648,"Shuffle Write Time":291924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5349,"Index":339,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5345,"Index":335,"Attempt":0,"Launch Time":1427397600245,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9650,"Shuffle Write Time":315389,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5350,"Index":340,"Attempt":0,"Launch Time":1427397600266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5322,"Index":312,"Attempt":0,"Launch Time":1427397600175,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":568711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5351,"Index":341,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5332,"Index":322,"Attempt":0,"Launch Time":1427397600209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600270,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":53,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1664609,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5352,"Index":342,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5353,"Index":343,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5354,"Index":344,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5309,"Index":299,"Attempt":0,"Launch Time":1427397600143,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":52,"Executor Run Time":66,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5348,"Index":338,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600271,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":293283,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5355,"Index":345,"Attempt":0,"Launch Time":1427397600272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5338,"Index":328,"Attempt":0,"Launch Time":1427397600221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":472664,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5347,"Index":337,"Attempt":0,"Launch Time":1427397600249,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600272,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9651,"Shuffle Write Time":321184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5356,"Index":346,"Attempt":0,"Launch Time":1427397600279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5353,"Index":343,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600279,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":288908,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5357,"Index":347,"Attempt":0,"Launch Time":1427397600280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5351,"Index":341,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600280,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":332661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5358,"Index":348,"Attempt":0,"Launch Time":1427397600282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5355,"Index":345,"Attempt":0,"Launch Time":1427397600272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5359,"Index":349,"Attempt":0,"Launch Time":1427397600286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5356,"Index":346,"Attempt":0,"Launch Time":1427397600279,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600287,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5360,"Index":350,"Attempt":0,"Launch Time":1427397600288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5357,"Index":347,"Attempt":0,"Launch Time":1427397600280,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600288,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5361,"Index":351,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5323,"Index":313,"Attempt":0,"Launch Time":1427397600176,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600289,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":60,"Executor Run Time":32,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":3545023,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5362,"Index":352,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5358,"Index":348,"Attempt":0,"Launch Time":1427397600282,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332685,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5363,"Index":353,"Attempt":0,"Launch Time":1427397600294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5360,"Index":350,"Attempt":0,"Launch Time":1427397600288,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":343810,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5364,"Index":354,"Attempt":0,"Launch Time":1427397600295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5359,"Index":349,"Attempt":0,"Launch Time":1427397600286,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600295,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":350327,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5365,"Index":355,"Attempt":0,"Launch Time":1427397600296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5362,"Index":352,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600296,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299354,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5366,"Index":356,"Attempt":0,"Launch Time":1427397600297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5361,"Index":351,"Attempt":0,"Launch Time":1427397600289,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600297,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331606,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5367,"Index":357,"Attempt":0,"Launch Time":1427397600303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5365,"Index":355,"Attempt":0,"Launch Time":1427397600296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323827,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5368,"Index":358,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5369,"Index":359,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5370,"Index":360,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5371,"Index":361,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5366,"Index":356,"Attempt":0,"Launch Time":1427397600297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":323674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5364,"Index":354,"Attempt":0,"Launch Time":1427397600295,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":1111118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5363,"Index":353,"Attempt":0,"Launch Time":1427397600294,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":326137,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5352,"Index":342,"Attempt":0,"Launch Time":1427397600270,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":296595,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5372,"Index":362,"Attempt":0,"Launch Time":1427397600319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5368,"Index":358,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":337630,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5373,"Index":363,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5371,"Index":361,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":319045,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5374,"Index":364,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5369,"Index":359,"Attempt":0,"Launch Time":1427397600305,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600320,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":373076,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5375,"Index":365,"Attempt":0,"Launch Time":1427397600321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5349,"Index":339,"Attempt":0,"Launch Time":1427397600253,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600321,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":55,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9647,"Shuffle Write Time":623048,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5376,"Index":366,"Attempt":0,"Launch Time":1427397600323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5367,"Index":357,"Attempt":0,"Launch Time":1427397600303,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600323,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":331621,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5377,"Index":367,"Attempt":0,"Launch Time":1427397600326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5372,"Index":362,"Attempt":0,"Launch Time":1427397600319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600326,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307594,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5378,"Index":368,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5379,"Index":369,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5374,"Index":364,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":344447,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5373,"Index":363,"Attempt":0,"Launch Time":1427397600320,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":315318,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5380,"Index":370,"Attempt":0,"Launch Time":1427397600330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5370,"Index":360,"Attempt":0,"Launch Time":1427397600306,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600330,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":23,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1015849,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5381,"Index":371,"Attempt":0,"Launch Time":1427397600331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5376,"Index":366,"Attempt":0,"Launch Time":1427397600323,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600331,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":435803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5382,"Index":372,"Attempt":0,"Launch Time":1427397600333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5377,"Index":367,"Attempt":0,"Launch Time":1427397600326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600333,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":295740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5383,"Index":373,"Attempt":0,"Launch Time":1427397600335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5379,"Index":369,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296648,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5384,"Index":374,"Attempt":0,"Launch Time":1427397600336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5380,"Index":370,"Attempt":0,"Launch Time":1427397600330,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600336,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":308464,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5385,"Index":375,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5386,"Index":376,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5381,"Index":371,"Attempt":0,"Launch Time":1427397600331,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":313683,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5378,"Index":368,"Attempt":0,"Launch Time":1427397600327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":331629,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5387,"Index":377,"Attempt":0,"Launch Time":1427397600341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5382,"Index":372,"Attempt":0,"Launch Time":1427397600333,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299596,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5388,"Index":378,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5383,"Index":373,"Attempt":0,"Launch Time":1427397600335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":262532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5389,"Index":379,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5384,"Index":374,"Attempt":0,"Launch Time":1427397600336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":294948,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5390,"Index":380,"Attempt":0,"Launch Time":1427397600346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5386,"Index":376,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":355394,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5391,"Index":381,"Attempt":0,"Launch Time":1427397600348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5385,"Index":375,"Attempt":0,"Launch Time":1427397600338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600348,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":453212,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5392,"Index":382,"Attempt":0,"Launch Time":1427397600351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5389,"Index":379,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326506,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5393,"Index":383,"Attempt":0,"Launch Time":1427397600352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5388,"Index":378,"Attempt":0,"Launch Time":1427397600343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":403682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5394,"Index":384,"Attempt":0,"Launch Time":1427397600354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5390,"Index":380,"Attempt":0,"Launch Time":1427397600346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600354,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5395,"Index":385,"Attempt":0,"Launch Time":1427397600355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5387,"Index":377,"Attempt":0,"Launch Time":1427397600341,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600355,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":361570,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5396,"Index":386,"Attempt":0,"Launch Time":1427397600359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5392,"Index":382,"Attempt":0,"Launch Time":1427397600351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":318437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5397,"Index":387,"Attempt":0,"Launch Time":1427397600360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5393,"Index":383,"Attempt":0,"Launch Time":1427397600352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600360,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5398,"Index":388,"Attempt":0,"Launch Time":1427397600362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5375,"Index":365,"Attempt":0,"Launch Time":1427397600321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5399,"Index":389,"Attempt":0,"Launch Time":1427397600364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5391,"Index":381,"Attempt":0,"Launch Time":1427397600348,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":382391,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5400,"Index":390,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5394,"Index":384,"Attempt":0,"Launch Time":1427397600354,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600368,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5401,"Index":391,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5395,"Index":385,"Attempt":0,"Launch Time":1427397600355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":399273,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5402,"Index":392,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5403,"Index":393,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5397,"Index":387,"Attempt":0,"Launch Time":1427397600360,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":394729,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5396,"Index":386,"Attempt":0,"Launch Time":1427397600359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":944783,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5404,"Index":394,"Attempt":0,"Launch Time":1427397600372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5399,"Index":389,"Attempt":0,"Launch Time":1427397600364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600372,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":590343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5405,"Index":395,"Attempt":0,"Launch Time":1427397600376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5400,"Index":390,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600376,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":339710,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5406,"Index":396,"Attempt":0,"Launch Time":1427397600377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5401,"Index":391,"Attempt":0,"Launch Time":1427397600368,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":512986,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5407,"Index":397,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5403,"Index":393,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":303678,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5408,"Index":398,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5409,"Index":399,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5404,"Index":394,"Attempt":0,"Launch Time":1427397600372,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318530,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5402,"Index":392,"Attempt":0,"Launch Time":1427397600370,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600380,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309311,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5410,"Index":400,"Attempt":0,"Launch Time":1427397600384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5405,"Index":395,"Attempt":0,"Launch Time":1427397600376,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600384,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":289736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5411,"Index":401,"Attempt":0,"Launch Time":1427397600385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5406,"Index":396,"Attempt":0,"Launch Time":1427397600377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":360320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5412,"Index":402,"Attempt":0,"Launch Time":1427397600386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5407,"Index":397,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":288817,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5413,"Index":403,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5414,"Index":404,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5409,"Index":399,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":272356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5354,"Index":344,"Attempt":0,"Launch Time":1427397600271,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600387,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":17,"Executor Run Time":48,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1598398,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5415,"Index":405,"Attempt":0,"Launch Time":1427397600389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5408,"Index":398,"Attempt":0,"Launch Time":1427397600379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600390,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":292343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5416,"Index":406,"Attempt":0,"Launch Time":1427397600392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5410,"Index":400,"Attempt":0,"Launch Time":1427397600384,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600392,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5417,"Index":407,"Attempt":0,"Launch Time":1427397600407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5398,"Index":388,"Attempt":0,"Launch Time":1427397600362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600407,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":29,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":305432,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5418,"Index":408,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5350,"Index":340,"Attempt":0,"Launch Time":1427397600266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":83,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9649,"Shuffle Write Time":347891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5419,"Index":409,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5413,"Index":403,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600411,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":7,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317162,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5420,"Index":410,"Attempt":0,"Launch Time":1427397600417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5417,"Index":407,"Attempt":0,"Launch Time":1427397600407,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600417,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":388575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5421,"Index":411,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5419,"Index":409,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600418,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":360589,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5422,"Index":412,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5418,"Index":408,"Attempt":0,"Launch Time":1427397600410,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600419,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":532993,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5423,"Index":413,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5422,"Index":412,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317068,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5424,"Index":414,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5420,"Index":410,"Attempt":0,"Launch Time":1427397600417,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600426,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":296431,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5425,"Index":415,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5421,"Index":411,"Attempt":0,"Launch Time":1427397600418,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600427,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5426,"Index":416,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5423,"Index":413,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5427,"Index":417,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5424,"Index":414,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":286302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5428,"Index":418,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5425,"Index":415,"Attempt":0,"Launch Time":1427397600426,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600433,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":281605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5429,"Index":419,"Attempt":0,"Launch Time":1427397600439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5426,"Index":416,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600439,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":323159,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5430,"Index":420,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5427,"Index":417,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":340210,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5431,"Index":421,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5428,"Index":418,"Attempt":0,"Launch Time":1427397600433,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":423926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5432,"Index":422,"Attempt":0,"Launch Time":1427397600445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5429,"Index":419,"Attempt":0,"Launch Time":1427397600439,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":309757,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5433,"Index":423,"Attempt":0,"Launch Time":1427397600446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5430,"Index":420,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329550,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5434,"Index":424,"Attempt":0,"Launch Time":1427397600447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5431,"Index":421,"Attempt":0,"Launch Time":1427397600440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600447,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":332303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5435,"Index":425,"Attempt":0,"Launch Time":1427397600452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5432,"Index":422,"Attempt":0,"Launch Time":1427397600445,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600452,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309644,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5436,"Index":426,"Attempt":0,"Launch Time":1427397600453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5433,"Index":423,"Attempt":0,"Launch Time":1427397600446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":327459,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5437,"Index":427,"Attempt":0,"Launch Time":1427397600454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5434,"Index":424,"Attempt":0,"Launch Time":1427397600447,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":326269,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5438,"Index":428,"Attempt":0,"Launch Time":1427397600459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5435,"Index":425,"Attempt":0,"Launch Time":1427397600452,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600459,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309488,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5439,"Index":429,"Attempt":0,"Launch Time":1427397600460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5436,"Index":426,"Attempt":0,"Launch Time":1427397600453,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600460,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320789,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5440,"Index":430,"Attempt":0,"Launch Time":1427397600461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5437,"Index":427,"Attempt":0,"Launch Time":1427397600454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":340055,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5441,"Index":431,"Attempt":0,"Launch Time":1427397600462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5412,"Index":402,"Attempt":0,"Launch Time":1427397600386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600462,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":76,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":982436,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5442,"Index":432,"Attempt":0,"Launch Time":1427397600467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5438,"Index":428,"Attempt":0,"Launch Time":1427397600459,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600467,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":369914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5443,"Index":433,"Attempt":0,"Launch Time":1427397600468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5439,"Index":429,"Attempt":0,"Launch Time":1427397600460,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600468,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":373651,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5444,"Index":434,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5445,"Index":435,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5440,"Index":430,"Attempt":0,"Launch Time":1427397600461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":452749,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5414,"Index":404,"Attempt":0,"Launch Time":1427397600387,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":83,"Result Size":930,"JVM GC Time":9,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":352650,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5446,"Index":436,"Attempt":0,"Launch Time":1427397600474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5442,"Index":432,"Attempt":0,"Launch Time":1427397600467,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600474,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5447,"Index":437,"Attempt":0,"Launch Time":1427397600475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5443,"Index":433,"Attempt":0,"Launch Time":1427397600468,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":380711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5448,"Index":438,"Attempt":0,"Launch Time":1427397600477,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5444,"Index":434,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600477,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340148,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5449,"Index":439,"Attempt":0,"Launch Time":1427397600479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5445,"Index":435,"Attempt":0,"Launch Time":1427397600470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1029239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5450,"Index":440,"Attempt":0,"Launch Time":1427397600480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5446,"Index":436,"Attempt":0,"Launch Time":1427397600474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600480,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5451,"Index":441,"Attempt":0,"Launch Time":1427397600482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5447,"Index":437,"Attempt":0,"Launch Time":1427397600475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600482,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":301247,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5452,"Index":442,"Attempt":0,"Launch Time":1427397600483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5448,"Index":438,"Attempt":0,"Launch Time":1427397600477,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600483,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310076,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5453,"Index":443,"Attempt":0,"Launch Time":1427397600486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5450,"Index":440,"Attempt":0,"Launch Time":1427397600480,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600486,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":307069,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5454,"Index":444,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5451,"Index":441,"Attempt":0,"Launch Time":1427397600482,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324180,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5455,"Index":445,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5452,"Index":442,"Attempt":0,"Launch Time":1427397600483,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600489,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299513,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5456,"Index":446,"Attempt":0,"Launch Time":1427397600493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5453,"Index":443,"Attempt":0,"Launch Time":1427397600486,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600493,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":284443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5457,"Index":447,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5454,"Index":444,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":329378,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5458,"Index":448,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5455,"Index":445,"Attempt":0,"Launch Time":1427397600489,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600496,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9605,"Shuffle Write Time":326806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5459,"Index":449,"Attempt":0,"Launch Time":1427397600500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5456,"Index":446,"Attempt":0,"Launch Time":1427397600493,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600500,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287848,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5460,"Index":450,"Attempt":0,"Launch Time":1427397600515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5411,"Index":401,"Attempt":0,"Launch Time":1427397600385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600515,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":74684537,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5461,"Index":451,"Attempt":0,"Launch Time":1427397600516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5459,"Index":449,"Attempt":0,"Launch Time":1427397600500,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600516,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314348,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5462,"Index":452,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5457,"Index":447,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":362949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5463,"Index":453,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5458,"Index":448,"Attempt":0,"Launch Time":1427397600496,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5464,"Index":454,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5415,"Index":405,"Attempt":0,"Launch Time":1427397600389,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":130,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377101,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5465,"Index":455,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5449,"Index":439,"Attempt":0,"Launch Time":1427397600479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600521,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":41,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":452437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5466,"Index":456,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5467,"Index":457,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5416,"Index":406,"Attempt":0,"Launch Time":1427397600392,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":123,"Result Size":930,"JVM GC Time":14,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":365240,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5441,"Index":431,"Attempt":0,"Launch Time":1427397600462,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600527,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1418167,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5468,"Index":458,"Attempt":0,"Launch Time":1427397600529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5460,"Index":450,"Attempt":0,"Launch Time":1427397600515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600529,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":886998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5469,"Index":459,"Attempt":0,"Launch Time":1427397600530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5464,"Index":454,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1238234,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5470,"Index":460,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5471,"Index":461,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5467,"Index":457,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328704,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5468,"Index":458,"Attempt":0,"Launch Time":1427397600529,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600535,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":271657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5472,"Index":462,"Attempt":0,"Launch Time":1427397600536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5469,"Index":459,"Attempt":0,"Launch Time":1427397600530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5473,"Index":463,"Attempt":0,"Launch Time":1427397600541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5470,"Index":460,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600541,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":267939,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5474,"Index":464,"Attempt":0,"Launch Time":1427397600542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5471,"Index":461,"Attempt":0,"Launch Time":1427397600535,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600542,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":303188,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5475,"Index":465,"Attempt":0,"Launch Time":1427397600543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5472,"Index":462,"Attempt":0,"Launch Time":1427397600536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":317081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5476,"Index":466,"Attempt":0,"Launch Time":1427397600549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5473,"Index":463,"Attempt":0,"Launch Time":1427397600541,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9607,"Shuffle Write Time":406927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5477,"Index":467,"Attempt":0,"Launch Time":1427397600552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5474,"Index":464,"Attempt":0,"Launch Time":1427397600542,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318878,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5478,"Index":468,"Attempt":0,"Launch Time":1427397600554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5475,"Index":465,"Attempt":0,"Launch Time":1427397600543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600554,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":400941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5479,"Index":469,"Attempt":0,"Launch Time":1427397600557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5476,"Index":466,"Attempt":0,"Launch Time":1427397600549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600557,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366701,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5480,"Index":470,"Attempt":0,"Launch Time":1427397600560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5478,"Index":468,"Attempt":0,"Launch Time":1427397600554,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303974,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5481,"Index":471,"Attempt":0,"Launch Time":1427397600561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5477,"Index":467,"Attempt":0,"Launch Time":1427397600552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600561,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5482,"Index":472,"Attempt":0,"Launch Time":1427397600564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5479,"Index":469,"Attempt":0,"Launch Time":1427397600557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":270620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5483,"Index":473,"Attempt":0,"Launch Time":1427397600567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5480,"Index":470,"Attempt":0,"Launch Time":1427397600560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600567,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5484,"Index":474,"Attempt":0,"Launch Time":1427397600568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5481,"Index":471,"Attempt":0,"Launch Time":1427397600561,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600568,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":328893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5485,"Index":475,"Attempt":0,"Launch Time":1427397600571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5482,"Index":472,"Attempt":0,"Launch Time":1427397600564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5486,"Index":476,"Attempt":0,"Launch Time":1427397600573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5483,"Index":473,"Attempt":0,"Launch Time":1427397600567,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":302095,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5487,"Index":477,"Attempt":0,"Launch Time":1427397600575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5484,"Index":474,"Attempt":0,"Launch Time":1427397600568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600575,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":262146,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5488,"Index":478,"Attempt":0,"Launch Time":1427397600578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5485,"Index":475,"Attempt":0,"Launch Time":1427397600571,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600578,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":264853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5489,"Index":479,"Attempt":0,"Launch Time":1427397600579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5486,"Index":476,"Attempt":0,"Launch Time":1427397600573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":319588,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5490,"Index":480,"Attempt":0,"Launch Time":1427397600581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5487,"Index":477,"Attempt":0,"Launch Time":1427397600575,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600581,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":289320,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5491,"Index":481,"Attempt":0,"Launch Time":1427397600585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5489,"Index":479,"Attempt":0,"Launch Time":1427397600579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600585,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":250921,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5492,"Index":482,"Attempt":0,"Launch Time":1427397600586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5488,"Index":478,"Attempt":0,"Launch Time":1427397600578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600586,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":265215,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5493,"Index":483,"Attempt":0,"Launch Time":1427397600588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5490,"Index":480,"Attempt":0,"Launch Time":1427397600581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5494,"Index":484,"Attempt":0,"Launch Time":1427397600591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5491,"Index":481,"Attempt":0,"Launch Time":1427397600585,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269705,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5495,"Index":485,"Attempt":0,"Launch Time":1427397600593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5492,"Index":482,"Attempt":0,"Launch Time":1427397600586,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":304376,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5496,"Index":486,"Attempt":0,"Launch Time":1427397600594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5493,"Index":483,"Attempt":0,"Launch Time":1427397600588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600594,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305569,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5497,"Index":487,"Attempt":0,"Launch Time":1427397600597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5494,"Index":484,"Attempt":0,"Launch Time":1427397600591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":247407,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5498,"Index":488,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5495,"Index":485,"Attempt":0,"Launch Time":1427397600593,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600601,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":339427,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5499,"Index":489,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5496,"Index":486,"Attempt":0,"Launch Time":1427397600594,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347355,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5500,"Index":490,"Attempt":0,"Launch Time":1427397600604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5497,"Index":487,"Attempt":0,"Launch Time":1427397600597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":353405,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5501,"Index":491,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5499,"Index":489,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":355103,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5502,"Index":492,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5498,"Index":488,"Attempt":0,"Launch Time":1427397600601,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600609,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":322798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5503,"Index":493,"Attempt":0,"Launch Time":1427397600610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5500,"Index":490,"Attempt":0,"Launch Time":1427397600604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600610,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":306803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5504,"Index":494,"Attempt":0,"Launch Time":1427397600615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5501,"Index":491,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600615,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":287719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5505,"Index":495,"Attempt":0,"Launch Time":1427397600616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5502,"Index":492,"Attempt":0,"Launch Time":1427397600609,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600616,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":307727,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5506,"Index":496,"Attempt":0,"Launch Time":1427397600617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5503,"Index":493,"Attempt":0,"Launch Time":1427397600610,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600617,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5507,"Index":497,"Attempt":0,"Launch Time":1427397600618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5466,"Index":456,"Attempt":0,"Launch Time":1427397600526,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":89,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5508,"Index":498,"Attempt":0,"Launch Time":1427397600627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5504,"Index":494,"Attempt":0,"Launch Time":1427397600615,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600627,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":302455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5509,"Index":499,"Attempt":0,"Launch Time":1427397600631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5465,"Index":455,"Attempt":0,"Launch Time":1427397600521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":108,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":318366,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5510,"Index":500,"Attempt":0,"Launch Time":1427397600632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5506,"Index":496,"Attempt":0,"Launch Time":1427397600617,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":375647,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5511,"Index":501,"Attempt":0,"Launch Time":1427397600634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5508,"Index":498,"Attempt":0,"Launch Time":1427397600627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":290952,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5512,"Index":502,"Attempt":0,"Launch Time":1427397600637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5509,"Index":499,"Attempt":0,"Launch Time":1427397600631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":298579,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5513,"Index":503,"Attempt":0,"Launch Time":1427397600639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5510,"Index":500,"Attempt":0,"Launch Time":1427397600632,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600639,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308073,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5514,"Index":504,"Attempt":0,"Launch Time":1427397600640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5511,"Index":501,"Attempt":0,"Launch Time":1427397600634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600641,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":277358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5515,"Index":505,"Attempt":0,"Launch Time":1427397600644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5512,"Index":502,"Attempt":0,"Launch Time":1427397600637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600644,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318419,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5516,"Index":506,"Attempt":0,"Launch Time":1427397600645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5513,"Index":503,"Attempt":0,"Launch Time":1427397600639,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306978,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5517,"Index":507,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5518,"Index":508,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5514,"Index":504,"Attempt":0,"Launch Time":1427397600640,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315632,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5462,"Index":452,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":128,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306075,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5519,"Index":509,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5461,"Index":451,"Attempt":0,"Launch Time":1427397600516,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413676,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5520,"Index":510,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5515,"Index":505,"Attempt":0,"Launch Time":1427397600644,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600651,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":375360,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5521,"Index":511,"Attempt":0,"Launch Time":1427397600653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5463,"Index":453,"Attempt":0,"Launch Time":1427397600518,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600653,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":134,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5522,"Index":512,"Attempt":0,"Launch Time":1427397600656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5523,"Index":513,"Attempt":0,"Launch Time":1427397600657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5517,"Index":507,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":376807,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5507,"Index":497,"Attempt":0,"Launch Time":1427397600618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600657,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":28,"Executor Run Time":9,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":692998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5524,"Index":514,"Attempt":0,"Launch Time":1427397600658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5519,"Index":509,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600658,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370889,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5525,"Index":515,"Attempt":0,"Launch Time":1427397600659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5520,"Index":510,"Attempt":0,"Launch Time":1427397600651,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600659,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":336940,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5526,"Index":516,"Attempt":0,"Launch Time":1427397600660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5521,"Index":511,"Attempt":0,"Launch Time":1427397600653,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600661,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":301581,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5527,"Index":517,"Attempt":0,"Launch Time":1427397600667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5522,"Index":512,"Attempt":0,"Launch Time":1427397600656,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600667,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322468,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5528,"Index":518,"Attempt":0,"Launch Time":1427397600668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5525,"Index":515,"Attempt":0,"Launch Time":1427397600659,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600668,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5529,"Index":519,"Attempt":0,"Launch Time":1427397600669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5526,"Index":516,"Attempt":0,"Launch Time":1427397600660,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600669,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":361020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5530,"Index":520,"Attempt":0,"Launch Time":1427397600673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5527,"Index":517,"Attempt":0,"Launch Time":1427397600667,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600673,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":329988,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5531,"Index":521,"Attempt":0,"Launch Time":1427397600674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5523,"Index":513,"Attempt":0,"Launch Time":1427397600657,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353984,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5532,"Index":522,"Attempt":0,"Launch Time":1427397600677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5524,"Index":514,"Attempt":0,"Launch Time":1427397600658,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406040,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5533,"Index":523,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5534,"Index":524,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5530,"Index":520,"Attempt":0,"Launch Time":1427397600673,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":320119,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5529,"Index":519,"Attempt":0,"Launch Time":1427397600669,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600680,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":313739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5535,"Index":525,"Attempt":0,"Launch Time":1427397600681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5531,"Index":521,"Attempt":0,"Launch Time":1427397600674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600681,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319439,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5536,"Index":526,"Attempt":0,"Launch Time":1427397600684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5528,"Index":518,"Attempt":0,"Launch Time":1427397600668,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331628,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5537,"Index":527,"Attempt":0,"Launch Time":1427397600687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5534,"Index":524,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":306707,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5538,"Index":528,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5539,"Index":529,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5533,"Index":523,"Attempt":0,"Launch Time":1427397600680,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5535,"Index":525,"Attempt":0,"Launch Time":1427397600681,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600689,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313639,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5540,"Index":530,"Attempt":0,"Launch Time":1427397600690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5536,"Index":526,"Attempt":0,"Launch Time":1427397600684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600690,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281798,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5541,"Index":531,"Attempt":0,"Launch Time":1427397600693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5532,"Index":522,"Attempt":0,"Launch Time":1427397600677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600693,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":348913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5542,"Index":532,"Attempt":0,"Launch Time":1427397600694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5537,"Index":527,"Attempt":0,"Launch Time":1427397600687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600694,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321481,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5543,"Index":533,"Attempt":0,"Launch Time":1427397600695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5538,"Index":528,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600695,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":285832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5544,"Index":534,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5540,"Index":530,"Attempt":0,"Launch Time":1427397600690,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":357646,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5545,"Index":535,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5539,"Index":529,"Attempt":0,"Launch Time":1427397600689,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":620315,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5546,"Index":536,"Attempt":0,"Launch Time":1427397600700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5541,"Index":531,"Attempt":0,"Launch Time":1427397600693,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600700,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":386499,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5547,"Index":537,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5542,"Index":532,"Attempt":0,"Launch Time":1427397600694,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":460049,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5548,"Index":538,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5543,"Index":533,"Attempt":0,"Launch Time":1427397600695,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600703,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":372802,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5549,"Index":539,"Attempt":0,"Launch Time":1427397600707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5545,"Index":535,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600707,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":343070,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5550,"Index":540,"Attempt":0,"Launch Time":1427397600708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5544,"Index":534,"Attempt":0,"Launch Time":1427397600699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":332011,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5551,"Index":541,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5548,"Index":538,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600711,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304894,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5552,"Index":542,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5553,"Index":543,"Attempt":0,"Launch Time":1427397600719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5549,"Index":539,"Attempt":0,"Launch Time":1427397600707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600719,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":279532,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5554,"Index":544,"Attempt":0,"Launch Time":1427397600725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5552,"Index":542,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314359,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5555,"Index":545,"Attempt":0,"Launch Time":1427397600726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5553,"Index":543,"Attempt":0,"Launch Time":1427397600719,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":298472,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5556,"Index":546,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5554,"Index":544,"Attempt":0,"Launch Time":1427397600725,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":341941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5557,"Index":547,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5555,"Index":545,"Attempt":0,"Launch Time":1427397600726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":297617,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5558,"Index":548,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5557,"Index":547,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316231,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5559,"Index":549,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5556,"Index":546,"Attempt":0,"Launch Time":1427397600732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600739,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":368342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5560,"Index":550,"Attempt":0,"Launch Time":1427397600746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5558,"Index":548,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5547,"Index":537,"Attempt":0,"Launch Time":1427397600703,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5561,"Index":551,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5550,"Index":540,"Attempt":0,"Launch Time":1427397600708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":48,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":372575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5562,"Index":552,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5551,"Index":541,"Attempt":0,"Launch Time":1427397600711,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":503345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5563,"Index":553,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5546,"Index":536,"Attempt":0,"Launch Time":1427397600700,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":519886,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5564,"Index":554,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5560,"Index":550,"Attempt":0,"Launch Time":1427397600746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5565,"Index":555,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5505,"Index":495,"Attempt":0,"Launch Time":1427397600616,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":133,"Result Size":930,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":384212,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5566,"Index":556,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5559,"Index":549,"Attempt":0,"Launch Time":1427397600739,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":518178,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5567,"Index":557,"Attempt":0,"Launch Time":1427397600768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5563,"Index":553,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600768,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314169,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5568,"Index":558,"Attempt":0,"Launch Time":1427397600769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5565,"Index":555,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311372,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5569,"Index":559,"Attempt":0,"Launch Time":1427397600770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5561,"Index":551,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600770,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":322471,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5570,"Index":560,"Attempt":0,"Launch Time":1427397600771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5564,"Index":554,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600771,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310239,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5571,"Index":561,"Attempt":0,"Launch Time":1427397600776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5568,"Index":558,"Attempt":0,"Launch Time":1427397600769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5572,"Index":562,"Attempt":0,"Launch Time":1427397600777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5562,"Index":552,"Attempt":0,"Launch Time":1427397600761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600777,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340991,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5573,"Index":563,"Attempt":0,"Launch Time":1427397600782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5571,"Index":561,"Attempt":0,"Launch Time":1427397600776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600782,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":281853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5574,"Index":564,"Attempt":0,"Launch Time":1427397600785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5572,"Index":562,"Attempt":0,"Launch Time":1427397600777,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600785,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5575,"Index":565,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5518,"Index":508,"Attempt":0,"Launch Time":1427397600647,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600788,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311336,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5576,"Index":566,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5573,"Index":563,"Attempt":0,"Launch Time":1427397600782,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600789,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":284600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5577,"Index":567,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5578,"Index":568,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5574,"Index":564,"Attempt":0,"Launch Time":1427397600785,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5570,"Index":560,"Attempt":0,"Launch Time":1427397600771,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600792,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314314,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5579,"Index":569,"Attempt":0,"Launch Time":1427397600795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5576,"Index":566,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":312722,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5580,"Index":570,"Attempt":0,"Launch Time":1427397600796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5575,"Index":565,"Attempt":0,"Launch Time":1427397600788,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600796,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":308973,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5581,"Index":571,"Attempt":0,"Launch Time":1427397600802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5579,"Index":569,"Attempt":0,"Launch Time":1427397600795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5582,"Index":572,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5578,"Index":568,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335740,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5583,"Index":573,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5580,"Index":570,"Attempt":0,"Launch Time":1427397600796,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600804,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":348128,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5584,"Index":574,"Attempt":0,"Launch Time":1427397600809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5581,"Index":571,"Attempt":0,"Launch Time":1427397600802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600810,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5585,"Index":575,"Attempt":0,"Launch Time":1427397600811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5582,"Index":572,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600811,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":313182,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5586,"Index":576,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5516,"Index":506,"Attempt":0,"Launch Time":1427397600645,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":306739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5587,"Index":577,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5584,"Index":574,"Attempt":0,"Launch Time":1427397600809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":312191,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5588,"Index":578,"Attempt":0,"Launch Time":1427397600818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5583,"Index":573,"Attempt":0,"Launch Time":1427397600804,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":2032966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5589,"Index":579,"Attempt":0,"Launch Time":1427397600821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5585,"Index":575,"Attempt":0,"Launch Time":1427397600811,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600821,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":278725,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5590,"Index":580,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5587,"Index":577,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5591,"Index":581,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5588,"Index":578,"Attempt":0,"Launch Time":1427397600818,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600824,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293936,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5592,"Index":582,"Attempt":0,"Launch Time":1427397600825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5577,"Index":567,"Attempt":0,"Launch Time":1427397600792,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600825,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319899,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5593,"Index":583,"Attempt":0,"Launch Time":1427397600830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5590,"Index":580,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600830,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9611,"Shuffle Write Time":314620,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5594,"Index":584,"Attempt":0,"Launch Time":1427397600832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5589,"Index":579,"Attempt":0,"Launch Time":1427397600821,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600832,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":347934,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5595,"Index":585,"Attempt":0,"Launch Time":1427397600833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5591,"Index":581,"Attempt":0,"Launch Time":1427397600824,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336839,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5596,"Index":586,"Attempt":0,"Launch Time":1427397600837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5593,"Index":583,"Attempt":0,"Launch Time":1427397600830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600838,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":348005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5597,"Index":587,"Attempt":0,"Launch Time":1427397600839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5594,"Index":584,"Attempt":0,"Launch Time":1427397600832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600840,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":353228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5598,"Index":588,"Attempt":0,"Launch Time":1427397600859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5566,"Index":556,"Attempt":0,"Launch Time":1427397600762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":95,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327089,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5599,"Index":589,"Attempt":0,"Launch Time":1427397600860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5596,"Index":586,"Attempt":0,"Launch Time":1427397600837,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600860,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309888,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5600,"Index":590,"Attempt":0,"Launch Time":1427397600862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5567,"Index":557,"Attempt":0,"Launch Time":1427397600768,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600862,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":94,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":374043,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5601,"Index":591,"Attempt":0,"Launch Time":1427397600863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5595,"Index":585,"Attempt":0,"Launch Time":1427397600833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600863,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":386742,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5602,"Index":592,"Attempt":0,"Launch Time":1427397600869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5599,"Index":589,"Attempt":0,"Launch Time":1427397600860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":327507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5603,"Index":593,"Attempt":0,"Launch Time":1427397600870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5598,"Index":588,"Attempt":0,"Launch Time":1427397600859,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600870,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":379876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5604,"Index":594,"Attempt":0,"Launch Time":1427397600871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5592,"Index":582,"Attempt":0,"Launch Time":1427397600825,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600871,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":32,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387112,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5605,"Index":595,"Attempt":0,"Launch Time":1427397600877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5597,"Index":587,"Attempt":0,"Launch Time":1427397600839,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600877,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":429201,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5606,"Index":596,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5607,"Index":597,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5608,"Index":598,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5601,"Index":591,"Attempt":0,"Launch Time":1427397600863,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":351806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5586,"Index":576,"Attempt":0,"Launch Time":1427397600817,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600878,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":18,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":334342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5609,"Index":599,"Attempt":0,"Launch Time":1427397600879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5604,"Index":594,"Attempt":0,"Launch Time":1427397600871,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":373265,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5602,"Index":592,"Attempt":0,"Launch Time":1427397600869,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600879,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":286214,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5610,"Index":600,"Attempt":0,"Launch Time":1427397600884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5605,"Index":595,"Attempt":0,"Launch Time":1427397600877,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317358,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5611,"Index":601,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5606,"Index":596,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":310521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5612,"Index":602,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5608,"Index":598,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600885,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":311072,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5613,"Index":603,"Attempt":0,"Launch Time":1427397600892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5614,"Index":604,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5610,"Index":600,"Attempt":0,"Launch Time":1427397600884,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":285253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5607,"Index":597,"Attempt":0,"Launch Time":1427397600878,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":514505,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5615,"Index":605,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5603,"Index":593,"Attempt":0,"Launch Time":1427397600870,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600893,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345952,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5616,"Index":606,"Attempt":0,"Launch Time":1427397600899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5617,"Index":607,"Attempt":0,"Launch Time":1427397600900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5600,"Index":590,"Attempt":0,"Launch Time":1427397600862,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600900,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":322784,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5612,"Index":602,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":377697,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5618,"Index":608,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5619,"Index":609,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5615,"Index":605,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":324538,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5613,"Index":603,"Attempt":0,"Launch Time":1427397600892,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":293493,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5617,"Index":607,"Attempt":0,"Launch Time":1427397600900,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600912,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":272877,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5620,"Index":610,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5621,"Index":611,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5611,"Index":601,"Attempt":0,"Launch Time":1427397600885,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600913,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":19,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":830964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5622,"Index":612,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5623,"Index":613,"Attempt":0,"Launch Time":1427397600914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5616,"Index":606,"Attempt":0,"Launch Time":1427397600899,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297815,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5569,"Index":559,"Attempt":0,"Launch Time":1427397600770,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600914,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":25,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":558814,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5624,"Index":614,"Attempt":0,"Launch Time":1427397600922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5625,"Index":615,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5623,"Index":613,"Attempt":0,"Launch Time":1427397600914,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":338446,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5621,"Index":611,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":365329,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5626,"Index":616,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5620,"Index":610,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600923,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":416234,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5627,"Index":617,"Attempt":0,"Launch Time":1427397600926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5622,"Index":612,"Attempt":0,"Launch Time":1427397600913,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600926,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5628,"Index":618,"Attempt":0,"Launch Time":1427397600927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5618,"Index":608,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600927,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":343695,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5629,"Index":619,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5624,"Index":614,"Attempt":0,"Launch Time":1427397600922,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347365,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5630,"Index":620,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5625,"Index":615,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600930,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":606672,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5631,"Index":621,"Attempt":0,"Launch Time":1427397600931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5626,"Index":616,"Attempt":0,"Launch Time":1427397600923,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600931,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323586,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5632,"Index":622,"Attempt":0,"Launch Time":1427397600932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5627,"Index":617,"Attempt":0,"Launch Time":1427397600926,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600932,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291571,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5633,"Index":623,"Attempt":0,"Launch Time":1427397600934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5628,"Index":618,"Attempt":0,"Launch Time":1427397600927,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600934,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5634,"Index":624,"Attempt":0,"Launch Time":1427397600937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5630,"Index":620,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600937,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":291357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5635,"Index":625,"Attempt":0,"Launch Time":1427397600938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5629,"Index":619,"Attempt":0,"Launch Time":1427397600930,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600939,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":300524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5636,"Index":626,"Attempt":0,"Launch Time":1427397600940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5633,"Index":623,"Attempt":0,"Launch Time":1427397600934,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600941,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":333005,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5637,"Index":627,"Attempt":0,"Launch Time":1427397600944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5634,"Index":624,"Attempt":0,"Launch Time":1427397600937,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600944,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5638,"Index":628,"Attempt":0,"Launch Time":1427397600946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5635,"Index":625,"Attempt":0,"Launch Time":1427397600938,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600946,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320957,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5639,"Index":629,"Attempt":0,"Launch Time":1427397600947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5636,"Index":626,"Attempt":0,"Launch Time":1427397600940,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600947,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5640,"Index":630,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5637,"Index":627,"Attempt":0,"Launch Time":1427397600944,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600953,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386992,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5641,"Index":631,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5638,"Index":628,"Attempt":0,"Launch Time":1427397600946,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600954,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":373423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5642,"Index":632,"Attempt":0,"Launch Time":1427397600961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5639,"Index":629,"Attempt":0,"Launch Time":1427397600947,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600962,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":398455,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5643,"Index":633,"Attempt":0,"Launch Time":1427397600963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5609,"Index":599,"Attempt":0,"Launch Time":1427397600879,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600963,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":70,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":13193761,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5644,"Index":634,"Attempt":0,"Launch Time":1427397600971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5641,"Index":631,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":338669,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5645,"Index":635,"Attempt":0,"Launch Time":1427397600972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5640,"Index":630,"Attempt":0,"Launch Time":1427397600953,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600972,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":18,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":504964,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5646,"Index":636,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5632,"Index":622,"Attempt":0,"Launch Time":1427397600932,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":316622,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5647,"Index":637,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5631,"Index":621,"Attempt":0,"Launch Time":1427397600931,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600973,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354021,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5648,"Index":638,"Attempt":0,"Launch Time":1427397600978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5644,"Index":634,"Attempt":0,"Launch Time":1427397600971,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316674,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5649,"Index":639,"Attempt":0,"Launch Time":1427397600980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5645,"Index":635,"Attempt":0,"Launch Time":1427397600972,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600980,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337963,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5650,"Index":640,"Attempt":0,"Launch Time":1427397600981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5619,"Index":609,"Attempt":0,"Launch Time":1427397600912,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600982,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":331513,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5651,"Index":641,"Attempt":0,"Launch Time":1427397600982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5646,"Index":636,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600982,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":425806,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5652,"Index":642,"Attempt":0,"Launch Time":1427397600983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5647,"Index":637,"Attempt":0,"Launch Time":1427397600973,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600983,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1147698,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5653,"Index":643,"Attempt":0,"Launch Time":1427397600985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5648,"Index":638,"Attempt":0,"Launch Time":1427397600978,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600985,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5654,"Index":644,"Attempt":0,"Launch Time":1427397600986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5649,"Index":639,"Attempt":0,"Launch Time":1427397600980,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600986,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301553,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5655,"Index":645,"Attempt":0,"Launch Time":1427397600988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5642,"Index":632,"Attempt":0,"Launch Time":1427397600961,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600988,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":26,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2147133,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5656,"Index":646,"Attempt":0,"Launch Time":1427397600989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5651,"Index":641,"Attempt":0,"Launch Time":1427397600982,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":309613,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5657,"Index":647,"Attempt":0,"Launch Time":1427397600991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5652,"Index":642,"Attempt":0,"Launch Time":1427397600983,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600991,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319525,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5658,"Index":648,"Attempt":0,"Launch Time":1427397600993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5654,"Index":644,"Attempt":0,"Launch Time":1427397600986,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600993,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327012,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5659,"Index":649,"Attempt":0,"Launch Time":1427397600994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5653,"Index":643,"Attempt":0,"Launch Time":1427397600985,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600994,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":316000,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5660,"Index":650,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5656,"Index":646,"Attempt":0,"Launch Time":1427397600989,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":306852,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5661,"Index":651,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5657,"Index":647,"Attempt":0,"Launch Time":1427397600991,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397600998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":319928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5662,"Index":652,"Attempt":0,"Launch Time":1427397601000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5658,"Index":648,"Attempt":0,"Launch Time":1427397600993,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601000,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310972,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5663,"Index":653,"Attempt":0,"Launch Time":1427397601001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5655,"Index":645,"Attempt":0,"Launch Time":1427397600988,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":377893,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5664,"Index":654,"Attempt":0,"Launch Time":1427397601002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5659,"Index":649,"Attempt":0,"Launch Time":1427397600994,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423917,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5665,"Index":655,"Attempt":0,"Launch Time":1427397601006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5661,"Index":651,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601006,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":363293,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5666,"Index":656,"Attempt":0,"Launch Time":1427397601009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5660,"Index":650,"Attempt":0,"Launch Time":1427397600998,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601009,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":538437,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5667,"Index":657,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5668,"Index":658,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5662,"Index":652,"Attempt":0,"Launch Time":1427397601000,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":344585,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5663,"Index":653,"Attempt":0,"Launch Time":1427397601001,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601010,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5669,"Index":659,"Attempt":0,"Launch Time":1427397601011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5650,"Index":640,"Attempt":0,"Launch Time":1427397600981,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601011,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":332925,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5670,"Index":660,"Attempt":0,"Launch Time":1427397601012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5664,"Index":654,"Attempt":0,"Launch Time":1427397601002,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5671,"Index":661,"Attempt":0,"Launch Time":1427397601013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5665,"Index":655,"Attempt":0,"Launch Time":1427397601006,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318657,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5672,"Index":662,"Attempt":0,"Launch Time":1427397601016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5666,"Index":656,"Attempt":0,"Launch Time":1427397601009,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601016,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324542,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5673,"Index":663,"Attempt":0,"Launch Time":1427397601018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5667,"Index":657,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601018,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":530865,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5674,"Index":664,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5670,"Index":660,"Attempt":0,"Launch Time":1427397601012,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338171,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5675,"Index":665,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5676,"Index":666,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5668,"Index":658,"Attempt":0,"Launch Time":1427397601010,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601019,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":299614,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5671,"Index":661,"Attempt":0,"Launch Time":1427397601013,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601020,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299418,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5677,"Index":667,"Attempt":0,"Launch Time":1427397601021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5614,"Index":604,"Attempt":0,"Launch Time":1427397600893,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601021,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":19,"Executor Run Time":108,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":309135,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5678,"Index":668,"Attempt":0,"Launch Time":1427397601025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5643,"Index":633,"Attempt":0,"Launch Time":1427397600963,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601026,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275038,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5679,"Index":669,"Attempt":0,"Launch Time":1427397601031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5677,"Index":667,"Attempt":0,"Launch Time":1427397601021,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601031,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325754,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5680,"Index":670,"Attempt":0,"Launch Time":1427397601032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5678,"Index":668,"Attempt":0,"Launch Time":1427397601025,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601032,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":324233,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5681,"Index":671,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5680,"Index":670,"Attempt":0,"Launch Time":1427397601032,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334514,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5682,"Index":672,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5669,"Index":659,"Attempt":0,"Launch Time":1427397601011,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601040,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":328498,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5683,"Index":673,"Attempt":0,"Launch Time":1427397601042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5679,"Index":669,"Attempt":0,"Launch Time":1427397601031,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601042,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":338256,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5684,"Index":674,"Attempt":0,"Launch Time":1427397601047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5682,"Index":672,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":298502,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5685,"Index":675,"Attempt":0,"Launch Time":1427397601049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5681,"Index":671,"Attempt":0,"Launch Time":1427397601040,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":354326,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5686,"Index":676,"Attempt":0,"Launch Time":1427397601050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5683,"Index":673,"Attempt":0,"Launch Time":1427397601042,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601050,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":463184,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5687,"Index":677,"Attempt":0,"Launch Time":1427397601056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5684,"Index":674,"Attempt":0,"Launch Time":1427397601047,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":468696,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5688,"Index":678,"Attempt":0,"Launch Time":1427397601068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5685,"Index":675,"Attempt":0,"Launch Time":1427397601049,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601068,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":365367,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5689,"Index":679,"Attempt":0,"Launch Time":1427397601075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5688,"Index":678,"Attempt":0,"Launch Time":1427397601068,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316379,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5690,"Index":680,"Attempt":0,"Launch Time":1427397601090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5689,"Index":679,"Attempt":0,"Launch Time":1427397601075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601090,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":387118,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5691,"Index":681,"Attempt":0,"Launch Time":1427397601091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5686,"Index":676,"Attempt":0,"Launch Time":1427397601050,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601091,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":39,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":378631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5692,"Index":682,"Attempt":0,"Launch Time":1427397601095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5687,"Index":677,"Attempt":0,"Launch Time":1427397601056,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601095,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":36,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5693,"Index":683,"Attempt":0,"Launch Time":1427397601097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5675,"Index":665,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601097,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":41,"Executor Run Time":8,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":308809,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5694,"Index":684,"Attempt":0,"Launch Time":1427397601098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5676,"Index":666,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601098,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":749091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5695,"Index":685,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5691,"Index":681,"Attempt":0,"Launch Time":1427397601091,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325165,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5696,"Index":686,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5690,"Index":680,"Attempt":0,"Launch Time":1427397601090,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601100,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":324433,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5697,"Index":687,"Attempt":0,"Launch Time":1427397601101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5692,"Index":682,"Attempt":0,"Launch Time":1427397601095,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601101,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":307665,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5698,"Index":688,"Attempt":0,"Launch Time":1427397601106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5694,"Index":684,"Attempt":0,"Launch Time":1427397601098,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601106,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":337131,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5699,"Index":689,"Attempt":0,"Launch Time":1427397601107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5696,"Index":686,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601107,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":332967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5700,"Index":690,"Attempt":0,"Launch Time":1427397601108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5697,"Index":687,"Attempt":0,"Launch Time":1427397601101,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":328421,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5701,"Index":691,"Attempt":0,"Launch Time":1427397601109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5674,"Index":664,"Attempt":0,"Launch Time":1427397601019,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601109,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":50,"Executor Run Time":33,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358222,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5702,"Index":692,"Attempt":0,"Launch Time":1427397601110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5693,"Index":683,"Attempt":0,"Launch Time":1427397601097,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601110,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319844,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5703,"Index":693,"Attempt":0,"Launch Time":1427397601114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5698,"Index":688,"Attempt":0,"Launch Time":1427397601106,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601114,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":395414,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5704,"Index":694,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5699,"Index":689,"Attempt":0,"Launch Time":1427397601107,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601115,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":294895,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5705,"Index":695,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5673,"Index":663,"Attempt":0,"Launch Time":1427397601018,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":8,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5706,"Index":696,"Attempt":0,"Launch Time":1427397601116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5700,"Index":690,"Attempt":0,"Launch Time":1427397601108,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601116,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":314132,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5707,"Index":697,"Attempt":0,"Launch Time":1427397601117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5695,"Index":685,"Attempt":0,"Launch Time":1427397601100,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601117,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":536305,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5708,"Index":698,"Attempt":0,"Launch Time":1427397601118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5702,"Index":692,"Attempt":0,"Launch Time":1427397601110,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601118,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":368248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5709,"Index":699,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5705,"Index":695,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":338077,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5710,"Index":700,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5706,"Index":696,"Attempt":0,"Launch Time":1427397601116,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601123,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349851,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5711,"Index":701,"Attempt":0,"Launch Time":1427397601124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5704,"Index":694,"Attempt":0,"Launch Time":1427397601115,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601124,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":349835,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5712,"Index":702,"Attempt":0,"Launch Time":1427397601128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5708,"Index":698,"Attempt":0,"Launch Time":1427397601118,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601128,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":320898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5713,"Index":703,"Attempt":0,"Launch Time":1427397601130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5709,"Index":699,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601130,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364914,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5714,"Index":704,"Attempt":0,"Launch Time":1427397601132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5711,"Index":701,"Attempt":0,"Launch Time":1427397601124,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601132,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":347125,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5715,"Index":705,"Attempt":0,"Launch Time":1427397601134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5707,"Index":697,"Attempt":0,"Launch Time":1427397601117,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9612,"Shuffle Write Time":285440,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5716,"Index":706,"Attempt":0,"Launch Time":1427397601137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5713,"Index":703,"Attempt":0,"Launch Time":1427397601130,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601137,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":299467,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5717,"Index":707,"Attempt":0,"Launch Time":1427397601139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5714,"Index":704,"Attempt":0,"Launch Time":1427397601132,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601139,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318096,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5718,"Index":708,"Attempt":0,"Launch Time":1427397601140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5712,"Index":702,"Attempt":0,"Launch Time":1427397601128,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601140,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323709,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5719,"Index":709,"Attempt":0,"Launch Time":1427397601141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5710,"Index":700,"Attempt":0,"Launch Time":1427397601123,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601141,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":282744,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5720,"Index":710,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5717,"Index":707,"Attempt":0,"Launch Time":1427397601139,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362438,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5721,"Index":711,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5722,"Index":712,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5716,"Index":706,"Attempt":0,"Launch Time":1427397601137,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":364709,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5715,"Index":705,"Attempt":0,"Launch Time":1427397601134,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9615,"Shuffle Write Time":403208,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5723,"Index":713,"Attempt":0,"Launch Time":1427397601147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5718,"Index":708,"Attempt":0,"Launch Time":1427397601140,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601147,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":378301,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5724,"Index":714,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5701,"Index":691,"Attempt":0,"Launch Time":1427397601109,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300864,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5725,"Index":715,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5672,"Index":662,"Attempt":0,"Launch Time":1427397601016,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601148,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":131,"Result Size":930,"JVM GC Time":4,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":387955,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5726,"Index":716,"Attempt":0,"Launch Time":1427397601154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5703,"Index":693,"Attempt":0,"Launch Time":1427397601114,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601154,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":339508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5727,"Index":717,"Attempt":0,"Launch Time":1427397601157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5725,"Index":715,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601157,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387737,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5728,"Index":718,"Attempt":0,"Launch Time":1427397601160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5719,"Index":709,"Attempt":0,"Launch Time":1427397601141,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601160,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":17,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":401215,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5729,"Index":719,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5730,"Index":720,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5726,"Index":716,"Attempt":0,"Launch Time":1427397601154,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601163,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5724,"Index":714,"Attempt":0,"Launch Time":1427397601148,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":553085,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5731,"Index":721,"Attempt":0,"Launch Time":1427397601164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5727,"Index":717,"Attempt":0,"Launch Time":1427397601157,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601164,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314562,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5732,"Index":722,"Attempt":0,"Launch Time":1427397601167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5728,"Index":718,"Attempt":0,"Launch Time":1427397601160,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601167,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314281,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5733,"Index":723,"Attempt":0,"Launch Time":1427397601168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5723,"Index":713,"Attempt":0,"Launch Time":1427397601147,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9613,"Shuffle Write Time":316394,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5734,"Index":724,"Attempt":0,"Launch Time":1427397601181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5730,"Index":720,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601181,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":16,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":347976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5735,"Index":725,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5729,"Index":719,"Attempt":0,"Launch Time":1427397601163,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":23586631,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5736,"Index":726,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5731,"Index":721,"Attempt":0,"Launch Time":1427397601164,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601196,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":29,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":300287,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5737,"Index":727,"Attempt":0,"Launch Time":1427397601199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5732,"Index":722,"Attempt":0,"Launch Time":1427397601167,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601199,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":356003,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5738,"Index":728,"Attempt":0,"Launch Time":1427397601207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5737,"Index":727,"Attempt":0,"Launch Time":1427397601199,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":359402,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5739,"Index":729,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5736,"Index":726,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":499241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5740,"Index":730,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5735,"Index":725,"Attempt":0,"Launch Time":1427397601196,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601209,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":920670,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5741,"Index":731,"Attempt":0,"Launch Time":1427397601210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5734,"Index":724,"Attempt":0,"Launch Time":1427397601181,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601210,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1412322,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5742,"Index":732,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5743,"Index":733,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5733,"Index":723,"Attempt":0,"Launch Time":1427397601168,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":18,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":523902,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5738,"Index":728,"Attempt":0,"Launch Time":1427397601207,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":275163,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5744,"Index":734,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5739,"Index":729,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301051,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5745,"Index":735,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5740,"Index":730,"Attempt":0,"Launch Time":1427397601209,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601215,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":301771,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5746,"Index":736,"Attempt":0,"Launch Time":1427397601217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5721,"Index":711,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":318947,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5747,"Index":737,"Attempt":0,"Launch Time":1427397601220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5741,"Index":731,"Attempt":0,"Launch Time":1427397601210,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601220,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":556661,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5748,"Index":738,"Attempt":0,"Launch Time":1427397601221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5744,"Index":734,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":310357,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5749,"Index":739,"Attempt":0,"Launch Time":1427397601222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5745,"Index":735,"Attempt":0,"Launch Time":1427397601215,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365423,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5750,"Index":740,"Attempt":0,"Launch Time":1427397601223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5720,"Index":710,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":57,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1258151,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5751,"Index":741,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5743,"Index":733,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":303927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5742,"Index":732,"Attempt":0,"Launch Time":1427397601214,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327533,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5752,"Index":742,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5753,"Index":743,"Attempt":0,"Launch Time":1427397601230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5749,"Index":739,"Attempt":0,"Launch Time":1427397601222,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601230,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":301566,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5754,"Index":744,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5752,"Index":742,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":305516,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5755,"Index":745,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5747,"Index":737,"Attempt":0,"Launch Time":1427397601220,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601231,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312951,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5756,"Index":746,"Attempt":0,"Launch Time":1427397601233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5748,"Index":738,"Attempt":0,"Launch Time":1427397601221,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601234,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1031507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5757,"Index":747,"Attempt":0,"Launch Time":1427397601238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5751,"Index":741,"Attempt":0,"Launch Time":1427397601224,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601238,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":293037,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5758,"Index":748,"Attempt":0,"Launch Time":1427397601239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5753,"Index":743,"Attempt":0,"Launch Time":1427397601230,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601240,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325769,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5759,"Index":749,"Attempt":0,"Launch Time":1427397601243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5755,"Index":745,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316122,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5760,"Index":750,"Attempt":0,"Launch Time":1427397601244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5754,"Index":744,"Attempt":0,"Launch Time":1427397601231,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601244,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":358312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5761,"Index":751,"Attempt":0,"Launch Time":1427397601246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5756,"Index":746,"Attempt":0,"Launch Time":1427397601233,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601246,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":321368,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5762,"Index":752,"Attempt":0,"Launch Time":1427397601248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5758,"Index":748,"Attempt":0,"Launch Time":1427397601239,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601248,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1079928,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5763,"Index":753,"Attempt":0,"Launch Time":1427397601250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5757,"Index":747,"Attempt":0,"Launch Time":1427397601238,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601250,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1580937,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5764,"Index":754,"Attempt":0,"Launch Time":1427397601251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5760,"Index":750,"Attempt":0,"Launch Time":1427397601244,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":366931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5765,"Index":755,"Attempt":0,"Launch Time":1427397601255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5759,"Index":749,"Attempt":0,"Launch Time":1427397601243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601256,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":398845,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5766,"Index":756,"Attempt":0,"Launch Time":1427397601256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5761,"Index":751,"Attempt":0,"Launch Time":1427397601246,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601257,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":754150,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5767,"Index":757,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5768,"Index":758,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5763,"Index":753,"Attempt":0,"Launch Time":1427397601250,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":368832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5762,"Index":752,"Attempt":0,"Launch Time":1427397601248,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":439831,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5769,"Index":759,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5764,"Index":754,"Attempt":0,"Launch Time":1427397601251,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601258,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":341142,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5770,"Index":760,"Attempt":0,"Launch Time":1427397601259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5750,"Index":740,"Attempt":0,"Launch Time":1427397601223,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601259,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":34,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":3856503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5771,"Index":761,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5772,"Index":762,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5773,"Index":763,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5774,"Index":764,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5765,"Index":755,"Attempt":0,"Launch Time":1427397601255,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":320071,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5766,"Index":756,"Attempt":0,"Launch Time":1427397601256,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601266,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":329400,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5768,"Index":758,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":310537,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5769,"Index":759,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601267,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":289026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5775,"Index":765,"Attempt":0,"Launch Time":1427397601269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5767,"Index":757,"Attempt":0,"Launch Time":1427397601258,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601269,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":295198,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5776,"Index":766,"Attempt":0,"Launch Time":1427397601272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5722,"Index":712,"Attempt":0,"Launch Time":1427397601146,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601274,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":81,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":345394,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5777,"Index":767,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5778,"Index":768,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5770,"Index":760,"Attempt":0,"Launch Time":1427397601259,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":321004,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5746,"Index":736,"Attempt":0,"Launch Time":1427397601217,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601275,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":315876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5779,"Index":769,"Attempt":0,"Launch Time":1427397601290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5776,"Index":766,"Attempt":0,"Launch Time":1427397601272,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601290,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":317284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5780,"Index":770,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5772,"Index":762,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601300,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":406392,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5781,"Index":771,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5779,"Index":769,"Attempt":0,"Launch Time":1427397601290,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601301,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":342522,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5782,"Index":772,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5783,"Index":773,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5773,"Index":763,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325521,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5778,"Index":768,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":31,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":370232,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5784,"Index":774,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5780,"Index":770,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":340782,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5785,"Index":775,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5777,"Index":767,"Attempt":0,"Launch Time":1427397601274,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601310,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317306,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5786,"Index":776,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5781,"Index":771,"Attempt":0,"Launch Time":1427397601300,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":329765,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5787,"Index":777,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5771,"Index":761,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601313,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":488134,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5788,"Index":778,"Attempt":0,"Launch Time":1427397601316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5775,"Index":765,"Attempt":0,"Launch Time":1427397601269,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601316,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1743013,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5789,"Index":779,"Attempt":0,"Launch Time":1427397601317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5790,"Index":780,"Attempt":0,"Launch Time":1427397601318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5785,"Index":775,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":321260,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5783,"Index":773,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601318,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":336804,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5791,"Index":781,"Attempt":0,"Launch Time":1427397601319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5784,"Index":774,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601319,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":335211,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5792,"Index":782,"Attempt":0,"Launch Time":1427397601321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5774,"Index":764,"Attempt":0,"Launch Time":1427397601266,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601322,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":54,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1091896,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5793,"Index":783,"Attempt":0,"Launch Time":1427397601326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5790,"Index":780,"Attempt":0,"Launch Time":1427397601318,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":317926,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5794,"Index":784,"Attempt":0,"Launch Time":1427397601327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5791,"Index":781,"Attempt":0,"Launch Time":1427397601319,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601327,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5795,"Index":785,"Attempt":0,"Launch Time":1427397601334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5789,"Index":779,"Attempt":0,"Launch Time":1427397601317,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601334,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":710871,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5796,"Index":786,"Attempt":0,"Launch Time":1427397601335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5794,"Index":784,"Attempt":0,"Launch Time":1427397601327,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601335,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343508,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5797,"Index":787,"Attempt":0,"Launch Time":1427397601336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5793,"Index":783,"Attempt":0,"Launch Time":1427397601326,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1347933,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5798,"Index":788,"Attempt":0,"Launch Time":1427397601338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5788,"Index":778,"Attempt":0,"Launch Time":1427397601316,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601338,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":21,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":2153296,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5799,"Index":789,"Attempt":0,"Launch Time":1427397601342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5800,"Index":790,"Attempt":0,"Launch Time":1427397601343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5795,"Index":785,"Attempt":0,"Launch Time":1427397601334,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":478773,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5796,"Index":786,"Attempt":0,"Launch Time":1427397601335,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601343,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324285,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5801,"Index":791,"Attempt":0,"Launch Time":1427397601344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5797,"Index":787,"Attempt":0,"Launch Time":1427397601336,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":336545,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5802,"Index":792,"Attempt":0,"Launch Time":1427397601346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5786,"Index":776,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601346,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":29,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":432144,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5803,"Index":793,"Attempt":0,"Launch Time":1427397601349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5798,"Index":788,"Attempt":0,"Launch Time":1427397601338,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601349,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":580656,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5804,"Index":794,"Attempt":0,"Launch Time":1427397601351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5801,"Index":791,"Attempt":0,"Launch Time":1427397601344,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601351,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":362029,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5805,"Index":795,"Attempt":0,"Launch Time":1427397601352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5792,"Index":782,"Attempt":0,"Launch Time":1427397601321,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601352,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":343516,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5806,"Index":796,"Attempt":0,"Launch Time":1427397601353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5782,"Index":772,"Attempt":0,"Launch Time":1427397601310,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601353,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":23,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":349339,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5807,"Index":797,"Attempt":0,"Launch Time":1427397601355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5800,"Index":790,"Attempt":0,"Launch Time":1427397601343,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601356,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352394,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5808,"Index":798,"Attempt":0,"Launch Time":1427397601358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5804,"Index":794,"Attempt":0,"Launch Time":1427397601351,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601358,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":367453,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5809,"Index":799,"Attempt":0,"Launch Time":1427397601359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5799,"Index":789,"Attempt":0,"Launch Time":1427397601342,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601359,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":589876,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5810,"Index":800,"Attempt":0,"Launch Time":1427397601361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5802,"Index":792,"Attempt":0,"Launch Time":1427397601346,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601361,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":14,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5811,"Index":801,"Attempt":0,"Launch Time":1427397601362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5807,"Index":797,"Attempt":0,"Launch Time":1427397601355,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601362,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":275020,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5812,"Index":802,"Attempt":0,"Launch Time":1427397601364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5803,"Index":793,"Attempt":0,"Launch Time":1427397601349,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601364,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":520528,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5813,"Index":803,"Attempt":0,"Launch Time":1427397601365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5808,"Index":798,"Attempt":0,"Launch Time":1427397601358,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601365,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":307010,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5814,"Index":804,"Attempt":0,"Launch Time":1427397601366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5809,"Index":799,"Attempt":0,"Launch Time":1427397601359,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601369,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":342688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5815,"Index":805,"Attempt":0,"Launch Time":1427397601369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5811,"Index":801,"Attempt":0,"Launch Time":1427397601362,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601370,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":283342,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5816,"Index":806,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5813,"Index":803,"Attempt":0,"Launch Time":1427397601365,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":278805,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5817,"Index":807,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5812,"Index":802,"Attempt":0,"Launch Time":1427397601364,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601371,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":287047,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5818,"Index":808,"Attempt":0,"Launch Time":1427397601377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5815,"Index":805,"Attempt":0,"Launch Time":1427397601369,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601377,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":306922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5819,"Index":809,"Attempt":0,"Launch Time":1427397601378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5814,"Index":804,"Attempt":0,"Launch Time":1427397601366,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601378,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":327891,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5820,"Index":810,"Attempt":0,"Launch Time":1427397601379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5817,"Index":807,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601379,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":312922,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5821,"Index":811,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5822,"Index":812,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5819,"Index":809,"Attempt":0,"Launch Time":1427397601378,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335487,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5818,"Index":808,"Attempt":0,"Launch Time":1427397601377,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601385,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323511,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5823,"Index":813,"Attempt":0,"Launch Time":1427397601386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5820,"Index":810,"Attempt":0,"Launch Time":1427397601379,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601386,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":334259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5824,"Index":814,"Attempt":0,"Launch Time":1427397601436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5787,"Index":777,"Attempt":0,"Launch Time":1427397601313,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":57,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":424299,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5825,"Index":815,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5805,"Index":795,"Attempt":0,"Launch Time":1427397601352,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":38,"Executor Run Time":7,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328797,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5826,"Index":816,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5806,"Index":796,"Attempt":0,"Launch Time":1427397601353,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601437,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328248,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5816,"Index":806,"Attempt":0,"Launch Time":1427397601371,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601438,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":323084,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5827,"Index":817,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5828,"Index":818,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5829,"Index":819,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5830,"Index":820,"Attempt":0,"Launch Time":1427397601440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5810,"Index":800,"Attempt":0,"Launch Time":1427397601361,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":22269690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5821,"Index":811,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":305665,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5822,"Index":812,"Attempt":0,"Launch Time":1427397601385,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601441,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":27,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":338356,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5831,"Index":821,"Attempt":0,"Launch Time":1427397601442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5823,"Index":813,"Attempt":0,"Launch Time":1427397601386,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601442,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":333221,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5832,"Index":822,"Attempt":0,"Launch Time":1427397601443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5825,"Index":815,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601443,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":291100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5833,"Index":823,"Attempt":0,"Launch Time":1427397601446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5828,"Index":818,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601446,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":291997,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5834,"Index":824,"Attempt":0,"Launch Time":1427397601448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5831,"Index":821,"Attempt":0,"Launch Time":1427397601442,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":5,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":281603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5835,"Index":825,"Attempt":0,"Launch Time":1427397601449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5832,"Index":822,"Attempt":0,"Launch Time":1427397601443,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601449,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":300053,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5836,"Index":826,"Attempt":0,"Launch Time":1427397601454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5833,"Index":823,"Attempt":0,"Launch Time":1427397601446,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":337180,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5837,"Index":827,"Attempt":0,"Launch Time":1427397601456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5834,"Index":824,"Attempt":0,"Launch Time":1427397601448,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601456,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":346668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5838,"Index":828,"Attempt":0,"Launch Time":1427397601461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5830,"Index":820,"Attempt":0,"Launch Time":1427397601440,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601461,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":318690,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5839,"Index":829,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5829,"Index":819,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601463,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":562135,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5840,"Index":830,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5836,"Index":826,"Attempt":0,"Launch Time":1427397601454,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601464,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":319985,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5841,"Index":831,"Attempt":0,"Launch Time":1427397601466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5827,"Index":817,"Attempt":0,"Launch Time":1427397601438,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601466,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":337756,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5842,"Index":832,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5838,"Index":828,"Attempt":0,"Launch Time":1427397601461,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":316605,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5843,"Index":833,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5826,"Index":816,"Attempt":0,"Launch Time":1427397601437,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601470,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":325449,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5844,"Index":834,"Attempt":0,"Launch Time":1427397601471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5840,"Index":830,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601471,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309718,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5845,"Index":835,"Attempt":0,"Launch Time":1427397601474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5839,"Index":829,"Attempt":0,"Launch Time":1427397601463,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":361345,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5846,"Index":836,"Attempt":0,"Launch Time":1427397601475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5841,"Index":831,"Attempt":0,"Launch Time":1427397601466,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601475,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":376527,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5847,"Index":837,"Attempt":0,"Launch Time":1427397601478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5844,"Index":834,"Attempt":0,"Launch Time":1427397601471,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601478,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313873,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5848,"Index":838,"Attempt":0,"Launch Time":1427397601479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5842,"Index":832,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601479,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":362593,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5849,"Index":839,"Attempt":0,"Launch Time":1427397601481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5843,"Index":833,"Attempt":0,"Launch Time":1427397601470,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601481,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":309066,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5847,"Index":837,"Attempt":0,"Launch Time":1427397601478,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601484,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":264853,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5850,"Index":840,"Attempt":0,"Launch Time":1427397601484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5851,"Index":841,"Attempt":0,"Launch Time":1427397601485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5845,"Index":835,"Attempt":0,"Launch Time":1427397601474,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601485,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":386081,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5852,"Index":842,"Attempt":0,"Launch Time":1427397601491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5848,"Index":838,"Attempt":0,"Launch Time":1427397601479,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601491,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333862,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5853,"Index":843,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5851,"Index":841,"Attempt":0,"Launch Time":1427397601485,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":357682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5854,"Index":844,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5850,"Index":840,"Attempt":0,"Launch Time":1427397601484,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601492,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":387018,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5855,"Index":845,"Attempt":0,"Launch Time":1427397601494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5849,"Index":839,"Attempt":0,"Launch Time":1427397601481,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601494,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":413960,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5856,"Index":846,"Attempt":0,"Launch Time":1427397601499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5852,"Index":842,"Attempt":0,"Launch Time":1427397601491,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601499,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349457,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5857,"Index":847,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5858,"Index":848,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5855,"Index":845,"Attempt":0,"Launch Time":1427397601494,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601501,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":345943,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5853,"Index":843,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":331276,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5859,"Index":849,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5860,"Index":850,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5854,"Index":844,"Attempt":0,"Launch Time":1427397601492,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":350834,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5846,"Index":836,"Attempt":0,"Launch Time":1427397601475,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601502,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":10,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322731,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5856,"Index":846,"Attempt":0,"Launch Time":1427397601499,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601508,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":404911,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5861,"Index":851,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5862,"Index":852,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5857,"Index":847,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601510,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":305156,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5863,"Index":853,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5835,"Index":825,"Attempt":0,"Launch Time":1427397601449,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601511,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":62,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":54537507,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5864,"Index":854,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5858,"Index":848,"Attempt":0,"Launch Time":1427397601501,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601512,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":363062,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5865,"Index":855,"Attempt":0,"Launch Time":1427397601514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5860,"Index":850,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601514,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":451953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5866,"Index":856,"Attempt":0,"Launch Time":1427397601517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5861,"Index":851,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601518,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302267,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5867,"Index":857,"Attempt":0,"Launch Time":1427397601530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5866,"Index":856,"Attempt":0,"Launch Time":1427397601517,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601530,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313749,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5868,"Index":858,"Attempt":0,"Launch Time":1427397601531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5837,"Index":827,"Attempt":0,"Launch Time":1427397601456,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601531,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":75,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":406011,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5869,"Index":859,"Attempt":0,"Launch Time":1427397601536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5867,"Index":857,"Attempt":0,"Launch Time":1427397601530,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601536,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":349223,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5870,"Index":860,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5871,"Index":861,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5824,"Index":814,"Attempt":0,"Launch Time":1427397601436,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601538,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":400443,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5868,"Index":858,"Attempt":0,"Launch Time":1427397601531,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601539,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":333966,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5872,"Index":862,"Attempt":0,"Launch Time":1427397601540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5863,"Index":853,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601540,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":22,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1276417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5873,"Index":863,"Attempt":0,"Launch Time":1427397601543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5869,"Index":859,"Attempt":0,"Launch Time":1427397601536,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601543,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":303941,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5874,"Index":864,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5870,"Index":860,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":278998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5875,"Index":865,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5871,"Index":861,"Attempt":0,"Launch Time":1427397601538,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601545,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":313675,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5876,"Index":866,"Attempt":0,"Launch Time":1427397601546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5872,"Index":862,"Attempt":0,"Launch Time":1427397601540,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601546,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":304444,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5877,"Index":867,"Attempt":0,"Launch Time":1427397601549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5873,"Index":863,"Attempt":0,"Launch Time":1427397601543,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601549,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315082,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5878,"Index":868,"Attempt":0,"Launch Time":1427397601552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5874,"Index":864,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601552,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":376826,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5879,"Index":869,"Attempt":0,"Launch Time":1427397601553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5875,"Index":865,"Attempt":0,"Launch Time":1427397601545,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601553,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":374981,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5880,"Index":870,"Attempt":0,"Launch Time":1427397601557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5877,"Index":867,"Attempt":0,"Launch Time":1427397601549,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601558,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":445353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5881,"Index":871,"Attempt":0,"Launch Time":1427397601559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5876,"Index":866,"Attempt":0,"Launch Time":1427397601546,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601559,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":383645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5882,"Index":872,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5883,"Index":873,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5878,"Index":868,"Attempt":0,"Launch Time":1427397601552,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":310859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5879,"Index":869,"Attempt":0,"Launch Time":1427397601553,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601560,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":319117,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5884,"Index":874,"Attempt":0,"Launch Time":1427397601564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5880,"Index":870,"Attempt":0,"Launch Time":1427397601557,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601564,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":271710,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5885,"Index":875,"Attempt":0,"Launch Time":1427397601565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5881,"Index":871,"Attempt":0,"Launch Time":1427397601559,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601565,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326337,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5886,"Index":876,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5882,"Index":872,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":301949,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5887,"Index":877,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5883,"Index":873,"Attempt":0,"Launch Time":1427397601560,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601566,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":283259,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5888,"Index":878,"Attempt":0,"Launch Time":1427397601568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5864,"Index":854,"Attempt":0,"Launch Time":1427397601511,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601569,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":56,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":360252,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5889,"Index":879,"Attempt":0,"Launch Time":1427397601570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5884,"Index":874,"Attempt":0,"Launch Time":1427397601564,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601570,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":293958,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5890,"Index":880,"Attempt":0,"Launch Time":1427397601572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5885,"Index":875,"Attempt":0,"Launch Time":1427397601565,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601572,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":320393,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5891,"Index":881,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5886,"Index":876,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294694,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5892,"Index":882,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5887,"Index":877,"Attempt":0,"Launch Time":1427397601566,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601573,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":371283,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5893,"Index":883,"Attempt":0,"Launch Time":1427397601577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5889,"Index":879,"Attempt":0,"Launch Time":1427397601570,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601577,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":353478,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5894,"Index":884,"Attempt":0,"Launch Time":1427397601579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5890,"Index":880,"Attempt":0,"Launch Time":1427397601572,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327953,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5895,"Index":885,"Attempt":0,"Launch Time":1427397601580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5892,"Index":882,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601580,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":321587,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5896,"Index":886,"Attempt":0,"Launch Time":1427397601581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5891,"Index":881,"Attempt":0,"Launch Time":1427397601573,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601582,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320780,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5897,"Index":887,"Attempt":0,"Launch Time":1427397601584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5893,"Index":883,"Attempt":0,"Launch Time":1427397601577,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601584,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":328612,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5898,"Index":888,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5859,"Index":849,"Attempt":0,"Launch Time":1427397601502,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":16,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":450832,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5899,"Index":889,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5895,"Index":885,"Attempt":0,"Launch Time":1427397601580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601587,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":322909,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5900,"Index":890,"Attempt":0,"Launch Time":1427397601588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5894,"Index":884,"Attempt":0,"Launch Time":1427397601579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601588,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":345484,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5901,"Index":891,"Attempt":0,"Launch Time":1427397601589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5896,"Index":886,"Attempt":0,"Launch Time":1427397601581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601589,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":314895,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5902,"Index":892,"Attempt":0,"Launch Time":1427397601591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5897,"Index":887,"Attempt":0,"Launch Time":1427397601584,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601591,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":367026,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5903,"Index":893,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5899,"Index":889,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":316207,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5904,"Index":894,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5898,"Index":888,"Attempt":0,"Launch Time":1427397601587,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601595,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":323312,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5905,"Index":895,"Attempt":0,"Launch Time":1427397601596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5900,"Index":890,"Attempt":0,"Launch Time":1427397601588,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601596,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":324353,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5902,"Index":892,"Attempt":0,"Launch Time":1427397601591,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601597,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294203,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5906,"Index":896,"Attempt":0,"Launch Time":1427397601597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5907,"Index":897,"Attempt":0,"Launch Time":1427397601599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5901,"Index":891,"Attempt":0,"Launch Time":1427397601589,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601600,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":512061,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5908,"Index":898,"Attempt":0,"Launch Time":1427397601602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5904,"Index":894,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601602,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":361575,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5909,"Index":899,"Attempt":0,"Launch Time":1427397601603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5903,"Index":893,"Attempt":0,"Launch Time":1427397601595,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601603,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":397645,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5910,"Index":900,"Attempt":0,"Launch Time":1427397601604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5905,"Index":895,"Attempt":0,"Launch Time":1427397601596,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601604,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":357409,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5911,"Index":901,"Attempt":0,"Launch Time":1427397601605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5906,"Index":896,"Attempt":0,"Launch Time":1427397601597,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601605,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":358856,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5912,"Index":902,"Attempt":0,"Launch Time":1427397601606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5862,"Index":852,"Attempt":0,"Launch Time":1427397601509,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601606,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1052228,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5913,"Index":903,"Attempt":0,"Launch Time":1427397601608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5907,"Index":897,"Attempt":0,"Launch Time":1427397601599,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601614,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":476214,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5914,"Index":904,"Attempt":0,"Launch Time":1427397601618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5911,"Index":901,"Attempt":0,"Launch Time":1427397601605,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601618,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":303230,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5915,"Index":905,"Attempt":0,"Launch Time":1427397601620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5913,"Index":903,"Attempt":0,"Launch Time":1427397601608,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601620,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9608,"Shuffle Write Time":288534,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5916,"Index":906,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5912,"Index":902,"Attempt":0,"Launch Time":1427397601606,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601624,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":371898,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5917,"Index":907,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5914,"Index":904,"Attempt":0,"Launch Time":1427397601618,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601625,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":296050,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5918,"Index":908,"Attempt":0,"Launch Time":1427397601626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5908,"Index":898,"Attempt":0,"Launch Time":1427397601602,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601626,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":355942,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5919,"Index":909,"Attempt":0,"Launch Time":1427397601627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5915,"Index":905,"Attempt":0,"Launch Time":1427397601620,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601628,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":342350,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5920,"Index":910,"Attempt":0,"Launch Time":1427397601631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5917,"Index":907,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601631,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":276717,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5921,"Index":911,"Attempt":0,"Launch Time":1427397601633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5918,"Index":908,"Attempt":0,"Launch Time":1427397601626,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":276190,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5922,"Index":912,"Attempt":0,"Launch Time":1427397601634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5919,"Index":909,"Attempt":0,"Launch Time":1427397601627,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":317568,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5923,"Index":913,"Attempt":0,"Launch Time":1427397601637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5920,"Index":910,"Attempt":0,"Launch Time":1427397601631,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":321600,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5924,"Index":914,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5925,"Index":915,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5926,"Index":916,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5927,"Index":917,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5928,"Index":918,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5929,"Index":919,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5909,"Index":899,"Attempt":0,"Launch Time":1427397601603,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":334503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5921,"Index":911,"Attempt":0,"Launch Time":1427397601633,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":323503,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5916,"Index":906,"Attempt":0,"Launch Time":1427397601624,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":423652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5923,"Index":913,"Attempt":0,"Launch Time":1427397601637,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601649,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5865,"Index":855,"Attempt":0,"Launch Time":1427397601514,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":131,"Result Size":930,"JVM GC Time":10,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1710924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5910,"Index":900,"Attempt":0,"Launch Time":1427397601604,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601674,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":44,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":314658,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5930,"Index":920,"Attempt":0,"Launch Time":1427397601674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5931,"Index":921,"Attempt":0,"Launch Time":1427397601675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5932,"Index":922,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5888,"Index":878,"Attempt":0,"Launch Time":1427397601568,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":7,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":282241,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5928,"Index":918,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352028,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5929,"Index":919,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325335,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5933,"Index":923,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5926,"Index":916,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601676,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":352533,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5934,"Index":924,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5922,"Index":912,"Attempt":0,"Launch Time":1427397601634,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302255,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5935,"Index":925,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5925,"Index":915,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601677,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":434343,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5936,"Index":926,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5934,"Index":924,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325924,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5937,"Index":927,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5938,"Index":928,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5935,"Index":925,"Attempt":0,"Launch Time":1427397601677,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336611,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5927,"Index":917,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601684,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":27,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":324284,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5939,"Index":929,"Attempt":0,"Launch Time":1427397601685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5933,"Index":923,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601685,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339803,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5940,"Index":930,"Attempt":0,"Launch Time":1427397601687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5932,"Index":922,"Attempt":0,"Launch Time":1427397601676,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601687,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":365739,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5941,"Index":931,"Attempt":0,"Launch Time":1427397601691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5936,"Index":926,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601691,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":320046,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5942,"Index":932,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5943,"Index":933,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5938,"Index":928,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":302603,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5939,"Index":929,"Attempt":0,"Launch Time":1427397601685,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312120,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5944,"Index":934,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5937,"Index":927,"Attempt":0,"Launch Time":1427397601684,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601692,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331688,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5945,"Index":935,"Attempt":0,"Launch Time":1427397601697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5941,"Index":931,"Attempt":0,"Launch Time":1427397601691,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601697,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317303,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5946,"Index":936,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5943,"Index":933,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601698,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":279709,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5947,"Index":937,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5942,"Index":932,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":336711,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5948,"Index":938,"Attempt":0,"Launch Time":1427397601699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5944,"Index":934,"Attempt":0,"Launch Time":1427397601692,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601699,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":340583,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5949,"Index":939,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5945,"Index":935,"Attempt":0,"Launch Time":1427397601697,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":381968,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5950,"Index":940,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5940,"Index":930,"Attempt":0,"Launch Time":1427397601687,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601705,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":15,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":405310,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5951,"Index":941,"Attempt":0,"Launch Time":1427397601706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5946,"Index":936,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601706,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":401859,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5952,"Index":942,"Attempt":0,"Launch Time":1427397601707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5948,"Index":938,"Attempt":0,"Launch Time":1427397601699,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":475668,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5953,"Index":943,"Attempt":0,"Launch Time":1427397601708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5947,"Index":937,"Attempt":0,"Launch Time":1427397601698,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601708,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":343751,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5954,"Index":944,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5950,"Index":940,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601713,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":317547,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5955,"Index":945,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5949,"Index":939,"Attempt":0,"Launch Time":1427397601705,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339417,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5956,"Index":946,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5951,"Index":941,"Attempt":0,"Launch Time":1427397601706,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":338679,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5957,"Index":947,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5953,"Index":943,"Attempt":0,"Launch Time":1427397601708,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601714,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":314879,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5958,"Index":948,"Attempt":0,"Launch Time":1427397601715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5952,"Index":942,"Attempt":0,"Launch Time":1427397601707,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601715,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":314719,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5959,"Index":949,"Attempt":0,"Launch Time":1427397601726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5954,"Index":944,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601726,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":390987,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5960,"Index":950,"Attempt":0,"Launch Time":1427397601727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5955,"Index":945,"Attempt":0,"Launch Time":1427397601713,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601727,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":12,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318332,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5961,"Index":951,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5956,"Index":946,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":312216,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5962,"Index":952,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5957,"Index":947,"Attempt":0,"Launch Time":1427397601714,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601728,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":7135543,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5963,"Index":953,"Attempt":0,"Launch Time":1427397601729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5958,"Index":948,"Attempt":0,"Launch Time":1427397601715,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601729,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326466,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5964,"Index":954,"Attempt":0,"Launch Time":1427397601732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5959,"Index":949,"Attempt":0,"Launch Time":1427397601726,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601732,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":275559,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5965,"Index":955,"Attempt":0,"Launch Time":1427397601735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5961,"Index":951,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601735,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":325253,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5966,"Index":956,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5967,"Index":957,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5960,"Index":950,"Attempt":0,"Launch Time":1427397601727,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601736,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":376175,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5962,"Index":952,"Attempt":0,"Launch Time":1427397601728,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601737,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":344264,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5968,"Index":958,"Attempt":0,"Launch Time":1427397601737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5963,"Index":953,"Attempt":0,"Launch Time":1427397601729,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":325364,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5969,"Index":959,"Attempt":0,"Launch Time":1427397601738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5964,"Index":954,"Attempt":0,"Launch Time":1427397601732,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601738,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":318980,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5970,"Index":960,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5971,"Index":961,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5965,"Index":955,"Attempt":0,"Launch Time":1427397601735,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601742,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":311846,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5966,"Index":956,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":286302,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5972,"Index":962,"Attempt":0,"Launch Time":1427397601743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5967,"Index":957,"Attempt":0,"Launch Time":1427397601736,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601743,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":261913,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5973,"Index":963,"Attempt":0,"Launch Time":1427397601744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5968,"Index":958,"Attempt":0,"Launch Time":1427397601737,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601744,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":322064,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5974,"Index":964,"Attempt":0,"Launch Time":1427397601746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5969,"Index":959,"Attempt":0,"Launch Time":1427397601738,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601746,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":269804,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5975,"Index":965,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5976,"Index":966,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5971,"Index":961,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601749,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":292976,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5970,"Index":960,"Attempt":0,"Launch Time":1427397601742,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":300652,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5977,"Index":967,"Attempt":0,"Launch Time":1427397601750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5972,"Index":962,"Attempt":0,"Launch Time":1427397601743,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601750,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":334258,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5978,"Index":968,"Attempt":0,"Launch Time":1427397601752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5973,"Index":963,"Attempt":0,"Launch Time":1427397601744,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601752,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":326338,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5979,"Index":969,"Attempt":0,"Launch Time":1427397601754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5974,"Index":964,"Attempt":0,"Launch Time":1427397601746,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601754,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":717100,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5980,"Index":970,"Attempt":0,"Launch Time":1427397601757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5975,"Index":965,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601757,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":563338,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5981,"Index":971,"Attempt":0,"Launch Time":1427397601758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5977,"Index":967,"Attempt":0,"Launch Time":1427397601750,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601758,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":393504,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5982,"Index":972,"Attempt":0,"Launch Time":1427397601759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5978,"Index":968,"Attempt":0,"Launch Time":1427397601752,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601759,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":336977,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5983,"Index":973,"Attempt":0,"Launch Time":1427397601761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5979,"Index":969,"Attempt":0,"Launch Time":1427397601754,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601761,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":352564,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5984,"Index":974,"Attempt":0,"Launch Time":1427397601762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5976,"Index":966,"Attempt":0,"Launch Time":1427397601749,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601762,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":13,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":549610,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5985,"Index":975,"Attempt":0,"Launch Time":1427397601765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5980,"Index":970,"Attempt":0,"Launch Time":1427397601757,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601765,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":315598,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5986,"Index":976,"Attempt":0,"Launch Time":1427397601766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5982,"Index":972,"Attempt":0,"Launch Time":1427397601759,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601766,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327931,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5987,"Index":977,"Attempt":0,"Launch Time":1427397601767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5981,"Index":971,"Attempt":0,"Launch Time":1427397601758,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601767,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":760770,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5988,"Index":978,"Attempt":0,"Launch Time":1427397601769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5983,"Index":973,"Attempt":0,"Launch Time":1427397601761,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601769,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":376998,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5989,"Index":979,"Attempt":0,"Launch Time":1427397601772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5985,"Index":975,"Attempt":0,"Launch Time":1427397601765,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601772,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":331460,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5990,"Index":980,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5991,"Index":981,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5986,"Index":976,"Attempt":0,"Launch Time":1427397601766,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":397959,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5984,"Index":974,"Attempt":0,"Launch Time":1427397601762,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601773,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":335074,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5992,"Index":982,"Attempt":0,"Launch Time":1427397601774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5987,"Index":977,"Attempt":0,"Launch Time":1427397601767,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601774,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":352278,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5993,"Index":983,"Attempt":0,"Launch Time":1427397601776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5930,"Index":920,"Attempt":0,"Launch Time":1427397601674,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601776,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":76,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":311850,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5994,"Index":984,"Attempt":0,"Launch Time":1427397601780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5990,"Index":980,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601780,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":308524,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5995,"Index":985,"Attempt":0,"Launch Time":1427397601784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5931,"Index":921,"Attempt":0,"Launch Time":1427397601675,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601784,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":327681,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5996,"Index":986,"Attempt":0,"Launch Time":1427397601787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5993,"Index":983,"Attempt":0,"Launch Time":1427397601776,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601787,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":294967,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5997,"Index":987,"Attempt":0,"Launch Time":1427397601789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5994,"Index":984,"Attempt":0,"Launch Time":1427397601780,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601790,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":9,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":392663,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5998,"Index":988,"Attempt":0,"Launch Time":1427397601794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5924,"Index":914,"Attempt":0,"Launch Time":1427397601648,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601794,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":121,"Result Size":930,"JVM GC Time":5,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":335682,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":5999,"Index":989,"Attempt":0,"Launch Time":1427397601795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5996,"Index":986,"Attempt":0,"Launch Time":1427397601787,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601795,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332819,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6000,"Index":990,"Attempt":0,"Launch Time":1427397601797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5997,"Index":987,"Attempt":0,"Launch Time":1427397601789,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601797,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":315687,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6001,"Index":991,"Attempt":0,"Launch Time":1427397601798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5995,"Index":985,"Attempt":0,"Launch Time":1427397601784,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601798,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":11,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":480787,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6002,"Index":992,"Attempt":0,"Launch Time":1427397601802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5998,"Index":988,"Attempt":0,"Launch Time":1427397601794,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601802,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":366030,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6003,"Index":993,"Attempt":0,"Launch Time":1427397601803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5999,"Index":989,"Attempt":0,"Launch Time":1427397601795,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601803,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":340388,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6004,"Index":994,"Attempt":0,"Launch Time":1427397601805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6000,"Index":990,"Attempt":0,"Launch Time":1427397601797,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601805,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":371736,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6005,"Index":995,"Attempt":0,"Launch Time":1427397601809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6001,"Index":991,"Attempt":0,"Launch Time":1427397601798,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601809,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":7,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":327916,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6006,"Index":996,"Attempt":0,"Launch Time":1427397601826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6002,"Index":992,"Attempt":0,"Launch Time":1427397601802,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601826,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":330535,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6007,"Index":997,"Attempt":0,"Launch Time":1427397601828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6003,"Index":993,"Attempt":0,"Launch Time":1427397601803,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601828,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":325841,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6008,"Index":998,"Attempt":0,"Launch Time":1427397601829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6004,"Index":994,"Attempt":0,"Launch Time":1427397601805,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601829,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":24,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":332396,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":6009,"Index":999,"Attempt":0,"Launch Time":1427397601833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6006,"Index":996,"Attempt":0,"Launch Time":1427397601826,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601833,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":299895,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6007,"Index":997,"Attempt":0,"Launch Time":1427397601828,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":299349,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6008,"Index":998,"Attempt":0,"Launch Time":1427397601829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601835,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":6,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":297091,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5988,"Index":978,"Attempt":0,"Launch Time":1427397601769,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":13,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":388214,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6009,"Index":999,"Attempt":0,"Launch Time":1427397601833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":8,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":384820,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6005,"Index":995,"Attempt":0,"Launch Time":1427397601809,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601841,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":32,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":346979,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5992,"Index":982,"Attempt":0,"Launch Time":1427397601774,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601846,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":273927,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5989,"Index":979,"Attempt":0,"Launch Time":1427397601772,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601851,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":339777,"Shuffle Records Written":100}}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5991,"Index":981,"Attempt":0,"Launch Time":1427397601773,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601854,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":42,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":326549,"Shuffle Records Written":100}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"map at :21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:32)\n$line15.$read$$iwC$$iwC$$iwC.(:34)\n$line15.$read$$iwC$$iwC.(:36)\n$line15.$read$$iwC.(:38)\n$line15.$read.(:40)\n$line15.$read$.(:44)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397599571,"Completion Time":1427397601854,"Accumulables":[]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":21,"Stage Attempt ID":0,"Task Info":{"Task ID":6010,"Index":0,"Attempt":0,"Launch Time":1427397601860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":21,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6010,"Index":0,"Attempt":0,"Launch Time":1427397601860,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397601971,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":109,"Result Size":1060,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1000,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":191840,"Total Records Read":2000}}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"next at :26","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.collection.Iterator$$anon$13.next(Iterator.scala:372)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:26)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:31)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.(:33)\n$line40.$read$$iwC$$iwC$$iwC$$iwC$$iwC.(:35)\n$line40.$read$$iwC$$iwC$$iwC$$iwC.(:37)\n$line40.$read$$iwC$$iwC$$iwC.(:39)\n$line40.$read$$iwC$$iwC.(:41)\n$line40.$read$$iwC.(:43)\n$line40.$read.(:45)\n$line40.$read$.(:49)\n$line40.$read$.()\n$line40.$eval$.(:7)\n$line40.$eval$.()\n$line40.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Submission Time":1427397601860,"Completion Time":1427397601971,"Accumulables":[]}} +{"Event":"SparkListenerJobEnd","Job ID":10,"Completion Time":1427397601971,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1427397602949} diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index bd0f8bdefa171..75399461f2a5f 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -27,19 +27,20 @@ import org.scalatest.Matchers class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { - implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { - def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[A]) : mutable.Set[A] = { - new mutable.HashSet[A]() + implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = + new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } } - } test ("basic accumulation"){ sc = new SparkContext("local", "test") @@ -49,11 +50,10 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { d.foreach{x => acc += x} acc.value should be (210) - - val longAcc = sc.accumulator(0l) + val longAcc = sc.accumulator(0L) val maxInt = Integer.MAX_VALUE.toLong d.foreach{x => longAcc += maxInt + x} - longAcc.value should be (210l + maxInt * 20) + longAcc.value should be (210L + maxInt * 20) } test ("value not assignable from tasks") { diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 4b25c200a695a..668ddf9f5f0a9 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -45,16 +45,17 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(split) override val getDependencies = List[Dependency[_]]() - override def compute(split: Partition, context: TaskContext) = Array(1, 2, 3, 4).iterator + override def compute(split: Partition, context: TaskContext): Iterator[Int] = + Array(1, 2, 3, 4).iterator } rdd2 = new RDD[Int](sc, List(new OneToOneDependency(rdd))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() rdd3 = new RDD[Int](sc, List(new OneToOneDependency(rdd2))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() } @@ -64,7 +65,7 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0, null) val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) @@ -76,7 +77,7 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) when(blockManager.get(RDDBlockId(0, 0))).thenReturn(Some(result)) - val context = new TaskContextImpl(0, 0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0, null) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -85,14 +86,14 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf // Local computation should not persist the resulting value, so don't expect a put(). when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - val context = new TaskContextImpl(0, 0, 0, 0, true) + val context = new TaskContextImpl(0, 0, 0, 0, null, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0, null) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) } diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 32abc65385267..e1faddeabec79 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -75,7 +75,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result) assert(parCollection.dependencies != Nil) assert(parCollection.partitions.length === numPartitions) - assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList) + assert(parCollection.partitions.toList === + parCollection.checkpointData.get.getPartitions.toList) assert(parCollection.collect() === result) } @@ -102,13 +103,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("UnionRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(_.union(otherRDD)) testRDDPartitions(_.union(otherRDD)) } test("CartesianRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(new CartesianRDD(sc, _, otherRDD)) testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) @@ -223,7 +224,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val partitionAfterCheckpoint = serializeDeserialize( unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition]) assert( - partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass, + partitionBeforeCheckpoint.parents.head.getClass != + partitionAfterCheckpoint.parents.head.getClass, "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed" ) } @@ -358,7 +360,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * Generate an pair RDD (with partitioner) such that both the RDD and its partitions * have large size. */ - def generateFatPairRDD() = { + def generateFatPairRDD(): RDD[(Int, Int)] = { new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) } @@ -445,7 +447,8 @@ class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like // the PairRDDFunctions.cogroup() - def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { + def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) + : RDD[(K, Array[Iterable[V]])] = { new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index cdfaacee7da40..c7868ddcf770f 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager @@ -64,7 +65,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha } } - //------ Helper functions ------ + // ------ Helper functions ------ protected def newRDD() = sc.makeRDD(1 to 10) protected def newPairRDD() = newRDD().map(_ -> 1) @@ -205,6 +206,52 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { postGCTester.assertCleanup() } + test("automatically cleanup checkpoint") { + val checkpointDir = java.io.File.createTempFile("temp", "") + checkpointDir.deleteOnExit() + checkpointDir.delete() + var rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).isDefined) + val path = RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + sc.stop() + val conf = new SparkConf().setMaster("local[2]").setAppName("cleanupCheckpoint"). + set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + sc = new SparkContext(conf) + rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + } + test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly @@ -359,18 +406,20 @@ class CleanerTester( sc: SparkContext, rddIds: Seq[Int] = Seq.empty, shuffleIds: Seq[Int] = Seq.empty, - broadcastIds: Seq[Long] = Seq.empty) + broadcastIds: Seq[Long] = Seq.empty, + checkpointIds: Seq[Long] = Seq.empty) extends Logging { val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds + val toBeCheckpointIds = new HashSet[Long] with SynchronizedSet[Long] ++= checkpointIds val isDistributed = !sc.isLocal val cleanerListener = new CleanerListener { def rddCleaned(rddId: Int): Unit = { toBeCleanedRDDIds -= rddId - logInfo("RDD "+ rddId + " cleaned") + logInfo("RDD " + rddId + " cleaned") } def shuffleCleaned(shuffleId: Int): Unit = { @@ -380,12 +429,17 @@ class CleanerTester( def broadcastCleaned(broadcastId: Long): Unit = { toBeCleanedBroadcstIds -= broadcastId - logInfo("Broadcast" + broadcastId + " cleaned") + logInfo("Broadcast " + broadcastId + " cleaned") } def accumCleaned(accId: Long): Unit = { logInfo("Cleaned accId " + accId + " cleaned") } + + def checkpointCleaned(rddId: Long): Unit = { + toBeCheckpointIds -= rddId + logInfo("checkpoint " + rddId + " cleaned") + } } val MAX_VALIDATION_ATTEMPTS = 10 @@ -409,7 +463,8 @@ class CleanerTester( /** Verify that RDDs, shuffles, etc. occupy resources */ private def preCleanupValidate() { - assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty || + checkpointIds.nonEmpty, "Nothing to cleanup") // Verify the RDDs have been persisted and blocks are present rddIds.foreach { rddId => @@ -500,7 +555,8 @@ class CleanerTester( private def isAllCleanedUp = toBeCleanedRDDIds.isEmpty && toBeCleanedShuffleIds.isEmpty && - toBeCleanedBroadcstIds.isEmpty + toBeCleanedBroadcstIds.isEmpty && + toBeCheckpointIds.isEmpty private def getRDDBlocks(rddId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 97ea3578aa8ba..96a9c207ad022 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -77,7 +77,7 @@ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { } test("groupByKey where map output sizes exceed maxMbInFlight") { - val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1") + val conf = new SparkConf().set("spark.reducer.maxSizeInFlight", "1m") sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 9bd5dfec8703a..c42dfbc82ada4 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { - test("driver should exit after finishing without cleanup (SPARK-530)") { + ignore("driver should exit after finishing without cleanup (SPARK-530)") { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val masters = Table("master", "local", "local-cluster[2,1,512]") forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index abfcee75728dc..84f787ee3715d 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import scala.collection.mutable -import org.scalatest.{FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -28,10 +28,20 @@ import org.apache.spark.util.ManualClock /** * Test add and remove behavior of ExecutorAllocationManager. */ -class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { +class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter { import ExecutorAllocationManager._ import ExecutorAllocationManagerSuite._ + private val contexts = new mutable.ListBuffer[SparkContext]() + + before { + contexts.clear() + } + + after { + contexts.foreach(_.stop()) + } + test("verify min/max executors") { val conf = new SparkConf() .setMaster("local") @@ -39,25 +49,20 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") val sc0 = new SparkContext(conf) + contexts += sc0 assert(sc0.executorAllocationManager.isDefined) sc0.stop() // Min < 0 val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") - intercept[SparkException] { new SparkContext(conf1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() + intercept[SparkException] { contexts += new SparkContext(conf1) } // Max < 0 val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") - intercept[SparkException] { new SparkContext(conf2) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() + intercept[SparkException] { contexts += new SparkContext(conf2) } // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) @@ -73,7 +78,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { test("starting state") { sc = createSparkContext() val manager = sc.executorAllocationManager.get - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 1) assert(executorsPendingToRemove(manager).isEmpty) assert(executorIds(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) @@ -86,108 +91,108 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Keep adding until the limit is reached - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 1) + assert(numExecutorsTarget(manager) === 2) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 2) - assert(numExecutorsPending(manager) === 3) + assert(numExecutorsTarget(manager) === 4) assert(numExecutorsToAdd(manager) === 4) assert(addExecutors(manager) === 4) - assert(numExecutorsPending(manager) === 7) + assert(numExecutorsTarget(manager) === 8) assert(numExecutorsToAdd(manager) === 8) - assert(addExecutors(manager) === 3) // reached the limit of 10 - assert(numExecutorsPending(manager) === 10) + assert(addExecutors(manager) === 2) // reached the limit of 10 + assert(numExecutorsTarget(manager) === 10) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 0) - assert(numExecutorsPending(manager) === 10) + assert(numExecutorsTarget(manager) === 10) assert(numExecutorsToAdd(manager) === 1) // Register previously requested executors onExecutorAdded(manager, "first") - assert(numExecutorsPending(manager) === 9) + assert(numExecutorsTarget(manager) === 10) onExecutorAdded(manager, "second") onExecutorAdded(manager, "third") onExecutorAdded(manager, "fourth") - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsTarget(manager) === 10) onExecutorAdded(manager, "first") // duplicates should not count onExecutorAdded(manager, "second") - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsTarget(manager) === 10) // Try adding again // This should still fail because the number pending + running is still at the limit assert(addExecutors(manager) === 0) - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsTarget(manager) === 10) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 0) - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsTarget(manager) === 10) assert(numExecutorsToAdd(manager) === 1) } test("add executors capped by num pending tasks") { - sc = createSparkContext(1, 10) + sc = createSparkContext(0, 10) val manager = sc.executorAllocationManager.get sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 5))) // Verify that we're capped at number of tasks in the stage - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 1) + assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 2) - assert(numExecutorsPending(manager) === 3) + assert(numExecutorsTarget(manager) === 3) assert(numExecutorsToAdd(manager) === 4) assert(addExecutors(manager) === 2) - assert(numExecutorsPending(manager) === 5) + assert(numExecutorsTarget(manager) === 5) assert(numExecutorsToAdd(manager) === 1) - // Verify that running a task reduces the cap + // Verify that running a task doesn't affect the target sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) sc.listenerBus.postToAll(SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsPending(manager) === 4) + assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 5) + assert(numExecutorsTarget(manager) === 6) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 2) - assert(numExecutorsPending(manager) === 7) + assert(numExecutorsTarget(manager) === 8) assert(numExecutorsToAdd(manager) === 4) assert(addExecutors(manager) === 0) - assert(numExecutorsPending(manager) === 7) + assert(numExecutorsTarget(manager) === 8) assert(numExecutorsToAdd(manager) === 1) - // Verify that re-running a task doesn't reduce the cap further + // Verify that re-running a task doesn't blow things up sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 3))) sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 8) + assert(numExecutorsTarget(manager) === 9) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 9) + assert(numExecutorsTarget(manager) === 10) assert(numExecutorsToAdd(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) assert(addExecutors(manager) === 0) - assert(numExecutorsPending(manager) === 9) + assert(numExecutorsTarget(manager) === 10) } test("cancel pending executors when no longer needed") { - sc = createSparkContext(1, 10) + sc = createSparkContext(0, 10) val manager = sc.executorAllocationManager.get sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 5))) - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 1) + assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 2) - assert(numExecutorsPending(manager) === 3) + assert(numExecutorsTarget(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task1Info)) @@ -261,7 +266,6 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Add a few executors assert(addExecutors(manager) === 1) assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) onExecutorAdded(manager, "1") onExecutorAdded(manager, "2") onExecutorAdded(manager, "3") @@ -269,55 +273,57 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onExecutorAdded(manager, "5") onExecutorAdded(manager, "6") onExecutorAdded(manager, "7") - assert(executorIds(manager).size === 7) + onExecutorAdded(manager, "8") + assert(executorIds(manager).size === 8) // Remove until limit assert(removeExecutor(manager, "1")) assert(removeExecutor(manager, "2")) - assert(!removeExecutor(manager, "3")) // lower limit reached - assert(!removeExecutor(manager, "4")) + assert(removeExecutor(manager, "3")) + assert(!removeExecutor(manager, "4")) // lower limit reached + assert(!removeExecutor(manager, "5")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") assert(executorIds(manager).size === 5) // Add until limit - assert(addExecutors(manager) === 5) // upper limit reached + assert(addExecutors(manager) === 2) // upper limit reached assert(addExecutors(manager) === 0) - assert(!removeExecutor(manager, "3")) // still at lower limit - assert(!removeExecutor(manager, "4")) - onExecutorAdded(manager, "8") + assert(!removeExecutor(manager, "4")) // still at lower limit + assert(!removeExecutor(manager, "5")) onExecutorAdded(manager, "9") onExecutorAdded(manager, "10") onExecutorAdded(manager, "11") onExecutorAdded(manager, "12") + onExecutorAdded(manager, "13") assert(executorIds(manager).size === 10) // Remove succeeds again, now that we are no longer at the lower limit - assert(removeExecutor(manager, "3")) assert(removeExecutor(manager, "4")) assert(removeExecutor(manager, "5")) assert(removeExecutor(manager, "6")) + assert(removeExecutor(manager, "7")) assert(executorIds(manager).size === 10) - assert(addExecutors(manager) === 1) - onExecutorRemoved(manager, "3") + assert(addExecutors(manager) === 0) onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") assert(executorIds(manager).size === 8) - // Add succeeds again, now that we are no longer at the upper limit - // Number of executors added restarts at 1 - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 1) // upper limit reached + // Number of executors pending restarts at 1 + assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 0) assert(executorIds(manager).size === 8) - onExecutorRemoved(manager, "5") onExecutorRemoved(manager, "6") - onExecutorAdded(manager, "13") + onExecutorRemoved(manager, "7") onExecutorAdded(manager, "14") + onExecutorAdded(manager, "15") assert(executorIds(manager).size === 8) assert(addExecutors(manager) === 0) // still at upper limit - onExecutorAdded(manager, "15") onExecutorAdded(manager, "16") + onExecutorAdded(manager, "17") assert(executorIds(manager).size === 10) + assert(numExecutorsTarget(manager) === 10) } test("starting/canceling add timer") { @@ -400,33 +406,33 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { } test("mock polling loop with no events") { - sc = createSparkContext(1, 20) + sc = createSparkContext(0, 20) val manager = sc.executorAllocationManager.get val clock = new ManualClock(2020L) manager.setClock(clock) // No events - we should not be adding or removing - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) schedule(manager) - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(100L) schedule(manager) - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(1000L) schedule(manager) - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(10000L) schedule(manager) - assert(numExecutorsPending(manager) === 0) + assert(numExecutorsTarget(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) } test("mock polling loop add behavior") { - sc = createSparkContext(1, 20) + sc = createSparkContext(0, 20) val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -436,43 +442,43 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsPending(manager) === 0) // timer not exceeded yet + assert(numExecutorsTarget(manager) === 0) // timer not exceeded yet clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 1) // first timer exceeded + assert(numExecutorsTarget(manager) === 1) // first timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet + assert(numExecutorsTarget(manager) === 1) // second timer not exceeded yet clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded + assert(numExecutorsTarget(manager) === 1 + 2) // second timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded + assert(numExecutorsTarget(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 7) // timer is canceled + assert(numExecutorsTarget(manager) === 7) // timer is canceled clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 7) + assert(numExecutorsTarget(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 7 + 1) // timer restarted + assert(numExecutorsTarget(manager) === 7 + 1) // timer restarted clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 7 + 1 + 2) + assert(numExecutorsTarget(manager) === 7 + 1 + 2) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4) + assert(numExecutorsTarget(manager) === 7 + 1 + 2 + 4) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsPending(manager) === 20) // limit reached + assert(numExecutorsTarget(manager) === 20) // limit reached } test("mock polling loop remove behavior") { @@ -665,16 +671,31 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).contains("executor-2")) assert(!removeTimes(manager).contains("executor-1")) } -} -/** - * Helper methods for testing ExecutorAllocationManager. - * This includes methods to access private methods and fields in ExecutorAllocationManager. - */ -private object ExecutorAllocationManagerSuite extends PrivateMethodTester { - private val schedulerBacklogTimeout = 1L - private val sustainedSchedulerBacklogTimeout = 2L - private val executorIdleTimeout = 3L + test("avoid ramp up when target < running executors") { + sc = createSparkContext(0, 100000) + val manager = sc.executorAllocationManager.get + val stage1 = createStageInfo(0, 1000) + sc.listenerBus.postToAll(SparkListenerStageSubmitted(stage1)) + + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 4) + assert(addExecutors(manager) === 8) + assert(numExecutorsTarget(manager) === 15) + (0 until 15).foreach { i => + onExecutorAdded(manager, s"executor-$i") + } + assert(executorIds(manager).size === 15) + sc.listenerBus.postToAll(SparkListenerStageCompleted(stage1)) + + adjustRequestedExecutors(manager) + assert(numExecutorsTarget(manager) === 0) + + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 1000))) + addExecutors(manager) + assert(numExecutorsTarget(manager) === 16) + } private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = { val conf = new SparkConf() @@ -683,16 +704,30 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", + s"${schedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", - sustainedSchedulerBacklogTimeout.toString) - .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + s"${sustainedSchedulerBacklogTimeout.toString}s") + .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") - new SparkContext(conf) + val sc = new SparkContext(conf) + contexts += sc + sc } +} + +/** + * Helper methods for testing ExecutorAllocationManager. + * This includes methods to access private methods and fields in ExecutorAllocationManager. + */ +private object ExecutorAllocationManagerSuite extends PrivateMethodTester { + private val schedulerBacklogTimeout = 1L + private val sustainedSchedulerBacklogTimeout = 2L + private val executorIdleTimeout = 3L + private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = { - new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details") + new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details") } private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { @@ -704,7 +739,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { * ------------------------------------------------------- */ private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) - private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending) + private val _numExecutorsTarget = PrivateMethod[Int]('numExecutorsTarget) private val _maxNumExecutorsNeeded = PrivateMethod[Int]('maxNumExecutorsNeeded) private val _executorsPendingToRemove = PrivateMethod[collection.Set[String]]('executorsPendingToRemove) @@ -713,7 +748,8 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) private val _schedule = PrivateMethod[Unit]('schedule) private val _addExecutors = PrivateMethod[Int]('addExecutors) - private val _addOrCancelExecutorRequests = PrivateMethod[Int]('addOrCancelExecutorRequests) + private val _updateAndSyncNumExecutorsTarget = + PrivateMethod[Int]('updateAndSyncNumExecutorsTarget) private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) @@ -726,8 +762,8 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _numExecutorsToAdd() } - private def numExecutorsPending(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsPending() + private def numExecutorsTarget(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _numExecutorsTarget() } private def executorsPendingToRemove( @@ -757,7 +793,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { } private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _addOrCancelExecutorRequests(0L) + manager invokePrivate _updateAndSyncNumExecutorsTarget(0L) } private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 5fdf6bc2777e3..c0439f934813e 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,10 +20,9 @@ package org.apache.spark import java.io._ import java.net.URI import java.util.jar.{JarEntry, JarOutputStream} -import javax.net.ssl.SSLHandshakeException +import javax.net.ssl.SSLException -import com.google.common.io.ByteStreams -import org.apache.commons.io.{FileUtils, IOUtils} +import com.google.common.io.{ByteStreams, Files} import org.apache.commons.lang3.RandomUtils import org.scalatest.FunSuite @@ -228,7 +227,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { try { server.initialize() - intercept[SSLHandshakeException] { + intercept[SSLException] { fileTransferTest(server) } } finally { @@ -239,7 +238,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = { val randomContent = RandomUtils.nextBytes(100) val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir) - FileUtils.writeByteArrayToFile(file, randomContent) + Files.write(randomContent, file) server.addFile(file) val uri = new URI(server.serverUri + "/files/" + file.getName) @@ -254,7 +253,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { Utils.setupSecureURLConnection(connection, sm) } - val buf = IOUtils.toByteArray(connection.getInputStream) + val buf = ByteStreams.toByteArray(connection.getInputStream) assert(buf === randomContent) } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 7acd27c735727..c8f08eed47c76 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -222,7 +222,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) val output = - sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) + sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } @@ -451,7 +451,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } @@ -459,8 +460,10 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) @@ -471,16 +474,20 @@ class FileSuite extends FunSuite with LocalSparkContext { val sf = new SparkConf() sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") sc = new SparkContext(sf) - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) @@ -492,7 +499,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala new file mode 100644 index 0000000000000..b789912e9ebef --- /dev/null +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -0,0 +1,81 @@ +/* + * 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 scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import org.scalatest.FunSuite +import org.mockito.Mockito.{mock, spy, verify, when} +import org.mockito.Matchers +import org.mockito.Matchers._ + +import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.util.RpcUtils +import org.scalatest.concurrent.Eventually._ + +class HeartbeatReceiverSuite extends FunSuite with LocalSparkContext { + + test("HeartbeatReceiver") { + sc = spy(new SparkContext("local[2]", "test")) + val scheduler = mock(classOf[TaskScheduler]) + when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(true) + when(sc.taskScheduler).thenReturn(scheduler) + + val heartbeatReceiver = new HeartbeatReceiver(sc) + sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver).send(TaskSchedulerIsSet) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(heartbeatReceiver.scheduler != null) + } + val receiverRef = RpcUtils.makeDriverRef("heartbeat", sc.conf, sc.env.rpcEnv) + + val metrics = new TaskMetrics + val blockManagerId = BlockManagerId("executor-1", "localhost", 12345) + val response = receiverRef.askWithRetry[HeartbeatResponse]( + Heartbeat("executor-1", Array(1L -> metrics), blockManagerId)) + + verify(scheduler).executorHeartbeatReceived( + Matchers.eq("executor-1"), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + assert(false === response.reregisterBlockManager) + } + + test("HeartbeatReceiver re-register") { + sc = spy(new SparkContext("local[2]", "test")) + val scheduler = mock(classOf[TaskScheduler]) + when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(false) + when(sc.taskScheduler).thenReturn(scheduler) + + val heartbeatReceiver = new HeartbeatReceiver(sc) + sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver).send(TaskSchedulerIsSet) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(heartbeatReceiver.scheduler != null) + } + val receiverRef = RpcUtils.makeDriverRef("heartbeat", sc.conf, sc.env.rpcEnv) + + val metrics = new TaskMetrics + val blockManagerId = BlockManagerId("executor-1", "localhost", 12345) + val response = receiverRef.askWithRetry[HeartbeatResponse]( + Heartbeat("executor-1", Array(1L -> metrics), blockManagerId)) + + verify(scheduler).executorHeartbeatReceived( + Matchers.eq("executor-1"), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + assert(true === response.reregisterBlockManager) + } +} diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index d895230ecf330..51348c039b5c9 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -51,7 +51,7 @@ private object ImplicitOrderingSuite { override def compare(o: OrderedClass): Int = ??? } - def basicMapExpectations(rdd: RDD[Int]) = { + def basicMapExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.map(x => (x, x)).keyOrdering.isDefined, "rdd.map(x => (x, x)).keyOrdering.isDefined"), (rdd.map(x => (1, x)).keyOrdering.isDefined, @@ -68,7 +68,7 @@ private object ImplicitOrderingSuite { "rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined")) } - def otherRDDMethodExpectations(rdd: RDD[Int]) = { + def otherRDDMethodExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.groupBy(x => x).keyOrdering.isDefined, "rdd.groupBy(x => x).keyOrdering.isDefined"), (rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty, @@ -82,4 +82,4 @@ private object ImplicitOrderingSuite { (rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined, "rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined")) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 21487bc24d58a..ae17fc60e4a43 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -141,6 +141,41 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter assert(jobB.get() === 100) } + test("inherited job group (SPARK-6629)") { + sc = new SparkContext("local[2]", "test") + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + sc.setJobGroup("jobA", "this is a job to be cancelled") + @volatile var exception: Exception = null + val jobA = new Thread() { + // The job group should be inherited by this thread + override def run(): Unit = { + exception = intercept[SparkException] { + sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() + } + } + } + jobA.start() + + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + sc.cancelJobGroup("jobA") + jobA.join(10000) + assert(!jobA.isAlive) + assert(exception.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + val jobB = sc.parallelize(1 to 100, 2).countAsync() + assert(jobB.get() === 100) + } + test("job group with interruption") { sc = new SparkContext("local[2]", "test") @@ -188,7 +223,7 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter val rdd = sc.parallelize(1 to 10, 2).map { i => JobCancellationSuite.twoJobsSharingStageSemaphore.acquire() (i, i) - }.reduceByKey(_+_) + }.reduceByKey(_ + _) val f1 = rdd.collectAsync() val f2 = rdd.countAsync() diff --git a/core/src/test/scala/org/apache/spark/JsonTestUtils.scala b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala new file mode 100644 index 0000000000000..ba367cd476146 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala @@ -0,0 +1,34 @@ +/* + * 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 org.json4s._ +import org.json4s.jackson.JsonMethods + +trait JsonTestUtils { + def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { + val Diff(c, a, d) = validateJson.diff(expectedJson) + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + import org.scalactic.TripleEquals._ + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDeleted:\n${JsonMethods.pretty(d)}") + } + +} diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 53e367a61715b..8bf2e55defd02 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -37,7 +37,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self super.afterEach() } - def resetSparkContext() = { + def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) sc = null } @@ -54,7 +54,7 @@ object LocalSparkContext { } /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ - def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { + def withSpark[T](sc: SparkContext)(f: SparkContext => T): T = { try { f(sc) } finally { diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index ccfe0678cb1c3..6ed057a7cab97 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -17,34 +17,37 @@ package org.apache.spark -import scala.concurrent.Await - -import akka.actor._ -import akka.testkit.TestActorRef +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, isA} import org.scalatest.FunSuite +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite { private val conf = new SparkConf + def createRpcEnv(name: String, host: String = "localhost", port: Int = 0, + securityManager: SecurityManager = new SecurityManager(conf)): RpcEnv = { + RpcEnv.create(name, host, port, conf, securityManager) + } + test("master start and stop") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register shuffle and fetch") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) assert(tracker.containsShuffle(10)) val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) @@ -57,13 +60,14 @@ class MapOutputTrackerSuite extends FunSuite { assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), (BlockManagerId("b", "hostB", 1000), size10000))) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register and unregister shuffle") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) @@ -78,14 +82,14 @@ class MapOutputTrackerSuite extends FunSuite { assert(tracker.getServerStatuses(10, 0).isEmpty) tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("master register shuffle and unregister map output and fetch") { - val actorSystem = ActorSystem("test") + val rpcEnv = createRpcEnv("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = - actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) @@ -104,25 +108,21 @@ class MapOutputTrackerSuite extends FunSuite { intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) } tracker.stop() - actorSystem.shutdown() + rpcEnv.shutdown() } test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, - securityManager = new SecurityManager(conf)) + val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, - securityManager = new SecurityManager(conf)) + val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() @@ -147,43 +147,47 @@ class MapOutputTrackerSuite extends FunSuite { masterTracker.stop() slaveTracker.stop() - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch below akka frame size") { val newConf = new SparkConf newConf.set("spark.akka.frameSize", "1") - newConf.set("spark.akka.askTimeout", "1") // Fail fast + newConf.set("spark.rpc.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) - val actorSystem = ActorSystem("test") - val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) - val masterActor = actorRef.underlyingActor + val rpcEnv = createRpcEnv("spark") + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) // Frame size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) - masterActor.receive(GetMapOutputStatuses(10)) + val sender = mock(classOf[RpcEndpointRef]) + when(sender.address).thenReturn(RpcAddress("localhost", 12345)) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.sender).thenReturn(sender) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(10)) + verify(rpcCallContext).reply(any()) + verify(rpcCallContext, never()).sendFailure(any()) // masterTracker.stop() // this throws an exception - actorSystem.shutdown() + rpcEnv.shutdown() } test("remote fetch exceeds akka frame size") { val newConf = new SparkConf newConf.set("spark.akka.frameSize", "1") - newConf.set("spark.akka.askTimeout", "1") // Fail fast + newConf.set("spark.rpc.askTimeout", "1") // Fail fast val masterTracker = new MapOutputTrackerMaster(conf) - val actorSystem = ActorSystem("test") - val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) - val masterActor = actorRef.underlyingActor + val rpcEnv = createRpcEnv("test") + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) - // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. + // Frame 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) @@ -191,9 +195,15 @@ class MapOutputTrackerSuite extends FunSuite { masterTracker.registerMapOutput(20, i, new CompressedMapStatus( BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } - intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } + val sender = mock(classOf[RpcEndpointRef]) + when(sender.address).thenReturn(RpcAddress("localhost", 12345)) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.sender).thenReturn(sender) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) + verify(rpcCallContext, never()).reply(any()) + verify(rpcCallContext).sendFailure(isA(classOf[SparkException])) // masterTracker.stop() // this throws an exception - actorSystem.shutdown() + rpcEnv.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index b7532314ada01..47e3bf6e1ac41 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -92,7 +92,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet test("RangePartitioner for keys that are not Comparable (but with Ordering)") { // Row does not extend Comparable, but has an implicit Ordering defined. implicit object RowOrdering extends Ordering[Row] { - override def compare(x: Row, y: Row) = x.value - y.value + override def compare(x: Row, y: Row): Int = x.value - y.value } val rdd = sc.parallelize(1 to 4500).map(x => (Row(x), Row(x))) @@ -212,20 +212,24 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet val arrPairs: RDD[(Array[Int], Int)] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => (Array(x), x)) - assert(intercept[SparkException]{ arrs.distinct() }.getMessage.contains("array")) + def verify(testFun: => Unit): Unit = { + intercept[SparkException](testFun).getMessage.contains("array") + } + + verify(arrs.distinct()) // We can't catch all usages of arrays, since they might occur inside other collections: // assert(fails { arrPairs.distinct() }) - assert(intercept[SparkException]{ arrPairs.partitionBy(new HashPartitioner(2)) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.rightOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.fullOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.groupByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKeyApprox(1) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.cogroup(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) + verify(arrPairs.partitionBy(new HashPartitioner(2))) + verify(arrPairs.join(arrPairs)) + verify(arrPairs.leftOuterJoin(arrPairs)) + verify(arrPairs.rightOuterJoin(arrPairs)) + verify(arrPairs.fullOuterJoin(arrPairs)) + verify(arrPairs.groupByKey()) + verify(arrPairs.countByKey()) + verify(arrPairs.countByKeyApprox(1)) + verify(arrPairs.cogroup(arrPairs)) + verify(arrPairs.reduceByKeyLocally(_ + _)) + verify(arrPairs.reduceByKey(_ + _)) } test("zero-length partitions should be correctly handled") { diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 444a33371bd71..93f46ef11c0e2 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -36,7 +36,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val opts = SSLOptions.parse(conf, "spark.ssl") @@ -52,7 +53,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test resolving property with defaults specified ") { @@ -66,7 +68,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) @@ -83,7 +86,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test whether defaults can be overridden ") { @@ -99,7 +103,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") conf.set("spark.ssl.protocol", "SSLv3") diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala index ace8123a8961f..308b9ea17708d 100644 --- a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -21,10 +21,11 @@ import java.io.File object SSLSampleConfigs { val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath - val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath + val untrustedKeyStorePath = new File( + this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath - def sparkSSLConfig() = { + def sparkSSLConfig(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", keyStorePath) @@ -38,7 +39,7 @@ object SSLSampleConfigs { conf } - def sparkSSLConfigUntrusted() = { + def sparkSSLConfigUntrusted(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", untrustedKeyStorePath) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index f57921b768310..d7180516029d5 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -142,7 +142,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("shuffle on mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new ShuffledRDD[Int, Int, Int](pairs, @@ -155,7 +155,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // This is not in SortingSuite because of the local cluster setup. // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) @@ -169,7 +169,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("cogroup using mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) @@ -196,7 +196,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("subtract mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) @@ -242,14 +242,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex shuffleSpillCompress <- Set(true, false); shuffleCompress <- Set(true, false) ) { - val conf = new SparkConf() + val myConf = conf.clone() .setAppName("test") .setMaster("local") .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) .set("spark.shuffle.compress", shuffleCompress.toString) .set("spark.shuffle.memoryFraction", "0.001") resetSparkContext() - sc = new SparkContext(conf) + sc = new SparkContext(myConf) try { sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect() } catch { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index e08210ae60d17..68d08e32f9aa4 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,14 +19,35 @@ package org.apache.spark import java.util.concurrent.{TimeUnit, Executors} +import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.{Try, Random} import org.scalatest.FunSuite +import org.apache.spark.network.util.ByteUnit import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} -import org.apache.spark.util.ResetSystemProperties +import org.apache.spark.util.{RpcUtils, ResetSystemProperties} import com.esotericsoftware.kryo.Kryo class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { + test("Test byteString conversion") { + val conf = new SparkConf() + // Simply exercise the API, we don't need a complete conversion test since that's handled in + // UtilsSuite.scala + assert(conf.getSizeAsBytes("fake","1k") === ByteUnit.KiB.toBytes(1)) + assert(conf.getSizeAsKb("fake","1k") === ByteUnit.KiB.toKiB(1)) + assert(conf.getSizeAsMb("fake","1k") === ByteUnit.KiB.toMiB(1)) + assert(conf.getSizeAsGb("fake","1k") === ByteUnit.KiB.toGiB(1)) + } + + test("Test timeString conversion") { + val conf = new SparkConf() + // Simply exercise the API, we don't need a complete conversion test since that's handled in + // UtilsSuite.scala + assert(conf.getTimeAsMs("fake","1ms") === TimeUnit.MILLISECONDS.toMillis(1)) + assert(conf.getTimeAsSeconds("fake","1000ms") === TimeUnit.MILLISECONDS.toSeconds(1000)) + } + test("loading from system properties") { System.setProperty("spark.test.testProperty", "2") val conf = new SparkConf() @@ -197,6 +218,51 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro serializer.newInstance().serialize(new StringBuffer()) } + test("deprecated configs") { + val conf = new SparkConf() + val newName = "spark.history.fs.update.interval" + + assert(!conf.contains(newName)) + + conf.set("spark.history.updateInterval", "1") + assert(conf.get(newName) === "1") + + conf.set("spark.history.fs.updateInterval", "2") + assert(conf.get(newName) === "2") + + conf.set("spark.history.fs.update.interval.seconds", "3") + assert(conf.get(newName) === "3") + + conf.set(newName, "4") + assert(conf.get(newName) === "4") + + val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size + assert(count === 4) + + conf.set("spark.yarn.applicationMaster.waitTries", "42") + assert(conf.getTimeAsSeconds("spark.yarn.am.waitTime") === 420) + } + + test("akka deprecated configs") { + val conf = new SparkConf() + + assert(!conf.contains("spark.rpc.numRetries")) + assert(!conf.contains("spark.rpc.retry.wait")) + assert(!conf.contains("spark.rpc.askTimeout")) + assert(!conf.contains("spark.rpc.lookupTimeout")) + + conf.set("spark.akka.num.retries", "1") + assert(RpcUtils.numRetries(conf) === 1) + + conf.set("spark.akka.retry.wait", "2") + assert(RpcUtils.retryWaitMs(conf) === 2L) + + conf.set("spark.akka.askTimeout", "3") + assert(RpcUtils.askTimeout(conf) === (3 seconds)) + + conf.set("spark.akka.lookupTimeout", "4") + assert(RpcUtils.lookupTimeout(conf) === (4 seconds)) + } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index b07c4d93db4e6..9049db7755358 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,16 +18,21 @@ package org.apache.spark import java.io.File +import java.util.concurrent.TimeUnit import com.google.common.base.Charsets._ import com.google.common.io.Files import org.scalatest.FunSuite -import org.apache.hadoop.io.BytesWritable - +import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} +import org.apache.hadoop.mapred.TextInputFormat +import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.apache.spark.util.Utils +import scala.concurrent.Await +import scala.concurrent.duration.Duration + class SparkContextSuite extends FunSuite with LocalSparkContext { test("Only one SparkContext may be active at a time") { @@ -64,6 +69,26 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { } } + test("Test getOrCreate") { + var sc2: SparkContext = null + SparkContext.clearActiveContext() + val conf = new SparkConf().setAppName("test").setMaster("local") + + sc = SparkContext.getOrCreate(conf) + + assert(sc.getConf.get("spark.app.name").equals("test")) + sc2 = SparkContext.getOrCreate(new SparkConf().setAppName("test2").setMaster("local")) + assert(sc2.getConf.get("spark.app.name").equals("test")) + assert(sc === sc2) + assert(sc eq sc2) + + // Try creating second context to confirm that it's still possible, if desired + sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true")) + + sc2.stop() + } + test("BytesWritable implicit conversion is correct") { // Regression test for SPARK-3121 val bytesWritable = new BytesWritable() @@ -111,11 +136,13 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { if (length1 != gotten1.length()) { throw new SparkException( - s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + s"file has different length $length1 than added file ${gotten1.length()} : " + + absolutePath1) } if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) + s"file has different length $length2 than added file ${gotten2.length()} : " + + absolutePath2) } if (absolutePath1 == gotten1.getAbsolutePath) { @@ -173,4 +200,78 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } + + test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val future = sc.parallelize(Seq(0)).foreachAsync(_ => {Thread.sleep(1000L)}) + sc.cancelJobGroup("nonExistGroupId") + Await.ready(future, Duration(2, TimeUnit.SECONDS)) + + // In SPARK-6414, sc.cancelJobGroup will cause NullPointerException and cause + // SparkContext to shutdown, so the following assertion will fail. + assert(sc.parallelize(1 to 10).count() == 10L) + } finally { + sc.stop() + } + } + + test("Comma separated paths for newAPIHadoopFile/wholeTextFiles/binaryFiles (SPARK-7155)") { + // Regression test for SPARK-7155 + // dir1 and dir2 are used for wholeTextFiles and binaryFiles + val dir1 = Utils.createTempDir() + val dir2 = Utils.createTempDir() + + val dirpath1=dir1.getAbsolutePath + val dirpath2=dir2.getAbsolutePath + + // file1 and file2 are placed inside dir1, they are also used for + // textFile, hadoopFile, and newAPIHadoopFile + // file3, file4 and file5 are placed inside dir2, they are used for + // textFile, hadoopFile, and newAPIHadoopFile as well + val file1 = new File(dir1, "part-00000") + val file2 = new File(dir1, "part-00001") + val file3 = new File(dir2, "part-00000") + val file4 = new File(dir2, "part-00001") + val file5 = new File(dir2, "part-00002") + + val filepath1=file1.getAbsolutePath + val filepath2=file2.getAbsolutePath + val filepath3=file3.getAbsolutePath + val filepath4=file4.getAbsolutePath + val filepath5=file5.getAbsolutePath + + + try { + // Create 5 text files. + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, UTF_8) + Files.write("someline1 in file2\nsomeline2 in file2", file2, UTF_8) + Files.write("someline1 in file3", file3, UTF_8) + Files.write("someline1 in file4\nsomeline2 in file4", file4, UTF_8) + Files.write("someline1 in file2\nsomeline2 in file5", file5, UTF_8) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file1 and file2 + assert(sc.textFile(filepath1 + "," + filepath2).count() == 5L) + assert(sc.hadoopFile(filepath1 + "," + filepath2, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath1 + "," + filepath2, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file3, file4, and file5 + assert(sc.textFile(filepath3 + "," + filepath4 + "," + filepath5).count() == 5L) + assert(sc.hadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test wholeTextFiles, and binaryFiles for dir1 and dir2 + assert(sc.wholeTextFiles(dirpath1 + "," + dirpath2).count() == 5L) + assert(sc.binaryFiles(dirpath1 + "," + dirpath2).count() == 5L) + + } finally { + sc.stop() + } + } } diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 41d6ea29d5b06..084eb237d70d1 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -82,7 +82,8 @@ class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext { secondJobFuture.jobIds.head } eventually(timeout(10 seconds)) { - sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId)) + sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be ( + Set(firstJobId, secondJobId)) } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index b5383d553add1..10917c866cc7d 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark -import java.util.concurrent.Semaphore +import java.util.concurrent.{TimeUnit, Semaphore} import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger +import org.apache.spark.scheduler._ import org.scalatest.FunSuite /** @@ -189,4 +190,47 @@ class ThreadingSuite extends FunSuite with LocalSparkContext { assert(sc.getLocalProperty("test") === "parent") assert(sc.getLocalProperty("Foo") === null) } + + test("mutations to local properties should not affect submitted jobs (SPARK-6629)") { + val jobStarted = new Semaphore(0) + val jobEnded = new Semaphore(0) + @volatile var jobResult: JobResult = null + + sc = new SparkContext("local", "test") + sc.setJobGroup("originalJobGroupId", "description") + sc.addSparkListener(new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobStarted.release() + } + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + jobResult = jobEnd.jobResult + jobEnded.release() + } + }) + + // Create a new thread which will inherit the current thread's properties + val thread = new Thread() { + override def run(): Unit = { + assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "originalJobGroupId") + // Sleeps for a total of 10 seconds, but allows cancellation to interrupt the task + try { + sc.parallelize(1 to 100).foreach { x => + Thread.sleep(100) + } + } catch { + case s: SparkException => // ignored so that we don't print noise in test logs + } + } + } + thread.start() + // Wait for the job to start, then mutate the original properties, which should have been + // inherited by the running job but hopefully defensively copied or snapshotted: + jobStarted.tryAcquire(10, TimeUnit.SECONDS) + sc.setJobGroup("modifiedJobGroupId", "description") + // Canceling the original job group should cancel the running job. In other words, the + // modification of the properties object should not affect the properties of running jobs + sc.cancelJobGroup("originalJobGroupId") + jobEnded.tryAcquire(10, TimeUnit.SECONDS) + assert(jobResult.isInstanceOf[JobFailed]) + } } diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index af3272692d7a1..c8fdfa693912e 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -33,7 +33,7 @@ class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { val broadcast = rdd.context.broadcast(list) val bid = broadcast.id - def doSomething() = { + def doSomething(): Set[(Int, Boolean)] = { rdd.map { x => val bm = SparkEnv.get.blockManager // Check if broadcast block was fetched diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 518073dcbb64e..745f9eeee7536 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -46,5 +46,4 @@ class ClientSuite extends FunSuite with Matchers { // Invalid syntax. ClientArguments.isValidJarUrl("hdfs:") should be (false) } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 68b5776fc6515..e04a79284175c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,9 +28,9 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} -class JsonProtocolSuite extends FunSuite { +class JsonProtocolSuite extends FunSuite with JsonTestUtils { test("writeApplicationInfo") { val output = JsonProtocol.writeApplicationInfo(createAppInfo()) @@ -100,13 +100,13 @@ class JsonProtocolSuite extends FunSuite { appInfo } - def createDriverCommand() = new Command( + def createDriverCommand(): Command = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) - def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, - false, createDriverCommand()) + def createDriverDesc(): DriverDescription = + new DriverDescription("hdfs://some-dir/some.jar", 100, 3, false, createDriverCommand()) def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date()) @@ -124,8 +124,9 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) } def assertValidJson(json: JValue) { @@ -135,16 +136,6 @@ class JsonProtocolSuite extends FunSuite { case e: JsonParseException => fail("Invalid Json detected", e) } } - - def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { - val Diff(c, a, d) = validateJson diff expectedJson - val validatePretty = JsonMethods.pretty(validateJson) - val expectedPretty = JsonMethods.pretty(expectedJson) - val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" - assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") - assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") - assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") - } } object JsonConstants { diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 54dd7c9c45c61..c93d16f8a1586 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.net.URL +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.io.Source @@ -56,7 +57,7 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") { val SPARK_PUBLIC_DNS = "public_dns" class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS else super.getenv(name) } @@ -65,16 +66,17 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { new MySparkConf().setAll(getAll) } } - val conf = new MySparkConf() + val conf = new MySparkConf().set( + "spark.extraListeners", classOf[SaveExecutorInfo].getName) sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - val listener = new SaveExecutorInfo - sc.addSparkListener(listener) - // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) info.logUrlMap.values.foreach { logUrl => @@ -82,12 +84,12 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { } } } +} - private class SaveExecutorInfo extends SparkListener { - val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() +private[spark] class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { - addedExecutorInfos(executor.executorId) = executor.executorInfo - } + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4561e5b8e9663..61c95419aedcf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -231,7 +231,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val childArgsStr = childArgs.mkString(" ") if (useRest) { childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.rest.StandaloneRestClient") + mainClass should be ("org.apache.spark.deploy.rest.RestSubmissionClient") } else { childArgsStr should startWith ("--supervise --memory 4g --cores 5") childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" @@ -320,7 +320,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties runSparkSubmit(args) } - test("includes jars passed in through --jars") { + ignore("includes jars passed in through --jars") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) @@ -334,7 +334,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties runSparkSubmit(args) } - test("includes jars passed in through --packages") { + ignore("includes jars passed in through --packages") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val packagesString = "com.databricks:spark-csv_2.10:0.1,com.databricks:spark-avro_2.10:0.1" val args = Seq( diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 8bcca926097a1..2df2597e058cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy import java.io.{PrintStream, OutputStream, File} +import org.apache.ivy.core.settings.IvySettings + import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -56,24 +58,23 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { } test("create repo resolvers") { - val resolver1 = SparkSubmitUtils.createRepoResolvers(None) + val settings = new IvySettings + val res1 = SparkSubmitUtils.createRepoResolvers(None, settings) // should have central and spark-packages by default - assert(resolver1.getResolvers.size() === 2) - assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central") - assert(resolver1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "spark-packages") + assert(res1.getResolvers.size() === 4) + assert(res1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "local-m2-cache") + assert(res1.getResolvers.get(1).asInstanceOf[IBiblioResolver].getName === "local-ivy-cache") + assert(res1.getResolvers.get(2).asInstanceOf[IBiblioResolver].getName === "central") + assert(res1.getResolvers.get(3).asInstanceOf[IBiblioResolver].getName === "spark-packages") val repos = "a/1,b/2,c/3" - val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos)) - assert(resolver2.getResolvers.size() === 5) + val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos), settings) + assert(resolver2.getResolvers.size() === 7) val expected = repos.split(",").map(r => s"$r/") resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: IBiblioResolver, i) => - if (i == 0) { - assert(resolver.getName === "central") - } else if (i == 1) { - assert(resolver.getName === "spark-packages") - } else { - assert(resolver.getName === s"repo-${i - 1}") - assert(resolver.getRoot === expected(i - 2)) + if (i > 3) { + assert(resolver.getName === s"repo-${i - 3}") + assert(resolver.getRoot === expected(i - 4)) } } } @@ -116,7 +117,7 @@ class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { } } - test("neglects Spark and Spark's dependencies") { + ignore("neglects Spark and Spark's dependencies") { val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index e908ba604ebed..a0a0afa48833e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI +import java.util.concurrent.TimeUnit import scala.io.Source @@ -30,7 +31,7 @@ import org.scalatest.Matchers import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io._ import org.apache.spark.scheduler._ -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.{JsonProtocol, ManualClock, Utils} class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { @@ -47,10 +48,11 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers /** Create a fake log file using the new log format used in Spark 1.3+ */ private def newLogFile( appId: String, + appAttemptId: Option[String], inProgress: Boolean, codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, appId) + val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId) val logPath = new URI(logUri).getPath + ip new File(logPath) } @@ -59,22 +61,23 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val newAppComplete = newLogFile("new1", inProgress = false) + val newAppComplete = newLogFile("new1", None, inProgress = false) writeFile(newAppComplete, true, None, - SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), + SparkListenerApplicationStart("new-app-complete", None, 1L, "test", None), SparkListenerApplicationEnd(5L) ) // Write a new-style application log. - val newAppCompressedComplete = newLogFile("new1compressed", inProgress = false, Some("lzf")) + val newAppCompressedComplete = newLogFile("new1compressed", None, inProgress = false, + Some("lzf")) writeFile(newAppCompressedComplete, true, None, - SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test"), + SparkListenerApplicationStart("new-app-compressed-complete", None, 1L, "test", None), SparkListenerApplicationEnd(4L)) // Write an unfinished app, new-style. - val newAppIncomplete = newLogFile("new2", inProgress = true) + val newAppIncomplete = newLogFile("new2", None, inProgress = true) writeFile(newAppIncomplete, true, None, - SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") + SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test", None) ) // Write an old-style application log. @@ -82,7 +85,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers oldAppComplete.mkdir() createEmptyFile(new File(oldAppComplete, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(oldAppComplete, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("old-app-complete", None, 2L, "test"), + SparkListenerApplicationStart("old-app-complete", None, 2L, "test", None), SparkListenerApplicationEnd(3L) ) createEmptyFile(new File(oldAppComplete, provider.APPLICATION_COMPLETE)) @@ -96,33 +99,45 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers oldAppIncomplete.mkdir() createEmptyFile(new File(oldAppIncomplete, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(oldAppIncomplete, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test") + SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test", None) ) // Force a reload of data from the log directory, and check that both logs are loaded. // Take the opportunity to check that the offset checks work as expected. - provider.checkForLogs() + updateAndCheck(provider) { list => + list.size should be (5) + list.count(_.attempts.head.completed) should be (3) + + def makeAppInfo( + id: String, + name: String, + start: Long, + end: Long, + lastMod: Long, + user: String, + completed: Boolean): ApplicationHistoryInfo = { + ApplicationHistoryInfo(id, name, + List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed))) + } - val list = provider.getListing().toSeq - list should not be (null) - list.size should be (5) - list.count(_.completed) should be (3) - - list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, - newAppComplete.lastModified(), "test", true)) - list(1) should be (ApplicationHistoryInfo(newAppCompressedComplete.getName(), - "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) - list(2) should be (ApplicationHistoryInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, - oldAppComplete.lastModified(), "test", true)) - list(3) should be (ApplicationHistoryInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, - -1L, oldAppIncomplete.lastModified(), "test", false)) - list(4) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, - -1L, newAppIncomplete.lastModified(), "test", false)) - - // Make sure the UI can be rendered. - list.foreach { case info => - val appUi = provider.getAppUI(info.id) - appUi should not be null + list(0) should be (makeAppInfo(newAppComplete.getName(), "new-app-complete", 1L, 5L, + newAppComplete.lastModified(), "test", true)) + list(1) should be (makeAppInfo(newAppCompressedComplete.getName(), + "new-app-compressed-complete", 1L, 4L, newAppCompressedComplete.lastModified(), "test", + true)) + list(2) should be (makeAppInfo(oldAppComplete.getName(), "old-app-complete", 2L, 3L, + oldAppComplete.lastModified(), "test", true)) + list(3) should be (makeAppInfo(oldAppIncomplete.getName(), "old-app-incomplete", 2L, -1L, + oldAppIncomplete.lastModified(), "test", false)) + list(4) should be (makeAppInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, -1L, + newAppIncomplete.lastModified(), "test", false)) + + // Make sure the UI can be rendered. + list.foreach { case info => + val appUi = provider.getAppUI(info.id, None) + appUi should not be null + appUi should not be None + } } } @@ -138,7 +153,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers logDir.mkdir() createEmptyFile(new File(logDir, provider.SPARK_VERSION_PREFIX + "1.0")) writeFile(new File(logDir, provider.LOG_PREFIX + "1"), false, Option(codec), - SparkListenerApplicationStart("app2", None, 2L, "test"), + SparkListenerApplicationStart("app2", None, 2L, "test", None), SparkListenerApplicationEnd(3L) ) createEmptyFile(new File(logDir, provider.COMPRESSION_CODEC_PREFIX + codecName)) @@ -159,52 +174,52 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-3697: ignore directories that cannot be read.") { - val logFile1 = newLogFile("new1", inProgress = false) + val logFile1 = newLogFile("new1", None, inProgress = false) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1-1", None, 1L, "test"), + SparkListenerApplicationStart("app1-1", None, 1L, "test", None), SparkListenerApplicationEnd(2L) ) - val logFile2 = newLogFile("new2", inProgress = false) + val logFile2 = newLogFile("new2", None, inProgress = false) writeFile(logFile2, true, None, - SparkListenerApplicationStart("app1-2", None, 1L, "test"), + SparkListenerApplicationStart("app1-2", None, 1L, "test", None), SparkListenerApplicationEnd(2L) ) logFile2.setReadable(false, false) val provider = new FsHistoryProvider(createTestConf()) - provider.checkForLogs() - - val list = provider.getListing().toSeq - list should not be (null) - list.size should be (1) + updateAndCheck(provider) { list => + list.size should be (1) + } } test("history file is renamed from inprogress to completed") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", inProgress = true) + val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L) ) - provider.checkForLogs() - val appListBeforeRename = provider.getListing() - appListBeforeRename.size should be (1) - appListBeforeRename.head.logPath should endWith(EventLoggingListener.IN_PROGRESS) + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should + endWith(EventLoggingListener.IN_PROGRESS) + } - logFile1.renameTo(newLogFile("app1", inProgress = false)) - provider.checkForLogs() - val appListAfterRename = provider.getListing() - appListAfterRename.size should be (1) - appListAfterRename.head.logPath should not endWith(EventLoggingListener.IN_PROGRESS) + logFile1.renameTo(newLogFile("app1", None, inProgress = false)) + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should not + endWith(EventLoggingListener.IN_PROGRESS) + } } test("SPARK-5582: empty log directory") { val provider = new FsHistoryProvider(createTestConf()) - val logFile1 = newLogFile("app1", inProgress = true) + val logFile1 = newLogFile("app1", None, inProgress = true) writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", None), SparkListenerApplicationEnd(2L)) val oldLog = new File(testDir, "old1") @@ -215,6 +230,126 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers appListAfterRename.size should be (1) } + test("apps with multiple attempts") { + val provider = new FsHistoryProvider(createTestConf()) + + val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(attempt1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), + SparkListenerApplicationEnd(2L) + ) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + } + + val attempt2 = newLogFile("app1", Some("attempt2"), inProgress = true) + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")) + ) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (2) + list.head.attempts.head.attemptId should be (Some("attempt2")) + } + + val completedAttempt2 = newLogFile("app1", Some("attempt2"), inProgress = false) + attempt2.delete() + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")), + SparkListenerApplicationEnd(4L) + ) + + updateAndCheck(provider) { list => + list should not be (null) + list.size should be (1) + list.head.attempts.size should be (2) + list.head.attempts.head.attemptId should be (Some("attempt2")) + } + + val app2Attempt1 = newLogFile("app2", Some("attempt1"), inProgress = false) + writeFile(attempt2, true, None, + SparkListenerApplicationStart("app2", Some("app2"), 5L, "test", Some("attempt1")), + SparkListenerApplicationEnd(6L) + ) + + updateAndCheck(provider) { list => + list.size should be (2) + list.head.attempts.size should be (1) + list.last.attempts.size should be (2) + list.head.attempts.head.attemptId should be (Some("attempt1")) + + list.foreach { case app => + app.attempts.foreach { attempt => + val appUi = provider.getAppUI(app.id, attempt.attemptId) + appUi should not be null + } + } + + } + } + + test("log cleaner") { + val maxAge = TimeUnit.SECONDS.toMillis(10) + val clock = new ManualClock(maxAge / 2) + val provider = new FsHistoryProvider( + createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + + val log1 = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(log1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), + SparkListenerApplicationEnd(2L) + ) + log1.setLastModified(0L) + + val log2 = newLogFile("app1", Some("attempt2"), inProgress = false) + writeFile(log2, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 3L, "test", Some("attempt2")), + SparkListenerApplicationEnd(4L) + ) + log2.setLastModified(clock.getTimeMillis()) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (2) + } + + // Move the clock forward so log1 exceeds the max age. + clock.advance(maxAge) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + list.head.attempts.head.attemptId should be (Some("attempt2")) + } + assert(!log1.exists()) + + // Do the same for the other log. + clock.advance(maxAge) + + updateAndCheck(provider) { list => + list.size should be (0) + } + assert(!log2.exists()) + } + + /** + * Asks the provider to check for logs and calls a function to perform checks on the updated + * app list. Example: + * + * updateAndCheck(provider) { list => + * // asserts + * } + */ + private def updateAndCheck(provider: FsHistoryProvider) + (checkFn: Seq[ApplicationHistoryInfo] => Unit): Unit = { + provider.checkForLogs() + provider.cleanLogs() + checkFn(provider.getListing().toSeq) + } + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], events: SparkListenerEvent*) = { val fstream = new FileOutputStream(file) @@ -224,9 +359,9 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers EventLoggingListener.initEventLog(new FileOutputStream(file)) } val writer = new OutputStreamWriter(bstream, "UTF-8") - try { + Utils.tryWithSafeFinally { events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) - } finally { + } { writer.close() } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 3a9963a5ce7b7..0744b68c695f9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -14,38 +14,178 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - -import scala.collection.mutable +import java.io.{File, FileInputStream, FileWriter, IOException} +import java.net.{HttpURLConnection, URL} +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} -import org.apache.hadoop.fs.Path -import org.mockito.Mockito.{when} -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.apache.commons.io.{FileUtils, IOUtils} +import org.mockito.Mockito.when +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.mock.MockitoSugar +import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf} import org.apache.spark.ui.SparkUI -class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { +/** + * A collection of tests against the historyserver, including comparing responses from the json + * metrics api to a set of known "golden files". If new endpoints / parameters are added, + * cases should be added to this test suite. The expected outcomes can be genered by running + * the HistoryServerSuite.main. Note that this will blindly generate new expectation files matching + * the current behavior -- the developer must verify that behavior is correct. + * + * Similarly, if the behavior is changed, HistoryServerSuite.main can be run to update the + * expectations. However, in general this should be done with extreme caution, as the metrics + * are considered part of Spark's public api. + */ +class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with MockitoSugar + with JsonTestUtils { + + private val logDir = new File("src/test/resources/spark-events") + private val expRoot = new File("src/test/resources/HistoryServerExpectations/") + + private var provider: FsHistoryProvider = null + private var server: HistoryServer = null + private var port: Int = -1 + + def init(): Unit = { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) + .set("spark.history.fs.updateInterval", "0") + .set("spark.testing", "true") + provider = new FsHistoryProvider(conf) + provider.checkForLogs() + val securityManager = new SecurityManager(conf) + + server = new HistoryServer(conf, provider, securityManager, 18080) + server.initialize() + server.bind() + port = server.boundPort + } + + def stop(): Unit = { + server.stop() + } + + before { + init() + } + + after{ + stop() + } + + val cases = Seq( + "application list json" -> "applications", + "completed app list json" -> "applications?status=completed", + "running app list json" -> "applications?status=running", + "minDate app list json" -> "applications?minDate=2015-02-10", + "maxDate app list json" -> "applications?maxDate=2015-02-10", + "maxDate2 app list json" -> "applications?maxDate=2015-02-03T10:42:40.000CST", + "one app json" -> "applications/local-1422981780767", + "one app multi-attempt json" -> "applications/local-1426533911241", + "job list json" -> "applications/local-1422981780767/jobs", + "job list from multi-attempt app json(1)" -> "applications/local-1426533911241/1/jobs", + "job list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/jobs", + "one job json" -> "applications/local-1422981780767/jobs/0", + "succeeded job list json" -> "applications/local-1422981780767/jobs?status=succeeded", + "succeeded&failed job list json" -> + "applications/local-1422981780767/jobs?status=succeeded&status=failed", + "executor list json" -> "applications/local-1422981780767/executors", + "stage list json" -> "applications/local-1422981780767/stages", + "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", + "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", + "one stage json" -> "applications/local-1422981780767/stages/1", + "one stage attempt json" -> "applications/local-1422981780767/stages/1/0", + + "stage task summary" -> "applications/local-1427397477963/stages/20/0/taskSummary", + "stage task summary w/ custom quantiles" -> + "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99", + + "stage task list" -> "applications/local-1427397477963/stages/20/0/taskList", + "stage task list w/ offset & length" -> + "applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50", + "stage task list w/ sortBy" -> + "applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME", + "stage task list w/ sortBy short names: -runtime" -> + "applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime", + "stage task list w/ sortBy short names: runtime" -> + "applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime", + + "stage list with accumulable json" -> "applications/local-1426533911241/1/stages", + "stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0", + "stage task list from multi-attempt app json(1)" -> + "applications/local-1426533911241/1/stages/0/0/taskList", + "stage task list from multi-attempt app json(2)" -> + "applications/local-1426533911241/2/stages/0/0/taskList", + + "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", + "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" + ) + + // run a bunch of characterization tests -- just verify the behavior is the same as what is saved + // in the test resource folder + cases.foreach { case (name, path) => + test(name) { + val (code, jsonOpt, errOpt) = getContentAndCode(path) + code should be (HttpServletResponse.SC_OK) + jsonOpt should be ('defined) + errOpt should be (None) + val json = jsonOpt.get + val exp = IOUtils.toString(new FileInputStream( + new File(expRoot, path + "/json_expectation"))) + // compare the ASTs so formatting differences don't cause failures + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + val jsonAst = parse(json) + val expAst = parse(exp) + assertValidDataInJson(jsonAst, expAst) + } + } + + test("response codes on bad paths") { + val badAppId = getContentAndCode("applications/foobar") + badAppId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badAppId._3 should be (Some("unknown app: foobar")) + + val badStageId = getContentAndCode("applications/local-1422981780767/stages/12345") + badStageId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageId._3 should be (Some("unknown stage: 12345")) + + val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1") + badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageAttemptId._3 should be (Some("unknown attempt for stage 1. Found attempts: [0]")) + + val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam") + badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND) + // will take some mucking w/ jersey to get a better error msg in this case + + val badQuantiles = getContentAndCode( + "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=foo,0.1") + badQuantiles._1 should be (HttpServletResponse.SC_BAD_REQUEST) + badQuantiles._3 should be (Some("Bad value for parameter \"quantiles\". Expected a double, " + + "got \"foo\"")) + + getContentAndCode("foobar")._1 should be (HttpServletResponse.SC_NOT_FOUND) + } test("generate history page with relative links") { val historyServer = mock[HistoryServer] val request = mock[HttpServletRequest] val ui = mock[SparkUI] val link = "/history/app1" - val info = new ApplicationHistoryInfo("app1", "app1", 0, 2, 1, "xxx", true) + val info = new ApplicationHistoryInfo("app1", "app1", + List(ApplicationAttemptInfo(None, 0, 2, 1, "xxx", true))) when(historyServer.getApplicationList()).thenReturn(Seq(info)) when(ui.basePath).thenReturn(link) when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) val page = new HistoryPage(historyServer) - //when + // when val response = page.render(request) - //then + // then val links = response \\ "a" val justHrefs = for { l <- links @@ -53,4 +193,70 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { } yield (attrs.toString) justHrefs should contain(link) } + + def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { + HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path")) + } + + def getUrl(path: String): String = { + HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path")) + } + + def generateExpectation(path: String): Unit = { + val json = getUrl(path) + val dir = new File(expRoot, path) + dir.mkdirs() + val out = new FileWriter(new File(dir, "json_expectation")) + out.write(json) + out.close() + } +} + +object HistoryServerSuite { + def main(args: Array[String]): Unit = { + // generate the "expected" results for the characterization tests. Just blindly assume the + // current behavior is correct, and write out the returned json to the test/resource files + + val suite = new HistoryServerSuite + FileUtils.deleteDirectory(suite.expRoot) + suite.expRoot.mkdirs() + try { + suite.init() + suite.cases.foreach { case (_, path) => + suite.generateExpectation(path) + } + } finally { + suite.stop() + } + } + + def getContentAndCode(url: URL): (Int, Option[String], Option[String]) = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod("GET") + connection.connect() + val code = connection.getResponseCode() + val inString = try { + val in = Option(connection.getInputStream()) + in.map{IOUtils.toString} + } catch { + case io: IOException => None + } + val errString = try { + val err = Option(connection.getErrorStream()) + err.map{IOUtils.toString} + } catch { + case io: IOException => None + } + (code, inString, errString) + } + + def getUrl(path: URL): String = { + val (code, resultOpt, error) = getContentAndCode(path) + if (code == 200) { + resultOpt.get + } else { + throw new RuntimeException( + "got code: " + code + " when getting " + path + " w/ error: " + error) + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 2fa90e3bd1c63..f4d548d9e7720 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -39,9 +39,8 @@ import org.apache.spark.deploy.master.DriverState._ * Tests for the REST application submission protocol used in standalone cluster mode. */ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { - private val client = new StandaloneRestClient private var actorSystem: Option[ActorSystem] = None - private var server: Option[StandaloneRestServer] = None + private var server: Option[RestSubmissionServer] = None override def afterEach() { actorSystem.foreach(_.shutdown()) @@ -52,7 +51,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val appArgs = Array("one", "two", "three") val sparkProperties = Map("spark.app.name" -> "pi") val environmentVariables = Map("SPARK_ONE" -> "UN", "SPARK_TWO" -> "DEUX") - val request = client.constructSubmitRequest( + val request = new RestSubmissionClient("spark://host:port").constructSubmitRequest( "my-app-resource", "my-main-class", appArgs, sparkProperties, environmentVariables) assert(request.action === Utils.getFormattedClassName(request)) assert(request.clientSparkVersion === SPARK_VERSION) @@ -71,7 +70,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val request = constructSubmitRequest(masterUrl, appArgs) assert(request.appArgs === appArgs) assert(request.sparkProperties("spark.master") === masterUrl) - val response = client.createSubmission(masterUrl, request) + val response = new RestSubmissionClient(masterUrl).createSubmission(request) val submitResponse = getSubmitResponse(response) assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) assert(submitResponse.serverSparkVersion === SPARK_VERSION) @@ -89,7 +88,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { conf.set("spark.app.name", "dreamer") val appArgs = Array("one", "two", "six") // main method calls this - val response = StandaloneRestClient.run("app-resource", "main-class", appArgs, conf) + val response = RestSubmissionClient.run("app-resource", "main-class", appArgs, conf) val submitResponse = getSubmitResponse(response) assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) assert(submitResponse.serverSparkVersion === SPARK_VERSION) @@ -102,7 +101,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val submissionId = "my-lyft-driver" val killMessage = "your driver is killed" val masterUrl = startDummyServer(killMessage = killMessage) - val response = client.killSubmission(masterUrl, submissionId) + val response = new RestSubmissionClient(masterUrl).killSubmission(submissionId) val killResponse = getKillResponse(response) assert(killResponse.action === Utils.getFormattedClassName(killResponse)) assert(killResponse.serverSparkVersion === SPARK_VERSION) @@ -116,7 +115,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val submissionState = KILLED val submissionException = new Exception("there was an irresponsible mix of alcohol and cars") val masterUrl = startDummyServer(state = submissionState, exception = Some(submissionException)) - val response = client.requestSubmissionStatus(masterUrl, submissionId) + val response = new RestSubmissionClient(masterUrl).requestSubmissionStatus(submissionId) val statusResponse = getStatusResponse(response) assert(statusResponse.action === Utils.getFormattedClassName(statusResponse)) assert(statusResponse.serverSparkVersion === SPARK_VERSION) @@ -129,13 +128,14 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("create then kill") { val masterUrl = startSmartServer() val request = constructSubmitRequest(masterUrl) - val response1 = client.createSubmission(masterUrl, request) + val client = new RestSubmissionClient(masterUrl) + val response1 = client.createSubmission(request) val submitResponse = getSubmitResponse(response1) assert(submitResponse.success) assert(submitResponse.submissionId != null) // kill submission that was just created val submissionId = submitResponse.submissionId - val response2 = client.killSubmission(masterUrl, submissionId) + val response2 = client.killSubmission(submissionId) val killResponse = getKillResponse(response2) assert(killResponse.success) assert(killResponse.submissionId === submissionId) @@ -144,13 +144,14 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("create then request status") { val masterUrl = startSmartServer() val request = constructSubmitRequest(masterUrl) - val response1 = client.createSubmission(masterUrl, request) + val client = new RestSubmissionClient(masterUrl) + val response1 = client.createSubmission(request) val submitResponse = getSubmitResponse(response1) assert(submitResponse.success) assert(submitResponse.submissionId != null) // request status of submission that was just created val submissionId = submitResponse.submissionId - val response2 = client.requestSubmissionStatus(masterUrl, submissionId) + val response2 = client.requestSubmissionStatus(submissionId) val statusResponse = getStatusResponse(response2) assert(statusResponse.success) assert(statusResponse.submissionId === submissionId) @@ -160,8 +161,9 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("create then kill then request status") { val masterUrl = startSmartServer() val request = constructSubmitRequest(masterUrl) - val response1 = client.createSubmission(masterUrl, request) - val response2 = client.createSubmission(masterUrl, request) + val client = new RestSubmissionClient(masterUrl) + val response1 = client.createSubmission(request) + val response2 = client.createSubmission(request) val submitResponse1 = getSubmitResponse(response1) val submitResponse2 = getSubmitResponse(response2) assert(submitResponse1.success) @@ -171,13 +173,13 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val submissionId1 = submitResponse1.submissionId val submissionId2 = submitResponse2.submissionId // kill only submission 1, but not submission 2 - val response3 = client.killSubmission(masterUrl, submissionId1) + val response3 = client.killSubmission(submissionId1) val killResponse = getKillResponse(response3) assert(killResponse.success) assert(killResponse.submissionId === submissionId1) // request status for both submissions: 1 should be KILLED but 2 should be RUNNING still - val response4 = client.requestSubmissionStatus(masterUrl, submissionId1) - val response5 = client.requestSubmissionStatus(masterUrl, submissionId2) + val response4 = client.requestSubmissionStatus(submissionId1) + val response5 = client.requestSubmissionStatus(submissionId2) val statusResponse1 = getStatusResponse(response4) val statusResponse2 = getStatusResponse(response5) assert(statusResponse1.submissionId === submissionId1) @@ -189,13 +191,14 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("kill or request status before create") { val masterUrl = startSmartServer() val doesNotExist = "does-not-exist" + val client = new RestSubmissionClient(masterUrl) // kill a non-existent submission - val response1 = client.killSubmission(masterUrl, doesNotExist) + val response1 = client.killSubmission(doesNotExist) val killResponse = getKillResponse(response1) assert(!killResponse.success) assert(killResponse.submissionId === doesNotExist) // request status for a non-existent submission - val response2 = client.requestSubmissionStatus(masterUrl, doesNotExist) + val response2 = client.requestSubmissionStatus(doesNotExist) val statusResponse = getStatusResponse(response2) assert(!statusResponse.success) assert(statusResponse.submissionId === doesNotExist) @@ -208,7 +211,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val json = constructSubmitRequest(masterUrl).toJson val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" @@ -238,7 +241,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("good request paths, bad requests") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill" val statusRequestPath = s"$httpUrl/$v/submissions/status" @@ -276,7 +279,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("bad request paths") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val (response1, code1) = sendHttpRequestWithResponse(httpUrl, "GET") val (response2, code2) = sendHttpRequestWithResponse(s"$httpUrl/", "GET") val (response3, code3) = sendHttpRequestWithResponse(s"$httpUrl/$v", "GET") @@ -292,7 +295,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(code5 === HttpServletResponse.SC_BAD_REQUEST) assert(code6 === HttpServletResponse.SC_BAD_REQUEST) assert(code7 === HttpServletResponse.SC_BAD_REQUEST) - assert(code8 === StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) + assert(code8 === RestSubmissionServer.SC_UNKNOWN_PROTOCOL_VERSION) // all responses should be error responses val errorResponse1 = getErrorResponse(response1) val errorResponse2 = getErrorResponse(response2) @@ -310,13 +313,13 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { assert(errorResponse5.highestProtocolVersion === null) assert(errorResponse6.highestProtocolVersion === null) assert(errorResponse7.highestProtocolVersion === null) - assert(errorResponse8.highestProtocolVersion === StandaloneRestServer.PROTOCOL_VERSION) + assert(errorResponse8.highestProtocolVersion === RestSubmissionServer.PROTOCOL_VERSION) } test("server returns unknown fields") { val masterUrl = startSmartServer() val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val oldJson = constructSubmitRequest(masterUrl).toJson val oldFields = parse(oldJson).asInstanceOf[JObject].obj @@ -339,8 +342,9 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { test("client handles faulty server") { val masterUrl = startFaultyServer() + val client = new RestSubmissionClient(masterUrl) val httpUrl = masterUrl.replace("spark://", "http://") - val v = StandaloneRestServer.PROTOCOL_VERSION + val v = RestSubmissionServer.PROTOCOL_VERSION val submitRequestPath = s"$httpUrl/$v/submissions/create" val killRequestPath = s"$httpUrl/$v/submissions/kill/anything" val statusRequestPath = s"$httpUrl/$v/submissions/status/anything" @@ -400,9 +404,9 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { val fakeMasterRef = _actorSystem.actorOf(Props(makeFakeMaster)) val _server = if (faulty) { - new FaultyStandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + new FaultyStandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077") } else { - new StandaloneRestServer(localhost, 0, fakeMasterRef, "spark://fake:7077", conf) + new StandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077") } val port = _server.start() // set these to clean them up after every test @@ -425,7 +429,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { mainJar) ++ appArgs val args = new SparkSubmitArguments(commandLineArgs) val (_, _, sparkProperties, _) = SparkSubmit.prepareSubmitEnvironment(args) - client.constructSubmitRequest( + new RestSubmissionClient("spark://host:port").constructSubmitRequest( mainJar, mainClass, appArgs, sparkProperties.toMap, Map.empty) } @@ -492,7 +496,7 @@ class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterEach { method: String, body: String = ""): (SubmitRestProtocolResponse, Int) = { val conn = sendHttpRequest(url, method, body) - (client.readResponse(conn), conn.getResponseCode) + (new RestSubmissionClient("spark://host:port").readResponse(conn), conn.getResponseCode) } } @@ -508,7 +512,7 @@ private class DummyMaster( exception: Option[Exception] = None) extends Actor { - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage) case RequestKillDriver(driverId) => @@ -531,7 +535,7 @@ private class SmarterMaster extends Actor { private var counter: Int = 0 private val submittedDrivers = new mutable.HashMap[String, DriverState] - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => val driverId = s"driver-$counter" submittedDrivers(driverId) = RUNNING @@ -563,20 +567,18 @@ private class SmarterMaster extends Actor { private class FaultyStandaloneRestServer( host: String, requestedPort: Int, + masterConf: SparkConf, masterActor: ActorRef, - masterUrl: String, - masterConf: SparkConf) - extends StandaloneRestServer(host, requestedPort, masterActor, masterUrl, masterConf) { + masterUrl: String) + extends RestSubmissionServer(host, requestedPort, masterConf) { - protected override val contextToServlet = Map[String, StandaloneRestServlet]( - s"$baseContext/create/*" -> new MalformedSubmitServlet, - s"$baseContext/kill/*" -> new InvalidKillServlet, - s"$baseContext/status/*" -> new ExplodingStatusServlet, - "/*" -> new ErrorServlet - ) + protected override val submitRequestServlet = new MalformedSubmitServlet + protected override val killRequestServlet = new InvalidKillServlet + protected override val statusRequestServlet = new ExplodingStatusServlet /** A faulty servlet that produces malformed responses. */ - class MalformedSubmitServlet extends SubmitRequestServlet(masterActor, masterUrl, masterConf) { + class MalformedSubmitServlet + extends StandaloneSubmitRequestServlet(masterActor, masterUrl, masterConf) { protected override def sendResponse( responseMessage: SubmitRestProtocolResponse, responseServlet: HttpServletResponse): Unit = { @@ -586,7 +588,7 @@ private class FaultyStandaloneRestServer( } /** A faulty servlet that produces invalid responses. */ - class InvalidKillServlet extends KillRequestServlet(masterActor, masterConf) { + class InvalidKillServlet extends StandaloneKillRequestServlet(masterActor, masterConf) { protected override def handleKill(submissionId: String): KillSubmissionResponse = { val k = super.handleKill(submissionId) k.submissionId = null @@ -595,7 +597,7 @@ private class FaultyStandaloneRestServer( } /** A faulty status servlet that explodes. */ - class ExplodingStatusServlet extends StatusRequestServlet(masterActor, masterConf) { + class ExplodingStatusServlet extends StandaloneStatusRequestServlet(masterActor, masterConf) { private def explode: Int = 1 / 0 protected override def handleStatus(submissionId: String): SubmissionStatusResponse = { val s = super.handleStatus(submissionId) diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 1d64ec201e647..61071ee17256c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -129,7 +129,8 @@ class SubmitRestProtocolSuite extends FunSuite { assert(newMessage.sparkProperties("spark.files") === "fireball.png") assert(newMessage.sparkProperties("spark.driver.memory") === "512m") assert(newMessage.sparkProperties("spark.driver.cores") === "180") - assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red") + assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === + " -Dslices=5 -Dcolor=mostly_red") assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar") assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") assert(newMessage.sparkProperties("spark.driver.supervise") === "false") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index aa6e4874cecde..2159fd8c16c6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,8 +33,9 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 6fca6321e5a1b..a8b9df227c996 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -35,7 +35,8 @@ class ExecutorRunnerTest extends FunSuite { val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) - val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) + val builder = CommandUtils.buildProcessBuilder( + appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala index 372d7aa453008..7cc2104281464 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala @@ -37,7 +37,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "50000" else super.getenv(name) } @@ -56,7 +56,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "5G" else super.getenv(name) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index 84e2fd7ad936d..450fba21f4b5c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -24,8 +24,10 @@ import org.scalatest.{Matchers, FunSuite} class WorkerSuite extends FunSuite with Matchers { - def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) - def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts) + def cmd(javaOpts: String*): Command = { + Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) + } + def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts) test("test isUseLocalNodeSSLConfig") { Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 5e538d6fab2a1..6a6f29dd613cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -17,32 +17,38 @@ package org.apache.spark.deploy.worker -import akka.actor.{ActorSystem, AddressFromURIString, Props} -import akka.testkit.TestActorRef -import akka.remote.DisassociatedEvent +import akka.actor.AddressFromURIString +import org.apache.spark.SparkConf +import org.apache.spark.SecurityManager +import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.scalatest.FunSuite class WorkerWatcherSuite extends FunSuite { test("WorkerWatcher shuts down on valid disassociation") { - val actorSystem = ActorSystem("test") - val targetWorkerUrl = "akka://1.2.3.4/user/Worker" + val conf = new SparkConf() + val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) + val targetWorkerUrl = "akka://test@1.2.3.4:1234/user/Worker" val targetWorkerAddress = AddressFromURIString(targetWorkerUrl) - val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) - val workerWatcher = actorRef.underlyingActor + val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl) workerWatcher.setTesting(testing = true) - actorRef.underlyingActor.receive(new DisassociatedEvent(null, targetWorkerAddress, false)) - assert(actorRef.underlyingActor.isShutDown) + rpcEnv.setupEndpoint("worker-watcher", workerWatcher) + workerWatcher.onDisconnected( + RpcAddress(targetWorkerAddress.host.get, targetWorkerAddress.port.get)) + assert(workerWatcher.isShutDown) + rpcEnv.shutdown() } test("WorkerWatcher stays alive on invalid disassociation") { - val actorSystem = ActorSystem("test") - val targetWorkerUrl = "akka://1.2.3.4/user/Worker" - val otherAkkaURL = "akka://4.3.2.1/user/OtherActor" + val conf = new SparkConf() + val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) + val targetWorkerUrl = "akka://test@1.2.3.4:1234/user/Worker" + val otherAkkaURL = "akka://test@4.3.2.1:1234/user/OtherActor" val otherAkkaAddress = AddressFromURIString(otherAkkaURL) - val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem) - val workerWatcher = actorRef.underlyingActor + val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl) workerWatcher.setTesting(testing = true) - actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) - assert(!actorRef.underlyingActor.isShutDown) + rpcEnv.setupEndpoint("worker-watcher", workerWatcher) + workerWatcher.onDisconnected(RpcAddress(otherAkkaAddress.host.get, otherAkkaAddress.port.get)) + assert(!workerWatcher.isShutDown) + rpcEnv.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 78fa98a3b9065..ef3e213f1fcce 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -21,7 +21,7 @@ import java.io.{File, FileWriter, PrintWriter} import scala.collection.mutable.ArrayBuffer -import org.apache.commons.lang.math.RandomUtils +import org.apache.commons.lang3.RandomUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{LongWritable, Text} @@ -60,7 +60,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(tmpFile)) for (x <- 1 to numRecords) { - pw.println(RandomUtils.nextInt(numBuckets)) + pw.println(RandomUtils.nextInt(0, numBuckets)) } pw.close() @@ -238,7 +238,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext sc.textFile(tmpFilePath, 4) .map(key => (key, 1)) - .reduceByKey(_+_) + .reduceByKey(_ + _) .saveAsTextFile("file://" + tmpFile.getAbsolutePath) sc.listenerBus.waitUntilEmpty(500) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index 37e528435aa5d..100ac77dec1f7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -35,7 +35,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val property = conf.getInstance("random") assert(property.size() === 2) - assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(property.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(property.getProperty("sink.servlet.path") === "/metrics/json") } @@ -47,16 +48,20 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(masterProp.size() === 5) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minutes") - assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(masterProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(masterProp.getProperty("sink.servlet.path") === "/metrics/master/json") val workerProp = conf.getInstance("worker") assert(workerProp.size() === 5) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "seconds") - assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(workerProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(workerProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(workerProp.getProperty("sink.servlet.path") === "/metrics/json") } diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 94bfa67451892..46d2e5173acae 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.network.netty +import java.io.InputStreamReader import java.nio._ +import java.nio.charset.Charset import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import scala.concurrent.{Await, Promise} import scala.util.{Failure, Success, Try} -import org.apache.commons.io.IOUtils +import com.google.common.io.CharStreams import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.network.{BlockDataManager, BlockTransferService} @@ -32,7 +34,7 @@ import org.apache.spark.storage.{BlockId, ShuffleBlockId} import org.apache.spark.{SecurityManager, SparkConf} import org.mockito.Mockito._ import org.scalatest.mock.MockitoSugar -import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, ShouldMatchers} +import org.scalatest.{FunSuite, ShouldMatchers} class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with ShouldMatchers { test("security default off") { @@ -113,7 +115,9 @@ class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with Sh val result = fetchBlock(exec0, exec1, "1", blockId) match { case Success(buf) => - IOUtils.toString(buf.createInputStream()) should equal(blockString) + val actualString = CharStreams.toString( + new InputStreamReader(buf.createInputStream(), Charset.forName("UTF-8"))) + actualString should equal(blockString) buf.release() Success() case Failure(t) => diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 716f875d30b8a..02424c59d6831 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -260,8 +260,8 @@ class ConnectionManagerSuite extends FunSuite { test("sendMessageReliably timeout") { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") - val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + val ackTimeoutS = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeoutS}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) @@ -272,7 +272,7 @@ class ConnectionManagerSuite extends FunSuite { val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { // sleep 60 sec > ack timeout for simulating server slow down or hang up - Thread.sleep(ackTimeout * 3 * 1000) + Thread.sleep(ackTimeoutS * 3 * 1000) None }) @@ -287,7 +287,7 @@ class ConnectionManagerSuite extends FunSuite { // Otherwise TimeoutExcepton is thrown from Await.result. // We expect TimeoutException is not thrown. intercept[IOException] { - Await.result(future, (ackTimeout * 2) second) + Await.result(future, (ackTimeoutS * 2) second) } manager.stop() diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 4cd0f97368ca3..01039b9449daf 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -22,6 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { + test("sum") { + assert(sc.parallelize(Seq.empty[Double]).sum() === 0.0) + assert(sc.parallelize(Seq(1.0)).sum() === 1.0) + assert(sc.parallelize(Seq(1.0, 2.0)).sum() === 3.0) + } + // Verify tests on the histogram functionality. We test with both evenly // and non-evenly spaced buckets as the bucket lookup function changes. test("WorksOnEmpty") { @@ -235,6 +241,12 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithDoubleValuesAtMinMax") { + val rdd = sc.parallelize(Seq(1, 1, 1, 2, 3, 3)) + assert(Array(3, 0, 1, 2) === rdd.map(_.toDouble).histogram(4)._2) + assert(Array(3, 1, 2) === rdd.map(_.toDouble).histogram(3)._2) + } + test("WorksWithoutBucketsWithMoreRequestedThanElements") { // Verify the basic case of one bucket and all elements in that bucket works val rdd = sc.parallelize(Seq(1, 2)) @@ -248,7 +260,7 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { } test("WorksWithoutBucketsForLargerDatasets") { - // Verify the case of slighly larger datasets + // Verify the case of slightly larger datasets val rdd = sc.parallelize(6 to 99) val (histogramBuckets, histogramResults) = rdd.histogram(8) val expectedHistogramResults = @@ -259,17 +271,27 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } - test("WorksWithoutBucketsWithIrrationalBucketEdges") { - // Verify the case of buckets with irrational edges. See #SPARK-2862. + test("WorksWithoutBucketsWithNonIntegralBucketEdges") { + // Verify the case of buckets with nonintegral edges. See #SPARK-2862. val rdd = sc.parallelize(6 to 99) val (histogramBuckets, histogramResults) = rdd.histogram(9) + // Buckets are 6.0, 16.333333333333336, 26.666666666666668, 37.0, 47.333333333333336 ... val expectedHistogramResults = - Array(11, 10, 11, 10, 10, 11, 10, 10, 11) + Array(11, 10, 10, 11, 10, 10, 11, 10, 11) assert(histogramResults === expectedHistogramResults) assert(histogramBuckets(0) === 6.0) assert(histogramBuckets(9) === 99.0) } + test("WorksWithHugeRange") { + val rdd = sc.parallelize(Array(0, 1.0e24, 1.0e30)) + val histogramResults = rdd.histogram(1000000)._2 + assert(histogramResults(0) === 1) + assert(histogramResults(1) === 1) + assert(histogramResults.last === 1) + assert((2 to histogramResults.length - 2).forall(i => histogramResults(i) == 0)) + } + // Test the failure mode with an invalid RDD test("ThrowsExceptionOnInvalidRDDs") { // infinity diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 0dc59888f7304..be8467354b222 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -80,7 +80,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 10100) + assert(rdd.reduce(_ + _) === 10100) } test("large id overflow") { @@ -92,7 +92,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { 1131544775L, 567279358897692673L, 20, (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 5050) + assert(rdd.reduce(_ + _) === 5050) } after { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 108f70af43f37..ca0d953d306d8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -168,13 +168,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() + val sums = pairs.reduceByKey(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("reduceByKey with collectAsMap") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() + val sums = pairs.reduceByKey(_ + _).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) assert(sums(2) === 1) @@ -182,7 +182,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey with many output partitons") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() + val sums = pairs.reduceByKey(_ + _, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -192,7 +192,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def getPartition(key: Any) = key.asInstanceOf[Int] } val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) + val sums = pairs.reduceByKey(_ + _) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) // count the dependencies to make sure there is only 1 ShuffledRDD @@ -208,7 +208,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("countApproxDistinctByKey") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is * only a statistical bound, the tests can fail for large values of relativeSD. We will be using @@ -465,7 +465,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("foldByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() + val sums = pairs.foldByKey(0)(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -505,7 +505,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { conf.setOutputCommitter(classOf[FakeOutputCommitter]) FakeOutputCommitter.ran = false - pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + pairs.saveAsHadoopFile( + "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) assert(FakeOutputCommitter.ran, "OutputCommitter was never called") } @@ -552,7 +553,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } private object StratifiedAuxiliary { - def stratifier (fractionPositive: Double) = { + def stratifier (fractionPositive: Double): (Int) => String = { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } @@ -572,7 +573,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSampleExact(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, true, samplingRate, seed, n) testPoisson(stratifiedData, true, samplingRate, seed, n) } @@ -580,7 +581,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSample(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, false, samplingRate, seed, n) testPoisson(stratifiedData, false, samplingRate, seed, n) } @@ -590,7 +591,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey() .mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -612,7 +613,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -701,27 +702,27 @@ class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { */ class NewFakeWriter extends NewRecordWriter[Integer, Integer] { - def close(p1: NewTaskAttempContext) = () + def close(p1: NewTaskAttempContext): Unit = () - def write(p1: Integer, p2: Integer) = () + def write(p1: Integer, p2: Integer): Unit = () } class NewFakeCommitter extends NewOutputCommitter { - def setupJob(p1: NewJobContext) = () + def setupJob(p1: NewJobContext): Unit = () def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: NewTaskAttempContext) = () + def setupTask(p1: NewTaskAttempContext): Unit = () - def commitTask(p1: NewTaskAttempContext) = () + def commitTask(p1: NewTaskAttempContext): Unit = () - def abortTask(p1: NewTaskAttempContext) = () + def abortTask(p1: NewTaskAttempContext): Unit = () } class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: NewJobContext) = () + def checkOutputSpecs(p1: NewJobContext): Unit = () def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { new NewFakeWriter() @@ -735,7 +736,7 @@ class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false - def setConf(p1: Configuration) = { + def setConf(p1: Configuration): Unit = { setConfCalled = true () } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index cd193ae4f5238..1880364581c1a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -100,7 +100,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[Range])) } @@ -108,7 +108,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[Range])) } @@ -139,7 +139,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(i).isInstanceOf[Range]) val range = slices(i).asInstanceOf[Range] assert(range.start === i * (N / 40), "slice " + i + " start") - assert(range.end === (i+1) * (N / 40), "slice " + i + " end") + assert(range.end === (i + 1) * (N / 40), "slice " + i + " end") assert(range.step === 1, "slice " + i + " step") } } @@ -156,7 +156,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size /n + 1)) } check(prop) } @@ -174,7 +174,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -192,7 +192,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -209,7 +209,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -217,7 +217,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -225,7 +225,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala index 8408d7e785c65..465068c6cbb16 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.{Partition, SharedSparkContext, TaskContext} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { - test("Pruned Partitions inherit locality prefs correctly") { val rdd = new RDD[Int](sc, Nil) { @@ -74,8 +73,6 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { } class TestPartition(i: Int, value: Int) extends Partition with Serializable { - def index = i - - def testValue = this.value - + def index: Int = i + def testValue: Int = this.value } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index a0483886f8db3..0d1369c19c69e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -35,7 +35,7 @@ class MockSampler extends RandomSampler[Long, Long] { Iterator(s) } - override def clone = new MockSampler + override def clone: MockSampler = new MockSampler } class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { 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 aea76c1adcc09..85eb2a1d07ba4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -176,7 +176,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContextImpl(0, 0, 0, 0) + val tContext = new TaskContextImpl(0, 0, 0, 0, null) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDOperationScopeSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDOperationScopeSuite.scala new file mode 100644 index 0000000000000..d75ecbf1f0b4d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/RDDOperationScopeSuite.scala @@ -0,0 +1,133 @@ +/* + * 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.rdd + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.{TaskContext, Partition, SparkContext} + +/** + * + */ +class RDDOperationScopeSuite extends FunSuite with BeforeAndAfter { + private var sc: SparkContext = null + private val scope1 = new RDDOperationScope("scope1") + private val scope2 = new RDDOperationScope("scope2", parent = Some(scope1)) + private val scope3 = new RDDOperationScope("scope3", parent = Some(scope2)) + + before { + sc = new SparkContext("local", "test") + } + + after { + sc.stop() + } + + test("getAllScopes") { + assert(scope1.getAllScopes === Seq(scope1)) + assert(scope2.getAllScopes === Seq(scope1, scope2)) + assert(scope3.getAllScopes === Seq(scope1, scope2, scope3)) + } + + test("json de/serialization") { + val scope1Json = scope1.toJson + val scope2Json = scope2.toJson + val scope3Json = scope3.toJson + assert(scope1Json === s"""{"id":${scope1.id},"name":"scope1"}""") + assert(scope2Json === s"""{"id":${scope2.id},"name":"scope2","parent":$scope1Json}""") + assert(scope3Json === s"""{"id":${scope3.id},"name":"scope3","parent":$scope2Json}""") + assert(RDDOperationScope.fromJson(scope1Json) === scope1) + assert(RDDOperationScope.fromJson(scope2Json) === scope2) + assert(RDDOperationScope.fromJson(scope3Json) === scope3) + } + + test("withScope") { + val rdd0: MyCoolRDD = new MyCoolRDD(sc) + var rdd1: MyCoolRDD = null + var rdd2: MyCoolRDD = null + var rdd3: MyCoolRDD = null + RDDOperationScope.withScope(sc, "scope1", allowNesting = false) { + rdd1 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope2", allowNesting = false) { + rdd2 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope3", allowNesting = false) { + rdd3 = new MyCoolRDD(sc) + } + } + } + assert(rdd0.scope.isEmpty) + assert(rdd1.scope.isDefined) + assert(rdd2.scope.isDefined) + assert(rdd3.scope.isDefined) + assert(rdd1.scope.get.getAllScopes.map(_.name) === Seq("scope1")) + assert(rdd2.scope.get.getAllScopes.map(_.name) === Seq("scope1")) + assert(rdd3.scope.get.getAllScopes.map(_.name) === Seq("scope1")) + } + + test("withScope with partial nesting") { + val rdd0: MyCoolRDD = new MyCoolRDD(sc) + var rdd1: MyCoolRDD = null + var rdd2: MyCoolRDD = null + var rdd3: MyCoolRDD = null + RDDOperationScope.withScope(sc, "scope1", allowNesting = true) { // allow nesting here + rdd1 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope2", allowNesting = false) { // stop nesting here + rdd2 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope3", allowNesting = false) { + rdd3 = new MyCoolRDD(sc) + } + } + } + assert(rdd0.scope.isEmpty) + assert(rdd1.scope.isDefined) + assert(rdd2.scope.isDefined) + assert(rdd3.scope.isDefined) + assert(rdd1.scope.get.getAllScopes.map(_.name) === Seq("scope1")) + assert(rdd2.scope.get.getAllScopes.map(_.name) === Seq("scope1", "scope2")) + assert(rdd3.scope.get.getAllScopes.map(_.name) === Seq("scope1", "scope2")) + } + + test("withScope with multiple layers of nesting") { + val rdd0: MyCoolRDD = new MyCoolRDD(sc) + var rdd1: MyCoolRDD = null + var rdd2: MyCoolRDD = null + var rdd3: MyCoolRDD = null + RDDOperationScope.withScope(sc, "scope1", allowNesting = true) { + rdd1 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope2", allowNesting = true) { + rdd2 = new MyCoolRDD(sc) + RDDOperationScope.withScope(sc, "scope3", allowNesting = true) { + rdd3 = new MyCoolRDD(sc) + } + } + } + assert(rdd0.scope.isEmpty) + assert(rdd1.scope.isDefined) + assert(rdd2.scope.isDefined) + assert(rdd3.scope.isDefined) + assert(rdd1.scope.get.getAllScopes.map(_.name) === Seq("scope1")) + assert(rdd2.scope.get.getAllScopes.map(_.name) === Seq("scope1", "scope2")) + assert(rdd3.scope.get.getAllScopes.map(_.name) === Seq("scope1", "scope2", "scope3")) + } + +} + +private class MyCoolRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { + override def getPartitions: Array[Partition] = Array.empty + override def compute(p: Partition, context: TaskContext): Iterator[Int] = { Nil.toIterator } +} 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 bede1ffb3e2d0..ef8c36a28655b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -82,7 +82,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("countApproxDistinct") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble val size = 1000 val uniformDistro = for (i <- 1 to 5000) yield i % size @@ -99,8 +99,29 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + 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)) + val unionRdd = sc.union(rddWithNoPartitioner, rddWithPartitioner) + assert(unionRdd.isInstanceOf[UnionRDD[_]]) + } + + test("SparkContext.union creates PartitionAwareUnionRDD if all RDDs have partitioners") { + val rddWithPartitioner = sc.parallelize(Seq(1->true)).partitionBy(new HashPartitioner(1)) + val unionRdd = sc.union(rddWithPartitioner, rddWithPartitioner) + assert(unionRdd.isInstanceOf[PartitionerAwareUnionRDD[_]]) + } + + test("PartitionAwareUnionRDD raises exception 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)) + intercept[IllegalArgumentException] { + new PartitionerAwareUnionRDD(sc, Seq(rddWithNoPartitioner, rddWithPartitioner)) + } + } + test("partitioner aware union") { - def makeRDDWithPartitioner(seq: Seq[Int]) = { + def makeRDDWithPartitioner(seq: Seq[Int]): RDD[Int] = { sc.makeRDD(seq, 1) .map(x => (x, null)) .partitionBy(new HashPartitioner(2)) @@ -159,8 +180,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("treeAggregate") { val rdd = sc.makeRDD(-1000 until 1000, 10) - def seqOp = (c: Long, x: Int) => c + x - def combOp = (c1: Long, c2: Long) => c1 + c2 + def seqOp: (Long, Int) => Long = (c: Long, x: Int) => c + x + def combOp: (Long, Long) => Long = (c1: Long, c2: Long) => c1 + c2 for (depth <- 1 until 10) { val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) assert(sum === -1000L) @@ -204,7 +225,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(empty.collect().size === 0) val thrown = intercept[UnsupportedOperationException]{ - empty.reduce(_+_) + empty.reduce(_ + _) } assert(thrown.getMessage.contains("empty")) @@ -321,7 +342,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 - val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) + val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i + 2)).map{ j => "m" + (j%6)}))) val coalesced1 = data.coalesce(3) assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") @@ -921,15 +942,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("task serialization exception should not hang scheduler") { class BadSerializable extends Serializable { @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream): Unit = throw new KryoException("Bad serialization") + private def writeObject(out: ObjectOutputStream): Unit = + throw new KryoException("Bad serialization") @throws(classOf[IOException]) private def readObject(in: ObjectInputStream): Unit = {} } - // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if there were - // more threads in the Spark Context than there were number of objects in this sequence. + // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if + // there were more threads in the Spark Context than there were number of objects in this + // sequence. intercept[Throwable] { - sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect + sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect() } // Check that the context has not crashed sc.parallelize(1 to 100).map(x => x*2).collect diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala index 4762fc17855ce..fe695d85e29dd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala @@ -21,11 +21,11 @@ object RDDSuiteUtils { case class Person(first: String, last: String, age: Int) object AgeOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = a.age compare b.age + def compare(a:Person, b:Person): Int = a.age.compare(b.age) } object NameOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = + def compare(a:Person, b:Person): Int = implicitly[Ordering[Tuple2[String,String]]].compare((a.last, a.first), (b.last, b.first)) } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala new file mode 100644 index 0000000000000..ae3339d80f9c6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -0,0 +1,548 @@ +/* + * 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.rpc + +import java.util.concurrent.{TimeUnit, CountDownLatch, TimeoutException} + +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{SparkException, SparkConf} + +/** + * Common tests for an RpcEnv implementation. + */ +abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { + + var env: RpcEnv = _ + + override def beforeAll(): Unit = { + val conf = new SparkConf() + env = createRpcEnv(conf, "local", 12345) + } + + override def afterAll(): Unit = { + if(env != null) { + env.shutdown() + } + } + + def createRpcEnv(conf: SparkConf, name: String, port: Int): RpcEnv + + test("send a message locally") { + @volatile var message: String = null + val rpcEndpointRef = env.setupEndpoint("send-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case msg: String => message = msg + } + }) + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(10 millis)) { + assert("hello" === message) + } + } + + test("send a message remotely") { + @volatile var message: String = null + // Set up a RpcEndpoint using env + env.setupEndpoint("send-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => message = msg + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote" ,13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "send-remotely") + try { + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(10 millis)) { + assert("hello" === message) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("send a RpcEndpointRef") { + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case "Hello" => context.reply(self) + case "Echo" => context.reply("Echo") + } + } + val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) + + val newRpcEndpointRef = rpcEndpointRef.askWithRetry[RpcEndpointRef]("Hello") + val reply = newRpcEndpointRef.askWithRetry[String]("Echo") + assert("Echo" === reply) + } + + test("ask a message locally") { + val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => { + context.reply(msg) + } + } + }) + val reply = rpcEndpointRef.askWithRetry[String]("hello") + assert("hello" === reply) + } + + test("ask a message remotely") { + env.setupEndpoint("ask-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => { + context.reply(msg) + } + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-remotely") + try { + val reply = rpcEndpointRef.askWithRetry[String]("hello") + assert("hello" === reply) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("ask a message timeout") { + env.setupEndpoint("ask-timeout", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => { + Thread.sleep(100) + context.reply(msg) + } + } + }) + + val conf = new SparkConf() + conf.set("spark.rpc.retry.wait", "0") + conf.set("spark.rpc.numRetries", "1") + val anotherEnv = createRpcEnv(conf, "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout") + try { + val e = intercept[Exception] { + rpcEndpointRef.askWithRetry[String]("hello", 1 millis) + } + assert(e.isInstanceOf[TimeoutException] || e.getCause.isInstanceOf[TimeoutException]) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("onStart and onStop") { + val stopLatch = new CountDownLatch(1) + val calledMethods = mutable.ArrayBuffer[String]() + + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + calledMethods += "start" + } + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => + } + + override def onStop(): Unit = { + calledMethods += "stop" + stopLatch.countDown() + } + } + val rpcEndpointRef = env.setupEndpoint("start-stop-test", endpoint) + env.stop(rpcEndpointRef) + stopLatch.await(10, TimeUnit.SECONDS) + assert(List("start", "stop") === calledMethods) + } + + test("onError: error in onStart") { + @volatile var e: Throwable = null + env.setupEndpoint("onError-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + throw new RuntimeException("Oops!") + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in onStop") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + + override def onStop(): Unit = { + throw new RuntimeException("Oops!") + } + }) + + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in receive") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => throw new RuntimeException("Oops!") + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5 seconds), interval(10 millis)) { + assert(e.getMessage === "Oops!") + } + } + + test("self: call in onStart") { + @volatile var callSelfSuccessfully = false + + env.setupEndpoint("self-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + self + callSelfSuccessfully = true + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + }) + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `onStart` is fine + assert(callSelfSuccessfully === true) + } + } + + test("self: call in receive") { + @volatile var callSelfSuccessfully = false + + val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => { + self + callSelfSuccessfully = true + } + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `receive` is fine + assert(callSelfSuccessfully === true) + } + } + + test("self: call in onStop") { + @volatile var selfOption: Option[RpcEndpointRef] = null + + val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + selfOption = Option(self) + } + + override def onError(cause: Throwable): Unit = { + } + }) + + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(10 millis)) { + // Calling `self` in `onStop` will return null, so selfOption will be None + assert(selfOption == None) + } + } + + test("call receive in sequence") { + // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it + for(i <- 0 until 100) { + @volatile var result = 0 + val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => result += 1 + } + + }) + + (0 until 10) foreach { _ => + new Thread { + override def run() { + (0 until 100) foreach { _ => + endpointRef.send("Hello") + } + } + }.start() + } + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(result == 1000) + } + + env.stop(endpointRef) + } + } + + test("stop(RpcEndpointRef) reentrant") { + @volatile var onStopCount = 0 + val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + onStopCount += 1 + } + }) + + env.stop(endpointRef) + env.stop(endpointRef) + + eventually(timeout(5 seconds), interval(5 millis)) { + // Calling stop twice should only trigger onStop once. + assert(onStopCount == 1) + } + } + + test("sendWithReply") { + val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val f = endpointRef.ask[String]("Hi") + val ack = Await.result(f, 5 seconds) + assert("ack" === ack) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely") { + env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "sendWithReply-remotely") + try { + val f = rpcEndpointRef.ask[String]("hello") + val ack = Await.result(f, 5 seconds) + assert("ack" === ack) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("sendWithReply: error") { + val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.sendFailure(new SparkException("Oops")) + } + }) + + val f = endpointRef.ask[String]("Hi") + val e = intercept[SparkException] { + Await.result(f, 5 seconds) + } + assert("Oops" === e.getMessage) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely error") { + env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new SparkException("Oops")) + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "sendWithReply-remotely-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + val e = intercept[SparkException] { + Await.result(f, 5 seconds) + } + assert("Oops" === e.getMessage) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("network events") { + val events = new mutable.ArrayBuffer[(Any, Any)] with mutable.SynchronizedBuffer[(Any, Any)] + env.setupEndpoint("network-events", new ThreadSafeRpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case "hello" => + case m => events += "receive" -> m + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + events += "onConnected" -> remoteAddress + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + events += "onDisconnected" -> remoteAddress + } + + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + events += "onNetworkError" -> remoteAddress + } + + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "network-events") + val remoteAddress = anotherEnv.address + rpcEndpointRef.send("hello") + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events === List(("onConnected", remoteAddress))) + } + + anotherEnv.shutdown() + anotherEnv.awaitTermination() + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events === List( + ("onConnected", remoteAddress), + ("onNetworkError", remoteAddress), + ("onDisconnected", remoteAddress))) + } + } + + test("sendWithReply: unserializable error") { + env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new UnserializableException) + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef( + "local", env.address, "sendWithReply-unserializable-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + intercept[TimeoutException] { + Await.result(f, 1 seconds) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + +} + +class UnserializableClass + +class UnserializableException extends Exception { + private val unserializableField = new UnserializableClass +} diff --git a/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala new file mode 100644 index 0000000000000..a33a83db7bc9e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala @@ -0,0 +1,50 @@ +/* + * 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.rpc.akka + +import org.apache.spark.rpc._ +import org.apache.spark.{SecurityManager, SparkConf} + +class AkkaRpcEnvSuite extends RpcEnvSuite { + + override def createRpcEnv(conf: SparkConf, name: String, port: Int): RpcEnv = { + new AkkaRpcEnvFactory().create( + RpcEnvConfig(conf, name, "localhost", port, new SecurityManager(conf))) + } + + test("setupEndpointRef: systemName, address, endpointName") { + val ref = env.setupEndpoint("test_endpoint", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case _ => + } + }) + val conf = new SparkConf() + val newRpcEnv = new AkkaRpcEnvFactory().create( + RpcEnvConfig(conf, "test", "localhost", 12346, new SecurityManager(conf))) + try { + val newRef = newRpcEnv.setupEndpointRef("local", ref.address, "test_endpoint") + assert(s"akka.tcp://local@${env.address}/user/test_endpoint" === + newRef.asInstanceOf[AkkaRpcEndpointRef].actorRef.path.toString) + } finally { + newRpcEnv.shutdown() + } + } + +} 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 63360a0f189a3..6a8ae29aae675 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -57,20 +57,18 @@ class MyRDD( locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new RuntimeException("should not be reached") - override def getPartitions = (0 until numPartitions).map(i => new Partition { - override def index = i + override def getPartitions: Array[Partition] = (0 until numPartitions).map(i => new Partition { + override def index: Int = i }).toArray override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil + if (locations.isDefinedAt(split.index)) locations(split.index) else Nil override def toString: String = "DAGSchedulerSuiteRDD " + id } class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite + extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -97,6 +95,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def applicationAttemptId(): Option[String] = None } /** Length of time to wait while draining listener events. */ @@ -175,6 +174,10 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) } + after { + scheduler.stop() + } + override def afterAll() { super.afterAll() } @@ -209,7 +212,8 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) } } } @@ -261,29 +265,32 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) + val jobId = submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) assert(numResults === 0) + cancel(jobId) } test("run trivial job") { submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job") { val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator - override def getPartitions = Array( new Partition { override def index = 0 } ) - override def getPreferredLocations(split: Partition) = Nil - override def toString = "DAGSchedulerSuite Local RDD" + override def getPartitions: Array[Partition] = + Array( new Partition { override def index: Int = 0 } ) + override def getPreferredLocations(split: Partition): List[String] = Nil + override def toString: String = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job oom") { @@ -295,9 +302,10 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results.size == 0) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial job w/ dependency") { @@ -306,7 +314,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cache location preferences w/ dependency") { @@ -319,7 +327,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("regression test for getCacheLocs") { @@ -335,7 +343,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("avoid exponential blowup when getting preferred locs list") { - // Build up a complex dependency graph with repeated zip operations, without preferred locations. + // Build up a complex dependency graph with repeated zip operations, without preferred locations var rdd: RDD[_] = new MyRDD(sc, 1, Nil) (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. @@ -357,7 +365,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job failure") { @@ -367,7 +375,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job cancellation") { @@ -378,7 +386,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("job cancellation no-kill backend") { @@ -387,19 +395,22 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar val noKillTaskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE - override def start() = {} - override def stop() = {} - override def submitTasks(taskSet: TaskSet) = { + override def start(): Unit = {} + override def stop(): Unit = {} + override def submitTasks(taskSet: TaskSet): Unit = { taskSets += taskSet } override def cancelTasks(stageId: Int, interruptThread: Boolean) { throw new UnsupportedOperationException } - override def setDAGScheduler(dagScheduler: DAGScheduler) = {} - override def defaultParallelism() = 2 - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} + override def defaultParallelism(): Int = 2 + override def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def applicationAttemptId(): Option[String] = None } val noKillScheduler = new DAGScheduler( sc, @@ -422,7 +433,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // When the task set completes normally, state should be correctly updated. complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.isEmpty) @@ -442,7 +453,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with fetch failure") { @@ -465,10 +476,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial shuffle with multiple fetch failures") { @@ -521,19 +533,23 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(newEpoch > oldEpoch) val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run shuffle with map stage failure") { @@ -552,7 +568,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.toSet === Set(0)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -586,7 +602,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar class FailureRecordingJobListener() extends JobListener { var failureMessage: String = _ override def taskSucceeded(index: Int, result: Any) {} - override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + override def jobFailed(exception: Exception): Unit = { failureMessage = exception.getMessage } } val listener1 = new FailureRecordingJobListener() val listener2 = new FailureRecordingJobListener() @@ -606,7 +622,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with out-of-band failure and retry") { @@ -629,7 +645,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("recursive shuffle failures") { @@ -658,7 +674,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cached post-shuffle") { @@ -690,7 +706,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { @@ -742,7 +758,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("accumulator not calculated for resubmitted result stage") { - //just for register + // just for register val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) @@ -754,7 +770,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(accVal === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -774,7 +790,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) - private def assertDataStructuresEmpty = { + private def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) @@ -783,6 +799,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) + assert(scheduler.outputCommitCoordinator.isEmpty) } // Nothing in this test should break if the task info's fields are null, but diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 448258a754153..b52a8d11d147d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -61,7 +61,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Verify log file exist") { // Verify logging directory exists val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", testDirPath.toUri().toString(), conf) + val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) eventLogger.start() val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) @@ -95,7 +95,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef } test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.getAbsolutePath, "test") + val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) val logPath = new URI(logUri).getPath // Create file before writing the event log new FileOutputStream(new File(logPath)).close() @@ -107,16 +107,19 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef test("Event log name") { // without compression - assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath("/base-dir", "app1")) + assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath( + Utils.resolveURI("/base-dir"), "app1", None)) // with compression assert(s"file:/base-dir/app1.lzf" === - EventLoggingListener.getLogPath("/base-dir", "app1", Some("lzf"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", None, Some("lzf"))) // illegal characters in app ID assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === - EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1")) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + "a fine:mind$dollar{bills}.1", None)) // illegal characters in app ID with compression assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === - EventLoggingListener.getLogPath("/base-dir", "a fine:mind$dollar{bills}.1", Some("lz4"))) + EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } /* ----------------- * @@ -137,10 +140,10 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef val conf = getLoggingConf(testDirPath, compressionCodec) extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") - val eventLogger = new EventLoggingListener(logName, testDirPath.toUri().toString(), conf) + val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") + 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite @@ -173,14 +176,17 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef * This runs a simple Spark job and asserts that the expected events are logged when expected. */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { + // Set defaultFS to something that would cause an exception, to make sure we don't run + // into SPARK-6688. val conf = getLoggingConf(testDirPath, compressionCodec) + .set("spark.hadoop.fs.defaultFS", "unsupported://example.com") val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val eventLogPath = eventLogger.logPath - val expectedLogDir = testDir.toURI().toString() + val expectedLogDir = testDir.toURI() assert(eventLogPath === EventLoggingListener.getLogPath( - expectedLogDir, sc.applicationId, compressionCodec.map(CompressionCodec.getShortName))) + expectedLogDir, sc.applicationId, None, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) @@ -262,7 +268,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef object EventLoggingListenerSuite { /** Get a SparkConf with event logging enabled. */ - def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { val conf = new SparkConf conf.set("spark.eventLog.enabled", "true") conf.set("spark.eventLog.testing", "true") @@ -274,5 +280,5 @@ object EventLoggingListenerSuite { conf } - def getUniqueApplicationId = "test-" + System.currentTimeMillis + def getUniqueApplicationId: String = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 6b75c98839e03..9b92f8de56759 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -24,7 +24,9 @@ import org.apache.spark.TaskContext /** * A Task implementation that fails to serialize. */ -private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) extends Task[Array[Byte]](stageId, 0) { +private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) + extends Task[Array[Byte]](stageId, 0) { + override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index c8c957856247a..cf97707946706 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -161,6 +161,31 @@ class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { } assert(tempDir.list().size === 0) } + + test("Only authorized committer failures can clear the authorized committer lock (SPARK-6614)") { + val stage: Int = 1 + val partition: Long = 2 + val authorizedCommitter: Long = 3 + val nonAuthorizedCommitter: Long = 100 + outputCommitCoordinator.stageStart(stage) + assert(outputCommitCoordinator.canCommit(stage, partition, attempt = authorizedCommitter)) + assert(!outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter)) + // The non-authorized committer fails + outputCommitCoordinator.taskCompleted( + stage, partition, attempt = nonAuthorizedCommitter, reason = TaskKilled) + // New tasks should still not be able to commit because the authorized committer has not failed + assert( + !outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 1)) + // The authorized committer now fails, clearing the lock + outputCommitCoordinator.taskCompleted( + stage, partition, attempt = authorizedCommitter, reason = TaskKilled) + // A new task should now be allowed to become the authorized committer + assert( + outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 2)) + // There can only be one authorized committer + assert( + !outputCommitCoordinator.canCommit(stage, partition, attempt = nonAuthorizedCommitter + 3)) + } } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 601694f57aad0..dabe4574b6456 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, PrintWriter} +import java.net.URI import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -49,7 +50,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val fstream = fileSystem.create(logFilePath) val writer = new PrintWriter(fstream) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, - 125L, "Mickey") + 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) @@ -145,7 +146,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * log the events. */ private class EventMonster(conf: SparkConf) - extends EventLoggingListener("test", "testdir", conf) { + extends EventLoggingListener("test", None, new URI("testdir"), conf) { override def start() { } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 627c9a4ddfffc..825c616c0c3e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -85,7 +85,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val stopperReturned = new Semaphore(0) class BlockingListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { listenerStarted.release() listenerWait.acquire() drained = true @@ -206,8 +206,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time val w = { i: Int => - if (i == 0) + if (i == 0) { Thread.sleep(100) + } i } @@ -247,12 +248,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => - taskMetrics.resultSize should be > (0l) + taskMetrics.resultSize should be > (0L) if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.inputMetrics should not be ('defined) taskMetrics.outputMetrics should not be ('defined) taskMetrics.shuffleWriteMetrics should be ('defined) - taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) + taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0L) } if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) @@ -260,7 +261,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sm.totalBlocksFetched should be (128) sm.localBlocksFetched should be (128) sm.remoteBlocksFetched should be (0) - sm.remoteBytesRead should be (0l) + sm.remoteBytesRead should be (0L) } } } @@ -406,12 +407,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { startedTasks += taskStart.taskInfo.index notify() } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { endedTasks += taskEnd.taskInfo.index notify() } @@ -425,7 +426,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers * A simple listener that throws an exception on job end. */ private class BadListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { throw new Exception } } } @@ -438,10 +439,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ private class BasicJobCounter extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } 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 057e226916027..83ae8701243e5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -51,7 +51,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } test("all TaskCompletionListeners should be called even if some fail") { - val context = new TaskContextImpl(0, 0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0, null) val listener = mock(classOf[TaskCompletionListener]) context.addTaskCompletionListener(_ => throw new Exception("blah")) context.addTaskCompletionListener(listener) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index add13f5b21765..ffa4381969b68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import java.util.Properties - import org.scalatest.FunSuite import org.apache.spark._ @@ -27,7 +25,7 @@ class FakeSchedulerBackend extends SchedulerBackend { def start() {} def stop() {} def reviveOffers() {} - def defaultParallelism() = 1 + def defaultParallelism(): Int = 1 } class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { @@ -115,7 +113,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } val numFreeCores = 1 taskScheduler.setDAGScheduler(dagScheduler) - var taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -123,7 +122,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(0 === taskDescriptions.length) // Now check that we can still submit tasks - // Even if one of the tasks has not-serializable tasks, the other task set should still be processed without error + // Even if one of the tasks has not-serializable tasks, the other task set should + // still be processed without error taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(FakeTask.createTaskSet(1)) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten 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 12330d8f63c40..6198cea46ddf8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -27,7 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -67,7 +66,7 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRackForHost(host: String) = { + def getRackForHost(host: String): Option[String] = { hostToRack.get(host) } } @@ -152,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) + val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 override def beforeEach() { @@ -240,7 +239,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) @@ -251,7 +250,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } @@ -292,7 +291,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host1 again: nothing should get chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -306,7 +305,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Now that we've launched a local task, we should no longer launch the task for host3 assert(manager.resourceOffer("exec2", "host2", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) @@ -327,8 +326,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - // After this, nothing should get chosen, because we have separated tasks with unavailable preference - // from the noPrefPendingTasks + // After this, nothing should get chosen, because we have separated tasks with unavailable + // preference from the noPrefPendingTasks assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead @@ -338,7 +337,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT * 2) + clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -499,7 +498,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sched.addExecutor("execC", "host2") manager.executorAdded() // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL and ANY - assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) + assert(manager.myLocalityLevels.sameElements( + Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) // test if the valid locality is recomputed when the executor is lost sched.removeExecutor("execC") manager.executorLost("execC", "host2") @@ -527,7 +527,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) @@ -569,7 +569,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { @@ -582,7 +583,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") sc = new SparkContext("local", "test", conf) - def genBytes(size: Int) = { (x: Int) => + def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => val bytes = Array.ofDim[Byte](size) scala.util.Random.nextBytes(bytes) bytes @@ -605,7 +606,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("speculative and noPref task should be scheduled after node-local") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2"), TaskLocation("host1")), @@ -619,19 +621,21 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } - test("node-local tasks should be scheduled right away when there are only node-local and no-preference tasks") { + test("node-local tasks should be scheduled right away " + + "when there are only node-local and no-preference tasks") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -650,7 +654,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } - test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") { + test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") + { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) val taskSet = FakeTask.createTaskSet(4, @@ -710,13 +715,13 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) assert(manager.resourceOffer("execA", "host1", ANY) !== None) - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1") manager.executorLost("execB.2", "host2") - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala similarity index 69% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index afbaa9ade811f..ab863f3d8d672 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import java.util @@ -24,21 +24,20 @@ import java.util.Collections import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.Scalar -import org.mockito.Mockito._ +import org.apache.mesos.Protos._ +import org.apache.mesos.SchedulerDriver import org.mockito.Matchers._ +import org.mockito.Mockito._ import org.mockito.{ArgumentCaptor, Matchers} import org.scalatest.FunSuite import org.scalatest.mock.MockitoSugar -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.cluster.mesos.{MesosSchedulerBackend, MemoryUtils} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with MockitoSugar { @@ -64,16 +63,64 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo // uri is null. val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo.getCommand.getValue === + s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") // uri exists. conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") val executorInfo1 = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo1.getCommand.getValue === + s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") + } + + test("spark docker properties correctly populate the DockerInfo message") { + val taskScheduler = mock[TaskSchedulerImpl] + + val conf = new SparkConf() + .set("spark.mesos.executor.docker.image", "spark/mock") + .set("spark.mesos.executor.docker.volumes", "/a,/b:/b,/c:/c:rw,/d:ro,/e:/e:ro") + .set("spark.mesos.executor.docker.portmaps", "80:8080,53:53:tcp") + + val listenerBus = mock[LiveListenerBus] + listenerBus.post( + SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + + val sc = mock[SparkContext] + when(sc.executorMemory).thenReturn(100) + when(sc.getSparkHome()).thenReturn(Option("/spark-home")) + when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) + when(sc.conf).thenReturn(conf) + when(sc.listenerBus).thenReturn(listenerBus) + + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + + val execInfo = backend.createExecutorInfo("mockExecutor") + assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock")) + val portmaps = execInfo.getContainer.getDocker.getPortMappingsList + assert(portmaps.get(0).getHostPort.equals(80)) + assert(portmaps.get(0).getContainerPort.equals(8080)) + assert(portmaps.get(0).getProtocol.equals("tcp")) + assert(portmaps.get(1).getHostPort.equals(53)) + assert(portmaps.get(1).getContainerPort.equals(53)) + assert(portmaps.get(1).getProtocol.equals("tcp")) + val volumes = execInfo.getContainer.getVolumesList + assert(volumes.get(0).getContainerPath.equals("/a")) + assert(volumes.get(0).getMode.equals(Volume.Mode.RW)) + assert(volumes.get(1).getContainerPath.equals("/b")) + assert(volumes.get(1).getHostPath.equals("/b")) + assert(volumes.get(1).getMode.equals(Volume.Mode.RW)) + assert(volumes.get(2).getContainerPath.equals("/c")) + assert(volumes.get(2).getHostPath.equals("/c")) + assert(volumes.get(2).getMode.equals(Volume.Mode.RW)) + assert(volumes.get(3).getContainerPath.equals("/d")) + assert(volumes.get(3).getMode.equals(Volume.Mode.RO)) + assert(volumes.get(4).getContainerPath.equals("/e")) + assert(volumes.get(4).getHostPath.equals("/e")) + assert(volumes.get(4).getMode.equals(Volume.Mode.RO)) } test("mesos resource offers result in launching tasks") { - def createOffer(id: Int, mem: Int, cpu: Int) = { + def createOffer(id: Int, mem: Int, cpu: Int): Offer = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -83,8 +130,10 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) + .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) + .setHostname(s"host${id.toString}").build() } val driver = mock[SchedulerDriver] @@ -115,12 +164,12 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(0).getSlaveId.getValue, mesosOffers.get(0).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(2).getSlaveId.getValue, mesosOffers.get(2).getHostname, - 2 + (minCpu - backend.mesosExecutorCores).toInt )) val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala similarity index 92% rename from core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala index 86a42a7398e4d..eebcba40f8a1c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala @@ -15,14 +15,12 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer import org.scalatest.FunSuite -import org.apache.spark.scheduler.cluster.mesos.MesosTaskLaunchData - class MesosTaskLaunchDataSuite extends FunSuite { test("serialize and deserialize data must be same") { val serializedTask = ByteBuffer.allocate(40) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala new file mode 100644 index 0000000000000..f28e29e9b8d8e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala @@ -0,0 +1,76 @@ +/* + * 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.scheduler.mesos + +import java.util.Date + +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.deploy.Command +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.scheduler.cluster.mesos._ +import org.apache.spark.{LocalSparkContext, SparkConf} + + +class MesosClusterSchedulerSuite extends FunSuite with LocalSparkContext with MockitoSugar { + + private val command = new Command("mainClass", Seq("arg"), null, null, null, null) + + test("can queue drivers") { + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("spark mesos") + val scheduler = new MesosClusterScheduler( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) { + override def start(): Unit = { ready = true } + } + scheduler.start() + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", 1000, 1, true, + command, Map[String, String](), "s1", new Date())) + assert(response.success) + val response2 = + scheduler.submitDriver(new MesosDriverDescription( + "d1", "jar", 1000, 1, true, command, Map[String, String](), "s2", new Date())) + assert(response2.success) + val state = scheduler.getSchedulerState() + val queuedDrivers = state.queuedDrivers.toList + assert(queuedDrivers(0).submissionId == response.submissionId) + assert(queuedDrivers(1).submissionId == response2.submissionId) + } + + test("can kill queued drivers") { + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("spark mesos") + val scheduler = new MesosClusterScheduler( + new BlackHoleMesosClusterPersistenceEngineFactory, conf) { + override def start(): Unit = { ready = true } + } + scheduler.start() + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", 1000, 1, true, + command, Map[String, String](), "s1", new Date())) + assert(response.success) + val killResponse = scheduler.killDriver(response.submissionId) + assert(killResponse.success) + val state = scheduler.getSchedulerState() + assert(state.queuedDrivers.isEmpty) + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala index 967c9e9899c9d..da98d09184735 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -33,8 +33,8 @@ class KryoSerializerResizableOutputSuite extends FunSuite { test("kryo without resizable output buffer should fail on large array") { val conf = new SparkConf(false) conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "1") + conf.set("spark.kryoserializer.buffer", "1m") + conf.set("spark.kryoserializer.buffer.max", "1m") val sc = new SparkContext("local", "test", conf) intercept[SparkException](sc.parallelize(x).collect()) LocalSparkContext.stop(sc) @@ -43,8 +43,8 @@ class KryoSerializerResizableOutputSuite extends FunSuite { test("kryo with resizable output buffer should succeed on large array") { val conf = new SparkConf(false) conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "2") + conf.set("spark.kryoserializer.buffer", "1m") + conf.set("spark.kryoserializer.buffer.max", "2m") val sc = new SparkContext("local", "test", conf) assert(sc.parallelize(x).collect() === x) LocalSparkContext.stop(sc) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 6198df84fab3d..778a7eee73b23 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -106,7 +106,9 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1->"one",2->"two",3->"three"))) } test("ranges") { @@ -169,7 +171,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("kryo with collect") { val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + val result = sc.parallelize(control, 2) + .map(new ClassWithoutNoArgConstructor(_)) + .collect() + .map(_.x) assert(control === result.toSeq) } @@ -237,7 +242,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { // Set a special, broken ClassLoader and make sure we get an exception on deserialization ser.setDefaultClassLoader(new ClassLoader() { - override def loadClass(name: String) = throw new UnsupportedOperationException + override def loadClass(name: String): Class[_] = throw new UnsupportedOperationException }) intercept[UnsupportedOperationException] { ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) @@ -264,7 +269,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("serialization buffer overflow reporting") { import org.apache.spark.SparkException - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max.mb" + val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" val largeObject = (1 to 1000000).toArray @@ -275,6 +280,15 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { val thrown = intercept[SparkException](ser.serialize(largeObject)) assert(thrown.getMessage.contains(kryoBufferMaxProperty)) } + + test("getAutoReset") { + val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] + assert(ser.getAutoReset) + val conf = new SparkConf().set("spark.kryo.registrator", + classOf[RegistratorWithoutAutoReset].getName) + val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] + assert(!ser2.getAutoReset) + } } @@ -287,14 +301,14 @@ object KryoTest { class ClassWithNoArgConstructor { var x: Int = 0 - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithNoArgConstructor => x == c.x case _ => false } } class ClassWithoutNoArgConstructor(val x: Int) { - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithoutNoArgConstructor => x == c.x case _ => false } @@ -308,4 +322,10 @@ object KryoTest { k.register(classOf[java.util.HashMap[_, _]]) } } + + class RegistratorWithoutAutoReset extends KryoRegistrator { + override def registerClasses(k: Kryo) { + k.setAutoReset(false) + } + } } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index d037e2c19a64d..433fd6bb4a11d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -24,14 +24,16 @@ import org.apache.spark.rdd.RDD /* A trivial (but unserializable) container for trivial functions */ class UnserializableClass { - def op[T](x: T) = x.toString + def op[T](x: T): String = x.toString - def pred[T](x: T) = x.toString.length % 2 == 0 + def pred[T](x: T): Boolean = x.toString.length % 2 == 0 } class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + def fixture: (RDD[String], UnserializableClass) = { + (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + } test("throws expected serialization exceptions on actions") { val (data, uc) = fixture diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala index 0ade1bab18d7e..86fcf447287f7 100644 --- a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -24,10 +24,10 @@ import scala.reflect.ClassTag /** - * A serializer implementation that always return a single element in a deserialization stream. + * A serializer implementation that always returns two elements in a deserialization stream. */ class TestSerializer extends Serializer { - override def newInstance() = new TestSerializerInstance + override def newInstance(): TestSerializerInstance = new TestSerializerInstance } @@ -36,7 +36,8 @@ class TestSerializerInstance extends SerializerInstance { override def serializeStream(s: OutputStream): SerializationStream = ??? - override def deserializeStream(s: InputStream) = new TestDeserializationStream + override def deserializeStream(s: InputStream): TestDeserializationStream = + new TestDeserializationStream override def deserialize[T: ClassTag](bytes: ByteBuffer): T = ??? @@ -50,7 +51,7 @@ class TestDeserializationStream extends DeserializationStream { override def readObject[T: ClassTag](): T = { count += 1 - if (count == 2) { + if (count == 3) { throw new EOFException } new Object().asInstanceOf[T] diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 6790388f96603..84384bb48999a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -54,13 +54,13 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test", conf) val shuffleBlockManager = - SparkEnv.get.shuffleManager.shuffleBlockManager.asInstanceOf[FileShuffleBlockManager] + SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[FileShuffleBlockManager] val shuffle1 = shuffleBlockManager.forMapTask(1, 1, 1, new JavaSerializer(conf), new ShuffleWriteMetrics) for (writer <- shuffle1.writers) { - writer.write("test1") - writer.write("test2") + writer.write("test1", "value") + writer.write("test2", "value") } for (writer <- shuffle1.writers) { writer.commitAndClose() @@ -73,8 +73,8 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { new ShuffleWriteMetrics) for (writer <- shuffle2.writers) { - writer.write("test3") - writer.write("test4") + writer.write("test3", "value") + writer.write("test4", "vlue") } for (writer <- shuffle2.writers) { writer.commitAndClose() @@ -85,14 +85,14 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { // Now comes the test : // Write to shuffle 3; and close it, but before registering it, check if the file lengths for // previous task (forof shuffle1) is the same as 'segments'. Earlier, we were inferring length - // of block based on remaining data in file : which could mess things up when there is concurrent read - // and writes happening to the same shuffle group. + // of block based on remaining data in file : which could mess things up when there is + // concurrent read and writes happening to the same shuffle group. val shuffle3 = shuffleBlockManager.forMapTask(1, 3, 1, new JavaSerializer(testConf), new ShuffleWriteMetrics) for (writer <- shuffle3.writers) { - writer.write("test3") - writer.write("test4") + writer.write("test3", "value") + writer.write("test4", "value") } for (writer <- shuffle3.writers) { writer.commitAndClose() diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala new file mode 100644 index 0000000000000..5274df904d395 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala @@ -0,0 +1,29 @@ +/* + * 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.status.api.v1 + +import org.scalatest.{Matchers, FunSuite} + +class SimpleDateParamTest extends FunSuite with Matchers { + + test("date parsing") { + new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L) + new SimpleDateParam("2015-02-20T17:21:17.190CST").timestamp should be (1424474477190L) + new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L) // GMT + } + +} 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 c2903c8597997..f647200402ecb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -22,11 +22,11 @@ import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps -import akka.actor.{ActorSystem, Props} import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.network.BlockTransferService import org.apache.spark.network.nio.NioBlockTransferService @@ -34,13 +34,12 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.StorageLevel._ -import org.apache.spark.util.{AkkaUtils, SizeEstimator} /** Testsuite that tests block replication in BlockManager */ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAndAfter { private val conf = new SparkConf(false) - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -51,7 +50,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd val allStores = new ArrayBuffer[BlockManager] // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer", "1m") val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. @@ -61,7 +60,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) - val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + val store = new BlockManager(name, rpcEnv, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) store.initialize("app-id") allStores += store @@ -69,32 +68,29 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd } before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.authenticate", "false") - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1") + conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) allStores.clear() } after { allStores.foreach { _.stop() } allStores.clear() - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null master = null } @@ -262,7 +258,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, + val failableStore = new BlockManager("failable-store", rpcEnv, master, serializer, 10000, conf, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test 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 3fdbe99b5d02b..151955ef7f435 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,24 +19,18 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays -import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps -import akka.actor._ -import akka.pattern.ask -import akka.util.Timeout - import org.mockito.Mockito.{mock, when} - import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.executor.DataReadMethod import org.apache.spark.network.nio.NioBlockTransferService @@ -53,7 +47,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) @@ -61,39 +55,36 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val shuffleManager = new HashShuffleManager(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer", "1m") val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + def rdd(rddId: Int, splitId: Int): RDDBlockId = RDDBlockId(rddId, splitId) private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) - val manager = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + val manager = new BlockManager(name, rpcEnv, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) manager.initialize("app-id") manager } override def beforeEach(): Unit = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") - conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.driver.port", rpcEnv.address.port.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -108,16 +99,18 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store2.stop() store2 = null } - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null master = null } test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, false, 3) - val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object + // this should return the same object as level1 + val level2 = StorageLevel(false, false, false, false, 3) + // this should return a different object + val level3 = StorageLevel(false, false, false, false, 2) assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -148,6 +141,12 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(id2_.eq(id1), "Deserialized id2 is not the same object as original id1") } + test("BlockManagerId.isDriver() backwards-compatibility with legacy driver ids (SPARK-6716)") { + assert(BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "XXX", 1).isDriver) + assert(BlockManagerId(SparkContext.LEGACY_DRIVER_IDENTIFIER, "XXX", 1).isDriver) + assert(!BlockManagerId("notADriverIdentifier", "XXX", 1).isDriver) + } + test("master + 1 manager interaction") { store = makeBlockManager(20000) val a1 = new Array[Byte](4000) @@ -170,8 +169,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(master.getLocations("a3").size === 0, "master was told about a3") // Drop a1 and a2 from memory; this should be reported back to the master - store.dropFromMemory("a1", null) - store.dropFromMemory("a2", null) + store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) + store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) assert(store.getSingle("a1") === None, "a1 not removed from store") assert(store.getSingle("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") @@ -357,10 +356,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - implicit val timeout = Timeout(30, TimeUnit.SECONDS) - val reregister = !Await.result( - master.driverActor ? BlockManagerHeartbeat(store.blockManagerId), - timeout.duration).asInstanceOf[Boolean] + val reregister = !master.driverEndpoint.askWithRetry[Boolean]( + BlockManagerHeartbeat(store.blockManagerId)) assert(reregister == true) } @@ -413,8 +410,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach t2.join() t3.join() - store.dropFromMemory("a1", null) - store.dropFromMemory("a2", null) + store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) + store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) store.waitForAsyncReregister() } } @@ -431,19 +428,19 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) - assert(list1Get.get.inputMetrics.bytesRead === list1SizeEstimate) - assert(list1Get.get.inputMetrics.readMethod === DataReadMethod.Memory) + assert(list1Get.get.bytes === list1SizeEstimate) + assert(list1Get.get.readMethod === DataReadMethod.Memory) val list2MemoryGet = store.get("list2memory") assert(list2MemoryGet.isDefined, "list2memory expected to be in store") assert(list2MemoryGet.get.data.size === 3) - assert(list2MemoryGet.get.inputMetrics.bytesRead === list2SizeEstimate) - assert(list2MemoryGet.get.inputMetrics.readMethod === DataReadMethod.Memory) + assert(list2MemoryGet.get.bytes === list2SizeEstimate) + assert(list2MemoryGet.get.readMethod === DataReadMethod.Memory) val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) // We don't know the exact size of the data on disk, but it should certainly be > 0. - assert(list2DiskGet.get.inputMetrics.bytesRead > 0) - assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) + assert(list2DiskGet.get.bytes > 0) + assert(list2DiskGet.get.readMethod === DataReadMethod.Disk) } test("in-memory LRU storage") { @@ -529,6 +526,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach test("tachyon storage") { // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) + conf.set(ExternalBlockStore.BLOCK_MANAGER_NAME, ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME) if (tachyonUnitTestEnabled) { store = makeBlockManager(1200) val a1 = new Array[Byte](400) @@ -785,7 +783,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach test("block store put failure") { // Use Java serializer so we can create an unserializable error. val transfer = new NioBlockTransferService(conf, securityMgr) - store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, actorSystem, master, + store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) @@ -807,7 +805,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // Create a non-trivial (not all zeros) byte array var counter = 0.toByte - def incr = {counter = (counter + 1).toByte; counter;} + def incr: Byte = {counter = (counter + 1).toByte; counter;} val bytes = Array.fill[Byte](1000)(incr) val byteBuffer = ByteBuffer.wrap(bytes) @@ -817,14 +815,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // be nice to refactor classes involved in disk storage in a way that // allows for easier testing. val blockManager = mock(classOf[BlockManager]) - when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) + when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0")) val diskBlockManager = new DiskBlockManager(blockManager, conf) val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) val mapped = diskStoreMapped.getBytes(blockId).get - when(blockManager.conf).thenReturn(conf.clone.set(confKey, (1000 * 1000).toString)) + when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) val notMapped = diskStoreNotMapped.getBytes(blockId).get @@ -961,8 +959,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) - assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size + === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size + === 1) // insert some more blocks store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) @@ -970,8 +970,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size + === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size + === 3) val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => @@ -1095,8 +1097,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with plenty of space. This should succeed and cache both blocks. @@ -1149,8 +1151,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) store.putIterator("b1", smallIterator, memAndDisk) @@ -1192,7 +1194,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memOnly = StorageLevel.MEMORY_ONLY val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // All unroll memory used is released because unrollSafely returned an array @@ -1223,4 +1225,30 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) } + + test("lazily create a big ByteBuffer to avoid OOM if it cannot be put into MemoryStore") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + val blockId = BlockId("rdd_3_10") + val result = memoryStore.putBytes(blockId, 13000, () => { + fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") + }) + assert(result.size === 13000) + assert(result.data === null) + assert(result.droppedBlocks === Nil) + } + + test("put a small ByteBuffer to MemoryStore") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + val blockId = BlockId("rdd_3_10") + var bytes: ByteBuffer = null + val result = memoryStore.putBytes(blockId, 10000, () => { + bytes = ByteBuffer.allocate(10000) + bytes + }) + assert(result.size === 10000) + assert(result.data === Right(bytes)) + assert(result.droppedBlocks === Nil) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala index 78bbc4ec2c620..43ef469c1fd48 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -30,9 +30,9 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) - writer.write(Long.box(20)) + writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write @@ -40,7 +40,7 @@ class BlockObjectWriterSuite extends FunSuite { // After 32 writes, metrics should update for (i <- 0 until 32) { writer.flush() - writer.write(Long.box(i)) + writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) assert(writeMetrics.shuffleRecordsWritten === 33) @@ -52,9 +52,9 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) - writer.write(Long.box(20)) + writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write @@ -62,7 +62,7 @@ class BlockObjectWriterSuite extends FunSuite { // After 32 writes, metrics should update for (i <- 0 until 32) { writer.flush() - writer.write(Long.box(i)) + writer.write(Long.box(i), Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) assert(writeMetrics.shuffleRecordsWritten === 33) @@ -75,7 +75,7 @@ class BlockObjectWriterSuite extends FunSuite { val file = new File(Utils.createTempDir(), "somefile") val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, - new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) writer.open() writer.close() diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index 82a82e23eecf2..b47157f8331cc 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -47,7 +47,7 @@ class LocalDirsSuite extends FunSuite with BeforeAndAfter { assert(!new File("/NONEXISTENT_DIR").exists()) // SPARK_LOCAL_DIRS is a valid directory: class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir") else super.getenv(name) } diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 37b593b2c5f79..2080c432d77db 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -89,7 +89,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0, 0), + new TaskContextImpl(0, 0, 0, 0, null), transfer, blockManager, blocksByAddress, @@ -154,7 +154,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0, null) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, @@ -217,7 +217,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0, null) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index ef5c55f91c39a..17193ddbfd894 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -240,11 +240,11 @@ class StorageSuite extends FunSuite { assert(status.rddBlocksById(1000).size === status.numRddBlocksById(1000)) } - test("storage status memUsed, diskUsed, tachyonUsed") { + test("storage status memUsed, diskUsed, externalBlockStoreUsed") { val status = storageStatus2 def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum - def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).sum + def actualOffHeapUsed: Long = status.blocks.values.map(_.externalBlockStoreSize).sum assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) assert(status.offHeapUsed === actualOffHeapUsed) @@ -287,8 +287,8 @@ class StorageSuite extends FunSuite { // For testing StorageUtils.updateRddInfo private def stockRDDInfos: Seq[RDDInfo] = { - val info0 = new RDDInfo(0, "0", 10, memAndDisk) - val info1 = new RDDInfo(1, "1", 3, memAndDisk) + val info0 = new RDDInfo(0, "0", 10, memAndDisk, Seq(3)) + val info1 = new RDDInfo(1, "1", 3, memAndDisk, Seq(4)) Seq(info0, info1) } @@ -300,12 +300,12 @@ class StorageSuite extends FunSuite { assert(rddInfos(0).numCachedPartitions === 5) assert(rddInfos(0).memSize === 5L) assert(rddInfos(0).diskSize === 10L) - assert(rddInfos(0).tachyonSize === 0L) + assert(rddInfos(0).externalBlockStoreSize === 0L) assert(rddInfos(1).storageLevel === memAndDisk) assert(rddInfos(1).numCachedPartitions === 3) assert(rddInfos(1).memSize === 3L) assert(rddInfos(1).diskSize === 6L) - assert(rddInfos(1).tachyonSize === 0L) + assert(rddInfos(1).externalBlockStoreSize === 0L) } test("StorageUtils.getRddBlockLocations") { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 0d155982a8c54..117b2c3960820 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.ui -import javax.servlet.http.HttpServletRequest +import java.net.{HttpURLConnection, URL} +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.collection.JavaConversions._ import scala.xml.Node +import org.json4s._ +import org.json4s.jackson.JsonMethods import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.openqa.selenium.{By, WebDriver} import org.scalatest._ @@ -32,8 +35,9 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.LocalSparkContext._ import org.apache.spark._ import org.apache.spark.api.java.StorageLevels +import org.apache.spark.deploy.history.HistoryServerSuite import org.apache.spark.shuffle.FetchFailedException - +import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} /** * Selenium tests for the Spark Web UI. @@ -41,6 +45,8 @@ import org.apache.spark.shuffle.FetchFailedException class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll { implicit var webDriver: WebDriver = _ + implicit val formats = DefaultFormats + override def beforeAll(): Unit = { webDriver = new HtmlUnitDriver @@ -56,12 +62,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before * Create a test SparkContext with the SparkUI enabled. * It is safe to `get` the SparkUI directly from the SparkContext returned here. */ - private def newSparkContext(): SparkContext = { + private def newSparkContext(killEnabled: Boolean = true): SparkContext = { val conf = new SparkConf() .setMaster("local") .setAppName("test") .set("spark.ui.enabled", "true") .set("spark.ui.port", "0") + .set("spark.ui.killEnabled", killEnabled.toString) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -74,28 +81,42 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before val rdd = sc.parallelize(Seq(1, 2, 3)) rdd.persist(StorageLevels.DISK_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage") + goToUi(ui, "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + goToUi(ui, "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.DISK_ONLY.description) } + val storageJson = getJson(ui, "storage/rdd") + storageJson.children.length should be (1) + (storageJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) + val rddJson = getJson(ui, "storage/rdd/0") + (rddJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description) + rdd.unpersist() rdd.persist(StorageLevels.MEMORY_ONLY).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage") + goToUi(ui, "/storage") val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + goToUi(ui, "/storage/rdd/?id=0") val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq tableRowText should contain (StorageLevels.MEMORY_ONLY.description) } + + val updatedStorageJson = getJson(ui, "storage/rdd") + updatedStorageJson.children.length should be (1) + (updatedStorageJson \ "storageLevel").extract[String] should be ( + StorageLevels.MEMORY_ONLY.description) + val updatedRddJson = getJson(ui, "storage/rdd/0") + (updatedRddJson \ "storageLevel").extract[String] should be ( + StorageLevels.MEMORY_ONLY.description) } } @@ -106,10 +127,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") find(id("active")) should be(None) // Since we hide empty tables find(id("failed")).get.text should be("Failed Stages (1)") } + val stageJson = getJson(sc.ui.get, "stages") + stageJson.children.length should be (1) + (stageJson \ "status").extract[String] should be (StageStatus.FAILED.name()) // Regression test for SPARK-2105 class NotSerializable @@ -118,42 +142,36 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") find(id("active")) should be(None) // Since we hide empty tables // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: find(id("failed")).get.text should be("Failed Stages (1)") } + + val updatedStageJson = getJson(sc.ui.get, "stages") + updatedStageJson should be (stageJson) } } test("spark.ui.killEnabled should properly control kill button display") { - def getSparkContext(killEnabled: Boolean): SparkContext = { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.ui.enabled", "true") - .set("spark.ui.killEnabled", killEnabled.toString) - new SparkContext(conf) - } - - def hasKillLink = find(className("kill-link")).isDefined + def hasKillLink: Boolean = find(className("kill-link")).isDefined def runSlowJob(sc: SparkContext) { sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } - withSpark(getSparkContext(killEnabled = true)) { sc => + withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") assert(hasKillLink) } } - withSpark(getSparkContext(killEnabled = false)) { sc => + withSpark(newSparkContext(killEnabled = false)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + goToUi(sc, "/stages") assert(!hasKillLink) } } @@ -164,7 +182,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // If no job has been run in a job group, then "(Job Group)" should not appear in the header sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should not contain "Job Id (Job Group)" } @@ -172,10 +190,22 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before sc.setJobGroup("my-job-group", "my-job-group-description") sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq tableHeaders should contain ("Job Id (Job Group)") } + + val jobJson = getJson(sc.ui.get, "jobs") + for { + job @ JObject(_) <- jobJson + JInt(jobId) <- job \ "jobId" + jobGroup = job \ "jobGroup" + } { + jobId.toInt match { + case 0 => jobGroup should be (JNothing) + case 1 => jobGroup should be (JString("my-job-group")) + } + } } } @@ -202,7 +232,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } mappedData.count() eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") // Ideally, the following test would pass, but currently we overcount completed tasks // if task recomputations occur: @@ -211,6 +241,32 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // of completed tasks may be higher: find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") } + val jobJson = getJson(sc.ui.get, "jobs") + (jobJson \ "numTasks").extract[Int]should be (2) + (jobJson \ "numCompletedTasks").extract[Int] should be (3) + (jobJson \ "numFailedTasks").extract[Int] should be (1) + (jobJson \ "numCompletedStages").extract[Int] should be (2) + (jobJson \ "numFailedStages").extract[Int] should be (1) + val stageJson = getJson(sc.ui.get, "stages") + + for { + stage @ JObject(_) <- stageJson + JString(status) <- stage \ "status" + JInt(stageId) <- stage \ "stageId" + JInt(attemptId) <- stage \ "attemptId" + } { + val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE + status should be (exp.name()) + } + + for { + stageId <- 0 to 1 + attemptId <- 0 to 1 + } { + val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE + val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId") + (stageJson \ "status").extract[String] should be (exp.name()) + } } } @@ -225,7 +281,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // Start the job: rdd.countAsync() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + goToUi(sc, "/jobs/job/?id=0") find(id("active")).get.text should be ("Active Stages (1)") find(id("pending")).get.text should be ("Pending Stages (2)") // Essentially, we want to check that none of the stage rows show @@ -233,7 +289,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before // because someone could change the error message and cause this test to pass by accident. // Instead, it's safer to check that each row contains a link to a stage details page. findAll(cssSelector("tbody tr")).foreach { row => - val link = row.underlying.findElement(By.xpath(".//a")) + val link = row.underlying.findElement(By.xpath("./td/div/a")) link.getAttribute("href") should include ("stage") } } @@ -251,7 +307,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") // The completed jobs table should have two rows. The first row will be the most recent job: val firstRow = find(cssSelector("tbody tr")).get.underlying val firstRowColumns = firstRow.findElements(By.tagName("td")) @@ -278,7 +334,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + goToUi(sc, "/jobs/job/?id=1") find(id("pending")) should be (None) find(id("active")) should be (None) find(id("failed")) should be (None) @@ -306,7 +362,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before rdd.count() rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + goToUi(sc, "/jobs") findAll(cssSelector("tbody tr a")).foreach { link => link.text.toLowerCase should include ("count") link.text.toLowerCase should not include "unknown" @@ -328,7 +384,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } sparkUI.attachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/")) + goToUi(sc, "") find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -337,12 +393,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check whether new page exists - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + goToUi(sc, "/foo") find(cssSelector("b")).get.text should include ("html magic") } sparkUI.detachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sc.ui.get.appUIAddress.stripSuffix("/")) + goToUi(sc, "") find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) @@ -351,9 +407,190 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check new page not exist - go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + goToUi(sc, "/foo") find(cssSelector("b")) should be(None) } } } + + test("kill stage POST/GET response is correct") { + def getResponseCode(url: URL, method: String): Int = { + val connection = url.openConnection().asInstanceOf[HttpURLConnection] + connection.setRequestMethod(method) + connection.connect() + val code = connection.getResponseCode() + connection.disconnect() + code + } + + withSpark(newSparkContext(killEnabled = true)) { sc => + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + val url = new URL( + sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0&terminate=true") + // SPARK-6846: should be POST only but YARN AM doesn't proxy POST + getResponseCode(url, "GET") should be (200) + getResponseCode(url, "POST") should be (200) + } + } + } + + test("stage & job retention") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.port", "0") + .set("spark.ui.retainedStages", "3") + .set("spark.ui.retainedJobs", "2") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + + withSpark(sc) { sc => + // run a few jobs & stages ... + (0 until 5).foreach { idx => + // NOTE: if we reverse the order, things don't really behave nicely + // we lose the stage for a job we keep, and then the job doesn't know + // about its last stage + sc.parallelize(idx to (idx + 3)).map(identity).groupBy(identity).map(identity) + .groupBy(identity).count() + sc.parallelize(idx to (idx + 3)).collect() + } + + val expJobInfo = Seq( + ("9", "collect"), + ("8", "count") + ) + + eventually(timeout(1 second), interval(50 milliseconds)) { + goToUi(sc, "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + find("completed-summary").get.text should be ("Completed Jobs: 10, only showing 2") + find("completed").get.text should be ("Completed Jobs (10, only showing 2)") + val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} + rows.size should be (expJobInfo.size) + for { + (row, idx) <- rows.zipWithIndex + columns = row.findElements(By.tagName("td")) + id = columns(0).getText() + description = columns(1).getText() + } { + id should be (expJobInfo(idx)._1) + description should include (expJobInfo(idx)._2) + } + } + + val jobsJson = getJson(sc.ui.get, "jobs") + jobsJson.children.size should be (expJobInfo.size) + for { + (job @ JObject(_),idx) <- jobsJson.children.zipWithIndex + id = (job \ "jobId").extract[String] + name = (job \ "name").extract[String] + } { + withClue(s"idx = $idx; id = $id; name = ${name.substring(0,20)}") { + id should be (expJobInfo(idx)._1) + name should include (expJobInfo(idx)._2) + } + } + + // what about when we query for a job that did exist, but has been cleared? + goToUi(sc, "/jobs/job/?id=7") + find("no-info").get.text should be ("No information to display for job 7") + + val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7")) + badJob._1 should be (HttpServletResponse.SC_NOT_FOUND) + badJob._2 should be (None) + badJob._3 should be (Some("unknown job: 7")) + + val expStageInfo = Seq( + ("19", "collect"), + ("18", "count"), + ("17", "groupBy") + ) + + eventually(timeout(1 second), interval(50 milliseconds)) { + goToUi(sc, "/stages") + find("completed-summary").get.text should be ("Completed Stages: 20, only showing 3") + find("completed").get.text should be ("Completed Stages (20, only showing 3)") + val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying} + rows.size should be (3) + for { + (row, idx) <- rows.zipWithIndex + columns = row.findElements(By.tagName("td")) + id = columns(0).getText() + description = columns(1).getText() + } { + id should be (expStageInfo(idx)._1) + description should include (expStageInfo(idx)._2) + } + } + + val stagesJson = getJson(sc.ui.get, "stages") + stagesJson.children.size should be (3) + for { + (stage @ JObject(_), idx) <- stagesJson.children.zipWithIndex + id = (stage \ "stageId").extract[String] + name = (stage \ "name").extract[String] + } { + id should be (expStageInfo(idx)._1) + name should include (expStageInfo(idx)._2) + } + + // nonexistent stage + + goToUi(sc, "/stages/stage/?id=12&attempt=0") + find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)") + val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0")) + badStage._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStage._2 should be (None) + badStage._3 should be (Some("unknown stage: 12")) + + val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15")) + badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND) + badAttempt._2 should be (None) + badAttempt._3 should be (Some("unknown attempt for stage 19. Found attempts: [0]")) + + val badStageAttemptList = HistoryServerSuite.getContentAndCode( + jsonUrl(sc.ui.get, "stages/12")) + badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND) + badStageAttemptList._2 should be (None) + badStageAttemptList._3 should be (Some("unknown stage: 12")) + } + } + + test("live UI json application list") { + withSpark(newSparkContext()) { sc => + val appListRawJson = HistoryServerSuite.getUrl(new URL( + sc.ui.get.appUIAddress + "/json/v1/applications")) + val appListJsonAst = JsonMethods.parse(appListRawJson) + appListJsonAst.children.length should be (1) + val attempts = (appListJsonAst \ "attempts").children + attempts.size should be (1) + (attempts(0) \ "completed").extract[Boolean] should be (false) + parseDate(attempts(0) \ "startTime") should be (sc.startTime) + parseDate(attempts(0) \ "endTime") should be (-1) + val oneAppJsonAst = getJson(sc.ui.get, "") + oneAppJsonAst should be (appListJsonAst.children(0)) + } + } + + def goToUi(sc: SparkContext, path: String): Unit = { + goToUi(sc.ui.get, path) + } + + def goToUi(ui: SparkUI, path: String): Unit = { + go to (ui.appUIAddress.stripSuffix("/") + path) + } + + def parseDate(json: JValue): Long = { + JacksonMessageWriter.makeISODateFormat.parse(json.extract[String]).getTime + } + + def getJson(ui: SparkUI, path: String): JValue = { + JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path))) + } + + def jsonUrl(ui: SparkUI, path: String): URL = { + new URL(ui.appUIAddress + "/json/v1/applications/test/" + path) + } } 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 730a4b54f5aa1..967dd0821ebd0 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 @@ -17,6 +17,8 @@ package org.apache.spark.ui.jobs +import java.util.Properties + import org.scalatest.FunSuite import org.scalatest.Matchers @@ -32,23 +34,31 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val jobCompletionTime = 1421191296660L private def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") SparkListenerStageSubmitted(stageInfo) } private def createStageEndEvent(stageId: Int, failed: Boolean = false) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") if (failed) { stageInfo.failureReason = Some("Failed!") } SparkListenerStageCompleted(stageInfo) } - private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = { + private def createJobStartEvent( + jobId: Int, + stageIds: Seq[Int], + jobGroup: Option[String] = None): SparkListenerJobStart = { val stageInfos = stageIds.map { stageId => - new StageInfo(stageId, 0, stageId.toString, 0, null, "") + new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") + } + val properties: Option[Properties] = jobGroup.map { groupId => + val props = new Properties() + props.setProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + props } - SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos) + SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos, properties.orNull) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { @@ -110,6 +120,23 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc listener.stageIdToActiveJobIds.size should be (0) } + test("test clearing of jobGroupToJobIds") { + val conf = new SparkConf() + conf.set("spark.ui.retainedJobs", 5.toString) + val listener = new JobProgressListener(conf) + + // Run 50 jobs, each with one stage + for (jobId <- 0 to 50) { + listener.onJobStart(createJobStartEvent(jobId, Seq(0), jobGroup = Some(jobId.toString))) + listener.onStageSubmitted(createStageStartEvent(0)) + listener.onStageCompleted(createStageEndEvent(0, failed = false)) + listener.onJobEnd(createJobEndEvent(jobId, false)) + } + assertActiveJobsStateIsEmpty(listener) + // This collection won't become empty, but it should be bounded by spark.ui.retainedJobs + listener.jobGroupToJobIds.size should be (5) + } + test("test LRU eviction of jobs") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) @@ -242,7 +269,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) val execId = "exe-1" - def makeTaskMetrics(base: Int) = { + def makeTaskMetrics(base: Int): TaskMetrics = { val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() val shuffleWriteMetrics = new ShuffleWriteMetrics() @@ -264,7 +291,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics } - def makeTaskInfo(taskId: Long, finishTime: Int = 0) = { + def makeTaskInfo(taskId: Long, finishTime: Int = 0): TaskInfo = { val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = finishTime diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index e1bc1379b5d80..7b38e6d9473e1 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -35,10 +35,10 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { private val none = StorageLevel.NONE private val taskInfo = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) private val taskInfo1 = new TaskInfo(1, 1, 1, 1, "big", "cat", TaskLocality.ANY, false) - private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly) - private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly) - private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk) - private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk) + private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly, Seq(10)) + private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly, Seq(10)) + private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk, Seq(10)) + private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk, Seq(10)) private val bm1 = BlockManagerId("big", "dog", 1) before { @@ -54,7 +54,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.isEmpty) // 2 RDDs are known, but none are cached - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0, rddInfo1), Seq.empty, "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.isEmpty) @@ -64,15 +64,16 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo3Cached = rddInfo3 rddInfo2Cached.numCachedPartitions = 1 rddInfo3Cached.numCachedPartitions = 1 - val stageInfo1 = new StageInfo(1, 0, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + val stageInfo1 = new StageInfo( + 1, 0, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), Seq.empty, "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) // Submitting RDDInfos with duplicate IDs does nothing - val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) + val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY, Seq(10)) rddInfo0Cached.numCachedPartitions = 1 - val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), "details") + val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), Seq.empty, "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) @@ -88,7 +89,8 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo1Cached = rddInfo1 rddInfo0Cached.numCachedPartitions = 1 rddInfo1Cached.numCachedPartitions = 1 - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + val stageInfo0 = new StageInfo( + 0, 0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), Seq.empty, "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.size === 2) @@ -107,7 +109,8 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + val stageInfo0 = new StageInfo( + 0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), Seq.empty, "details") bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) @@ -132,12 +135,12 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { bus.postToAll(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo, metrics1)) assert(storageListener._rddInfoMap(0).memSize === 800L) assert(storageListener._rddInfoMap(0).diskSize === 400L) - assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).externalBlockStoreSize === 200L) assert(storageListener._rddInfoMap(0).numCachedPartitions === 3) assert(storageListener._rddInfoMap(0).isCached) assert(storageListener._rddInfoMap(1).memSize === 0L) assert(storageListener._rddInfoMap(1).diskSize === 240L) - assert(storageListener._rddInfoMap(1).tachyonSize === 0L) + assert(storageListener._rddInfoMap(1).externalBlockStoreSize === 0L) assert(storageListener._rddInfoMap(1).numCachedPartitions === 1) assert(storageListener._rddInfoMap(1).isCached) assert(!storageListener._rddInfoMap(2).isCached) @@ -154,7 +157,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { bus.postToAll(SparkListenerTaskEnd(2, 0, "obliteration", Success, taskInfo, metrics2)) assert(storageListener._rddInfoMap(0).memSize === 400L) assert(storageListener._rddInfoMap(0).diskSize === 400L) - assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).externalBlockStoreSize === 200L) assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) assert(storageListener._rddInfoMap(0).isCached) assert(!storageListener._rddInfoMap(1).isCached) @@ -165,10 +168,10 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { test("verify StorageTab contains all cached rdds") { - val rddInfo0 = new RDDInfo(0, "rdd0", 1, memOnly) - val rddInfo1 = new RDDInfo(1, "rdd1", 1 ,memOnly) - val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo0), "details") - val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfo1), "details") + val rddInfo0 = new RDDInfo(0, "rdd0", 1, memOnly, Seq(4)) + val rddInfo1 = new RDDInfo(1, "rdd1", 1 ,memOnly, Seq(4)) + val stageInfo0 = new StageInfo(0, 0, "stage0", 1, Seq(rddInfo0), Seq.empty, "details") + val stageInfo1 = new StageInfo(1, 0, "stage1", 1, Seq(rddInfo1), Seq.empty, "details") val taskMetrics0 = new TaskMetrics val taskMetrics1 = new TaskMetrics val block0 = (RDDBlockId(0, 1), BlockStatus(memOnly, 100L, 0L, 0L)) diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 6250d50fb7036..bec79fc4dc8f7 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -19,14 +19,11 @@ package org.apache.spark.util import java.util.concurrent.TimeoutException -import scala.concurrent.Await -import scala.util.{Failure, Try} - -import akka.actor._ - +import akka.actor.ActorNotFound import org.scalatest.FunSuite import org.apache.spark._ +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId import org.apache.spark.SSLSampleConfigs._ @@ -39,39 +36,37 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro test("remote fetch security bad password") { val conf = new SparkConf + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf + badconf.set("spark.rpc", "akka") badconf.set("spark.authenticate", "true") badconf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = conf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, conf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security off") { @@ -81,28 +76,24 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = badconf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, badconf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -120,8 +111,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security pass") { @@ -131,15 +122,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") @@ -148,13 +138,10 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(securityManagerGood.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = goodconf, securityManager = securityManagerGood) + val slaveRpcEnv =RpcEnv.create("spark-slave", hostname, 0, goodconf, securityManagerGood) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() @@ -170,47 +157,45 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch security off client") { val conf = new SparkConf + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val badconf = new SparkConf + badconf.set("spark.rpc", "akka") badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === false) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = badconf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, badconf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch ssl on") { @@ -218,26 +203,22 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slaves", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -255,8 +236,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } @@ -267,28 +248,24 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() slaveConf.set("spark.authenticate", "true") slaveConf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) assert(securityManagerBad.isAuthenticationEnabled() === true) @@ -305,45 +282,43 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000))) - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } test("remote fetch ssl on and security enabled - bad credentials") { val conf = sparkSSLConfig() + conf.set("spark.rpc", "akka") conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === true) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() + slaveConf.set("spark.rpc", "akka") slaveConf.set("spark.authenticate", "true") slaveConf.set("spark.authenticate.secret", "bad") val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } @@ -352,35 +327,30 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val securityManager = new SecurityManager(conf) val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, - conf = conf, securityManager = securityManager) - System.setProperty("spark.hostPort", hostname + ":" + boundPort) + val rpcEnv = RpcEnv.create("spark", hostname, 0, conf, securityManager) + System.setProperty("spark.hostPort", rpcEnv.address.hostPort) assert(securityManager.isAuthenticationEnabled() === false) val masterTracker = new MapOutputTrackerMaster(conf) - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveConf = sparkSSLConfig() val securityManagerBad = new SecurityManager(slaveConf) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, - conf = slaveConf, securityManager = securityManagerBad) + val slaveRpcEnv = RpcEnv.create("spark-slave", hostname, 0, slaveConf, securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) - val selection = slaveSystem.actorSelection( - AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) - val timeout = AkkaUtils.lookupTimeout(conf) - val result = Try(Await.result(selection.resolveOne(timeout * 2), timeout)) - - result match { - case Failure(ex: ActorNotFound) => - case Failure(ex: TimeoutException) => - case r => fail(s"$r is neither Failure(ActorNotFound) nor Failure(TimeoutException)") + try { + slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + fail("should receive either ActorNotFound or TimeoutException") + } catch { + case e: ActorNotFound => + case e: TimeoutException => } - actorSystem.shutdown() - slaveSystem.shutdown() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 054ef54e746a5..446c3f24a74dd 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.util +import java.io.NotSerializableException + import org.scalatest.FunSuite import org.apache.spark.LocalSparkContext._ -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.{TaskContext, SparkContext, SparkException} +import org.apache.spark.partial.CountEvaluator +import org.apache.spark.rdd.RDD class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { @@ -50,24 +54,80 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } - + test("toplevel return statements in closures are identified at cleaning time") { - val ex = intercept[SparkException] { + intercept[ReturnStatementInClosureException] { TestObjectWithBogusReturns.run() } - - assert(ex.getMessage.contains("Return statements aren't allowed in Spark closures")) } test("return statements from named functions nested in closures don't raise exceptions") { val result = TestObjectWithNestedReturns.run() - assert(result == 1) + assert(result === 1) + } + + test("user provided closures are actually cleaned") { + + // We use return statements as an indication that a closure is actually being cleaned + // We expect closure cleaner to find the return statements in the user provided closures + def expectCorrectException(body: => Unit): Unit = { + try { + body + } catch { + case rse: ReturnStatementInClosureException => // Success! + case e @ (_: NotSerializableException | _: SparkException) => + fail(s"Expected ReturnStatementInClosureException, but got $e.\n" + + "This means the closure provided by user is not actually cleaned.") + } + } + + withSpark(new SparkContext("local", "test")) { sc => + val rdd = sc.parallelize(1 to 10) + val pairRdd = rdd.map { i => (i, i) } + expectCorrectException { TestUserClosuresActuallyCleaned.testMap(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testFlatMap(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testFilter(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testSortBy(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testGroupBy(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testKeyBy(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testMapPartitions(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testMapPartitionsWithIndex(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions2(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions3(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions4(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testForeach(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testForeachPartition(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testReduce(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testTreeReduce(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testFold(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testAggregate(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testTreeAggregate(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testCombineByKey(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testAggregateByKey(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testFoldByKey(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testReduceByKey(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testMapValues(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testFlatMapValues(pairRdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testForeachAsync(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testForeachPartitionAsync(rdd) } + expectCorrectException { TestUserClosuresActuallyCleaned.testRunJob1(sc) } + expectCorrectException { TestUserClosuresActuallyCleaned.testRunJob2(sc) } + expectCorrectException { TestUserClosuresActuallyCleaned.testRunApproximateJob(sc) } + expectCorrectException { TestUserClosuresActuallyCleaned.testSubmitJob(sc) } + } } } // A non-serializable class we create in closures to make sure that we aren't // keeping references to unneeded variables from our outer closures. -class NonSerializable {} +class NonSerializable(val id: Int = -1) { + override def equals(other: Any): Boolean = { + other match { + case o: NonSerializable => id == o.id + case _ => false + } + } +} object TestObject { def run(): Int = { @@ -83,7 +143,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -95,7 +155,7 @@ class TestClass extends Serializable { } class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -164,7 +224,7 @@ object TestObjectWithNesting { } class TestClassWithNesting(val y: Int) extends Serializable { - def getY = y + def getY: Int = y def run(): Int = { var nonSer = new NonSerializable @@ -180,3 +240,90 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } + +/** + * Test whether closures passed in through public APIs are actually cleaned. + * + * We put a return statement in each of these closures as a mechanism to detect whether the + * ClosureCleaner actually cleaned our closure. If it did, then it would throw an appropriate + * exception explicitly complaining about the return statement. Otherwise, we know the + * ClosureCleaner did not actually clean our closure, in which case we should fail the test. + */ +private object TestUserClosuresActuallyCleaned { + def testMap(rdd: RDD[Int]): Unit = { rdd.map { _ => return; 0 }.count() } + def testFlatMap(rdd: RDD[Int]): Unit = { rdd.flatMap { _ => return; Seq() }.count() } + def testFilter(rdd: RDD[Int]): Unit = { rdd.filter { _ => return; true }.count() } + def testSortBy(rdd: RDD[Int]): Unit = { rdd.sortBy { _ => return; 1 }.count() } + def testKeyBy(rdd: RDD[Int]): Unit = { rdd.keyBy { _ => return; 1 }.count() } + def testGroupBy(rdd: RDD[Int]): Unit = { rdd.groupBy { _ => return; 1 }.count() } + def testMapPartitions(rdd: RDD[Int]): Unit = { rdd.mapPartitions { it => return; it }.count() } + def testMapPartitionsWithIndex(rdd: RDD[Int]): Unit = { + rdd.mapPartitionsWithIndex { (_, it) => return; it }.count() + } + def testZipPartitions2(rdd: RDD[Int]): Unit = { + rdd.zipPartitions(rdd) { case (it1, it2) => return; it1 }.count() + } + def testZipPartitions3(rdd: RDD[Int]): Unit = { + rdd.zipPartitions(rdd, rdd) { case (it1, it2, it3) => return; it1 }.count() + } + def testZipPartitions4(rdd: RDD[Int]): Unit = { + rdd.zipPartitions(rdd, rdd, rdd) { case (it1, it2, it3, it4) => return; it1 }.count() + } + def testForeach(rdd: RDD[Int]): Unit = { rdd.foreach { _ => return } } + def testForeachPartition(rdd: RDD[Int]): Unit = { rdd.foreachPartition { _ => return } } + def testReduce(rdd: RDD[Int]): Unit = { rdd.reduce { case (_, _) => return; 1 } } + def testTreeReduce(rdd: RDD[Int]): Unit = { rdd.treeReduce { case (_, _) => return; 1 } } + def testFold(rdd: RDD[Int]): Unit = { rdd.fold(0) { case (_, _) => return; 1 } } + def testAggregate(rdd: RDD[Int]): Unit = { + rdd.aggregate(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }) + } + def testTreeAggregate(rdd: RDD[Int]): Unit = { + rdd.treeAggregate(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }) + } + + // Test pair RDD functions + def testCombineByKey(rdd: RDD[(Int, Int)]): Unit = { + rdd.combineByKey( + { _ => return; 1 }: Int => Int, + { case (_, _) => return; 1 }: (Int, Int) => Int, + { case (_, _) => return; 1 }: (Int, Int) => Int + ).count() + } + def testAggregateByKey(rdd: RDD[(Int, Int)]): Unit = { + rdd.aggregateByKey(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }).count() + } + def testFoldByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.foldByKey(0) { case (_, _) => return; 1 } } + def testReduceByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKey { case (_, _) => return; 1 } } + def testMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.mapValues { _ => return; 1 } } + def testFlatMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.flatMapValues { _ => return; Seq() } } + + // Test async RDD actions + def testForeachAsync(rdd: RDD[Int]): Unit = { rdd.foreachAsync { _ => return } } + def testForeachPartitionAsync(rdd: RDD[Int]): Unit = { rdd.foreachPartitionAsync { _ => return } } + + // Test SparkContext runJob + def testRunJob1(sc: SparkContext): Unit = { + val rdd = sc.parallelize(1 to 10, 10) + sc.runJob(rdd, { (ctx: TaskContext, iter: Iterator[Int]) => return; 1 } ) + } + def testRunJob2(sc: SparkContext): Unit = { + val rdd = sc.parallelize(1 to 10, 10) + sc.runJob(rdd, { iter: Iterator[Int] => return; 1 } ) + } + def testRunApproximateJob(sc: SparkContext): Unit = { + val rdd = sc.parallelize(1 to 10, 10) + val evaluator = new CountEvaluator(1, 0.5) + sc.runApproximateJob( + rdd, { (ctx: TaskContext, iter: Iterator[Int]) => return; 1L }, evaluator, 1000) + } + def testSubmitJob(sc: SparkContext): Unit = { + val rdd = sc.parallelize(1 to 10, 10) + sc.submitJob( + rdd, + { _ => return; 1 }: Iterator[Int] => Int, + Seq.empty, + { case (_, _) => return }: (Int, Int) => Unit, + { return } + ) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite2.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite2.scala new file mode 100644 index 0000000000000..59456790e89f0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite2.scala @@ -0,0 +1,571 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.NotSerializableException + +import scala.collection.mutable + +import org.scalatest.{BeforeAndAfterAll, FunSuite, PrivateMethodTester} + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.serializer.SerializerInstance + +/** + * Another test suite for the closure cleaner that is finer-grained. + * For tests involving end-to-end Spark jobs, see {{ClosureCleanerSuite}}. + */ +class ClosureCleanerSuite2 extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + + // Start a SparkContext so that the closure serializer is accessible + // We do not actually use this explicitly otherwise + private var sc: SparkContext = null + private var closureSerializer: SerializerInstance = null + + override def beforeAll(): Unit = { + sc = new SparkContext("local", "test") + closureSerializer = sc.env.closureSerializer.newInstance() + } + + override def afterAll(): Unit = { + sc.stop() + sc = null + closureSerializer = null + } + + // Some fields and methods to reference in inner closures later + private val someSerializableValue = 1 + private val someNonSerializableValue = new NonSerializable + private def someSerializableMethod() = 1 + private def someNonSerializableMethod() = new NonSerializable + + /** Assert that the given closure is serializable (or not). */ + private def assertSerializable(closure: AnyRef, serializable: Boolean): Unit = { + if (serializable) { + closureSerializer.serialize(closure) + } else { + intercept[NotSerializableException] { + closureSerializer.serialize(closure) + } + } + } + + /** + * Helper method for testing whether closure cleaning works as expected. + * This cleans the given closure twice, with and without transitive cleaning. + * + * @param closure closure to test cleaning with + * @param serializableBefore if true, verify that the closure is serializable + * before cleaning, otherwise assert that it is not + * @param serializableAfter if true, assert that the closure is serializable + * after cleaning otherwise assert that it is not + */ + private def verifyCleaning( + closure: AnyRef, + serializableBefore: Boolean, + serializableAfter: Boolean): Unit = { + verifyCleaning(closure, serializableBefore, serializableAfter, transitive = true) + verifyCleaning(closure, serializableBefore, serializableAfter, transitive = false) + } + + /** Helper method for testing whether closure cleaning works as expected. */ + private def verifyCleaning( + closure: AnyRef, + serializableBefore: Boolean, + serializableAfter: Boolean, + transitive: Boolean): Unit = { + assertSerializable(closure, serializableBefore) + // If the resulting closure is not serializable even after + // cleaning, we expect ClosureCleaner to throw a SparkException + if (serializableAfter) { + ClosureCleaner.clean(closure, checkSerializable = true, transitive) + } else { + intercept[SparkException] { + ClosureCleaner.clean(closure, checkSerializable = true, transitive) + } + } + assertSerializable(closure, serializableAfter) + } + + /** + * Return the fields accessed by the given closure by class. + * This also optionally finds the fields transitively referenced through methods invocations. + */ + private def findAccessedFields( + closure: AnyRef, + outerClasses: Seq[Class[_]], + findTransitively: Boolean): Map[Class[_], Set[String]] = { + val fields = new mutable.HashMap[Class[_], mutable.Set[String]] + outerClasses.foreach { c => fields(c) = new mutable.HashSet[String] } + ClosureCleaner.getClassReader(closure.getClass) + .accept(new FieldAccessFinder(fields, findTransitively), 0) + fields.mapValues(_.toSet).toMap + } + + // Accessors for private methods + private val _isClosure = PrivateMethod[Boolean]('isClosure) + private val _getInnerClosureClasses = PrivateMethod[List[Class[_]]]('getInnerClosureClasses) + private val _getOuterClasses = PrivateMethod[List[Class[_]]]('getOuterClasses) + private val _getOuterObjects = PrivateMethod[List[AnyRef]]('getOuterObjects) + + private def isClosure(obj: AnyRef): Boolean = { + ClosureCleaner invokePrivate _isClosure(obj) + } + + private def getInnerClosureClasses(closure: AnyRef): List[Class[_]] = { + ClosureCleaner invokePrivate _getInnerClosureClasses(closure) + } + + private def getOuterClasses(closure: AnyRef): List[Class[_]] = { + ClosureCleaner invokePrivate _getOuterClasses(closure) + } + + private def getOuterObjects(closure: AnyRef): List[AnyRef] = { + ClosureCleaner invokePrivate _getOuterObjects(closure) + } + + test("get inner closure classes") { + val closure1 = () => 1 + val closure2 = () => { () => 1 } + val closure3 = (i: Int) => { + (1 to i).map { x => x + 1 }.filter { x => x > 5 } + } + val closure4 = (j: Int) => { + (1 to j).flatMap { x => + (1 to x).flatMap { y => + (1 to y).map { z => z + 1 } + } + } + } + val inner1 = getInnerClosureClasses(closure1) + val inner2 = getInnerClosureClasses(closure2) + val inner3 = getInnerClosureClasses(closure3) + val inner4 = getInnerClosureClasses(closure4) + assert(inner1.isEmpty) + assert(inner2.size === 1) + assert(inner3.size === 2) + assert(inner4.size === 3) + assert(inner2.forall(isClosure)) + assert(inner3.forall(isClosure)) + assert(inner4.forall(isClosure)) + } + + test("get outer classes and objects") { + val localValue = someSerializableValue + val closure1 = () => 1 + val closure2 = () => localValue + val closure3 = () => someSerializableValue + val closure4 = () => someSerializableMethod() + val outerClasses1 = getOuterClasses(closure1) + val outerClasses2 = getOuterClasses(closure2) + val outerClasses3 = getOuterClasses(closure3) + val outerClasses4 = getOuterClasses(closure4) + val outerObjects1 = getOuterObjects(closure1) + val outerObjects2 = getOuterObjects(closure2) + val outerObjects3 = getOuterObjects(closure3) + val outerObjects4 = getOuterObjects(closure4) + + // The classes and objects should have the same size + assert(outerClasses1.size === outerObjects1.size) + assert(outerClasses2.size === outerObjects2.size) + assert(outerClasses3.size === outerObjects3.size) + assert(outerClasses4.size === outerObjects4.size) + + // These do not have $outer pointers because they reference only local variables + assert(outerClasses1.isEmpty) + assert(outerClasses2.isEmpty) + + // These closures do have $outer pointers because they ultimately reference `this` + // The first $outer pointer refers to the closure defines this test (see FunSuite#test) + // The second $outer pointer refers to ClosureCleanerSuite2 + assert(outerClasses3.size === 2) + assert(outerClasses4.size === 2) + assert(isClosure(outerClasses3(0))) + assert(isClosure(outerClasses4(0))) + assert(outerClasses3(0) === outerClasses4(0)) // part of the same "FunSuite#test" scope + assert(outerClasses3(1) === this.getClass) + assert(outerClasses4(1) === this.getClass) + assert(outerObjects3(1) === this) + assert(outerObjects4(1) === this) + } + + test("get outer classes and objects with nesting") { + val localValue = someSerializableValue + + val test1 = () => { + val x = 1 + val closure1 = () => 1 + val closure2 = () => x + val outerClasses1 = getOuterClasses(closure1) + val outerClasses2 = getOuterClasses(closure2) + val outerObjects1 = getOuterObjects(closure1) + val outerObjects2 = getOuterObjects(closure2) + assert(outerClasses1.size === outerObjects1.size) + assert(outerClasses2.size === outerObjects2.size) + // These inner closures only reference local variables, and so do not have $outer pointers + assert(outerClasses1.isEmpty) + assert(outerClasses2.isEmpty) + } + + val test2 = () => { + def y = 1 + val closure1 = () => 1 + val closure2 = () => y + val closure3 = () => localValue + val outerClasses1 = getOuterClasses(closure1) + val outerClasses2 = getOuterClasses(closure2) + val outerClasses3 = getOuterClasses(closure3) + val outerObjects1 = getOuterObjects(closure1) + val outerObjects2 = getOuterObjects(closure2) + val outerObjects3 = getOuterObjects(closure3) + assert(outerClasses1.size === outerObjects1.size) + assert(outerClasses2.size === outerObjects2.size) + assert(outerClasses3.size === outerObjects3.size) + // Same as above, this closure only references local variables + assert(outerClasses1.isEmpty) + // This closure references the "test2" scope because it needs to find the method `y` + // Scope hierarchy: "test2" < "FunSuite#test" < ClosureCleanerSuite2 + assert(outerClasses2.size === 3) + // This closure references the "test2" scope because it needs to find the `localValue` + // defined outside of this scope + assert(outerClasses3.size === 3) + assert(isClosure(outerClasses2(0))) + assert(isClosure(outerClasses3(0))) + assert(isClosure(outerClasses2(1))) + assert(isClosure(outerClasses3(1))) + assert(outerClasses2(0) === outerClasses3(0)) // part of the same "test2" scope + assert(outerClasses2(1) === outerClasses3(1)) // part of the same "FunSuite#test" scope + assert(outerClasses2(2) === this.getClass) + assert(outerClasses3(2) === this.getClass) + assert(outerObjects2(2) === this) + assert(outerObjects3(2) === this) + } + + test1() + test2() + } + + test("find accessed fields") { + val localValue = someSerializableValue + val closure1 = () => 1 + val closure2 = () => localValue + val closure3 = () => someSerializableValue + val outerClasses1 = getOuterClasses(closure1) + val outerClasses2 = getOuterClasses(closure2) + val outerClasses3 = getOuterClasses(closure3) + + val fields1 = findAccessedFields(closure1, outerClasses1, findTransitively = false) + val fields2 = findAccessedFields(closure2, outerClasses2, findTransitively = false) + val fields3 = findAccessedFields(closure3, outerClasses3, findTransitively = false) + assert(fields1.isEmpty) + assert(fields2.isEmpty) + assert(fields3.size === 2) + // This corresponds to the "FunSuite#test" closure. This is empty because the + // `someSerializableValue` belongs to its parent (i.e. ClosureCleanerSuite2). + assert(fields3(outerClasses3(0)).isEmpty) + // This corresponds to the ClosureCleanerSuite2. This is also empty, however, + // because accessing a `ClosureCleanerSuite2#someSerializableValue` actually involves a + // method call. Since we do not find fields transitively, we will not recursively trace + // through the fields referenced by this method. + assert(fields3(outerClasses3(1)).isEmpty) + + val fields1t = findAccessedFields(closure1, outerClasses1, findTransitively = true) + val fields2t = findAccessedFields(closure2, outerClasses2, findTransitively = true) + val fields3t = findAccessedFields(closure3, outerClasses3, findTransitively = true) + assert(fields1t.isEmpty) + assert(fields2t.isEmpty) + assert(fields3t.size === 2) + // Because we find fields transitively now, we are able to detect that we need the + // $outer pointer to get the field from the ClosureCleanerSuite2 + assert(fields3t(outerClasses3(0)).size === 1) + assert(fields3t(outerClasses3(0)).head === "$outer") + assert(fields3t(outerClasses3(1)).size === 1) + assert(fields3t(outerClasses3(1)).head.contains("someSerializableValue")) + } + + test("find accessed fields with nesting") { + val localValue = someSerializableValue + + val test1 = () => { + def a = localValue + 1 + val closure1 = () => 1 + val closure2 = () => a + val closure3 = () => localValue + val closure4 = () => someSerializableValue + val outerClasses1 = getOuterClasses(closure1) + val outerClasses2 = getOuterClasses(closure2) + val outerClasses3 = getOuterClasses(closure3) + val outerClasses4 = getOuterClasses(closure4) + + // First, find only fields accessed directly, not transitively, by these closures + val fields1 = findAccessedFields(closure1, outerClasses1, findTransitively = false) + val fields2 = findAccessedFields(closure2, outerClasses2, findTransitively = false) + val fields3 = findAccessedFields(closure3, outerClasses3, findTransitively = false) + val fields4 = findAccessedFields(closure4, outerClasses4, findTransitively = false) + assert(fields1.isEmpty) + // Note that the size here represents the number of outer classes, not the number of fields + // "test1" < parameter of "FunSuite#test" < ClosureCleanerSuite2 + assert(fields2.size === 3) + // Since we do not find fields transitively here, we do not look into what `def a` references + assert(fields2(outerClasses2(0)).isEmpty) // This corresponds to the "test1" scope + assert(fields2(outerClasses2(1)).isEmpty) // This corresponds to the "FunSuite#test" scope + assert(fields2(outerClasses2(2)).isEmpty) // This corresponds to the ClosureCleanerSuite2 + assert(fields3.size === 3) + // Note that `localValue` is a field of the "test1" scope because `def a` references it, + // but NOT a field of the "FunSuite#test" scope because it is only a local variable there + assert(fields3(outerClasses3(0)).size === 1) + assert(fields3(outerClasses3(0)).head.contains("localValue")) + assert(fields3(outerClasses3(1)).isEmpty) + assert(fields3(outerClasses3(2)).isEmpty) + assert(fields4.size === 3) + // Because `val someSerializableValue` is an instance variable, even an explicit reference + // here actually involves a method call to access the underlying value of the variable. + // Because we are not finding fields transitively here, we do not consider the fields + // accessed by this "method" (i.e. the val's accessor). + assert(fields4(outerClasses4(0)).isEmpty) + assert(fields4(outerClasses4(1)).isEmpty) + assert(fields4(outerClasses4(2)).isEmpty) + + // Now do the same, but find fields that the closures transitively reference + val fields1t = findAccessedFields(closure1, outerClasses1, findTransitively = true) + val fields2t = findAccessedFields(closure2, outerClasses2, findTransitively = true) + val fields3t = findAccessedFields(closure3, outerClasses3, findTransitively = true) + val fields4t = findAccessedFields(closure4, outerClasses4, findTransitively = true) + assert(fields1t.isEmpty) + assert(fields2t.size === 3) + assert(fields2t(outerClasses2(0)).size === 1) // `def a` references `localValue` + assert(fields2t(outerClasses2(0)).head.contains("localValue")) + assert(fields2t(outerClasses2(1)).isEmpty) + assert(fields2t(outerClasses2(2)).isEmpty) + assert(fields3t.size === 3) + assert(fields3t(outerClasses3(0)).size === 1) // as before + assert(fields3t(outerClasses3(0)).head.contains("localValue")) + assert(fields3t(outerClasses3(1)).isEmpty) + assert(fields3t(outerClasses3(2)).isEmpty) + assert(fields4t.size === 3) + // Through a series of method calls, we are able to detect that we ultimately access + // ClosureCleanerSuite2's field `someSerializableValue`. Along the way, we also accessed + // a few $outer parent pointers to get to the outermost object. + assert(fields4t(outerClasses4(0)) === Set("$outer")) + assert(fields4t(outerClasses4(1)) === Set("$outer")) + assert(fields4t(outerClasses4(2)).size === 1) + assert(fields4t(outerClasses4(2)).head.contains("someSerializableValue")) + } + + test1() + } + + test("clean basic serializable closures") { + val localValue = someSerializableValue + val closure1 = () => 1 + val closure2 = () => Array[String]("a", "b", "c") + val closure3 = (s: String, arr: Array[Long]) => s + arr.mkString(", ") + val closure4 = () => localValue + val closure5 = () => new NonSerializable(5) // we're just serializing the class information + val closure1r = closure1() + val closure2r = closure2() + val closure3r = closure3("g", Array(1, 5, 8)) + val closure4r = closure4() + val closure5r = closure5() + + verifyCleaning(closure1, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure2, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure3, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure4, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure5, serializableBefore = true, serializableAfter = true) + + // Verify that closures can still be invoked and the result still the same + assert(closure1() === closure1r) + assert(closure2() === closure2r) + assert(closure3("g", Array(1, 5, 8)) === closure3r) + assert(closure4() === closure4r) + assert(closure5() === closure5r) + } + + test("clean basic non-serializable closures") { + val closure1 = () => this // ClosureCleanerSuite2 is not serializable + val closure5 = () => someSerializableValue + val closure3 = () => someSerializableMethod() + val closure4 = () => someNonSerializableValue + val closure2 = () => someNonSerializableMethod() + + // These are not cleanable because they ultimately reference the ClosureCleanerSuite2 + verifyCleaning(closure1, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure2, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure3, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure4, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure5, serializableBefore = false, serializableAfter = false) + } + + test("clean basic nested serializable closures") { + val localValue = someSerializableValue + val closure1 = (i: Int) => { + (1 to i).map { x => x + localValue } // 1 level of nesting + } + val closure2 = (j: Int) => { + (1 to j).flatMap { x => + (1 to x).map { y => y + localValue } // 2 levels + } + } + val closure3 = (k: Int, l: Int, m: Int) => { + (1 to k).flatMap(closure2) ++ // 4 levels + (1 to l).flatMap(closure1) ++ // 3 levels + (1 to m).map { x => x + 1 } // 2 levels + } + val closure1r = closure1(1) + val closure2r = closure2(2) + val closure3r = closure3(3, 4, 5) + + verifyCleaning(closure1, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure2, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure3, serializableBefore = true, serializableAfter = true) + + // Verify that closures can still be invoked and the result still the same + assert(closure1(1) === closure1r) + assert(closure2(2) === closure2r) + assert(closure3(3, 4, 5) === closure3r) + } + + test("clean basic nested non-serializable closures") { + def localSerializableMethod(): Int = someSerializableValue + val localNonSerializableValue = someNonSerializableValue + // These closures ultimately reference the ClosureCleanerSuite2 + // Note that even accessing `val` that is an instance variable involves a method call + val closure1 = (i: Int) => { (1 to i).map { x => x + someSerializableValue } } + val closure2 = (j: Int) => { (1 to j).map { x => x + someSerializableMethod() } } + val closure4 = (k: Int) => { (1 to k).map { x => x + localSerializableMethod() } } + // This closure references a local non-serializable value + val closure3 = (l: Int) => { (1 to l).map { x => localNonSerializableValue } } + // This is non-serializable no matter how many levels we nest it + val closure5 = (m: Int) => { + (1 to m).foreach { x => + (1 to x).foreach { y => + (1 to y).foreach { z => + someSerializableValue + } + } + } + } + + verifyCleaning(closure1, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure2, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure3, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure4, serializableBefore = false, serializableAfter = false) + verifyCleaning(closure5, serializableBefore = false, serializableAfter = false) + } + + test("clean complicated nested serializable closures") { + val localValue = someSerializableValue + + // Here we assume that if the outer closure is serializable, + // then all inner closures must also be serializable + + // Reference local fields from all levels + val closure1 = (i: Int) => { + val a = 1 + (1 to i).flatMap { x => + val b = a + 1 + (1 to x).map { y => + y + a + b + localValue + } + } + } + + // Reference local fields and methods from all levels within the outermost closure + val closure2 = (i: Int) => { + val a1 = 1 + def a2 = 2 + (1 to i).flatMap { x => + val b1 = a1 + 1 + def b2 = a2 + 1 + (1 to x).map { y => + // If this references a method outside the outermost closure, then it will try to pull + // in the ClosureCleanerSuite2. This is why `localValue` here must be a local `val`. + y + a1 + a2 + b1 + b2 + localValue + } + } + } + + val closure1r = closure1(1) + val closure2r = closure2(2) + verifyCleaning(closure1, serializableBefore = true, serializableAfter = true) + verifyCleaning(closure2, serializableBefore = true, serializableAfter = true) + assert(closure1(1) == closure1r) + assert(closure2(2) == closure2r) + } + + test("clean complicated nested non-serializable closures") { + val localValue = someSerializableValue + + // Note that we are not interested in cleaning the outer closures here (they are not cleanable) + // The only reason why they exist is to nest the inner closures + + val test1 = () => { + val a = localValue + val b = sc + val inner1 = (x: Int) => x + a + b.hashCode() + val inner2 = (x: Int) => x + a + + // This closure explicitly references a non-serializable field + // There is no way to clean it + verifyCleaning(inner1, serializableBefore = false, serializableAfter = false) + + // This closure is serializable to begin with since it does not need a pointer to + // the outer closure (it only references local variables) + verifyCleaning(inner2, serializableBefore = true, serializableAfter = true) + } + + // Same as above, but the `val a` becomes `def a` + // The difference here is that all inner closures now have pointers to the outer closure + val test2 = () => { + def a = localValue + val b = sc + val inner1 = (x: Int) => x + a + b.hashCode() + val inner2 = (x: Int) => x + a + + // As before, this closure is neither serializable nor cleanable + verifyCleaning(inner1, serializableBefore = false, serializableAfter = false) + + // This closure is no longer serializable because it now has a pointer to the outer closure, + // which is itself not serializable because it has a pointer to the ClosureCleanerSuite2. + // If we do not clean transitively, we will not null out this indirect reference. + verifyCleaning( + inner2, serializableBefore = false, serializableAfter = false, transitive = false) + + // If we clean transitively, we will find that method `a` does not actually reference the + // outer closure's parent (i.e. the ClosureCleanerSuite), so we can additionally null out + // the outer closure's parent pointer. This will make `inner2` serializable. + verifyCleaning( + inner2, serializableBefore = false, serializableAfter = true, transitive = true) + } + + // Same as above, but with more levels of nesting + val test3 = () => { () => test1() } + val test4 = () => { () => test2() } + val test5 = () => { () => { () => test3() } } + val test6 = () => { () => { () => test4() } } + + test1() + test2() + test3()() + test4()() + test5()()() + test6()()() + } + +} diff --git a/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala index 1026cb2aa7cae..47b535206c949 100644 --- a/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala @@ -203,4 +203,76 @@ class EventLoopSuite extends FunSuite with Timeouts { assert(!eventLoop.isActive) } } + + test("EventLoop: stop() in onStart should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onStart(): Unit = { + stop() + } + + override def onReceive(event: Int): Unit = { + } + + override def onError(e: Throwable): Unit = { + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } + + test("EventLoop: stop() in onReceive should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + stop() + } + + override def onError(e: Throwable): Unit = { + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } + + test("EventLoop: stop() in onError should call onStop") { + @volatile var onStopCalled: Boolean = false + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + throw new RuntimeException("Oops") + } + + override def onError(e: Throwable): Unit = { + stop() + } + + override def onStop(): Unit = { + onStopCalled = true + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(5 millis)) { + assert(!eventLoop.isActive) + } + assert(onStopCalled) + } } diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 43b6a405cb68c..c05317534cddf 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -109,7 +109,8 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // verify whether the earliest file has been deleted val rolledOverFiles = allGeneratedFiles.filter { _ != testFile.toString }.toArray.sorted - logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + rolledOverFiles.mkString("\n")) + logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + + rolledOverFiles.mkString("\n")) assert(rolledOverFiles.size > 2) val earliestRolledOverFile = rolledOverFiles.head val existingRolledOverFiles = RollingFileAppender.getSortedRolledOverFiles( @@ -135,7 +136,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream) val appender = FileAppender(testInputStream, testFile, conf) - //assert(appender.getClass === classTag[ExpectedAppender].getClass) + // assert(appender.getClass === classTag[ExpectedAppender].getClass) assert(appender.getClass.getSimpleName === classTag[ExpectedAppender].runtimeClass.getSimpleName) if (appender.isInstanceOf[RollingFileAppender]) { @@ -153,9 +154,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { import RollingFileAppender._ - def rollingStrategy(strategy: String) = Seq(STRATEGY_PROPERTY -> strategy) - def rollingSize(size: String) = Seq(SIZE_PROPERTY -> size) - def rollingInterval(interval: String) = Seq(INTERVAL_PROPERTY -> interval) + def rollingStrategy(strategy: String): Seq[(String, String)] = + Seq(STRATEGY_PROPERTY -> strategy) + def rollingSize(size: String): Seq[(String, String)] = Seq(SIZE_PROPERTY -> size) + def rollingInterval(interval: String): Seq[(String, String)] = + Seq(INTERVAL_PROPERTY -> interval) val msInDay = 24 * 60 * 60 * 1000L val msInHour = 60 * 60 * 1000L diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a2be724254d7c..0c5221d10d79d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.storage._ @@ -74,7 +75,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") + val applicationStart = SparkListenerApplicationStart("The winner of all", Some("appId"), + 42L, "Garfield", Some("appAttempt")) val applicationEnd = SparkListenerApplicationEnd(42L) val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", @@ -161,7 +163,7 @@ class JsonProtocolSuite extends FunSuite { assertEquals(exceptionFailure, JsonProtocol.taskEndReasonFromJson(oldEvent)) } - test("StageInfo backward compatibility") { + test("StageInfo backward compatibility (details, accumulables)") { val info = makeStageInfo(1, 2, 3, 4L, 5L) val newJson = JsonProtocol.stageInfoToJson(info) @@ -274,9 +276,11 @@ class JsonProtocolSuite extends FunSuite { test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. - val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") + // SparkListenerApplicationStart pre-Spark 1.4 does not have "appAttemptId". + val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user", None) val oldEvent = JsonProtocol.applicationStartToJson(applicationStart) .removeField({ _._1 == "App ID" }) + .removeField({ _._1 == "App Attempt ID" }) assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) } @@ -294,7 +298,7 @@ class JsonProtocolSuite extends FunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) val dummyStageInfos = - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = @@ -320,6 +324,25 @@ class JsonProtocolSuite extends FunSuite { assertEquals(expectedJobEnd, JsonProtocol.jobEndFromJson(oldEndEvent)) } + test("RDDInfo backward compatibility (scope, parent IDs)") { + // Prior to Spark 1.4.0, RDDInfo did not have the "Scope" and "Parent IDs" properties + val rddInfo = new RDDInfo( + 1, "one", 100, StorageLevel.NONE, Seq(1, 6, 8), Some(new RDDOperationScope("fable"))) + val oldRddInfoJson = JsonProtocol.rddInfoToJson(rddInfo) + .removeField({ _._1 == "Parent IDs"}) + .removeField({ _._1 == "Scope"}) + val expectedRddInfo = new RDDInfo(1, "one", 100, StorageLevel.NONE, Seq.empty, scope = None) + assertEquals(expectedRddInfo, JsonProtocol.rddInfoFromJson(oldRddInfoJson)) + } + + test("StageInfo backward compatibility (parent IDs)") { + // Prior to Spark 1.4.0, StageInfo did not have the "Parent IDs" property + val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details") + val oldStageInfo = JsonProtocol.stageInfoToJson(stageInfo).removeField({ _._1 == "Parent IDs"}) + val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details") + assertEquals(expectedStageInfo, JsonProtocol.stageInfoFromJson(oldStageInfo)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -642,7 +665,7 @@ class JsonProtocolSuite extends FunSuite { } private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK) + val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK, Seq(1, 4, 7)) r.numCachedPartitions = c r.memSize = d r.diskSize = e @@ -651,7 +674,7 @@ class JsonProtocolSuite extends FunSuite { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details") val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 @@ -744,6 +767,7 @@ class JsonProtocolSuite extends FunSuite { | "Stage Name": "greetings", | "Number of Tasks": 200, | "RDD Info": [], + | "ParentIDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -782,20 +806,22 @@ class JsonProtocolSuite extends FunSuite { | { | "RDD ID": 101, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 201, | "Number of Cached Partitions": 301, | "Memory Size": 401, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 501 | } | ], + | "ParentIDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -969,12 +995,12 @@ class JsonProtocolSuite extends FunSuite { | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": false, | "Replication": 2 | }, | "Memory Size": 0, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 0 | } | } @@ -1052,12 +1078,12 @@ class JsonProtocolSuite extends FunSuite { | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": false, | "Replication": 2 | }, | "Memory Size": 0, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 0 | } | } @@ -1135,12 +1161,12 @@ class JsonProtocolSuite extends FunSuite { | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": false, | "Replication": 2 | }, | "Memory Size": 0, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 0 | } | } @@ -1165,20 +1191,22 @@ class JsonProtocolSuite extends FunSuite { | { | "RDD ID": 1, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 200, | "Number of Cached Partitions": 300, | "Memory Size": 400, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 500 | } | ], + | "Parent IDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -1204,36 +1232,39 @@ class JsonProtocolSuite extends FunSuite { | { | "RDD ID": 2, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 400, | "Number of Cached Partitions": 600, | "Memory Size": 800, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 1000 | }, | { | "RDD ID": 3, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 401, | "Number of Cached Partitions": 601, | "Memory Size": 801, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 1001 | } | ], + | "ParentIDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -1259,52 +1290,56 @@ class JsonProtocolSuite extends FunSuite { | { | "RDD ID": 3, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 600, | "Number of Cached Partitions": 900, | "Memory Size": 1200, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 1500 | }, | { | "RDD ID": 4, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 601, | "Number of Cached Partitions": 901, | "Memory Size": 1201, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 1501 | }, | { | "RDD ID": 5, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 602, | "Number of Cached Partitions": 902, | "Memory Size": 1202, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 1502 | } | ], + | "ParentIDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -1330,68 +1365,73 @@ class JsonProtocolSuite extends FunSuite { | { | "RDD ID": 4, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 800, | "Number of Cached Partitions": 1200, | "Memory Size": 1600, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 2000 | }, | { | "RDD ID": 5, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 801, | "Number of Cached Partitions": 1201, | "Memory Size": 1601, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 2001 | }, | { | "RDD ID": 6, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 802, | "Number of Cached Partitions": 1202, | "Memory Size": 1602, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 2002 | }, | { | "RDD ID": 7, | "Name": "mayor", + | "Parent IDs": [1, 4, 7], | "Storage Level": { | "Use Disk": true, | "Use Memory": true, - | "Use Tachyon": false, + | "Use ExternalBlockStore": false, | "Deserialized": true, | "Replication": 1 | }, | "Number of Partitions": 803, | "Number of Cached Partitions": 1203, | "Memory Size": 1603, - | "Tachyon Size": 0, + | "ExternalBlockStore Size": 0, | "Disk Size": 2003 | } | ], + | "ParentIDs" : [100, 200, 300], | "Details": "details", | "Accumulables": [ | { @@ -1497,8 +1537,10 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerApplicationStart", | "App Name": "The winner of all", + | "App ID": "appId", | "Timestamp": 42, - | "User": "Garfield" + | "User": "Garfield", + | "App Attempt ID": "appAttempt" |} """ diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 72e81f3f1a884..403dcb03bd6e5 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -71,7 +71,7 @@ class NextIteratorSuite extends FunSuite with Matchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - override def getNext() = { + override def getNext(): Int = { if (ints.size == 0) { finished = true 0 diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 7424c2e91d4f2..133a76f28e000 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import scala.collection.mutable.ArrayBuffer + import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite, PrivateMethodTester} class DummyClass1 {} @@ -34,6 +36,15 @@ class DummyClass4(val d: DummyClass3) { val x: Int = 0 } +// dummy class to show class field blocks alignment. +class DummyClass5 extends DummyClass1 { + val x: Boolean = true +} + +class DummyClass6 extends DummyClass5 { + val y: Boolean = true +} + object DummyString { def apply(str: String) : DummyString = new DummyString(str.toArray) } @@ -48,6 +59,7 @@ class SizeEstimatorSuite override def beforeEach() { // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + super.beforeEach() System.setProperty("os.arch", "amd64") System.setProperty("spark.test.useCompressedOops", "true") } @@ -60,6 +72,22 @@ class SizeEstimatorSuite assertResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) } + test("primitive wrapper objects") { + assertResult(16)(SizeEstimator.estimate(new java.lang.Boolean(true))) + assertResult(16)(SizeEstimator.estimate(new java.lang.Byte("1"))) + assertResult(16)(SizeEstimator.estimate(new java.lang.Character('1'))) + assertResult(16)(SizeEstimator.estimate(new java.lang.Short("1"))) + assertResult(16)(SizeEstimator.estimate(new java.lang.Integer(1))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Long(1))) + assertResult(16)(SizeEstimator.estimate(new java.lang.Float(1.0))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Double(1.0d))) + } + + test("class field blocks rounding") { + assertResult(16)(SizeEstimator.estimate(new DummyClass5)) + assertResult(24)(SizeEstimator.estimate(new DummyClass6)) + } + // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { @@ -96,10 +124,28 @@ class SizeEstimatorSuite // Past size 100, our samples 100 elements, but we should still get the right size. assertResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + + val arr = new Array[Char](100000) + assertResult(200016)(SizeEstimator.estimate(arr)) + assertResult(480032)(SizeEstimator.estimate(Array.fill(10000)(new DummyString(arr)))) + + val buf = new ArrayBuffer[DummyString]() + for (i <- 0 until 5000) { + buf.append(new DummyString(new Array[Char](10))) + } + assertResult(340016)(SizeEstimator.estimate(buf.toArray)) + + for (i <- 0 until 5000) { + buf.append(new DummyString(arr)) + } + assertResult(683912)(SizeEstimator.estimate(buf.toArray)) + // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + // 10 pointers plus 8-byte object + assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) + // 100 pointers plus 8-byte object + assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first @@ -135,5 +181,20 @@ class SizeEstimatorSuite assertResult(64)(SizeEstimator.estimate(DummyString("a"))) assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) + + // primitive wrapper classes + assertResult(24)(SizeEstimator.estimate(new java.lang.Boolean(true))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Byte("1"))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Character('1'))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Short("1"))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Integer(1))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Long(1))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Float(1.0))) + assertResult(24)(SizeEstimator.estimate(new java.lang.Double(1.0d))) + } + + test("class field blocks rounding on 64-bit VM without useCompressedOops") { + assertResult(24)(SizeEstimator.estimate(new DummyClass5)) + assertResult(32)(SizeEstimator.estimate(new DummyClass6)) } } diff --git a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala new file mode 100644 index 0000000000000..a3aa3e953fbec --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.util + +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import org.scalatest.FunSuite + +class ThreadUtilsSuite extends FunSuite { + + test("newDaemonSingleThreadExecutor") { + val executor = ThreadUtils.newDaemonSingleThreadExecutor("this-is-a-thread-name") + @volatile var threadName = "" + executor.submit(new Runnable { + override def run(): Unit = { + threadName = Thread.currentThread().getName() + } + }) + executor.shutdown() + executor.awaitTermination(10, TimeUnit.SECONDS) + assert(threadName === "this-is-a-thread-name") + } + + test("newDaemonSingleThreadScheduledExecutor") { + val executor = ThreadUtils.newDaemonSingleThreadScheduledExecutor("this-is-a-thread-name") + try { + val latch = new CountDownLatch(1) + @volatile var threadName = "" + executor.schedule(new Runnable { + override def run(): Unit = { + threadName = Thread.currentThread().getName() + latch.countDown() + } + }, 1, TimeUnit.MILLISECONDS) + latch.await(10, TimeUnit.SECONDS) + assert(threadName === "this-is-a-thread-name") + } finally { + executor.shutdownNow() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index c1c605cdb487b..8b72fe665c214 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -63,7 +63,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } @@ -93,7 +93,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 5d93086082189..651ead6ff1de2 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.util -import scala.util.Random - import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols +import java.util.concurrent.TimeUnit import java.util.Locale +import scala.collection.mutable.ListBuffer +import scala.util.Random + import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.scalatest.FunSuite @@ -32,9 +34,152 @@ import org.scalatest.FunSuite import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.Logging import org.apache.spark.SparkConf -class UtilsSuite extends FunSuite with ResetSystemProperties { +class UtilsSuite extends FunSuite with ResetSystemProperties with Logging { + + test("timeConversion") { + // Test -1 + assert(Utils.timeStringAsSeconds("-1") === -1) + + // Test zero + assert(Utils.timeStringAsSeconds("0") === 0) + + assert(Utils.timeStringAsSeconds("1") === 1) + assert(Utils.timeStringAsSeconds("1s") === 1) + assert(Utils.timeStringAsSeconds("1000ms") === 1) + assert(Utils.timeStringAsSeconds("1000000us") === 1) + assert(Utils.timeStringAsSeconds("1m") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) + + assert(Utils.timeStringAsMs("1") === 1) + assert(Utils.timeStringAsMs("1ms") === 1) + assert(Utils.timeStringAsMs("1000us") === 1) + assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1m") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) + assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) + + // Test invalid strings + intercept[NumberFormatException] { + Utils.timeStringAsMs("600l") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600s") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600ds") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("600s This breaks") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This 123s breaks") + } + } + + test("Test byteString conversion") { + // Test zero + assert(Utils.byteStringAsBytes("0") === 0) + + assert(Utils.byteStringAsGb("1") === 1) + assert(Utils.byteStringAsGb("1g") === 1) + assert(Utils.byteStringAsGb("1023m") === 0) + assert(Utils.byteStringAsGb("1024m") === 1) + assert(Utils.byteStringAsGb("1048575k") === 0) + assert(Utils.byteStringAsGb("1048576k") === 1) + assert(Utils.byteStringAsGb("1k") === 0) + assert(Utils.byteStringAsGb("1t") === ByteUnit.TiB.toGiB(1)) + assert(Utils.byteStringAsGb("1p") === ByteUnit.PiB.toGiB(1)) + + assert(Utils.byteStringAsMb("1") === 1) + assert(Utils.byteStringAsMb("1m") === 1) + assert(Utils.byteStringAsMb("1048575b") === 0) + assert(Utils.byteStringAsMb("1048576b") === 1) + assert(Utils.byteStringAsMb("1023k") === 0) + assert(Utils.byteStringAsMb("1024k") === 1) + assert(Utils.byteStringAsMb("3645k") === 3) + assert(Utils.byteStringAsMb("1024gb") === 1048576) + assert(Utils.byteStringAsMb("1g") === ByteUnit.GiB.toMiB(1)) + assert(Utils.byteStringAsMb("1t") === ByteUnit.TiB.toMiB(1)) + assert(Utils.byteStringAsMb("1p") === ByteUnit.PiB.toMiB(1)) + + assert(Utils.byteStringAsKb("1") === 1) + assert(Utils.byteStringAsKb("1k") === 1) + assert(Utils.byteStringAsKb("1m") === ByteUnit.MiB.toKiB(1)) + assert(Utils.byteStringAsKb("1g") === ByteUnit.GiB.toKiB(1)) + assert(Utils.byteStringAsKb("1t") === ByteUnit.TiB.toKiB(1)) + assert(Utils.byteStringAsKb("1p") === ByteUnit.PiB.toKiB(1)) + + assert(Utils.byteStringAsBytes("1") === 1) + assert(Utils.byteStringAsBytes("1k") === ByteUnit.KiB.toBytes(1)) + assert(Utils.byteStringAsBytes("1m") === ByteUnit.MiB.toBytes(1)) + assert(Utils.byteStringAsBytes("1g") === ByteUnit.GiB.toBytes(1)) + assert(Utils.byteStringAsBytes("1t") === ByteUnit.TiB.toBytes(1)) + assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1)) + + // Overflow handling, 1073741824p exceeds Long.MAX_VALUE if converted straight to Bytes + // This demonstrates that we can have e.g 1024^3 PB without overflowing. + assert(Utils.byteStringAsGb("1073741824p") === ByteUnit.PiB.toGiB(1073741824)) + assert(Utils.byteStringAsMb("1073741824p") === ByteUnit.PiB.toMiB(1073741824)) + + // Run this to confirm it doesn't throw an exception + assert(Utils.byteStringAsBytes("9223372036854775807") === 9223372036854775807L) + assert(ByteUnit.PiB.toPiB(9223372036854775807L) === 9223372036854775807L) + + // Test overflow exception + intercept[IllegalArgumentException] { + // This value exceeds Long.MAX when converted to bytes + Utils.byteStringAsBytes("9223372036854775808") + } + + // Test overflow exception + intercept[IllegalArgumentException] { + // This value exceeds Long.MAX when converted to TB + ByteUnit.PiB.toTiB(9223372036854775807L) + } + + // Test fractional string + intercept[NumberFormatException] { + Utils.byteStringAsMb("0.064") + } + + // Test fractional string + intercept[NumberFormatException] { + Utils.byteStringAsMb("0.064m") + } + + // Test invalid strings + intercept[NumberFormatException] { + Utils.byteStringAsBytes("500ub") + } + + // Test invalid strings + intercept[NumberFormatException] { + Utils.byteStringAsBytes("This breaks 600b") + } + + intercept[NumberFormatException] { + Utils.byteStringAsBytes("This breaks 600") + } + + intercept[NumberFormatException] { + Utils.byteStringAsBytes("600gb This breaks") + } + + intercept[NumberFormatException] { + Utils.byteStringAsBytes("This 123mb breaks") + } + } test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") @@ -106,7 +251,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { val second = 1000 val minute = second * 60 val hour = minute * 60 - def str = Utils.msDurationToString(_) + def str: (Long) => String = Utils.msDurationToString(_) val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator() @@ -199,7 +344,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() - val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + // The parent directory has two child directories + val child1: File = Utils.createTempDir(parent.getCanonicalPath) val child2: File = Utils.createTempDir(parent.getCanonicalPath) val child3: File = Utils.createTempDir(child1.getCanonicalPath) // set the last modified time of child1 to 30 secs old @@ -330,6 +476,15 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { } } + // Test for using the util function to change our log levels. + test("log4j log level change") { + Utils.setLogLevel(org.apache.log4j.Level.ALL) + assert(log.isInfoEnabled()) + Utils.setLogLevel(org.apache.log4j.Level.ERROR) + assert(!log.isInfoEnabled()) + assert(log.isErrorEnabled()) + } + test("deleteRecursively") { val tempDir1 = Utils.createTempDir() assert(tempDir1.exists()) @@ -421,4 +576,18 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { val newFileName = new File(testFileDir, testFileName) assert(newFileName.isFile()) } + + test("shutdown hook manager") { + val manager = new SparkShutdownHookManager() + val output = new ListBuffer[Int]() + + val hook1 = manager.add(1, () => output += 1) + manager.add(3, () => output += 3) + manager.add(2, () => output += 2) + manager.add(4, () => output += 4) + manager.remove(hook1) + + manager.runAll() + assert(output.toList === List(4, 3, 2)) + } } diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala index 794a55d61750b..ce2968728a996 100644 --- a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite @deprecated("suppress compile time deprecation warning", "1.0.0") class VectorSuite extends FunSuite { - def verifyVector(vector: Vector, expectedLength: Int) = { + def verifyVector(vector: Vector, expectedLength: Int): Unit = { assert(vector.length == expectedLength) assert(vector.elements.min > 0.0) assert(vector.elements.max < 1.0) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferSuite.scala new file mode 100644 index 0000000000000..c0c38cd4ac4ad --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ChainedBufferSuite.scala @@ -0,0 +1,143 @@ +/* + * 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.collection + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite +import org.scalatest.Matchers._ + +class ChainedBufferSuite extends FunSuite { + test("write and read at start") { + // write from start of source array + val buffer = new ChainedBuffer(8) + buffer.capacity should be (0) + verifyWriteAndRead(buffer, 0, 0, 0, 4) + buffer.capacity should be (8) + + // write from middle of source array + verifyWriteAndRead(buffer, 0, 5, 0, 4) + buffer.capacity should be (8) + + // read to middle of target array + verifyWriteAndRead(buffer, 0, 0, 5, 4) + buffer.capacity should be (8) + + // write up to border + verifyWriteAndRead(buffer, 0, 0, 0, 8) + buffer.capacity should be (8) + + // expand into second buffer + verifyWriteAndRead(buffer, 0, 0, 0, 12) + buffer.capacity should be (16) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 0, 0, 0, 28) + buffer.capacity should be (32) + } + + test("write and read at middle") { + val buffer = new ChainedBuffer(8) + + // fill to a middle point + verifyWriteAndRead(buffer, 0, 0, 0, 3) + + // write from start of source array + verifyWriteAndRead(buffer, 3, 0, 0, 4) + buffer.capacity should be (8) + + // write from middle of source array + verifyWriteAndRead(buffer, 3, 5, 0, 4) + buffer.capacity should be (8) + + // read to middle of target array + verifyWriteAndRead(buffer, 3, 0, 5, 4) + buffer.capacity should be (8) + + // write up to border + verifyWriteAndRead(buffer, 3, 0, 0, 5) + buffer.capacity should be (8) + + // expand into second buffer + verifyWriteAndRead(buffer, 3, 0, 0, 12) + buffer.capacity should be (16) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 3, 0, 0, 28) + buffer.capacity should be (32) + } + + test("write and read at later buffer") { + val buffer = new ChainedBuffer(8) + + // fill to a middle point + verifyWriteAndRead(buffer, 0, 0, 0, 11) + + // write from start of source array + verifyWriteAndRead(buffer, 11, 0, 0, 4) + buffer.capacity should be (16) + + // write from middle of source array + verifyWriteAndRead(buffer, 11, 5, 0, 4) + buffer.capacity should be (16) + + // read to middle of target array + verifyWriteAndRead(buffer, 11, 0, 5, 4) + buffer.capacity should be (16) + + // write up to border + verifyWriteAndRead(buffer, 11, 0, 0, 5) + buffer.capacity should be (16) + + // expand into second buffer + verifyWriteAndRead(buffer, 11, 0, 0, 12) + buffer.capacity should be (24) + + // expand into multiple buffers + verifyWriteAndRead(buffer, 11, 0, 0, 28) + buffer.capacity should be (40) + } + + + // Used to make sure we're writing different bytes each time + var rangeStart = 0 + + /** + * @param buffer The buffer to write to and read from. + * @param offsetInBuffer The offset to write to in the buffer. + * @param offsetInSource The offset in the array that the bytes are written from. + * @param offsetInTarget The offset in the array to read the bytes into. + * @param length The number of bytes to read and write + */ + def verifyWriteAndRead( + buffer: ChainedBuffer, + offsetInBuffer: Int, + offsetInSource: Int, + offsetInTarget: Int, + length: Int): Unit = { + val source = new Array[Byte](offsetInSource + length) + (rangeStart until rangeStart + length).map(_.toByte).copyToArray(source, offsetInSource) + buffer.write(offsetInBuffer, source, offsetInSource, length) + val target = new Array[Byte](offsetInTarget + length) + buffer.read(offsetInBuffer, target, offsetInTarget, length) + ByteBuffer.wrap(source, offsetInSource, length) should be + (ByteBuffer.wrap(target, offsetInTarget, length)) + + rangeStart += 100 + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 48f79ea651018..dff8f3ddc816f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -185,7 +185,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { // reduceByKey val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) - val result1 = rdd.reduceByKey(_+_).collect() + val result1 = rdd.reduceByKey(_ + _).collect() assert(result1.toSet === Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 72d96798b1141..7a98723bc6472 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -19,19 +19,24 @@ package org.apache.spark.util.collection import scala.collection.mutable.ArrayBuffer -import org.scalatest.{PrivateMethodTester, FunSuite} - -import org.apache.spark._ +import org.scalatest.{FunSuite, PrivateMethodTester} import scala.util.Random +import org.apache.spark._ +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} + class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester { - private def createSparkConf(loadDefaults: Boolean): SparkConf = { + private def createSparkConf(loadDefaults: Boolean, kryo: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) - // Make the Java serializer write a reset instruction (TC_RESET) after each object to test - // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + if (kryo) { + conf.set("spark.serializer", classOf[KryoSerializer].getName) + } else { + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", classOf[JavaSerializer].getName) + } // Ensure that we actually have multiple batches per spill file conf.set("spark.shuffle.spill.batchSize", "10") conf @@ -47,8 +52,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(!sorter.invokePrivate(bypassMergeSort()), "sorter bypassed merge-sort") } - test("empty data stream") { - val conf = new SparkConf(false) + test("empty data stream with kryo ser") { + emptyDataStream(createSparkConf(false, true)) + } + + test("empty data stream with java ser") { + emptyDataStream(createSparkConf(false, false)) + } + + def emptyDataStream(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -81,8 +93,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe sorter4.stop() } - test("few elements per partition") { - val conf = createSparkConf(false) + test("few elements per partition with kryo ser") { + fewElementsPerPartition(createSparkConf(false, true)) + } + + test("few elements per partition with java ser") { + fewElementsPerPartition(createSparkConf(false, false)) + } + + def fewElementsPerPartition(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -123,8 +142,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe sorter4.stop() } - test("empty partitions with spilling") { - val conf = createSparkConf(false) + test("empty partitions with spilling with kryo ser") { + emptyPartitionsWithSpilling(createSparkConf(false, true)) + } + + test("empty partitions with spilling with java ser") { + emptyPartitionsWithSpilling(createSparkConf(false, false)) + } + + def emptyPartitionsWithSpilling(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.spill.initialMemoryThreshold", "512") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") @@ -149,8 +175,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe sorter.stop() } - test("empty partitions with spilling, bypass merge-sort") { - val conf = createSparkConf(false) + test("empty partitions with spilling, bypass merge-sort with kryo ser") { + emptyPartitionerWithSpillingBypassMergeSort(createSparkConf(false, true)) + } + + test("empty partitions with spilling, bypass merge-sort with java ser") { + emptyPartitionerWithSpillingBypassMergeSort(createSparkConf(false, false)) + } + + def emptyPartitionerWithSpillingBypassMergeSort(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.spill.initialMemoryThreshold", "512") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") @@ -174,8 +207,17 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe sorter.stop() } - test("spilling in local cluster") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + test("spilling in local cluster with kryo ser") { + // Load defaults, otherwise SPARK_HOME is not found + testSpillingInLocalCluster(createSparkConf(true, true)) + } + + test("spilling in local cluster with java ser") { + // Load defaults, otherwise SPARK_HOME is not found + testSpillingInLocalCluster(createSparkConf(true, false)) + } + + def testSpillingInLocalCluster(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -245,8 +287,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(resultE === (0 until 100000).map(i => (i/4, i)).toSeq) } - test("spilling in local cluster with many reduce tasks") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + test("spilling in local cluster with many reduce tasks with kryo ser") { + spillingInLocalClusterWithManyReduceTasks(createSparkConf(true, true)) + } + + test("spilling in local cluster with many reduce tasks with java ser") { + spillingInLocalClusterWithManyReduceTasks(createSparkConf(true, false)) + } + + def spillingInLocalClusterWithManyReduceTasks(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local-cluster[2,1,512]", "test", conf) @@ -317,7 +366,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("cleanup of intermediate files in sorter") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true, false) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -328,7 +377,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe val sorter = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(3)), Some(ord), None) assertDidNotBypassMergeSort(sorter) - sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) + sorter.insertAll((0 until 120000).iterator.map(i => (i, i))) assert(diskBlockManager.getAllFiles().length > 0) sorter.stop() assert(diskBlockManager.getAllBlocks().length === 0) @@ -336,15 +385,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe val sorter2 = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(3)), Some(ord), None) assertDidNotBypassMergeSort(sorter2) - sorter2.insertAll((0 until 100000).iterator.map(i => (i, i))) + sorter2.insertAll((0 until 120000).iterator.map(i => (i, i))) assert(diskBlockManager.getAllFiles().length > 0) - assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) + assert(sorter2.iterator.toSet === (0 until 120000).map(i => (i, i)).toSet) sorter2.stop() assert(diskBlockManager.getAllBlocks().length === 0) } test("cleanup of intermediate files in sorter, bypass merge-sort") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true, false) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -367,7 +416,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("cleanup of intermediate files in sorter if there are errors") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true, false) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -379,8 +428,8 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe None, Some(new HashPartitioner(3)), Some(ord), None) assertDidNotBypassMergeSort(sorter) intercept[SparkException] { - sorter.insertAll((0 until 100000).iterator.map(i => { - if (i == 99990) { + sorter.insertAll((0 until 120000).iterator.map(i => { + if (i == 119990) { throw new SparkException("Intentional failure") } (i, i) @@ -392,7 +441,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("cleanup of intermediate files in sorter if there are errors, bypass merge-sort") { - val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true, false) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -414,7 +463,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("cleanup of intermediate files in shuffle") { - val conf = createSparkConf(false) + val conf = createSparkConf(false, false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -429,7 +478,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("cleanup of intermediate files in shuffle with errors") { - val conf = createSparkConf(false) + val conf = createSparkConf(false, false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -450,8 +499,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(diskBlockManager.getAllFiles().length === 2) } - test("no partial aggregation or sorting") { - val conf = createSparkConf(false) + test("no partial aggregation or sorting with kryo ser") { + noPartialAggregationOrSorting(createSparkConf(false, true)) + } + + test("no partial aggregation or sorting with java ser") { + noPartialAggregationOrSorting(createSparkConf(false, false)) + } + + def noPartialAggregationOrSorting(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -465,8 +521,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(results === expected) } - test("partial aggregation without spill") { - val conf = createSparkConf(false) + test("partial aggregation without spill with kryo ser") { + partialAggregationWithoutSpill(createSparkConf(false, true)) + } + + test("partial aggregation without spill with java ser") { + partialAggregationWithoutSpill(createSparkConf(false, false)) + } + + def partialAggregationWithoutSpill(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -481,8 +544,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(results === expected) } - test("partial aggregation with spill, no ordering") { - val conf = createSparkConf(false) + test("partial aggregation with spill, no ordering with kryo ser") { + partialAggregationWIthSpillNoOrdering(createSparkConf(false, true)) + } + + test("partial aggregation with spill, no ordering with java ser") { + partialAggregationWIthSpillNoOrdering(createSparkConf(false, false)) + } + + def partialAggregationWIthSpillNoOrdering(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -497,8 +567,16 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(results === expected) } - test("partial aggregation with spill, with ordering") { - val conf = createSparkConf(false) + test("partial aggregation with spill, with ordering with kryo ser") { + partialAggregationWithSpillWithOrdering(createSparkConf(false, true)) + } + + + test("partial aggregation with spill, with ordering with java ser") { + partialAggregationWithSpillWithOrdering(createSparkConf(false, false)) + } + + def partialAggregationWithSpillWithOrdering(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -506,7 +584,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) val ord = implicitly[Ordering[Int]] val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), Some(ord), None) - sorter.insertAll((0 until 100000).iterator.map(i => (i / 2, i))) + + // avoid combine before spill + sorter.insertAll((0 until 50000).iterator.map(i => (i , 2 * i))) + sorter.insertAll((0 until 50000).iterator.map(i => (i, 2 * i + 1))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet val expected = (0 until 3).map(p => { (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) @@ -514,8 +595,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(results === expected) } - test("sorting without aggregation, no spill") { - val conf = createSparkConf(false) + test("sorting without aggregation, no spill with kryo ser") { + sortingWithoutAggregationNoSpill(createSparkConf(false, true)) + } + + test("sorting without aggregation, no spill with java ser") { + sortingWithoutAggregationNoSpill(createSparkConf(false, false)) + } + + def sortingWithoutAggregationNoSpill(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -531,8 +619,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assert(results === expected) } - test("sorting without aggregation, with spill") { - val conf = createSparkConf(false) + test("sorting without aggregation, with spill with kryo ser") { + sortingWithoutAggregationWithSpill(createSparkConf(false, true)) + } + + test("sorting without aggregation, with spill with java ser") { + sortingWithoutAggregationWithSpill(createSparkConf(false, false)) + } + + def sortingWithoutAggregationWithSpill(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -549,14 +644,14 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("spilling with hash collisions") { - val conf = createSparkConf(true) + val conf = createSparkConf(true, false) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = - buffer1 ++= buffer2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) + : ArrayBuffer[String] = buffer1 ++= buffer2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner _, mergeValue _, mergeCombiners _) @@ -606,7 +701,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("spilling with many hash collisions") { - val conf = createSparkConf(true) + val conf = createSparkConf(true, false) conf.set("spark.shuffle.memoryFraction", "0.0001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -629,18 +724,21 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("spilling with hash collisions using the Int.MaxValue key") { - val conf = createSparkConf(true) + val conf = createSparkConf(true, false) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: Int) = ArrayBuffer[Int](i) - def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + def createCombiner(i: Int): ArrayBuffer[Int] = ArrayBuffer[Int](i) + def mergeValue(buffer: ArrayBuffer[Int], i: Int): ArrayBuffer[Int] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]): ArrayBuffer[Int] = { + buf1 ++= buf2 + } val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) - sorter.insertAll((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + sorter.insertAll( + (1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) val it = sorter.iterator while (it.hasNext) { @@ -650,13 +748,14 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("spilling with null keys and values") { - val conf = createSparkConf(true) + val conf = createSparkConf(true, false) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]) = buf1 ++= buf2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]): ArrayBuffer[String] = + buf1 ++= buf2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) @@ -678,7 +777,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe } test("conditions for bypassing merge-sort") { - val conf = createSparkConf(false) + val conf = createSparkConf(false, false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -711,8 +810,15 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe assertDidNotBypassMergeSort(sorter4) } - test("sort without breaking sorting contracts") { - val conf = createSparkConf(true) + test("sort without breaking sorting contracts with kryo ser") { + sortWithoutBreakingSortingContracts(createSparkConf(true, true)) + } + + test("sort without breaking sorting contracts with java ser") { + sortWithoutBreakingSortingContracts(createSparkConf(true, false)) + } + + def sortWithoutBreakingSortingContracts(conf: SparkConf) { conf.set("spark.shuffle.memoryFraction", "0.01") conf.set("spark.shuffle.manager", "sort") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -720,7 +826,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using wrongOrdering to show integer overflow introduced exception. val rand = new Random(100L) val wrongOrdering = new Ordering[String] { - override def compare(a: String, b: String) = { + override def compare(a: String, b: String): Int = { val h1 = if (a == null) 0 else a.hashCode() val h2 = if (b == null) 0 else b.hashCode() h1 - h2 @@ -742,9 +848,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using aggregation and external spill to make sure ExternalSorter using // partitionKeyComparator. - def createCombiner(i: String) = ArrayBuffer(i) - def mergeValue(c: ArrayBuffer[String], i: String) = c += i - def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]) = c1 ++= c2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer(i) + def mergeValue(c: ArrayBuffer[String], i: String): ArrayBuffer[String] = c += i + def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]): ArrayBuffer[String] = + c1 ++= c2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 6a70877356409..ef890d2ba60f3 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -176,4 +176,14 @@ class OpenHashMapSuite extends FunSuite with Matchers { assert(map(i.toString) === i.toString) } } + + test("contains") { + val map = new OpenHashMap[String, Int](2) + map("a") = 1 + assert(map.contains("a")) + assert(!map.contains("b")) + assert(!map.contains(null)) + map(null) = 0 + assert(map.contains(null)) + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala new file mode 100644 index 0000000000000..b5a2d9ef720c1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala @@ -0,0 +1,149 @@ +/* + * 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.collection + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream} + +import com.google.common.io.ByteStreams + +import org.scalatest.FunSuite +import org.scalatest.Matchers._ + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.storage.{FileSegment, BlockObjectWriter} + +class PartitionedSerializedPairBufferSuite extends FunSuite { + test("OrderedInputStream single record") { + val serializerInstance = new KryoSerializer(new SparkConf()).newInstance + + val buffer = new PartitionedSerializedPairBuffer[Int, SomeStruct](4, 32, serializerInstance) + val struct = SomeStruct("something", 5) + buffer.insert(4, 10, struct) + + val bytes = ByteStreams.toByteArray(buffer.orderedInputStream) + + val baos = new ByteArrayOutputStream() + val stream = serializerInstance.serializeStream(baos) + stream.writeObject(10) + stream.writeObject(struct) + stream.close() + + baos.toByteArray should be (bytes) + } + + test("insert single record") { + val serializerInstance = new KryoSerializer(new SparkConf()).newInstance + val buffer = new PartitionedSerializedPairBuffer[Int, SomeStruct](4, 32, serializerInstance) + val struct = SomeStruct("something", 5) + buffer.insert(4, 10, struct) + val elements = buffer.partitionedDestructiveSortedIterator(None).toArray + elements.size should be (1) + elements.head should be (((4, 10), struct)) + } + + test("insert multiple records") { + val serializerInstance = new KryoSerializer(new SparkConf()).newInstance + val buffer = new PartitionedSerializedPairBuffer[Int, SomeStruct](4, 32, serializerInstance) + val struct1 = SomeStruct("something1", 8) + buffer.insert(6, 1, struct1) + val struct2 = SomeStruct("something2", 9) + buffer.insert(4, 2, struct2) + val struct3 = SomeStruct("something3", 10) + buffer.insert(5, 3, struct3) + + val elements = buffer.partitionedDestructiveSortedIterator(None).toArray + elements.size should be (3) + elements(0) should be (((4, 2), struct2)) + elements(1) should be (((5, 3), struct3)) + elements(2) should be (((6, 1), struct1)) + } + + test("write single record") { + val serializerInstance = new KryoSerializer(new SparkConf()).newInstance + val buffer = new PartitionedSerializedPairBuffer[Int, SomeStruct](4, 32, serializerInstance) + val struct = SomeStruct("something", 5) + buffer.insert(4, 10, struct) + val it = buffer.destructiveSortedWritablePartitionedIterator(None) + val writer = new SimpleBlockObjectWriter + assert(it.hasNext) + it.nextPartition should be (4) + it.writeNext(writer) + assert(!it.hasNext) + + val stream = serializerInstance.deserializeStream(writer.getInputStream) + stream.readObject[AnyRef]() should be (10) + stream.readObject[AnyRef]() should be (struct) + } + + test("write multiple records") { + val serializerInstance = new KryoSerializer(new SparkConf()).newInstance + val buffer = new PartitionedSerializedPairBuffer[Int, SomeStruct](4, 32, serializerInstance) + val struct1 = SomeStruct("something1", 8) + buffer.insert(6, 1, struct1) + val struct2 = SomeStruct("something2", 9) + buffer.insert(4, 2, struct2) + val struct3 = SomeStruct("something3", 10) + buffer.insert(5, 3, struct3) + + val it = buffer.destructiveSortedWritablePartitionedIterator(None) + val writer = new SimpleBlockObjectWriter + assert(it.hasNext) + it.nextPartition should be (4) + it.writeNext(writer) + assert(it.hasNext) + it.nextPartition should be (5) + it.writeNext(writer) + assert(it.hasNext) + it.nextPartition should be (6) + it.writeNext(writer) + assert(!it.hasNext) + + val stream = serializerInstance.deserializeStream(writer.getInputStream) + val iter = stream.asIterator + iter.next() should be (2) + iter.next() should be (struct2) + iter.next() should be (3) + iter.next() should be (struct3) + iter.next() should be (1) + iter.next() should be (struct1) + assert(!iter.hasNext) + } +} + +case class SomeStruct(val str: String, val num: Int) + +class SimpleBlockObjectWriter extends BlockObjectWriter(null) { + val baos = new ByteArrayOutputStream() + + override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { + baos.write(bytes, offs, len) + } + + def getInputStream(): InputStream = new ByteArrayInputStream(baos.toByteArray) + + override def open(): BlockObjectWriter = this + override def close(): Unit = { } + override def isOpen: Boolean = true + override def commitAndClose(): Unit = { } + override def revertPartialWritesAndClose(): Unit = { } + override def fileSegment(): FileSegment = null + override def write(key: Any, value: Any): Unit = { } + override def recordWritten(): Unit = { } + override def write(b: Int): Unit = { } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index 8c7df7d73dcd3..caf378fec8b3e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -118,4 +118,11 @@ class PrimitiveKeyOpenHashMapSuite extends FunSuite with Matchers { assert(map(i.toLong) === i.toString) } } + + test("contains") { + val map = new PrimitiveKeyOpenHashMap[Int, Int](1) + map(0) = 0 + assert(map.contains(0)) + assert(!map.contains(1)) + } } diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index ef7178bcdf5c2..03f5f2d1b8528 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -28,7 +28,7 @@ import scala.language.reflectiveCalls class XORShiftRandomSuite extends FunSuite with Matchers { - def fixture = new { + def fixture: Object {val seed: Long; val hundMil: Int; val xorRand: XORShiftRandom} = new { val seed = 1L val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt diff --git a/dev/.gitignore b/dev/.gitignore new file mode 100644 index 0000000000000..4a6027429e0d3 --- /dev/null +++ b/dev/.gitignore @@ -0,0 +1 @@ +pep8*.py 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 d888de929fdda..cc86ef45858c9 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 @@ -36,8 +36,10 @@ object SparkSqlExample { val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ import sqlContext._ - val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF() people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh index 15e0c73b4295e..c4adb1f96b7d3 100755 --- a/dev/change-version-to-2.10.sh +++ b/dev/change-version-to-2.10.sh @@ -18,9 +18,9 @@ # # Note that this will not necessarily work as intended with non-GNU sed (e.g. OS X) - -find . -name 'pom.xml' | grep -v target \ +BASEDIR=$(dirname $0)/.. +find $BASEDIR -name 'pom.xml' | grep -v target \ | xargs -I {} sed -i -e 's/\(artifactId.*\)_2.11/\1_2.10/g' {} # Also update in parent POM -sed -i -e '0,/2.112.102.112.10 in parent POM -sed -i -e '0,/2.102.112.102.11 "$JAR_DL" && mv "$JAR_DL" "$JAR" - elif [ $(command -v wget) ]; then - wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" - else - printf "You do not have curl or wget installed, please install rat manually.\n" - exit -1 - fi - fi - - unzip -tq $JAR &> /dev/null - if [ $? -ne 0 ]; then - # We failed to download - printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + # Download rat launch jar if it hasn't been downloaded yet + if [ ! -f "$JAR" ]; then + # Download + printf "Attempting to fetch rat\n" + JAR_DL="${JAR}.part" + if [ $(command -v curl) ]; then + curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" + elif [ $(command -v wget) ]; then + wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" + else + printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi - printf "Launching rat from ${JAR}\n" + fi + + unzip -tq "$JAR" &> /dev/null + if [ $? -ne 0 ]; then + # We failed to download + rm "$JAR" + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 fi } @@ -71,6 +69,11 @@ mkdir -p "$FWDIR"/lib $java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt +if [ $? -ne 0 ]; then + echo "RAT exited abnormally" + exit 1 +fi + ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b5a67dd783b93..3dbb35f7054a2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -119,7 +119,7 @@ if [[ ! "$@" =~ --skip-publish ]]; then rm -rf $SPARK_REPO build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.11.sh diff --git a/dev/lint-python b/dev/lint-python index fded654893a7c..f50d149dc4d44 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -32,18 +32,19 @@ compile_status="${PIPESTATUS[0]}" #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 #+ TODOs: #+ - Download pep8 from PyPI. It's more "official". -PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" -PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.6.2/pep8.py" +PEP8_VERSION="1.6.2" +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8-$PEP8_VERSION.py" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/$PEP8_VERSION/pep8.py" -# if [ ! -e "$PEP8_SCRIPT_PATH" ]; then -curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" -curl_status="$?" +if [ ! -e "$PEP8_SCRIPT_PATH" ]; then + curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" + curl_status="$?" -if [ "$curl_status" -ne 0 ]; then - echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." - exit "$curl_status" + if [ "$curl_status" -ne 0 ]; then + echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." + exit "$curl_status" + fi fi -# fi # There is no need to write this output to a file #+ first, but we do so so that the check status can @@ -65,7 +66,7 @@ else echo "Python lint checks passed." fi -rm "$PEP8_SCRIPT_PATH" +# rm "$PEP8_SCRIPT_PATH" rm "$PYTHON_LINT_REPORT_PATH" exit "$lint_status" diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 3062e9c3c6651..b69cd15f99f63 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -55,8 +55,6 @@ # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" -os.chdir(SPARK_HOME) - def get_json(url): try: @@ -85,10 +83,6 @@ def continue_maybe(prompt): if result.lower() != "y": fail("Okay, exiting") - -original_head = run_cmd("git rev-parse HEAD")[:8] - - def clean_up(): print "Restoring head pointer to %s" % original_head run_cmd("git checkout %s" % original_head) @@ -101,7 +95,7 @@ def clean_up(): # merge the requested PR and return the merge hash -def merge_pr(pr_num, target_ref): +def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): pr_branch_name = "%s_MERGE_PR_%s" % (BRANCH_PREFIX, pr_num) target_branch_name = "%s_MERGE_PR_%s_%s" % (BRANCH_PREFIX, pr_num, target_ref.upper()) run_cmd("git fetch %s pull/%s/head:%s" % (PR_REMOTE_NAME, pr_num, pr_branch_name)) @@ -274,7 +268,7 @@ def get_version_json(version_str): asf_jira.transition_issue( jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) - print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) def resolve_jira_issues(title, merge_branches, comment): @@ -286,68 +280,155 @@ def resolve_jira_issues(title, merge_branches, comment): resolve_jira_issue(merge_branches, comment, jira_id) -branches = get_json("%s/branches" % GITHUB_API_BASE) -branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) -# Assumes branch names can be sorted lexicographically -latest_branch = sorted(branch_names, reverse=True)[0] - -pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) -pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) +def standardize_jira_ref(text): + """ + Standardize the [SPARK-XXXXX] [MODULE] prefix + Converts "[SPARK-XXX][mllib] Issue", "[MLLib] SPARK-XXX. Issue" or "SPARK XXX [MLLIB]: Issue" to "[SPARK-XXX] [MLLIB] Issue" + + >>> standardize_jira_ref("[SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful") + '[SPARK-5821] [SQL] ParquetRelation2 CTAS should check if delete is successful' + >>> standardize_jira_ref("[SPARK-4123][Project Infra][WIP]: Show new dependencies added in pull requests") + '[SPARK-4123] [PROJECT INFRA] [WIP] Show new dependencies added in pull requests' + >>> standardize_jira_ref("[MLlib] Spark 5954: Top by key") + '[SPARK-5954] [MLLIB] Top by key' + >>> standardize_jira_ref("[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl") + '[SPARK-979] a LRU scheduler for load balancing in TaskSchedulerImpl' + >>> standardize_jira_ref("SPARK-1094 Support MiMa for reporting binary compatibility accross versions.") + '[SPARK-1094] Support MiMa for reporting binary compatibility accross versions.' + >>> standardize_jira_ref("[WIP] [SPARK-1146] Vagrant support for Spark") + '[SPARK-1146] [WIP] Vagrant support for Spark' + >>> standardize_jira_ref("SPARK-1032. If Yarn app fails before registering, app master stays aroun...") + '[SPARK-1032] If Yarn app fails before registering, app master stays aroun...' + >>> standardize_jira_ref("[SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.") + '[SPARK-6250] [SPARK-6146] [SPARK-5911] [SQL] Types are now reserved words in DDL parser.' + >>> standardize_jira_ref("Additional information for users building from source code") + 'Additional information for users building from source code' + """ + jira_refs = [] + components = [] + + # If the string is compliant, no need to process any further + if (re.search(r'^\[SPARK-[0-9]{3,6}\] (\[[A-Z0-9_\s,]+\] )+\S+', text)): + return text + + # Extract JIRA ref(s): + pattern = re.compile(r'(SPARK[-\s]*[0-9]{3,6})+', re.IGNORECASE) + for ref in pattern.findall(text): + # Add brackets, replace spaces with a dash, & convert to uppercase + jira_refs.append('[' + re.sub(r'\s+', '-', ref.upper()) + ']') + text = text.replace(ref, '') + + # Extract spark component(s): + # Look for alphanumeric chars, spaces, dashes, periods, and/or commas + pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + for component in pattern.findall(text): + components.append(component.upper()) + text = text.replace(component, '') + + # Cleanup any remaining symbols: + pattern = re.compile(r'^\W+(.*)', re.IGNORECASE) + if (pattern.search(text) is not None): + text = pattern.search(text).groups()[0] + + # Assemble full text (JIRA ref(s), module(s), remaining text) + clean_text = ' '.join(jira_refs).strip() + " " + ' '.join(components).strip() + " " + text.strip() + + # Replace multiple spaces with a single space, e.g. if no jira refs and/or components were included + clean_text = re.sub(r'\s+', ' ', clean_text.strip()) + + return clean_text + +def main(): + global original_head + + os.chdir(SPARK_HOME) + original_head = run_cmd("git rev-parse HEAD")[:8] + + branches = get_json("%s/branches" % GITHUB_API_BASE) + branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) + # Assumes branch names can be sorted lexicographically + latest_branch = sorted(branch_names, reverse=True)[0] + + pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") + pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) + pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) + + url = pr["url"] + + # Decide whether to use the modified title or not + modified_title = standardize_jira_ref(pr["title"]) + if modified_title != pr["title"]: + print "I've re-written the title as follows to match the standard format:" + print "Original: %s" % pr["title"] + print "Modified: %s" % modified_title + result = raw_input("Would you like to use the modified title? (y/n): ") + if result.lower() == "y": + title = modified_title + print "Using modified title:" + else: + title = pr["title"] + print "Using original title:" + print title + else: + title = pr["title"] -url = pr["url"] -title = pr["title"] -body = pr["body"] -target_ref = pr["base"]["ref"] -user_login = pr["user"]["login"] -base_ref = pr["head"]["ref"] -pr_repo_desc = "%s/%s" % (user_login, base_ref) + body = pr["body"] + target_ref = pr["base"]["ref"] + user_login = pr["user"]["login"] + base_ref = pr["head"]["ref"] + pr_repo_desc = "%s/%s" % (user_login, base_ref) -# Merged pull requests don't appear as merged in the GitHub API; -# Instead, they're closed by asfgit. -merge_commits = \ - [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + # Merged pull requests don't appear as merged in the GitHub API; + # Instead, they're closed by asfgit. + merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] -if merge_commits: - merge_hash = merge_commits[0]["commit_id"] - message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] - print "Pull request %s has already been merged, assuming you want to backport" % pr_num - commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + print "Pull request %s has already been merged, assuming you want to backport" % pr_num + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', "%s^{commit}" % merge_hash]).strip() != "" - if not commit_is_downloaded: - fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) + if not commit_is_downloaded: + fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - print "Found commit %s:\n%s" % (merge_hash, message) - cherry_pick(pr_num, merge_hash, latest_branch) - sys.exit(0) + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) + sys.exit(0) -if not bool(pr["mergeable"]): - msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ - "Continue? (experts only!)" - continue_maybe(msg) + if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) -print ("\n=== Pull Request #%s ===" % pr_num) -print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( - title, pr_repo_desc, target_ref, url)) -continue_maybe("Proceed with merging pull request #%s?" % pr_num) + print ("\n=== Pull Request #%s ===" % pr_num) + print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( + title, pr_repo_desc, target_ref, url)) + continue_maybe("Proceed with merging pull request #%s?" % pr_num) -merged_refs = [target_ref] + merged_refs = [target_ref] -merge_hash = merge_pr(pr_num, target_ref) + merge_hash = merge_pr(pr_num, target_ref, title, body, pr_repo_desc) -pick_prompt = "Would you like to pick %s into another branch?" % merge_hash -while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": - merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + pick_prompt = "Would you like to pick %s into another branch?" % merge_hash + while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] -if JIRA_IMPORTED: - if JIRA_USERNAME and JIRA_PASSWORD: - continue_maybe("Would you like to update an associated JIRA?") - jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira_issues(title, merged_refs, jira_comment) + if JIRA_IMPORTED: + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira_issues(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." else: - print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." print "Exiting without trying to close the associated JIRA." -else: - print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." - print "Exiting without trying to close the associated JIRA." + +if __name__ == "__main__": + import doctest + doctest.testmod() + + main() diff --git a/dev/mima b/dev/mima index bed5cd042634e..2952fa65d42ff 100755 --- a/dev/mima +++ b/dev/mima @@ -27,16 +27,21 @@ cd "$FWDIR" echo -e "q\n" | build/sbt oldDeps/update rm -f .generated-mima* +generate_mima_ignore() { + SPARK_JAVA_OPTS="-XX:MaxPermSize=1g -Xmx2g" \ + ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +} + # Generate Mima Ignore is called twice, first with latest built jars # on the classpath and then again with previous version jars on the classpath. # Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath # it did not process the new classes (which are in assembly jar). -./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +generate_mima_ignore export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" -./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +generate_mima_ignore echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? diff --git a/dev/run-tests b/dev/run-tests index d6935a61c6d29..05c63bce4d40d 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -173,11 +173,20 @@ CURRENT_BLOCK=$BLOCK_BUILD build/mvn $HIVE_BUILD_ARGS clean package -DskipTests else echo -e "q\n" \ - | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \ + | build/sbt $HIVE_BUILD_ARGS package assembly/assembly streaming-kafka-assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" fi } +echo "" +echo "=========================================================================" +echo "Detecting binary incompatibilities with MiMa" +echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_MIMA + +./dev/mima + echo "" echo "=========================================================================" echo "Running Spark unit tests" @@ -226,13 +235,21 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS +# add path for python 3 in jenkins +export PATH="${PATH}:/home/anaconda/envs/py3k/bin" ./python/run-tests echo "" echo "=========================================================================" -echo "Detecting binary incompatibilities with MiMa" +echo "Running SparkR tests" echo "=========================================================================" -CURRENT_BLOCK=$BLOCK_MIMA +CURRENT_BLOCK=$BLOCK_SPARKR_UNIT_TESTS + +if [ $(command -v R) ]; then + ./R/install-dev.sh + ./R/run-tests.sh +else + echo "Ignoring SparkR tests as R was not found in PATH" +fi -./dev/mima diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh index 1348e0609dda4..154e01255b2ef 100644 --- a/dev/run-tests-codes.sh +++ b/dev/run-tests-codes.sh @@ -22,6 +22,7 @@ readonly BLOCK_RAT=11 readonly BLOCK_SCALA_STYLE=12 readonly BLOCK_PYTHON_STYLE=13 readonly BLOCK_BUILD=14 -readonly BLOCK_SPARK_UNIT_TESTS=15 -readonly BLOCK_PYSPARK_UNIT_TESTS=16 -readonly BLOCK_MIMA=17 +readonly BLOCK_MIMA=15 +readonly BLOCK_SPARK_UNIT_TESTS=16 +readonly BLOCK_PYSPARK_UNIT_TESTS=17 +readonly BLOCK_SPARKR_UNIT_TESTS=18 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 5f4000e83925c..f452ab66efcd8 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -47,7 +47,7 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" -TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +TESTS_TIMEOUT="150m" # format: http://linux.die.net/man/1/timeout # Array to capture all tests to run on the pull request. These tests are held under the #+ dev/tests/ directory. @@ -55,13 +55,14 @@ TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout # To write a PR test: #+ * the file must reside within the dev/tests directory #+ * be an executable bash script -#+ * accept two arguments on the command line, the first being the Github PR long commit -#+ hash and the second the Github SHA1 hash +#+ * accept three arguments on the command line, the first being the Github PR long commit +#+ hash, the second the Github SHA1 hash, and the final the current PR hash #+ * and, lastly, return string output to be included in the pr message output that will #+ be posted to Github PR_TESTS=( "pr_merge_ability" "pr_public_classes" +# DISABLED (pwendell) "pr_new_dependencies" ) function post_message () { @@ -146,34 +147,42 @@ function send_archived_logs () { fi } +# post start message +{ + start_message="\ + [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." + + post_message "$start_message" +} + # Environment variable to capture PR test output pr_message="" +# Ensure we save off the current HEAD to revert to +current_pr_head="`git rev-parse HEAD`" + +echo "HEAD: `git rev-parse HEAD`" +echo "GHPRB: $ghprbActualCommit" +echo "SHA1: $sha1" # Run pull request tests for t in "${PR_TESTS[@]}"; do this_test="${FWDIR}/dev/tests/${t}.sh" - # Ensure the test is a file and is executable - if [ -x "$this_test" ]; then - echo "ghprb: $ghprbActualCommit sha1: $sha1" - this_mssg="`bash \"${this_test}\" \"${ghprbActualCommit}\" \"${sha1}\" 2>/dev/null`" + # Ensure the test can be found and is a file + if [ -f "${this_test}" ]; then + echo "Running test: $t" + this_mssg="$(bash "${this_test}" "${ghprbActualCommit}" "${sha1}" "${current_pr_head}")" # Check if this is the merge test as we submit that note *before* and *after* # the tests run [ "$t" == "pr_merge_ability" ] && merge_note="${this_mssg}" pr_message="${pr_message}\n${this_mssg}" + # Ensure, after each test, that we're back on the current PR + git checkout -f "${current_pr_head}" &>/dev/null + else + echo "Cannot find test ${this_test}." fi done -# post start message -{ - start_message="\ - [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ - PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." - - start_message="${start_message}\n${merge_note}" - - post_message "$start_message" -} - # run tests { timeout "${TESTS_TIMEOUT}" ./dev/run-tests @@ -199,12 +208,14 @@ done failing_test="Python style tests" elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then failing_test="to build" + elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then + failing_test="MiMa tests" elif [ "$test_result" -eq "$BLOCK_SPARK_UNIT_TESTS" ]; then failing_test="Spark unit tests" elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then failing_test="PySpark unit tests" - elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then - failing_test="MiMa tests" + elif [ "$test_result" -eq "$BLOCK_SPARKR_UNIT_TESTS" ]; then + failing_test="SparkR unit tests" else failing_test="some tests" fi @@ -222,7 +233,7 @@ done PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" - result_message="${result_message}\n${pr_message}" + result_message="${result_message}${pr_message}" post_message "$result_message" } diff --git a/dev/scalastyle b/dev/scalastyle index 86919227ed1ab..4e03f89ed5d5d 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,9 +18,10 @@ # echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver test:scalastyle >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ - >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 test:scalastyle >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') rm scalastyle.txt diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh new file mode 100755 index 0000000000000..fdfb3c62aff58 --- /dev/null +++ b/dev/tests/pr_new_dependencies.sh @@ -0,0 +1,117 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# Arg3: Current PR Commit Hash +#+ the PR hash for the current commit +# + +ghprbActualCommit="$1" +sha1="$2" +current_pr_head="$3" + +MVN_BIN="build/mvn" +CURR_CP_FILE="my-classpath.txt" +MASTER_CP_FILE="master-classpath.txt" + +# First switch over to the master branch +git checkout -f master +# Find and copy all pom.xml files into a *.gate file that we can check +# against through various `git` changes +find -name "pom.xml" -exec cp {} {}.gate \; +# Switch back to the current PR +git checkout -f "${current_pr_head}" + +# Check if any *.pom files from the current branch are different from the master +difference_q="" +for p in $(find -name "pom.xml"); do + [[ -f "${p}" && -f "${p}.gate" ]] && \ + difference_q="${difference_q}$(diff $p.gate $p)" +done + +# If no pom files were changed we can easily say no new dependencies were added +if [ -z "${difference_q}" ]; then + echo " * This patch does not change any dependencies." +else + # Else we need to manually build spark to determine what, if any, dependencies + # were added into the Spark assembly jar + ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ + sed -n -e '/Building Spark Project Assembly/,$p' | \ + grep --context=1 -m 2 "Dependencies classpath:" | \ + head -n 3 | \ + tail -n 1 | \ + tr ":" "\n" | \ + rev | \ + cut -d "/" -f 1 | \ + rev | \ + sort > ${CURR_CP_FILE} + + # Checkout the master branch to compare against + git checkout -f master + + ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ + sed -n -e '/Building Spark Project Assembly/,$p' | \ + grep --context=1 -m 2 "Dependencies classpath:" | \ + head -n 3 | \ + tail -n 1 | \ + tr ":" "\n" | \ + rev | \ + cut -d "/" -f 1 | \ + rev | \ + sort > ${MASTER_CP_FILE} + + DIFF_RESULTS="`diff ${CURR_CP_FILE} ${MASTER_CP_FILE}`" + + if [ -z "${DIFF_RESULTS}" ]; then + echo " * This patch does not change any dependencies." + else + # Pretty print the new dependencies + added_deps=$(echo "${DIFF_RESULTS}" | grep "<" | cut -d' ' -f2 | awk '{printf " * \`"$1"\`\\n"}') + removed_deps=$(echo "${DIFF_RESULTS}" | grep ">" | cut -d' ' -f2 | awk '{printf " * \`"$1"\`\\n"}') + added_deps_text=" * This patch **adds the following new dependencies:**\n${added_deps}" + removed_deps_text=" * This patch **removes the following dependencies:**\n${removed_deps}" + + # Construct the final returned message with proper + return_mssg="" + [ -n "${added_deps}" ] && return_mssg="${added_deps_text}" + if [ -n "${removed_deps}" ]; then + if [ -n "${return_mssg}" ]; then + return_mssg="${return_mssg}\n${removed_deps_text}" + else + return_mssg="${removed_deps_text}" + fi + fi + echo "${return_mssg}" + fi + + # Remove the files we've left over + [ -f "${CURR_CP_FILE}" ] && rm -f "${CURR_CP_FILE}" + [ -f "${MASTER_CP_FILE}" ] && rm -f "${MASTER_CP_FILE}" + + # Clean up our mess from the Maven builds just in case + ${MVN_BIN} clean &>/dev/null +fi diff --git a/docker/spark-mesos/Dockerfile b/docker/spark-mesos/Dockerfile new file mode 100644 index 0000000000000..b90aef3655dee --- /dev/null +++ b/docker/spark-mesos/Dockerfile @@ -0,0 +1,30 @@ +# This is an example Dockerfile for creating a Spark image which can be +# references by the Spark property 'spark.mesos.executor.docker.image' +# +# 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 mesosphere/mesos:0.20.1 + +# Update the base ubuntu image with dependencies needed for Spark +RUN apt-get update && \ + apt-get install -y python libnss3 openjdk-7-jre-headless curl + +RUN mkdir /opt/spark && \ + curl http://www.apache.org/dyn/closer.cgi/spark/spark-1.4.0/spark-1.4.0-bin-hadoop2.4.tgz \ + | tar -xzC /opt +ENV SPARK_HOME /opt/spark +ENV MESOS_NATIVE_JAVA_LIBRARY /usr/local/lib/libmesos.so diff --git a/docs/README.md b/docs/README.md index 8a54724c4beae..5852f972a051d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -58,19 +58,25 @@ phase, use the following sytax: We use Sphinx to generate Python API docs, so you will need to install it by running `sudo pip install sphinx`. -## API Docs (Scaladoc and Sphinx) +## knitr, devtools -You can build just the Spark scaladoc by running `build/sbt doc` from the SPARK_PROJECT_ROOT directory. +SparkR documentation is written using `roxygen2` and we use `knitr`, `devtools` to generate +documentation. To install these packages you can run `install.packages(c("knitr", "devtools"))` from a +R console. + +## API Docs (Scaladoc, Sphinx, roxygen2) + +You can build just the Spark scaladoc by running `build/sbt unidoc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as -public in `__init__.py`. +public in `__init__.py`. The SparkR docs can be built by running SPARK_PROJECT_ROOT/R/create-docs.sh. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `build/sbt doc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `build/sbt unidoc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). -NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 +NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 2e88b3093652d..b92c75f90b11c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -84,6 +84,7 @@
  • Scala
  • Java
  • Python
  • +
  • R
  • diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 3c626a0b7f54b..0ea3f8eab461b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -78,5 +78,18 @@ puts "cp -r python/docs/_build/html/. docs/api/python" cp_r("python/docs/_build/html/.", "docs/api/python") - cd("..") + # Build SparkR API docs + puts "Moving to R directory and building roxygen docs." + cd("R") + puts `./create-docs.sh` + + puts "Moving back into home dir." + cd("../") + + puts "Making directory api/R" + mkdir_p "docs/api/R" + + puts "cp -r R/pkg/html/. docs/api/R" + cp_r("R/pkg/html/.", "docs/api/R") + end diff --git a/docs/building-spark.md b/docs/building-spark.md index ea79c5bc276d3..287fcd3c4034f 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -66,7 +66,6 @@ Because HDFS is not protocol-compatible across versions, if you want to read fro Hadoop versionProfile required - 0.23.xhadoop-0.23 1.x to 2.1.x(none) 2.2.xhadoop-2.2 2.3.xhadoop-2.3 @@ -82,9 +81,6 @@ mvn -Dhadoop.version=1.2.1 -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v1 mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package - -# Apache Hadoop 0.23.x -mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} You can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". Spark only supports YARN versions 2.2.0 and later. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 6a75d5c457f02..7079de546e2f5 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -33,7 +33,11 @@ There are several useful things to note about this architecture: 2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor processes, and these communicate with each other, it is relatively easy to run it even on a cluster manager that also supports other applications (e.g. Mesos/YARN). -3. Because the driver schedules tasks on the cluster, it should be run close to the worker +3. The driver program must listen for and accept incoming connections from its executors throughout + its lifetime (e.g., see [spark.driver.port and spark.fileserver.port in the network config + section](configuration.html#networking)). As such, the driver program must be network + addressable from the worker nodes. +4. Because the driver schedules tasks on the cluster, it should be run close to the worker nodes, preferably on the same local area network. If you'd like to send requests to the cluster remotely, it's better to open an RPC to the driver and have it submit operations from nearby than to run a driver far away from the worker nodes. diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..64066bc0d70cd 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1,4 +1,4 @@ ---- +-- layout: global displayTitle: Spark Configuration title: Configuration @@ -35,8 +35,29 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually -require one to prevent any sort of starvation issues. +Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may +actually require one to prevent any sort of starvation issues. + +Properties that specify some time duration should be configured with a unit of time. +The following format is accepted: + + 25ms (milliseconds) + 5s (seconds) + 10m or 10min (minutes) + 3h (hours) + 5d (days) + 1y (years) + + +Properties that specify a byte size should be configured with a unit of size. +The following format is accepted: + + 1b (bytes) + 1k or 1kb (kibibytes = 1024 bytes) + 1m or 1mb (mebibytes = 1024 kibibytes) + 1g or 1gb (gibibytes = 1024 mebibytes) + 1t or 1tb (tebibytes = 1024 gibibytes) + 1p or 1pb (pebibytes = 1024 tebibytes) ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For @@ -262,12 +283,11 @@ Apart from these, the following properties are also available, and may be useful - spark.executor.logs.rolling.size.maxBytes + spark.executor.logs.rolling.maxSize (none) Set the max size of the file by which the executor logs will be rolled over. - Rolling is disabled by default. Value is set in terms of bytes. - See spark.executor.logs.rolling.maxRetainedFiles + Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs. @@ -356,10 +376,10 @@ Apart from these, the following properties are also available, and may be useful - - + + @@ -393,10 +413,10 @@ Apart from these, the following properties are also available, and may be useful - - + + @@ -429,10 +449,10 @@ Apart from these, the following properties are also available, and may be useful - + @@ -572,18 +592,18 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + @@ -631,19 +651,19 @@ Apart from these, the following properties are also available, and may be useful - - + + - - + + @@ -688,9 +708,9 @@ Apart from these, the following properties are also available, and may be useful - + @@ -713,6 +733,17 @@ Apart from these, the following properties are also available, and may be useful this duration will be cleared as well. + + + + + - - + - + @@ -795,9 +826,9 @@ Apart from these, the following properties are also available, and may be useful - + @@ -812,19 +843,27 @@ Apart from these, the following properties are also available, and may be useful - + + + + + + - - + +
    Property NameDefaultMeaning
    spark.reducer.maxMbInFlight48spark.reducer.maxSizeInFlight48m - Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since + Maximum size of map outputs to fetch simultaneously from each reduce task. Since each output requires us to create a buffer to receive it, this represents a fixed memory overhead per reduce task, so keep it small unless you have a large amount of memory.
    spark.shuffle.file.buffer.kb32spark.shuffle.file.buffer32k - Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers + Size of the in-memory buffer for each shuffle file output stream. These buffers reduce the number of disk seeks and system calls made in creating intermediate shuffle files.
    spark.shuffle.io.retryWait55s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is 15 seconds by default, calculated as maxRetries * retryWait.
    spark.io.compression.lz4.block.size32768spark.io.compression.lz4.blockSize32k - Block size (in bytes) used in LZ4 compression, in the case when LZ4 compression codec + Block size used in LZ4 compression, in the case when LZ4 compression codec is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used.
    spark.io.compression.snappy.block.size32768spark.io.compression.snappy.blockSize32k - Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec + Block size used in Snappy compression, in the case when Snappy compression codec is used. Lowering this block size will also lower shuffle memory usage when Snappy is used.
    spark.kryoserializer.buffer.max.mb64spark.kryoserializer.buffer.max64m - Maximum allowable size of Kryo serialization buffer, in megabytes. This must be larger than any + Maximum allowable size of Kryo serialization buffer. This must be larger than any object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception inside Kryo.
    spark.kryoserializer.buffer.mb0.064spark.kryoserializer.buffer64k - Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer + Initial size of Kryo's serialization buffer. Note that there will be one buffer per core on each worker. This buffer will grow up to spark.kryoserializer.buffer.max.mb if needed.
    Property NameDefaultMeaning
    spark.broadcast.blockSize40964m - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Size of each piece of a block for TorrentBroadcastFactory. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit.
    spark.executor.cores1 in YARN mode, all the available cores on the worker in standalone mode. + The number of cores to use on each executor. For YARN and standalone mode only. + + In standalone mode, setting this parameter allows an application to run multiple executors on + the same worker, provided that there are enough cores on that worker. Otherwise, only one + executor per application will run on each worker. +
    spark.default.parallelism @@ -732,17 +763,17 @@ Apart from these, the following properties are also available, and may be useful
    spark.executor.heartbeatInterval10000Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10sInterval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks.
    spark.files.fetchTimeout6060s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver.
    spark.storage.memoryMapThreshold20971522m - Size of a block, in bytes, above which Spark memory maps when reading a block from disk. + Size of a block above which Spark memory maps when reading a block from disk. This prevents Spark from memory mapping very small blocks. In general, memory mapping has high overhead for blocks close to or below the page size of the operating system.
    spark.tachyonStore.baseDirspark.externalBlockStore.blockManagerorg.apache.spark.storage.TachyonBlockManager + Implementation of external block manager (file system) that store RDDs. The file system's URL is set by + spark.externalBlockStore.url. +
    spark.externalBlockStore.baseDir System.getProperty("java.io.tmpdir") - Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by - spark.tachyonStore.url. It can also be a comma-separated list of multiple + Directories of the external block store that store RDDs. The file system's URL is set by + spark.externalBlockStore.url It can also be a comma-separated list of multiple directories on Tachyon file system.
    spark.tachyonStore.urltachyon://localhost:19998spark.externalBlockStore.urltachyon://localhost:19998 for Tachyon - The URL of the underlying Tachyon file system in the TachyonStore. + The URL of the underlying external blocker file system in the external block store.
    @@ -853,11 +892,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value reduces network overhead and a smaller value ( ~ 1 s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case @@ -868,11 +907,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 6000 + 6000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -886,9 +925,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. @@ -938,12 +977,13 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, - spark.storage.blockManagerSlaveTimeoutMs or - spark.shuffle.io.connectionTimeout, if they are not configured. + Default timeout for all network interactions. This config will be used in place of + spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.storage.blockManagerSlaveTimeoutMs, + spark.shuffle.io.connectionTimeout, spark.rpc.askTimeout or + spark.rpc.lookupTimeout if they are not configured. @@ -961,6 +1001,35 @@ Apart from these, the following properties are also available, and may be useful This is only relevant for the Spark shell. + + spark.rpc.numRetries + 3 + Number of times to retry before an RPC task gives up. + An RPC task will run at most times of this number. + + + + + spark.rpc.retry.wait + 3s + + Duration for an RPC ask operation to wait before retrying. + + + + spark.rpc.askTimeout + 120s + + Duration for an RPC ask operation to wait before timing out. + + + + spark.rpc.lookupTimeout + 120s + Duration for an RPC remote endpoint lookup operation to wait before timing out. + + + #### Scheduling @@ -989,9 +1058,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3s - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1024,10 +1093,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30s - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. @@ -1054,10 +1122,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1s - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. @@ -1070,9 +1137,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. @@ -1127,10 +1194,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this description. @@ -1157,10 +1224,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration (in seconds), new executors will be requested. For more detail, see this + this duration, new executors will be requested. For more detail, see this description. @@ -1215,18 +1282,18 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. spark.core.connection.auth.wait.timeout - 30 + 30s - Number of seconds for the connection to wait for authentication to occur before timing + How long for the connection to wait for authentication to occur before timing out and giving up. @@ -1347,9 +1414,9 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. @@ -1397,6 +1464,16 @@ Apart from these, the following properties are also available, and may be useful for more details. + + spark.streaming.kafka.maxRetries + 1 + + Maximum number of consecutive retries the driver will make in order to find + the latest offsets on the leader of each partition (a default value of 1 + means that the driver will make a maximum of 2 attempts). Only applies to + the new Kafka direct stream API. + + #### Cluster Managers diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index c601d793a2e9a..3f10cb2dc3d2a 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -899,6 +899,8 @@ class VertexRDD[VD] extends RDD[(VertexID, VD)] { // Transform the values without changing the ids (preserves the internal index) def mapValues[VD2](map: VD => VD2): VertexRDD[VD2] def mapValues[VD2](map: (VertexId, VD) => VD2): VertexRDD[VD2] + // Show only vertices unique to this set based on their VertexId's + def minus(other: RDD[(VertexId, VD)]) // Remove vertices from this set that appear in the other set def diff(other: VertexRDD[VD]): VertexRDD[VD] // Join operators that take advantage of the internal indexing to accelerate joins (substantially) diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 87dcc58feb494..96bd69ca3b33b 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -29,9 +29,6 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. ReleaseVersion code CDH 4.X.X (YARN mode)2.0.0-cdh4.X.X CDH 4.X.X2.0.0-mr1-cdh4.X.X - CDH 3u60.20.2-cdh3u6 - CDH 3u50.20.2-cdh3u5 - CDH 3u40.20.2-cdh3u4 diff --git a/docs/img/cluster-overview.png b/docs/img/cluster-overview.png index 368274068e754..317554c5f2a5b 100644 Binary files a/docs/img/cluster-overview.png and b/docs/img/cluster-overview.png differ diff --git a/docs/img/cluster-overview.pptx b/docs/img/cluster-overview.pptx index af3c462cd904d..1b90d7ec5a7ae 100644 Binary files a/docs/img/cluster-overview.pptx and b/docs/img/cluster-overview.pptx differ diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 963e88a3e1d8f..8d9c2ba2041b2 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -32,7 +32,7 @@ Resource allocation can be configured as follows, based on the cluster type: * **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit the number of nodes an application uses by setting the `spark.cores.max` configuration property in it, - or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. + or change the default for applications that don't set this setting through `spark.deploy.defaultCores`. Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls its memory use. * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, diff --git a/docs/ml-guide.md b/docs/ml-guide.md index da6aef7f14c4c..771a07183e26f 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -408,31 +408,31 @@ import org.apache.spark.sql.SQLContext; // Labeled and unlabeled instance types. // Spark SQL can infer schema from Java Beans. public class Document implements Serializable { - private Long id; + private long id; private String text; - public Document(Long id, String text) { + public Document(long id, String text) { this.id = id; this.text = text; } - public Long getId() { return this.id; } - public void setId(Long id) { this.id = id; } + public long getId() { return this.id; } + public void setId(long id) { this.id = id; } public String getText() { return this.text; } public void setText(String text) { this.text = text; } } public class LabeledDocument extends Document implements Serializable { - private Double label; + private double label; - public LabeledDocument(Long id, String text, Double label) { + public LabeledDocument(long id, String text, double label) { super(id, text); this.label = label; } - public Double getLabel() { return this.label; } - public void setLabel(Double label) { this.label = label; } + public double getLabel() { return this.label; } + public void setLabel(double label) { this.label = label; } } // Set up contexts. @@ -493,7 +493,7 @@ from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.sql import Row, SQLContext sc = SparkContext(appName="SimpleTextClassificationPipeline") -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") @@ -565,6 +565,11 @@ import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.sql.{Row, SQLContext} +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from case classes. +case class LabeledDocument(id: Long, text: String, label: Double) +case class Document(id: Long, text: String) + val conf = new SparkConf().setAppName("CrossValidatorExample") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) @@ -655,6 +660,36 @@ import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from Java Beans. +public class Document implements Serializable { + private long id; + private String text; + + public Document(long id, String text) { + this.id = id; + this.text = text; + } + + public long getId() { return this.id; } + public void setId(long id) { this.id = id; } + + public String getText() { return this.text; } + public void setText(String text) { this.text = text; } +} + +public class LabeledDocument extends Document implements Serializable { + private double label; + + public LabeledDocument(long id, String text, double label) { + super(id, text); + this.label = label; + } + + public double getLabel() { return this.label; } + public void setLabel(double label) { this.label = label; } +} + SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); JavaSparkContext jsc = new JavaSparkContext(conf); SQLContext jsql = new SQLContext(jsc); diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 0b6db4fcb7b1f..f5aa15b7d9b79 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -173,6 +173,7 @@ to the algorithm. We then output the parameters of the mixture model. {% highlight scala %} import org.apache.spark.mllib.clustering.GaussianMixture +import org.apache.spark.mllib.clustering.GaussianMixtureModel import org.apache.spark.mllib.linalg.Vectors // Load and parse the data @@ -182,6 +183,10 @@ val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))) // Cluster the data into two classes using GaussianMixture val gmm = new GaussianMixture().setK(2).run(parsedData) +// Save and load model +gmm.save(sc, "myGMMModel") +val sameModel = GaussianMixtureModel.load(sc, "myGMMModel") + // output parameters of max-likelihood model for (i <- 0 until gmm.k) { println("weight=%f\nmu=%s\nsigma=\n%s\n" format @@ -231,6 +236,9 @@ public class GaussianMixtureExample { // Cluster the data into two classes using GaussianMixture GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd()); + // Save and load GaussianMixtureModel + gmm.save(sc, "myGMMModel") + GaussianMixtureModel sameModel = GaussianMixtureModel.load(sc, "myGMMModel") // Output the parameters of the mixture model for(int j=0; j A labeled point is represented by -[`LabeledPoint`](api/python/pyspark.mllib.regression.LabeledPoint-class.html). +[`LabeledPoint`](api/python/pyspark.mllib.html#pyspark.mllib.regression.LabeledPoint). {% highlight python %} from pyspark.mllib.linalg import SparseVector @@ -211,7 +211,7 @@ JavaRDD examples =
    -[`MLUtils.loadLibSVMFile`](api/python/pyspark.mllib.util.MLUtils-class.html) reads training +[`MLUtils.loadLibSVMFile`](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) reads training examples stored in LIBSVM format. {% highlight python %} diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md index 12fb29d426741..b521c2f27cd6e 100644 --- a/docs/mllib-isotonic-regression.md +++ b/docs/mllib-isotonic-regression.md @@ -1,6 +1,6 @@ --- layout: global -title: Naive Bayes - MLlib +title: Isotonic regression - MLlib displayTitle: MLlib - Regression --- @@ -152,4 +152,4 @@ Double meanSquaredError = new JavaDoubleRDD(predictionAndLabel.map( System.out.println("Mean Squared Error = " + meanSquaredError); {% endhighlight %}
    - \ No newline at end of file + diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 9270741d439d9..2b2be4d9d0273 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -377,7 +377,7 @@ references. Here is an [detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). -For multiclass classification problems, the algorithm will outputs a multinomial logistic regression +For multiclass classification problems, the algorithm will output a multinomial logistic regression model, which contains $K - 1$ binary logistic regression models regressed against the first class. Given a new data points, $K - 1$ models will be run, and the class with largest probability will be chosen as the predicted class. diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 55b8f2ce6c364..9780ea52c4994 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -13,12 +13,15 @@ compute the conditional probability distribution of label given an observation and use it for prediction. MLlib supports [multinomial naive -Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), -which is typically used for [document -classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +and [Bernoulli naive Bayes] (http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). +These models are typically used for [document classification] +(http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each -feature represents a term whose value is the frequency of the term. -Feature values must be nonnegative to represent term frequencies. +feature represents a term whose value is the frequency of the term (in multinomial naive Bayes) or +a zero or one indicating whether the term was found in the document (in Bernoulli naive Bayes). +Feature values must be nonnegative. The model type is selected with an optional parameter +"Multinomial" or "Bernoulli" with "Multinomial" as the default. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of @@ -32,7 +35,7 @@ sparsity. Since the training data is only used once, it is not necessary to cach [NaiveBayes](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements multinomial naive Bayes. It takes an RDD of [LabeledPoint](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional -smoothing parameter `lambda` as input, and output a +smoothing parameter `lambda` as input, an optional model type parameter (default is Multinomial), and outputs a [NaiveBayesModel](api/scala/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which can be used for evaluation and prediction. @@ -51,7 +54,7 @@ val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) val training = splits(0) val test = splits(1) -val model = NaiveBayes.train(training, lambda = 1.0) +val model = NaiveBayes.train(training, lambda = 1.0, model = "Multinomial") val predictionAndLabel = test.map(p => (model.predict(p.features), p.label)) val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() @@ -106,11 +109,11 @@ NaiveBayesModel sameModel = NaiveBayesModel.load(sc.sc(), "myModelPath");
    -[NaiveBayes](api/python/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +[NaiveBayes](api/python/pyspark.mllib.html#pyspark.mllib.classification.NaiveBayes) implements multinomial naive Bayes. It takes an RDD of -[LabeledPoint](api/python/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +[LabeledPoint](api/python/pyspark.mllib.html#pyspark.mllib.regression.LabeledPoint) and an optionally smoothing parameter `lambda` as input, and output a -[NaiveBayesModel](api/python/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +[NaiveBayesModel](api/python/pyspark.mllib.html#pyspark.mllib.classification.NaiveBayesModel), which can be used for evaluation and prediction. Note that the Python API does not yet support model save/load but will in the future. diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index ca8c29218f52d..887eae7f4f07b 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -81,8 +81,8 @@ System.out.println(summary.numNonzeros()); // number of nonzeros in each column
    -[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of -[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +[`colStats()`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics.colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.html#pyspark.mllib.stat.MultivariateStatisticalSummary), which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the total count. @@ -169,7 +169,7 @@ Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson");
    -[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +[`Statistics`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) provides methods to calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. @@ -258,7 +258,7 @@ JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); {% endhighlight %}
    -[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +[`sampleByKey()`](api/python/pyspark.html#pyspark.RDD.sampleByKey) allows users to sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of keys. @@ -476,7 +476,7 @@ JavaDoubleRDD v = u.map(
    -[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +[`RandomRDDs`](api/python/pyspark.mllib.html#pyspark.mllib.random.RandomRDDs) provides factory methods to generate random double RDDs or vector RDDs. The following example generates a random double RDD, whose values follows the standard normal distribution `N(0, 1)`, and then map it to `N(1, 4)`. diff --git a/docs/monitoring.md b/docs/monitoring.md index 6816671ffbf46..1e0fc150862fb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -86,10 +86,10 @@ follows: - spark.history.fs.update.interval.seconds - 10 + spark.history.fs.update.interval + 10s - The period, in seconds, at which information displayed by this history server is updated. + The period at which information displayed by this history server is updated. Each update checks for any changes made to the event logs in persisted storage. @@ -153,19 +153,18 @@ follows: - spark.history.fs.cleaner.interval.seconds - 86400 + spark.history.fs.cleaner.interval + 1d - How often the job history cleaner checks for files to delete, in seconds. Defaults to 86400 (one day). - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge.seconds. + How often the job history cleaner checks for files to delete. + Files are only deleted if they are older than spark.history.fs.cleaner.maxAge. - spark.history.fs.cleaner.maxAge.seconds - 3600 * 24 * 7 + spark.history.fs.cleaner.maxAge + 7d - Job history files older than this many seconds will be deleted when the history cleaner runs. - Defaults to 3600 * 24 * 7 (1 week). + Job history files older than this will be deleted when the history cleaner runs. @@ -175,6 +174,80 @@ making it easy to identify slow tasks, data skew, etc. Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI. +## REST API + +In addition to viewing the metrics in the UI, they are also available as JSON. This gives developers +an easy way to create new visualizations and monitoring tools for Spark. The JSON is available for +both running applications, and in the history server. The endpoints are mounted at `/json/v1`. Eg., +for the history server, they would typically be accessible at `http://:18080/json/v1`, and +for a running application, at `http://localhost:4040/json/v1`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    EndpointMeaning
    /applicationsA list of all applications
    /applications/[app-id]/jobsA list of all jobs for a given application
    /applications/[app-id]/jobs/[job-id]Details for the given job
    /applications/[app-id]/stagesA list of all stages for a given application
    /applications/[app-id]/stages/[stage-id]A list of all attempts for the given stage
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]Details for the given stage attempt
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummarySummary metrics of all tasks in the given stage attempt
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskListA list of all tasks for the given stage attempt
    /applications/[app-id]/executorsA list of all executors for the given application
    /applications/[app-id]/storage/rddA list of stored RDDs for the given application
    /applications/[app-id]/storage/rdd/[rdd-id]Details for the storage status of a given RDD
    + +When running on Yarn, each application has multiple attempts, so `[app-id]` is actually +`[app-id]/[attempt-id]` in all cases. + +These endpoints have been strongly versioned to make it easier to develop applications on top. + In particular, Spark guarantees: + +* Endpoints will never be removed from one version +* Individual fields will never be removed for any given endpoint +* New endpoints may be added +* New fields may be added to existing endpoints +* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`). New versions are *not* required to be backwards compatible. +* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version + +Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is +still required, though there is only one application available. Eg. to see the list of jobs for the +running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`. This is to +keep the paths consistent in both modes. + # Metrics Spark has a configurable metrics system based on the diff --git a/docs/programming-guide.md b/docs/programming-guide.md index eda3a95426182..27816515c5de2 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -142,8 +142,8 @@ JavaSparkContext sc = new JavaSparkContext(conf);
    -The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.context.SparkContext-class.html) object, which tells Spark -how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.conf.SparkConf-class.html) object +The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.html#pyspark.SparkConf) object that contains information about your application. {% highlight python %} @@ -237,9 +237,13 @@ You can customize the `ipython` command by setting `PYSPARK_DRIVER_PYTHON_OPTS`. the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: {% highlight bash %} -$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark +$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook" ./bin/pyspark {% endhighlight %} +After the IPython Notebook server is launched, you can create a new "Python 2" notebook from +the "Files" tab. Inside the notebook, you can input the command `%pylab inline` as part of +your notebook before you start to try Spark from the IPython notebook. +
    @@ -912,7 +916,7 @@ The following table lists some of the common transformations supported by Spark. RDD API doc ([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.rdd.RDD-class.html)) + [Python](api/python/pyspark.html#pyspark.RDD)) and pair RDD functions doc ([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) @@ -933,7 +937,7 @@ for details. Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item). - mapPartitions(func) + mapPartitions(func) Similar to map, but runs separately on each partition (block) of the RDD, so func must be of type Iterator<T> => Iterator<U> when running on an RDD of type T. @@ -960,7 +964,7 @@ for details. Return a new dataset that contains the distinct elements of the source dataset. - groupByKey([numTasks]) + groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
    Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will yield much better @@ -971,25 +975,25 @@ for details. - reduceByKey(func, [numTasks]) + reduceByKey(func, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, V) pairs where the values for each key are aggregated using the given reduce function func, which must be of type (V,V) => V. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument. - aggregateByKey(zeroValue)(seqOp, combOp, [numTasks]) + aggregateByKey(zeroValue)(seqOp, combOp, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, U) pairs where the values for each key are aggregated using the given combine functions and a neutral "zero" value. Allows an aggregated value type that is different than the input value type, while avoiding unnecessary allocations. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument. - sortByKey([ascending], [numTasks]) + sortByKey([ascending], [numTasks]) When called on a dataset of (K, V) pairs where K implements Ordered, returns a dataset of (K, V) pairs sorted by keys in ascending or descending order, as specified in the boolean ascending argument. - join(otherDataset, [numTasks]) + join(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin. - cogroup(otherDataset, [numTasks]) + cogroup(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith. @@ -1002,17 +1006,17 @@ for details. process's stdin and lines output to its stdout are returned as an RDD of strings. - coalesce(numPartitions) + coalesce(numPartitions) Decrease the number of partitions in the RDD to numPartitions. Useful for running operations more efficiently after filtering down a large dataset. repartition(numPartitions) Reshuffle the data in the RDD randomly to create either more or fewer partitions and balance it across them. - This always shuffles all data over the network. + This always shuffles all data over the network. - repartitionAndSortWithinPartitions(partitioner) + repartitionAndSortWithinPartitions(partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, sort records by their keys. This is more efficient than calling repartition and then sorting within each partition because it can push the sorting down into the shuffle machinery. @@ -1025,7 +1029,7 @@ The following table lists some of the common actions supported by Spark. Refer t RDD API doc ([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.rdd.RDD-class.html)) + [Python](api/python/pyspark.html#pyspark.RDD)) and pair RDD functions doc ([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) @@ -1076,7 +1080,7 @@ for details. SparkContext.objectFile(). - countByKey() + countByKey() Only available on RDDs of type (K, V). Returns a hashmap of (K, Int) pairs with the count of each key. @@ -1086,6 +1090,67 @@ for details. +### Shuffle operations + +Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's +mechanism for re-distributing data so that it's grouped differently across partitions. This typically +involves copying data across executors and machines, making the shuffle a complex and +costly operation. + +#### Background + +To understand what happens during the shuffle we can consider the example of the +[`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all +values for a single key are combined into a tuple - the key and the result of executing a reduce +function against all values associated with that key. The challenge is that not all values for a +single key necessarily reside on the same partition, or even the same machine, but they must be +co-located to compute the result. + +In Spark, data is generally not distributed across partitions to be in the necessary place for a +specific operation. During computations, a single task will operate on a single partition - thus, to +organize all the data for a single `reduceByKey` reduce task to execute, Spark needs to perform an +all-to-all operation. It must read from all partitions to find all the values for all keys, +and then bring together values across partitions to compute the final result for each key - +this is called the **shuffle**. + +Although the set of elements in each partition of newly shuffled data will be deterministic, and so +is the ordering of partitions themselves, the ordering of these elements is not. If one desires predictably +ordered data following shuffle then it's possible to use: + +* `mapPartitions` to sort each partition using, for example, `.sorted` +* `repartitionAndSortWithinPartitions` to efficiently sort partitions while simultaneously repartitioning +* `sortBy` to make a globally ordered RDD + +Operations which can cause a shuffle include **repartition** operations like +[`repartition`](#RepartitionLink), and [`coalesce`](#CoalesceLink), **'ByKey** operations +(except for counting) like [`groupByKey`](#GroupByLink) and [`reduceByKey`](#ReduceByLink), and +**join** operations like [`cogroup`](#CogroupLink) and [`join`](#JoinLink). + +#### Performance Impact +The **Shuffle** is an expensive operation since it involves disk I/O, data serialization, and +network I/O. To organize data for the shuffle, Spark generates sets of tasks - *map* tasks to +organize the data, and a set of *reduce* tasks to aggregate it. This nomenclature comes from +MapReduce and does not directly relate to Spark's `map` and `reduce` operations. + +Internally, results from individual map tasks are kept in memory until they can't fit. Then, these +are sorted based on the target partition and written to a single file. On the reduce side, tasks +read the relevant sorted blocks. + +Certain shuffle operations can consume significant amounts of heap memory since they employ +in-memory data structures to organize records before or after transferring them. Specifically, +`reduceByKey` and `aggregateByKey` create these structures on the map side and `'ByKey` operations +generate these on the reduce side. When data does not fit in memory Spark will spill these tables +to disk, incurring the additional overhead of disk I/O and increased garbage collection. + +Shuffle also generates a large number of intermediate files on disk. As of Spark 1.3, these files +are not cleaned up from Spark's temporary storage until Spark is stopped, which means that +long-running Spark jobs may consume available disk space. This is done so the shuffle doesn't need +to be re-computed if the lineage is re-computed. The temporary storage directory is specified by the +`spark.local.dir` configuration parameter when configuring the Spark context. + +Shuffle behavior can be tuned by adjusting a variety of configuration parameters. See the +'Shuffle Behavior' section within the [Spark Configuration Guide](configuration.html). + ## RDD Persistence One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory @@ -1105,7 +1170,7 @@ replicate it across nodes, or store it off-heap in [Tachyon](http://tachyon-proj These levels are set by passing a `StorageLevel` object ([Scala](api/scala/index.html#org.apache.spark.storage.StorageLevel), [Java](api/java/index.html?org/apache/spark/storage/StorageLevel.html), -[Python](api/python/pyspark.storagelevel.StorageLevel-class.html)) +[Python](api/python/pyspark.html#pyspark.StorageLevel)) to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The full set of storage levels is: @@ -1374,7 +1439,7 @@ scala> accum.value {% endhighlight %} While this code used the built-in support for accumulators of type Int, programmers can also -create their own types by subclassing [AccumulatorParam](api/python/pyspark.accumulators.AccumulatorParam-class.html). +create their own types by subclassing [AccumulatorParam](api/python/pyspark.html#pyspark.AccumulatorParam). The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class representing mathematical vectors, we could write: diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index c984639bd34cf..5f1d6daeb27f0 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -78,6 +78,9 @@ To verify that the Mesos cluster is ready for Spark, navigate to the Mesos maste To use Mesos from Spark, you need a Spark binary package available in a place accessible by Mesos, and a Spark driver program configured to connect to Mesos. +Alternatively, you can also install Spark in the same location in all the Mesos slaves, and configure +`spark.mesos.executor.home` (defaults to SPARK_HOME) to point to that location. + ## Uploading Spark Package When Mesos runs a task on a Mesos slave for the first time, that slave must have a Spark binary @@ -107,7 +110,11 @@ the `make-distribution.sh` script included in a Spark source tarball/checkout. The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooKeeper. -The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: +## Client Mode + +In client mode, a Spark Mesos framework is launched directly on the client machine and waits for the driver output. + +The driver needs some configuration in `spark-env.sh` to interact properly with Mesos: 1. In `spark-env.sh` set some environment variables: * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically @@ -129,8 +136,7 @@ val sc = new SparkContext(conf) {% endhighlight %} (You can also use [`spark-submit`](submitting-applications.html) and configure `spark.executor.uri` -in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file. Note -that `spark-submit` currently only supports deploying the Spark driver in `client` mode for Mesos.) +in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file.) When running a shell, the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so it does not need to be redundantly passed in as a system property. @@ -139,6 +145,17 @@ it does not need to be redundantly passed in as a system property. ./bin/spark-shell --master mesos://host:5050 {% endhighlight %} +## Cluster mode + +Spark on Mesos also supports cluster mode, where the driver is launched in the cluster and the client +can find the results of the driver from the Mesos Web UI. + +To use cluster mode, you must start the MesosClusterDispatcher in your cluster via the `sbin/start-mesos-dispatcher.sh` script, +passing in the Mesos master url (e.g: mesos://host:5050). + +From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master url +to the url of the MesosClusterDispatcher (e.g: mesos://dispatcher:7077). You can view driver statuses on the +Spark cluster Web UI. # Mesos Run Modes @@ -167,6 +184,16 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +# Mesos Docker Support + +Spark can make use of a Mesos Docker containerizer by setting the property `spark.mesos.executor.docker.image` +in your [SparkConf](configuration.html#spark-properties). + +The Docker image used must have an appropriate version of Spark already part of the image, or you can +have Mesos download Spark via the usual methods. + +Requires Mesos version 0.20.1 or later. + # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a @@ -210,6 +237,48 @@ See the [configuration page](configuration.html) for information on Spark config Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. + + spark.mesos.mesosExecutor.cores + 1.0 + + (Fine-grained mode only) Number of cores to give each Mesos executor. This does not + include the cores used to run the Spark tasks. In other words, even if no Spark task + is being run, each Mesos executor will occupy the number of cores configured here. + The value can be a floating point number. + + + + spark.mesos.executor.docker.image + (none) + + Set the name of the docker image that the Spark executors will run in. The selected + image must have Spark installed, as well as a compatible version of the Mesos library. + The installed path of Spark in the image can be specified with spark.mesos.executor.home; + the installed path of the Mesos library can be specified with spark.executorEnv.MESOS_NATIVE_LIBRARY. + + + + spark.mesos.executor.docker.volumes + (none) + + Set the list of volumes which will be mounted into the Docker image, which was set using + spark.mesos.executor.docker.image. The format of this property is a comma-separated list of + mappings following the form passed to docker run -v. That is they take the form: + +
    [host_path:]container_path[:ro|:rw]
    + + + + spark.mesos.executor.docker.portmaps + (none) + + Set the list of incoming ports exposed by the Docker image, which was set using + spark.mesos.executor.docker.image. The format of this property is a comma-separated list of + mappings which take the form: + +
    host_port:container_port[:tcp|:udp]
    + + spark.mesos.executor.home driver side SPARK_HOME diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 68b1aeb8ebd01..4fb4a90307ec8 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,9 +48,9 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000 + 100s - In yarn-cluster mode, time in milliseconds for the application master to wait for the + In yarn-cluster mode, time for the application master to wait for the SparkContext to be initialized. In yarn-client mode, time for the application master to wait for the driver to connect to it. @@ -87,7 +87,8 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.historyServer.address (none) - The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + For this property, YARN properties can be used as variables, and these are substituted by Spark at runtime. For eg, if the Spark history server runs on the same node as the YARN ResourceManager, it can be set to `${hadoopconf-yarn.resourcemanager.hostname}:18080`. @@ -132,6 +133,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes Same as spark.yarn.driver.memoryOverhead, but for the Application Master in client mode. + + spark.yarn.am.port + (random) + + Port for the YARN Application Master to listen on. In YARN client mode, this is used to communicate between the Spark driver running on a gateway and the Application Master running on YARN. In YARN cluster mode, this is used for the dynamic executor feature, where it handles the kill from the scheduler backend. + + spark.yarn.queue default @@ -188,6 +196,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes In cluster mode, use spark.driver.extraJavaOptions instead. + + spark.yarn.am.extraLibraryPath + (none) + + Set a special library path to use when launching the application master in client mode. + + spark.yarn.maxAppAttempts yarn.resourcemanager.am.max-attempts in YARN @@ -196,12 +211,25 @@ Most of the configs are the same for Spark on YARN as for other deployment modes It should be no larger than the global number of max attempts in the YARN configuration. + + spark.yarn.submit.waitAppCompletion + true + + In YARN cluster mode, controls whether the client waits to exit until the application completes. + If set to true, the client process will stay alive reporting the application's status. + Otherwise, the client process will exit after submission. + + # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. -These configs are used to write to the dfs and connect to the YARN ResourceManager. +These configs are used to write to the dfs and connect to the YARN ResourceManager. The +configuration contained in this directory will be distributed to the YARN cluster so that all +containers used by the application use the same configuration. If the configuration references +Java system properties or environment variables not managed by YARN, they should also be set in the +Spark application's configuration (driver, executors, and the AM when running in client mode). There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. @@ -274,6 +302,6 @@ If you need a reference to the proper location to put log files in the YARN so t # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. -- The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. +- In `yarn-cluster` mode, the local directories used by the Spark executors and the Spark driver will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. In `yarn-client` mode, the Spark executors will use the local directories configured for YARN while the Spark driver will use those defined in `spark.local.dir`. This is because the Spark driver does not run on the YARN cluster in `yarn-client` mode, only the Spark executors do. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `yarn-cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. diff --git a/docs/security.md b/docs/security.md index c034ba12ff1fc..d4ffa60e59a33 100644 --- a/docs/security.md +++ b/docs/security.md @@ -32,6 +32,8 @@ SSL must be configured on each node and configured for each component involved i ### YARN mode The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. +For long-running apps like Spark Streaming apps to be able to write to HDFS, it is possible to pass a principal and keytab to `spark-submit` via the `--principal` and `--keytab` parameters respectively. The keytab passed in will be copied over to the machine running the Application Master via the Hadoop Distributed Cache (securely - if YARN is configured with SSL and HDFS encryption is enabled). The Kerberos login will be periodically renewed using this principal and keytab and the delegation tokens required for HDFS will be generated periodically so the application can continue writing to HDFS. + ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 74d8653a8b845..0eed9adacf123 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -24,7 +24,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT + ./sbin/start-slave.sh Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). @@ -81,6 +81,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on. - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. +- `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. - `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2cbb4c967eb81..b8233ae06fdf3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -56,7 +56,7 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
    The entry point into all relational functionality in Spark is the -[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one +[`SQLContext`](api/python/pyspark.sql.html#pyspark.sql.SQLContext) class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight python %} @@ -193,8 +193,8 @@ df.groupBy("age").count().show()
    {% highlight java %} -val sc: JavaSparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +JavaSparkContext sc // An existing SparkContext. +SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc) // Create the DataFrame DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); @@ -308,8 +308,8 @@ val df = sqlContext.sql("SELECT * FROM table")
    {% highlight java %} -val sqlContext = ... // An existing SQLContext -val df = sqlContext.sql("SELECT * FROM table") +SQLContext sqlContext = ... // An existing SQLContext +DataFrame df = sqlContext.sql("SELECT * FROM table") {% endhighlight %}
    @@ -435,7 +435,7 @@ DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AN // 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. -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -509,8 +509,11 @@ val people = sc.textFile("examples/src/main/resources/people.txt") // The schema is encoded in a string val schemaString = "name age" -// Import Spark SQL data types and Row. -import org.apache.spark.sql._ +// Import Row. +import org.apache.spark.sql.Row; + +// Import Spark SQL data types +import org.apache.spark.sql.types.{StructType,StructField,StringType}; // Generate the schema based on the string of schema val schema = @@ -552,13 +555,16 @@ by `SQLContext`. For example: {% highlight java %} -// Import factory methods provided by DataType. -import org.apache.spark.sql.types.DataType; +import org.apache.spark.api.java.function.Function; +// Import factory methods provided by DataTypes. +import org.apache.spark.sql.types.DataTypes; // Import StructType and StructField import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.StructField; // Import Row. import org.apache.spark.sql.Row; +// Import RowFactory. +import org.apache.spark.sql.RowFactory; // sc is an existing JavaSparkContext. SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); @@ -572,16 +578,16 @@ String schemaString = "name age"; // Generate the schema based on the string of schema List fields = new ArrayList(); for (String fieldName: schemaString.split(" ")) { - fields.add(DataType.createStructField(fieldName, DataType.StringType, true)); + fields.add(DataTypes.createStructField(fieldName, DataTypes.StringType, true)); } -StructType schema = DataType.createStructType(fields); +StructType schema = DataTypes.createStructType(fields); // Convert records of the RDD (people) to Rows. JavaRDD rowRDD = people.map( new Function() { public Row call(String record) throws Exception { String[] fields = record.split(","); - return Row.create(fields[0], fields[1].trim()); + return RowFactory.create(fields[0], fields[1].trim()); } }); @@ -596,7 +602,7 @@ DataFrame results = sqlContext.sql("SELECT name FROM people"); // 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. -List names = results.map(new Function() { +List names = results.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -621,7 +627,8 @@ tuples or lists in the RDD created in the step 1. For example: {% highlight python %} # Import SQLContext and data types -from pyspark.sql import * +from pyspark.sql import SQLContext +from pyspark.sql.types import * # sc is an existing SparkContext. sqlContext = SQLContext(sc) @@ -674,8 +681,8 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    {% highlight scala %} -val df = sqlContext.load("people.parquet") -df.select("name", "age").save("namesAndAges.parquet") +val df = sqlContext.load("examples/src/main/resources/users.parquet") +df.select("name", "favorite_color").save("namesAndFavColors.parquet") {% endhighlight %}
    @@ -684,8 +691,8 @@ df.select("name", "age").save("namesAndAges.parquet") {% highlight java %} -DataFrame df = sqlContext.load("people.parquet"); -df.select("name", "age").save("namesAndAges.parquet"); +DataFrame df = sqlContext.load("examples/src/main/resources/users.parquet"); +df.select("name", "favorite_color").save("namesAndFavColors.parquet"); {% endhighlight %} @@ -695,8 +702,8 @@ df.select("name", "age").save("namesAndAges.parquet"); {% highlight python %} -df = sqlContext.load("people.parquet") -df.select("name", "age").save("namesAndAges.parquet") +df = sqlContext.load("examples/src/main/resources/users.parquet") +df.select("name", "favorite_color").save("namesAndFavColors.parquet") {% endhighlight %} @@ -715,7 +722,7 @@ using this syntax.
    {% highlight scala %} -val df = sqlContext.load("people.json", "json") +val df = sqlContext.load("examples/src/main/resources/people.json", "json") df.select("name", "age").save("namesAndAges.parquet", "parquet") {% endhighlight %} @@ -725,7 +732,7 @@ df.select("name", "age").save("namesAndAges.parquet", "parquet") {% highlight java %} -DataFrame df = sqlContext.load("people.json", "json"); +DataFrame df = sqlContext.load("examples/src/main/resources/people.json", "json"); df.select("name", "age").save("namesAndAges.parquet", "parquet"); {% endhighlight %} @@ -736,7 +743,7 @@ df.select("name", "age").save("namesAndAges.parquet", "parquet"); {% highlight python %} -df = sqlContext.load("people.json", "json") +df = sqlContext.load("examples/src/main/resources/people.json", "json") df.select("name", "age").save("namesAndAges.parquet", "parquet") {% endhighlight %} @@ -856,7 +863,7 @@ DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); -List teenagerNames = teenagers.map(new Function() { +List teenagerNames = teenagers.javaRDD().map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); } @@ -1357,7 +1364,7 @@ the Data Sources API. The following options are supported: driver - The class name of the JDBC driver needed to connect to this URL. This class with be loaded + The class name of the JDBC driver needed to connect to this URL. This class will be loaded on the master and workers before running an JDBC commands to allow the driver to register itself with the JDBC subsystem. @@ -1367,7 +1374,10 @@ the Data Sources API. The following options are supported: These options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple workers. - partitionColumn must be a numeric column from the table in question. + partitionColumn must be a numeric column from the table in question. Notice + that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. @@ -1402,7 +1412,7 @@ DataFrame jdbcDF = sqlContext.load("jdbc", options) {% highlight python %} -df = sqlContext.load("jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") +df = sqlContext.load(source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") {% endhighlight %} @@ -1638,7 +1648,7 @@ moved into the udf object in `SQLContext`.
    {% highlight java %} -sqlCtx.udf.register("strLen", (s: String) => s.length()) +sqlContext.udf.register("strLen", (s: String) => s.length()) {% endhighlight %}
    @@ -1646,7 +1656,7 @@ sqlCtx.udf.register("strLen", (s: String) => s.length())
    {% highlight java %} -sqlCtx.udf().register("strLen", (String s) -> { s.length(); }); +sqlContext.udf().register("strLen", (String s) -> { s.length(); }); {% endhighlight %}
    @@ -1780,6 +1790,7 @@ in Hive deployments. **Esoteric Hive Features** + * `UNION` type * Unique join * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 6d6229625f3f9..2f2fea53168a3 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -704,7 +704,7 @@ create a DStream using data from Twitter's stream of tweets, you have to do the {% highlight scala %} import org.apache.spark.streaming.twitter._ -TwitterUtils.createStream(ssc) +TwitterUtils.createStream(ssc, None) {% endhighlight %}
    @@ -1588,7 +1588,7 @@ See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more abo *** ## MLlib Operations -You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. (Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. +You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. *** diff --git a/docs/tuning.md b/docs/tuning.md index cbd227868b248..572c7270e4999 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -60,7 +60,7 @@ val sc = new SparkContext(conf) The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes more advanced registration options, such as adding custom serialization code. -If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb` +If your objects are large, you may also need to increase the `spark.kryoserializer.buffer` config property. The default is 2, but this value needs to be large enough to hold the *largest* object you will serialize. @@ -94,11 +94,13 @@ We will then cover tuning Spark's cache size and the Java garbage collector. ## Determining Memory Consumption -The best way to size the amount of memory consumption your dataset will require is to create an RDD, put it into cache, and look at the SparkContext logs on your driver program. The logs will tell you how much memory each partition is consuming, which you can aggregate to get the total size of the RDD. You will see messages like this: +The best way to size the amount of memory consumption a dataset will require is to create an RDD, put it +into cache, and look at the "Storage" page in the web UI. The page will tell you how much memory the RDD +is occupying. - INFO BlockManagerMasterActor: Added rdd_0_1 in memory on mbk.local:50311 (size: 717.5 KB, free: 332.3 MB) - -This means that partition 1 of RDD 0 consumed 717.5 KB. +To estimate the memory consumption of a particular object, use `SizeEstimator`'s `estimate` method +This is useful for experimenting with different data layouts to trim memory usage, as well as +determining the amount of space a broadcast variable will occupy on each executor heap. ## Tuning Data Structures diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index c467cd08ed742..87c0818279713 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -19,7 +19,7 @@ # limitations under the License. # -from __future__ import with_statement +from __future__ import with_statement, print_function import hashlib import itertools @@ -37,12 +37,17 @@ import tempfile import textwrap import time -import urllib2 import warnings from datetime import datetime from optparse import OptionParser from sys import stderr +if sys.version < "3": + from urllib2 import urlopen, Request, HTTPError +else: + from urllib.request import urlopen, Request + from urllib.error import HTTPError + SPARK_EC2_VERSION = "1.2.1" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -88,10 +93,10 @@ def setup_external_libs(libs): SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") if not os.path.exists(SPARK_EC2_LIB_DIR): - print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( + print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( path=SPARK_EC2_LIB_DIR - ) - print "This should be a one-time operation." + )) + print("This should be a one-time operation.") os.mkdir(SPARK_EC2_LIB_DIR) for lib in libs: @@ -100,8 +105,8 @@ def setup_external_libs(libs): if not os.path.isdir(lib_dir): tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print " - Downloading {lib}...".format(lib=lib["name"]) - download_stream = urllib2.urlopen( + print(" - Downloading {lib}...".format(lib=lib["name"])) + download_stream = urlopen( "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( prefix=PYPI_URL_PREFIX, first_letter=lib["name"][:1], @@ -113,13 +118,13 @@ def setup_external_libs(libs): tgz_file.write(download_stream.read()) with open(tgz_file_path) as tar: if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]) + print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) sys.exit(1) tar = tarfile.open(tgz_file_path) tar.extractall(path=SPARK_EC2_LIB_DIR) tar.close() os.remove(tgz_file_path) - print " - Finished downloading {lib}.".format(lib=lib["name"]) + print(" - Finished downloading {lib}.".format(lib=lib["name"])) sys.path.insert(1, lib_dir) @@ -282,6 +287,10 @@ def parse_args(): parser.add_option( "--vpc-id", default=None, help="VPC to launch instances in") + parser.add_option( + "--private-ips", action="store_true", default=False, + help="Use private IPs for instances rather than public if VPC/subnet " + + "requires that.") (opts, args) = parser.parse_args() if len(args) != 2: @@ -295,12 +304,12 @@ def parse_args(): if home_dir is None or not os.path.isfile(home_dir + '/.boto'): if not os.path.isfile('/etc/boto.cfg'): if os.getenv('AWS_ACCESS_KEY_ID') is None: - print >> stderr, ("ERROR: The environment variable AWS_ACCESS_KEY_ID " + - "must be set") + print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", + file=stderr) sys.exit(1) if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print >> stderr, ("ERROR: The environment variable AWS_SECRET_ACCESS_KEY " + - "must be set") + print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", + file=stderr) sys.exit(1) return (opts, action, cluster_name) @@ -312,7 +321,7 @@ def get_or_make_group(conn, name, vpc_id): if len(group) > 0: return group[0] else: - print "Creating security group " + name + print("Creating security group " + name) return conn.create_security_group(name, "Spark EC2 group", vpc_id) @@ -320,18 +329,19 @@ def get_validate_spark_version(version, repo): if "." in version: version = version.replace("v", "") if version not in VALID_SPARK_VERSIONS: - print >> stderr, "Don't know about Spark version: {v}".format(v=version) + print("Don't know about Spark version: {v}".format(v=version), file=stderr) sys.exit(1) return version else: github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) - request = urllib2.Request(github_commit_url) + request = Request(github_commit_url) request.get_method = lambda: 'HEAD' try: - response = urllib2.urlopen(request) - except urllib2.HTTPError, e: - print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url) - print >> stderr, "Received HTTP response code of {code}.".format(code=e.code) + response = urlopen(request) + except HTTPError as e: + print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), + file=stderr) + print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) sys.exit(1) return version @@ -390,8 +400,7 @@ def get_spark_ami(opts): instance_type = EC2_INSTANCE_TYPES[opts.instance_type] else: instance_type = "pvm" - print >> stderr,\ - "Don't recognize %s, assuming type is pvm" % opts.instance_type + print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) # URL prefix from which to fetch AMI information ami_prefix = "{r}/{b}/ami-list".format( @@ -400,10 +409,10 @@ def get_spark_ami(opts): ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) try: - ami = urllib2.urlopen(ami_path).read().strip() - print "Spark AMI: " + ami + ami = urlopen(ami_path).read().strip() + print("Spark AMI: " + ami) except: - print >> stderr, "Could not resolve AMI at: " + ami_path + print("Could not resolve AMI at: " + ami_path, file=stderr) sys.exit(1) return ami @@ -415,11 +424,11 @@ def get_spark_ami(opts): # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): if opts.identity_file is None: - print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections." + print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) sys.exit(1) if opts.key_pair is None: - print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." + print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) sys.exit(1) user_data_content = None @@ -427,7 +436,7 @@ def launch_cluster(conn, opts, cluster_name): with open(opts.user_data) as user_data_file: user_data_content = user_data_file.read() - print "Setting up security groups..." + print("Setting up security groups...") master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) authorized_address = opts.authorized_address @@ -456,6 +465,13 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50070, 50070, authorized_address) master_group.authorize('tcp', 60070, 60070, authorized_address) master_group.authorize('tcp', 4040, 4045, authorized_address) + # HDFS NFS gateway requires 111,2049,4242 for tcp & udp + master_group.authorize('tcp', 111, 111, authorized_address) + master_group.authorize('udp', 111, 111, authorized_address) + master_group.authorize('tcp', 2049, 2049, authorized_address) + master_group.authorize('udp', 2049, 2049, authorized_address) + master_group.authorize('tcp', 4242, 4242, authorized_address) + master_group.authorize('udp', 4242, 4242, authorized_address) if opts.ganglia: master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created @@ -486,8 +502,8 @@ def launch_cluster(conn, opts, cluster_name): existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print("ERROR: There are already instances running in group %s or %s" % + (master_group.name, slave_group.name), file=stderr) sys.exit(1) # Figure out Spark AMI @@ -500,12 +516,12 @@ def launch_cluster(conn, opts, cluster_name): additional_group_ids = [sg.id for sg in conn.get_all_security_groups() if opts.additional_security_group in (sg.name, sg.id)] - print "Launching instances..." + print("Launching instances...") try: image = conn.get_all_images(image_ids=[opts.ami])[0] except: - print >> stderr, "Could not find AMI " + opts.ami + print("Could not find AMI " + opts.ami, file=stderr) sys.exit(1) # Create block device mapping so that we can add EBS volumes if asked to. @@ -531,8 +547,8 @@ def launch_cluster(conn, opts, cluster_name): # Launch slaves if opts.spot_price is not None: # Launch spot instances with the requested price - print ("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) + print("Requesting %d slaves as spot instances with price $%.3f" % + (opts.slaves, opts.spot_price)) zones = get_zones(conn, opts) num_zones = len(zones) i = 0 @@ -555,7 +571,7 @@ def launch_cluster(conn, opts, cluster_name): my_req_ids += [req.id for req in slave_reqs] i += 1 - print "Waiting for spot instances to be granted..." + print("Waiting for spot instances to be granted...") try: while True: time.sleep(10) @@ -568,24 +584,24 @@ def launch_cluster(conn, opts, cluster_name): if i in id_to_req and id_to_req[i].state == "active": active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: - print "All %d slaves granted" % opts.slaves + print("All %d slaves granted" % opts.slaves) reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print("%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves)) except: - print "Canceling spot instance requests" + print("Canceling spot instance requests") conn.cancel_spot_instance_requests(my_req_ids) # Log a warning if any of these requests actually launched instances: (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) running = len(master_nodes) + len(slave_nodes) if running: - print >> stderr, ("WARNING: %d instances are still running" % running) + print(("WARNING: %d instances are still running" % running), file=stderr) sys.exit(0) else: # Launch non-spot instances @@ -607,16 +623,16 @@ def launch_cluster(conn, opts, cluster_name): placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances - print "Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id) + print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( + s=num_slaves_this_zone, + plural_s=('' if num_slaves_this_zone == 1 else 's'), + z=zone, + r=slave_res.id)) i += 1 # Launch or resume masters if existing_masters: - print "Starting master..." + print("Starting master...") for inst in existing_masters: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -639,10 +655,10 @@ def launch_cluster(conn, opts, cluster_name): user_data=user_data_content) master_nodes = master_res.instances - print "Launched master in %s, regid = %s" % (zone, master_res.id) + print("Launched master in %s, regid = %s" % (zone, master_res.id)) # This wait time corresponds to SPARK-4983 - print "Waiting for AWS to propagate instance metadata..." + print("Waiting for AWS to propagate instance metadata...") time.sleep(5) # Give the instances descriptive names for master in master_nodes: @@ -663,8 +679,8 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): Get the EC2 instances in an existing cluster if available. Returns a tuple of lists of EC2 instance objects for the masters and slaves. """ - print "Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region) + print("Searching for existing cluster {c} in region {r}...".format( + c=cluster_name, r=opts.region)) def get_instances(group_names): """ @@ -682,16 +698,15 @@ def get_instances(group_names): slave_instances = get_instances([cluster_name + "-slaves"]) if any((master_instances, slave_instances)): - print "Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's')) + print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( + m=len(master_instances), + plural_m=('' if len(master_instances) == 1 else 's'), + s=len(slave_instances), + plural_s=('' if len(slave_instances) == 1 else 's'))) if not master_instances and die_on_error: - print >> sys.stderr, \ - "ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region) + print("ERROR: Could not find a master for cluster {c} in region {r}.".format( + c=cluster_name, r=opts.region), file=sys.stderr) sys.exit(1) return (master_instances, slave_instances) @@ -700,9 +715,9 @@ def get_instances(group_names): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): - master = master_nodes[0].public_dns_name + master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: - print "Generating cluster's SSH key on master..." + print("Generating cluster's SSH key on master...") key_setup = """ [ -f ~/.ssh/id_rsa ] || (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && @@ -710,10 +725,11 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): """ ssh(master, opts, key_setup) dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print "Transferring cluster's SSH key to slaves..." + print("Transferring cluster's SSH key to slaves...") for slave in slave_nodes: - print slave.public_dns_name - ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) + slave_address = get_dns_name(slave, opts.private_ips) + print(slave_address) + ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone', 'tachyon'] @@ -726,8 +742,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch) + print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( + r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) ssh( host=master, opts=opts, @@ -737,7 +753,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): b=opts.spark_ec2_git_branch) ) - print "Deploying files to master..." + print("Deploying files to master...") deploy_files( conn=conn, root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", @@ -748,25 +764,25 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ) if opts.deploy_root_dir is not None: - print "Deploying {s} to master...".format(s=opts.deploy_root_dir) + print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) deploy_user_files( root_dir=opts.deploy_root_dir, opts=opts, master_nodes=master_nodes ) - print "Running setup on master..." + print("Running setup on master...") setup_spark_cluster(master, opts) - print "Done!" + print("Done!") def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") - print "Spark standalone cluster started at http://%s:8080" % master + print("Spark standalone cluster started at http://%s:8080" % master) if opts.ganglia: - print "Ganglia started at http://%s:5080/ganglia" % master + print("Ganglia started at http://%s:5080/ganglia" % master) def is_ssh_available(host, opts, print_ssh_output=True): @@ -783,7 +799,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): if s.returncode != 0 and print_ssh_output: # extra leading newline is for spacing in wait_for_cluster_state() - print textwrap.dedent("""\n + print(textwrap.dedent("""\n Warning: SSH connection error. (This could be temporary.) Host: {h} SSH return code: {r} @@ -792,7 +808,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): h=host, r=s.returncode, o=cmd_output.strip() - ) + )) return s.returncode == 0 @@ -802,7 +818,8 @@ def is_cluster_ssh_available(cluster_instances, opts): Check if SSH is available on all the instances in a cluster. """ for i in cluster_instances: - if not is_ssh_available(host=i.ip_address, opts=opts): + dns_name = get_dns_name(i, opts.private_ips) + if not is_ssh_available(host=dns_name, opts=opts): return False else: return True @@ -852,10 +869,10 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.write("\n") end_time = datetime.now() - print "Cluster is now in '{s}' state. Waited {t} seconds.".format( + print("Cluster is now in '{s}' state. Waited {t} seconds.".format( s=cluster_state, t=(end_time - start_time).seconds - ) + )) # Get number of local disks available for a given EC2 instance type. @@ -903,8 +920,8 @@ def get_num_disks(instance_type): if instance_type in disks_by_instance: return disks_by_instance[instance_type] else: - print >> stderr, ("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type) + print("WARNING: Don't know number of disks on instance type %s; assuming 1" + % instance_type, file=stderr) return 1 @@ -916,7 +933,7 @@ def get_num_disks(instance_type): # # root_dir should be an absolute path to the directory with the files we want to deploy. def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) num_disks = get_num_disks(opts.instance_type) hdfs_data_dirs = "/mnt/ephemeral-hdfs/data" @@ -938,13 +955,15 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) tachyon_v = "" - print "Deploying Spark via git hash; Tachyon won't be set up" + print("Deploying Spark via git hash; Tachyon won't be set up") modules = filter(lambda x: x != "tachyon", modules) + master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] + slave_addresses = [get_dns_name(i, opts.private_ips) for i in slave_nodes] template_vars = { - "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), + "master_list": '\n'.join(master_addresses), "active_master": active_master, - "slave_list": '\n'.join([i.public_dns_name for i in slave_nodes]), + "slave_list": '\n'.join(slave_addresses), "cluster_url": cluster_url, "hdfs_data_dirs": hdfs_data_dirs, "mapred_local_dirs": mapred_local_dirs, @@ -1004,7 +1023,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # # root_dir should be an absolute path. def deploy_user_files(root_dir, opts, master_nodes): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) command = [ 'rsync', '-rv', '-e', stringify_command(ssh_command(opts)), @@ -1052,8 +1071,8 @@ def ssh(host, opts, command): "--key-pair parameters and try again.".format(host)) else: raise e - print >> stderr, \ - "Error executing remote command, retrying after 30 seconds: {0}".format(e) + print("Error executing remote command, retrying after 30 seconds: {0}".format(e), + file=stderr) time.sleep(30) tries = tries + 1 @@ -1092,8 +1111,8 @@ def ssh_write(host, opts, command, arguments): elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: - print >> stderr, \ - "Error {0} while executing remote command, retrying after 30 seconds".format(status) + print("Error {0} while executing remote command, retrying after 30 seconds". + format(status), file=stderr) time.sleep(30) tries = tries + 1 @@ -1115,6 +1134,20 @@ def get_partition(total, num_partitions, current_partitions): return num_slaves_this_zone +# Gets the IP address, taking into account the --private-ips flag +def get_ip_address(instance, private_ips=False): + ip = instance.ip_address if not private_ips else \ + instance.private_ip_address + return ip + + +# Gets the DNS name, taking into account the --private-ips flag +def get_dns_name(instance, private_ips=False): + dns = instance.public_dns_name if not private_ips else \ + instance.private_ip_address + return dns + + def real_main(): (opts, action, cluster_name) = parse_args() @@ -1133,42 +1166,41 @@ def real_main(): if opts.identity_file is not None: if not os.path.exists(opts.identity_file): - print >> stderr,\ - "ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file) + print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), + file=stderr) sys.exit(1) file_mode = os.stat(opts.identity_file).st_mode if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print >> stderr, "ERROR: The identity file must be accessible only by you." - print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file) + print("ERROR: The identity file must be accessible only by you.", file=stderr) + print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), + file=stderr) sys.exit(1) if opts.instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type) + print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( + t=opts.instance_type), file=stderr) if opts.master_instance_type != "": if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, \ - "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type) + print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( + t=opts.master_instance_type), file=stderr) # Since we try instance types even if we can't resolve them, we check if they resolve first # and, if they do, see if they resolve to the same virtualization type. if opts.instance_type in EC2_INSTANCE_TYPES and \ opts.master_instance_type in EC2_INSTANCE_TYPES: if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: - print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves " + \ - "with different AMI virtualization types." - print >> stderr, "master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]) - print >> stderr, "slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]) + print("Error: spark-ec2 currently does not support having a master and slaves " + "with different AMI virtualization types.", file=stderr) + print("master instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) + print("slave instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) sys.exit(1) if opts.ebs_vol_num > 8: - print >> stderr, "ebs-vol-num cannot be greater than 8" + print("ebs-vol-num cannot be greater than 8", file=stderr) sys.exit(1) # Prevent breaking ami_prefix (/, .git and startswith checks) @@ -1177,23 +1209,22 @@ def real_main(): opts.spark_ec2_git_repo.endswith(".git") or \ not opts.spark_ec2_git_repo.startswith("https://github.com") or \ not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \ - "trailing / or .git. " \ - "Furthermore, we currently only support forks named spark-ec2." + print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " + "Furthermore, we currently only support forks named spark-ec2.", file=stderr) sys.exit(1) if not (opts.deploy_root_dir is None or (os.path.isabs(opts.deploy_root_dir) and os.path.isdir(opts.deploy_root_dir) and os.path.exists(opts.deploy_root_dir))): - print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \ - "on the local file system" + print("--deploy-root-dir must be an absolute path to a directory that exists " + "on the local file system", file=stderr) sys.exit(1) try: conn = ec2.connect_to_region(opts.region) except Exception as e: - print >> stderr, (e) + print((e), file=stderr) sys.exit(1) # Select an AZ at random if it was not specified. @@ -1202,7 +1233,7 @@ def real_main(): if action == "launch": if opts.slaves <= 0: - print >> sys.stderr, "ERROR: You have to start at least 1 slave" + print("ERROR: You have to start at least 1 slave", file=sys.stderr) sys.exit(1) if opts.resume: (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) @@ -1221,18 +1252,18 @@ def real_main(): conn, opts, cluster_name, die_on_error=False) if any(master_nodes + slave_nodes): - print "The following instances will be terminated:" + print("The following instances will be terminated:") for inst in master_nodes + slave_nodes: - print "> %s" % inst.public_dns_name - print "ALL DATA ON ALL NODES WILL BE LOST!!" + print("> %s" % get_dns_name(inst, opts.private_ips)) + print("ALL DATA ON ALL NODES WILL BE LOST!!") msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) response = raw_input(msg) if response == "y": - print "Terminating master..." + print("Terminating master...") for inst in master_nodes: inst.terminate() - print "Terminating slaves..." + print("Terminating slaves...") for inst in slave_nodes: inst.terminate() @@ -1245,16 +1276,16 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated' ) - print "Deleting security groups (this will take some time)..." + print("Deleting security groups (this will take some time)...") attempt = 1 while attempt <= 3: - print "Attempt %d" % attempt + print("Attempt %d" % attempt) groups = [g for g in conn.get_all_security_groups() if g.name in group_names] success = True # Delete individual rules in all groups before deleting groups to # remove dependencies between them for group in groups: - print "Deleting rules in security group " + group.name + print("Deleting rules in security group " + group.name) for rule in group.rules: for grant in rule.grants: success &= group.revoke(ip_protocol=rule.ip_protocol, @@ -1269,10 +1300,10 @@ def real_main(): try: # It is needed to use group_id to make it work with VPC conn.delete_security_group(group_id=group.id) - print "Deleted security group %s" % group.name + print("Deleted security group %s" % group.name) except boto.exception.EC2ResponseError: success = False - print "Failed to delete security group %s" % group.name + print("Failed to delete security group %s" % group.name) # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails @@ -1282,18 +1313,21 @@ def real_main(): attempt += 1 if not success: - print "Failed to delete all security groups after 3 tries." - print "Try re-running in a few minutes." + print("Failed to delete all security groups after 3 tries.") + print("Try re-running in a few minutes.") elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - master = master_nodes[0].public_dns_name - print "Logging into master " + master + "..." - proxy_opt = [] - if opts.proxy_port is not None: - proxy_opt = ['-D', opts.proxy_port] - subprocess.check_call( - ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) + if not master_nodes[0].public_dns_name and not opts.private_ips: + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") + else: + master = get_dns_name(master_nodes[0], opts.private_ips) + print("Logging into master " + master + "...") + proxy_opt = [] + if opts.proxy_port is not None: + proxy_opt = ['-D', opts.proxy_port] + subprocess.check_call( + ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) elif action == "reboot-slaves": response = raw_input( @@ -1303,15 +1337,18 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Rebooting slaves..." + print("Rebooting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: - print "Rebooting " + inst.id + print("Rebooting " + inst.id) inst.reboot() elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print master_nodes[0].public_dns_name + if not master_nodes[0].public_dns_name and not opts.private_ips: + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") + else: + print(get_dns_name(master_nodes[0], opts.private_ips)) elif action == "stop": response = raw_input( @@ -1324,11 +1361,11 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Stopping master..." + print("Stopping master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.stop() - print "Stopping slaves..." + print("Stopping slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: if inst.spot_instance_request_id: @@ -1338,11 +1375,11 @@ def real_main(): elif action == "start": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print "Starting slaves..." + print("Starting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - print "Starting master..." + print("Starting master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -1366,15 +1403,15 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: - print >> stderr, "Invalid action: %s" % action + print("Invalid action: %s" % action, file=stderr) sys.exit(1) def main(): try: real_main() - except UsageError, e: - print >> stderr, "\nError:\n", e + except UsageError as e: + print("\nError:\n", e, file=stderr) sys.exit(1) diff --git a/examples/pom.xml b/examples/pom.xml index 7e93f0eec0b91..5b04b4f8d6ca0 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -90,6 +90,12 @@ org.apache.spark spark-streaming-zeromq_${scala.binary.version} ${project.version} + + + org.spark-project.protobuf + protobuf-java + + org.apache.hbase @@ -234,11 +240,12 @@ org.apache.commons commons-math3 + provided com.twitter algebird-core_${scala.binary.version} - 0.8.1 + 0.9.0 org.scalacheck @@ -262,6 +269,22 @@ com.ning compress-lzf + + commons-cli + commons-cli + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-logging + commons-logging + io.netty netty @@ -270,10 +293,22 @@ jline jline + + net.jpountz.lz4 + lz4 + org.apache.cassandra.deps avro + + org.apache.commons + commons-math3 + + + org.apache.thrift + libthrift + @@ -281,6 +316,17 @@ scopt_${scala.binary.version} 3.2.0 + + + + org.scala-lang + scala-library + provided + + @@ -322,12 +368,6 @@ - - - org.apache.commons.math3 - org.spark-project.commons.math3 - - @@ -350,11 +390,6 @@ spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} - - org.apache.httpcomponents - httpclient - ${commons.httpclient.version} - diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index 1c8a20bf8f1ae..11a8cf09533ce 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -41,7 +41,7 @@ object DirectKafkaWordCount { | is a list of one or more Kafka brokers | is a list of one or more kafka topics to consume from | - """".stripMargin) + """.stripMargin) System.exit(1) } 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 19d0eb216848e..eac4f898a475d 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 @@ -28,8 +28,6 @@ import org.apache.spark.ml.classification.ClassificationModel; import org.apache.spark.ml.param.IntParam; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.param.Params; -import org.apache.spark.ml.param.Params$; import org.apache.spark.mllib.linalg.BLAS; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; @@ -100,11 +98,12 @@ public static void main(String[] args) throws Exception { /** * Example of defining a type of {@link Classifier}. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to + * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. + * However, this should still compile and run successfully. */ class MyJavaLogisticRegression - extends Classifier - implements Params { + extends Classifier { /** * Param for max number of iterations @@ -116,7 +115,7 @@ class MyJavaLogisticRegression */ IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - int getMaxIter() { return (Integer) get(maxIter); } + int getMaxIter() { return (Integer) getOrDefault(maxIter); } public MyJavaLogisticRegression() { setMaxIter(100); @@ -129,42 +128,37 @@ MyJavaLogisticRegression setMaxIter(int value) { // This method is used by fit(). // In Java, we have to make it public since Java does not understand Scala's protected modifier. - public MyJavaLogisticRegressionModel train(DataFrame dataset, ParamMap paramMap) { + public MyJavaLogisticRegressionModel train(DataFrame dataset) { // Extract columns from data using helper method. - JavaRDD oldDataset = extractLabeledPoints(dataset, paramMap).toJavaRDD(); + JavaRDD oldDataset = extractLabeledPoints(dataset).toJavaRDD(); // Do learning to estimate the weight vector. int numFeatures = oldDataset.take(1).get(0).features().size(); Vector weights = Vectors.zeros(numFeatures); // Learning would happen here. // Create a model, and return it. - return new MyJavaLogisticRegressionModel(this, paramMap, weights); + return new MyJavaLogisticRegressionModel(this, weights); } } /** * Example of defining a type of {@link ClassificationModel}. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to + * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. + * However, this should still compile and run successfully. */ class MyJavaLogisticRegressionModel - extends ClassificationModel implements Params { + extends ClassificationModel { private MyJavaLogisticRegression parent_; public MyJavaLogisticRegression parent() { return parent_; } - private ParamMap fittingParamMap_; - public ParamMap fittingParamMap() { return fittingParamMap_; } - private Vector weights_; public Vector weights() { return weights_; } - public MyJavaLogisticRegressionModel( - MyJavaLogisticRegression parent_, - ParamMap fittingParamMap_, - Vector weights_) { + public MyJavaLogisticRegressionModel(MyJavaLogisticRegression parent_, Vector weights_) { this.parent_ = parent_; - this.fittingParamMap_ = fittingParamMap_; this.weights_ = weights_; } @@ -208,10 +202,8 @@ public Vector predictRaw(Vector features) { * In Java, we have to make this method public since Java does not understand Scala's protected * modifier. */ - public MyJavaLogisticRegressionModel copy() { - MyJavaLogisticRegressionModel m = - new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); - Params$.MODULE$.inheritValues(this.paramMap(), this, m); - return m; + @Override + public MyJavaLogisticRegressionModel copy(ParamMap extra) { + return copyValues(new MyJavaLogisticRegressionModel(parent_, weights_), extra); } } 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 4e02acce696e6..29158d5c85651 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 @@ -71,7 +71,7 @@ public static void main(String[] args) { // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this // LogisticRegression instance. - System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap()); + System.out.println("Model 1 was fit using parameters: " + model1.parent().extractParamMap()); // We may alternatively specify parameters using a ParamMap. ParamMap paramMap = new ParamMap(); @@ -87,7 +87,7 @@ public static void main(String[] args) { // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); - System.out.println("Model 2 was fit using parameters: " + model2.fittingParamMap()); + System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. List localTest = Lists.newArrayList( diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java index 36207ae38d9a9..fd53c81cc4974 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java @@ -58,7 +58,7 @@ public Tuple2 call(Tuple2 doc_id) { corpus.cache(); // Cluster the documents into three topics using LDA - DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus); + DistributedLDAModel ldaModel = (DistributedLDAModel)new LDA().setK(3).run(corpus); // Output topics. Each is a distribution over words (matching word count vectors) System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() 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 dee794840a3e1..8159ffbe2d269 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 @@ -55,7 +55,7 @@ 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 sqlCtx = new SQLContext(ctx); + SQLContext sqlContext = new SQLContext(ctx); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -74,11 +74,11 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - DataFrame schemaPeople = sqlCtx.createDataFrame(people, Person.class); + DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - DataFrame teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + DataFrame teenagers = sqlContext.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. @@ -99,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. - DataFrame parquetFile = sqlCtx.parquetFile("people.parquet"); + DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers2 = - sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -120,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 - DataFrame peopleFromJsonFile = sqlCtx.jsonFile(path); + DataFrame peopleFromJsonFile = sqlContext.jsonFile(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. @@ -133,8 +133,8 @@ public String call(Row row) { // Register this DataFrame as a table. peopleFromJsonFile.registerTempTable("people"); - // SQL statements can be run by using the sql methods provided by sqlCtx. - DataFrame teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + // SQL statements can be run by using the sql methods provided by sqlContext. + DataFrame teenagers3 = sqlContext.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. @@ -151,7 +151,7 @@ public String call(Row row) { List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - DataFrame peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd()); + DataFrame peopleFromJsonRDD = sqlContext.jsonRDD(anotherPeopleRDD.rdd()); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -164,7 +164,7 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - DataFrame peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); + DataFrame peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 70b6146e39a87..1c3a787bd0e94 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -21,7 +21,8 @@ This example requires numpy (http://www.numpy.org/) """ -from os.path import realpath +from __future__ import print_function + import sys import numpy as np @@ -57,9 +58,9 @@ def update(i, vec, mat, ratings): Usage: als [M] [U] [F] [iterations] [partitions]" """ - print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + 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 - conventional use.""" + conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 @@ -68,8 +69,8 @@ def update(i, vec, mat, ratings): ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ - (M, U, F, ITERATIONS, partitions) + print("Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % + (M, U, F, ITERATIONS, partitions)) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -95,7 +96,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i - print "\nRMSE: %5.4f\n" % error + print("Iteration %d:" % i) + print("\nRMSE: %5.4f\n" % error) sc.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 4626bbb7e3b02..da368ac628a49 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -15,9 +15,12 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext +from functools import reduce """ Read data file users.avro in local Spark distro: @@ -49,7 +52,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 2 and len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: avro_inputformat [reader_schema_file] Run with example jar: @@ -57,7 +60,7 @@ /path/to/examples/avro_inputformat.py [reader_schema_file] Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -77,6 +80,6 @@ conf=conf) output = avro_rdd.map(lambda x: x[0]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 05f34b74df45a..93ca0cfcc9302 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, """ + print(""" Usage: cassandra_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -77,6 +79,6 @@ conf=conf) output = cass_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index d144539e58b8f..5d643eac92f94 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -46,7 +48,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: cassandra_outputformat Run with example jar: @@ -60,7 +62,7 @@ ... fname text, ... lname text ... ); - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3b16010f1cb97..5b82a14fba413 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,15 @@ """ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: hbase_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ - /path/to/examples/hbase_inputformat.py
    + /path/to/examples/hbase_inputformat.py
    [] Assumes you have some data in HBase already, running on , in
    - """ + optionally, you can specify parent znode for your hbase cluster - + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -62,6 +65,9 @@ sc = SparkContext(appName="HBaseInputFormat") conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table} + if len(sys.argv) > 3: + conf = {"hbase.zookeeper.quorum": host, "zookeeper.znode.parent": sys.argv[3], + "hbase.mapreduce.inputtable": table} keyConv = "org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter" valueConv = "org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter" @@ -74,6 +80,6 @@ conf=conf) output = hbase_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index abb425b1f886a..9e5641789a976 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -40,7 +42,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: hbase_outputformat
    Run with example jar: @@ -48,7 +50,7 @@ /path/to/examples/hbase_outputformat.py Assumes you have created
    with column family in HBase running on already - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 86ef6f32c84e8..1456c87312841 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -22,6 +22,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -47,12 +48,12 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + 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.""" + how to use MLlib's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) @@ -67,15 +68,15 @@ def closestPoint(p, centers): closest = data.map( lambda p: (closestPoint(p, kPoints), (p, 1))) pointStats = closest.reduceByKey( - lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) + lambda (p1, c1), (p2, c2): (p1 + p2, c1 + c2)) newPoints = pointStats.map( - lambda (x, (y, z)): (x, y / z)).collect() + lambda st: (st[0], st[1][0] / st[1][1])).collect() - tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) + tempDist = sum(np.sum((kPoints[iK] - p) ** 2) for (iK, p) in newPoints) - for (x, y) in newPoints: - kPoints[x] = y + for (iK, p) in newPoints: + kPoints[iK] = p - print "Final centers: " + str(kPoints) + print("Final centers: " + str(kPoints)) sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 3aa56b0528168..b318b7d87bfdc 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -22,10 +22,8 @@ In practice, one may prefer to use the LogisticRegression algorithm in MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ +from __future__ import print_function -from collections import namedtuple -from math import exp -from os.path import realpath import sys import numpy as np @@ -42,19 +40,19 @@ def readPointBatch(iterator): strs = list(iterator) matrix = np.zeros((len(strs), D + 1)) - for i in xrange(len(strs)): - matrix[i] = np.fromstring(strs[i].replace(',', ' '), dtype=np.float32, sep=' ') + for i, s in enumerate(strs): + matrix[i] = np.fromstring(s.replace(',', ' '), dtype=np.float32, sep=' ') return [matrix] if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + 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.""" + to see how MLlib's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() @@ -62,7 +60,7 @@ def readPointBatch(iterator): # Initialize w to a random value w = 2 * np.random.ranf(size=D) - 1 - print "Initial w: " + str(w) + print("Initial w: " + str(w)) # Compute logistic regression gradient for a matrix of data points def gradient(matrix, w): @@ -76,9 +74,9 @@ def add(x, y): return x for i in range(iterations): - print "On iteration %i" % (i + 1) + print("On iteration %i" % (i + 1)) w -= points.map(lambda m: gradient(m, w)).reduce(add) - print "Final w: " + str(w) + print("Final w: " + str(w)) sc.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 d281f4fa44282..fab21f003b233 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression @@ -33,14 +35,14 @@ if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ + 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() # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. @@ -54,16 +56,16 @@ # 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")]) \ + test = sc.parallelize([(4, "spark i j k"), + (5, "l m n"), + (6, "mapreduce spark"), + (7, "apache hadoop")]) \ .map(lambda x: Document(*x)).toDF() # Make predictions on test documents and print columns of interest. prediction = model.transform(test) selected = prediction.select("id", "text", "prediction") for row in selected.collect(): - print row + print(row) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 4218eca822a99..0e13546b88e67 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -18,6 +18,7 @@ """ Correlations using MLlib. """ +from __future__ import print_function import sys @@ -29,7 +30,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: correlations ()" + print("Usage: correlations ()", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonCorrelations") if len(sys.argv) == 2: @@ -41,20 +42,20 @@ points = MLUtils.loadLibSVMFile(sc, filepath)\ .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) - print - print 'Summary of data file: ' + filepath - print '%d data points' % points.count() + print() + print('Summary of data file: ' + filepath) + print('%d data points' % points.count()) # Statistics (correlations) - print - print 'Correlation (%s) between label and each feature' % corrType - print 'Feature\tCorrelation' + print() + print('Correlation (%s) between label and each feature' % corrType) + print('Feature\tCorrelation') numFeatures = points.take(1)[0].features.size labelRDD = points.map(lambda lp: lp.label) for i in range(numFeatures): featureRDD = points.map(lambda lp: lp.features[i]) corr = Statistics.corr(labelRDD, featureRDD, corrType) - print '%d\t%g' % (i, corr) - print + print('%d\t%g' % (i, corr)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index b5a70db2b9a3c..e23ecc0c5d302 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -19,6 +19,7 @@ An example of how to use DataFrame as a dataset for ML. Run with:: bin/spark-submit examples/src/main/python/mllib/dataset_example.py """ +from __future__ import print_function import os import sys @@ -32,31 +33,31 @@ def summarize(dataset): - print "schema: %s" % dataset.schema().json() + print("schema: %s" % dataset.schema().json()) labels = dataset.map(lambda r: r.label) - print "label average: %f" % labels.mean() + print("label average: %f" % labels.mean()) features = dataset.map(lambda r: r.features) summary = Statistics.colStats(features) - print "features average: %r" % summary.mean() + print("features average: %r" % summary.mean()) if __name__ == "__main__": if len(sys.argv) > 2: - print >> sys.stderr, "Usage: dataset_example.py " + print("Usage: dataset_example.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="DatasetExample") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) if len(sys.argv) == 2: input = sys.argv[1] else: input = "data/mllib/sample_libsvm_data.txt" points = MLUtils.loadLibSVMFile(sc, input) - dataset0 = sqlCtx.inferSchema(points).setName("dataset0").cache() + dataset0 = sqlContext.inferSchema(points).setName("dataset0").cache() summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) - print "Save dataset as a Parquet file to %s." % tempdir + print("Save dataset as a Parquet file to %s." % tempdir) dataset0.saveAsParquetFile(tempdir) - print "Load it back and summarize it again." - dataset1 = sqlCtx.parquetFile(tempdir).setName("dataset1").cache() + print("Load it back and summarize it again.") + dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index fccabd841b139..513ed8fd51450 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import numpy import os @@ -83,18 +84,17 @@ def reindexClassLabels(data): numClasses = len(classCounts) # origToNewLabels: class --> index in 0,...,numClasses-1 if (numClasses < 2): - print >> sys.stderr, \ - "Dataset for classification should have at least 2 classes." + \ - " The given dataset had only %d classes." % numClasses + print("Dataset for classification should have at least 2 classes." + " The given dataset had only %d classes." % numClasses, file=sys.stderr) exit(1) origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) - print "numClasses = %d" % numClasses - print "Per-class example fractions, counts:" - print "Class\tFrac\tCount" + print("numClasses = %d" % numClasses) + print("Per-class example fractions, counts:") + print("Class\tFrac\tCount") for c in sortedClasses: frac = classCounts[c] / (numExamples + 0.0) - print "%g\t%g\t%d" % (c, frac, classCounts[c]) + print("%g\t%g\t%d" % (c, frac, classCounts[c])) if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): return (data, origToNewLabels) @@ -105,8 +105,7 @@ def reindexClassLabels(data): def usage(): - print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]" + print("Usage: decision_tree_runner [libsvm format data filepath]", file=sys.stderr) exit(1) @@ -133,13 +132,13 @@ def usage(): model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. - print "Trained DecisionTree for classification:" - print " Model numNodes: %d" % model.numNodes() - print " Model depth: %d" % model.depth() - print " Training accuracy: %g" % getAccuracy(model, reindexedData) + print("Trained DecisionTree for classification:") + print(" Model numNodes: %d" % model.numNodes()) + print(" Model depth: %d" % model.depth()) + print(" Training accuracy: %g" % getAccuracy(model, reindexedData)) if model.numNodes() < 20: - print model.toDebugString() + print(model.toDebugString()) else: - print model + print(model) sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index a2cd626c9f19d..2cb8010cdc07f 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -18,7 +18,8 @@ """ A Gaussian Mixture Model clustering program using MLlib. """ -import sys +from __future__ import print_function + import random import argparse import numpy as np @@ -59,7 +60,7 @@ def parseVector(line): model = GaussianMixture.train(data, args.k, args.convergenceTol, args.maxIterations, args.seed) for i in range(args.k): - print ("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, - "sigma = ", model.gaussians[i].sigma.toArray()) - print ("Cluster labels (first 100): ", model.predict(data).take(100)) + print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, + "sigma = ", model.gaussians[i].sigma.toArray())) + print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosted_trees.py b/examples/src/main/python/mllib/gradient_boosted_trees.py index e647773ad9060..781bd61c9d2b5 100644 --- a/examples/src/main/python/mllib/gradient_boosted_trees.py +++ b/examples/src/main/python/mllib/gradient_boosted_trees.py @@ -18,6 +18,7 @@ """ Gradient boosted Trees classification and regression using MLlib. """ +from __future__ import print_function import sys @@ -34,7 +35,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() \ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count() \ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification ensemble model:') @@ -49,7 +50,7 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() \ + testMSE = labelsAndPredictions.map(lambda vp: (vp[0] - vp[1]) * (vp[0] - vp[1])).sum() \ / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression ensemble model:') @@ -58,7 +59,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: gradient_boosted_trees" + print("Usage: gradient_boosted_trees", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonGradientBoostedTrees") diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index 2eeb1abeeb12b..002fc75799648 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -34,12 +35,13 @@ def parseVector(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) sc = SparkContext(appName="KMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector) k = int(sys.argv[2]) model = KMeans.train(data, k) - print "Final centers: " + str(model.clusterCenters) + print("Final centers: " + str(model.clusterCenters)) + print("Total Cost: " + str(model.computeCost(data))) sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 8cae27fc4a52d..d4f1d34e2d8cf 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -20,11 +20,10 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function -from math import exp import sys -import numpy as np from pyspark import SparkContext from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import LogisticRegressionWithSGD @@ -42,12 +41,12 @@ def parsePoint(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).map(parsePoint) iterations = int(sys.argv[2]) model = LogisticRegressionWithSGD.train(points, iterations) - print "Final weights: " + str(model.weights) - print "Final intercept: " + str(model.intercept) + print("Final weights: " + str(model.weights)) + print("Final intercept: " + str(model.intercept)) sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py index d3c24f7664329..4cfdad868c66e 100755 --- a/examples/src/main/python/mllib/random_forest_example.py +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -22,6 +22,7 @@ For information on multiclass classification, please refer to the decision_tree_runner.py example. """ +from __future__ import print_function import sys @@ -43,7 +44,7 @@ def testClassification(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count()\ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count()\ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification forest model:') @@ -62,8 +63,8 @@ def testRegression(trainingData, testData): # Evaluate model on test instances and compute test error predictions = model.predict(testData.map(lambda x: x.features)) labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum()\ - / float(testData.count()) + testMSE = labelsAndPredictions.map(lambda v_p1: (v_p1[0] - v_p1[1]) * (v_p1[0] - v_p1[1]))\ + .sum() / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression forest model:') print(model.toDebugString()) @@ -71,7 +72,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: random_forest_example" + print("Usage: random_forest_example", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonRandomForestExample") diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index 1e8892741e714..729bae30b152c 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -18,6 +18,7 @@ """ Randomly generated RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: random_rdd_generation" + print("Usage: random_rdd_generation", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonRandomRDDGeneration") @@ -37,19 +38,19 @@ # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) - print 'Generated RDD of %d examples sampled from the standard normal distribution'\ - % normalRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples sampled from the standard normal distribution' + % normalRDD.count()) + print(' First 5 samples:') for sample in normalRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() # Example: RandomRDDs.normalVectorRDD normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) - print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count()) + print(' First 5 samples:') for sample in normalVectorRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index 92af3af5ebd1e..b7033ab7daeb3 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -18,6 +18,7 @@ """ Randomly sampled RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: sampled_rdds " + print("Usage: sampled_rdds ", file=sys.stderr) exit(-1) if len(sys.argv) == 2: datapath = sys.argv[1] @@ -41,24 +42,24 @@ examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() if numExamples == 0: - print >> sys.stderr, "Error: Data file had no samples to load." + print("Error: Data file had no samples to load.", file=sys.stderr) exit(1) - print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + print('Loaded data with %d examples from file: %s' % (numExamples, datapath)) # Example: RDD.sample() and RDD.takeSample() expectedSampleSize = int(numExamples * fraction) - print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ - % (fraction, expectedSampleSize) + print('Sampling RDD using fraction %g. Expected sample size = %d.' + % (fraction, expectedSampleSize)) sampledRDD = examples.sample(withReplacement=True, fraction=fraction) - print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + print(' RDD.sample(): sample has %d examples' % sampledRDD.count()) sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) - print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + print(' RDD.takeSample(): sample has %d examples' % len(sampledArray)) - print + print() # Example: RDD.sampleByKey() keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) - print ' Keyed data using label (Int) as key ==> Orig' + print(' Keyed data using label (Int) as key ==> Orig') # Count examples per label in original data. keyCountsA = keyedRDD.countByKey() @@ -69,18 +70,18 @@ sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) - print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ - % sizeB + print(' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' + % sizeB) # Compare samples - print ' \tFractions of examples with key' - print 'Key\tOrig\tSample' + print(' \tFractions of examples with key') + print('Key\tOrig\tSample') for k in sorted(keyCountsA.keys()): fracA = keyCountsA[k] / float(numExamples) if sizeB != 0: fracB = keyCountsB.get(k, 0) / float(sizeB) else: fracB = 0 - print '%d\t%g\t%g' % (k, fracA, fracB) + print('%d\t%g\t%g' % (k, fracA, fracB)) sc.stop() diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 99fef4276a369..40d1b887927e0 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -23,6 +23,7 @@ # grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines # This was done so that the example can be run in local mode +from __future__ import print_function import sys @@ -34,7 +35,7 @@ if __name__ == "__main__": if len(sys.argv) < 2: - print USAGE + print(USAGE) sys.exit("Argument for file not provided") file_path = sys.argv[1] sc = SparkContext(appName='Word2Vec') @@ -46,5 +47,5 @@ synonyms = model.findSynonyms('china', 40) for word, cosine_distance in synonyms: - print "{}: {}".format(word, cosine_distance) + print("{}: {}".format(word, cosine_distance)) sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index a5f25d78c1146..2fdc9773d4eb1 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -19,6 +19,7 @@ This is an example implementation of PageRank. For more conventional use, Please refer to PageRank implementation provided by graphx """ +from __future__ import print_function import re import sys @@ -42,11 +43,12 @@ def parseNeighbors(urls): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: pagerank " + print("Usage: pagerank ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of PageRank and is - given as an example! Please refer to PageRank implementation provided by graphx""" + print("""WARN: This is a naive implementation of PageRank and is + given as an example! Please refer to PageRank implementation provided by graphx""", + file=sys.stderr) # Initialize the spark context. sc = SparkContext(appName="PythonPageRank") @@ -62,19 +64,19 @@ def parseNeighbors(urls): links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() # Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - ranks = links.map(lambda (url, neighbors): (url, 1.0)) + ranks = links.map(lambda url_neighbors: (url_neighbors[0], 1.0)) # Calculates and updates URL ranks continuously using PageRank algorithm. - for iteration in xrange(int(sys.argv[2])): + for iteration in range(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. contribs = links.join(ranks).flatMap( - lambda (url, (urls, rank)): computeContribs(urls, rank)) + lambda url_urls_rank: computeContribs(url_urls_rank[1][0], url_urls_rank[1][1])) # Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(add).mapValues(lambda rank: rank * 0.85 + 0.15) # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): - print "%s has rank: %s." % (link, rank) + print("%s has rank: %s." % (link, rank)) sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index fa4c20ab20281..96ddac761d698 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,14 +36,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, """ + print(""" Usage: parquet_inputformat.py Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \\ /path/to/examples/parquet_inputformat.py Assumes you have Parquet data stored in . - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -56,6 +57,6 @@ valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') output = parquet_rdd.map(lambda x: x[1]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index a7c74e969cdb9..92e5cf45abc8b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,7 +36,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) - print "Pi is roughly %f" % (4.0 * count / n) + count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index bb686f17518a0..f6b0ecb02c100 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -22,7 +24,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: sort " + print("Usage: sort ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonSort") lines = sc.textFile(sys.argv[1], 1) @@ -33,6 +35,6 @@ # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() for (num, unitcount) in output: - print num + print(num) sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index 47202fde7510b..2c188759328f2 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -15,11 +15,14 @@ # limitations under the License. # +from __future__ import print_function + import os +import sys from pyspark import SparkContext from pyspark.sql import SQLContext -from pyspark.sql import Row, StructField, StructType, StringType, IntegerType +from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType if __name__ == "__main__": @@ -48,7 +51,11 @@ # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. - path = os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") + if len(sys.argv) < 2: + path = "file://" + \ + os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") + else: + path = sys.argv[1] # Create a DataFrame from the file(s) pointed to by path people = sqlContext.jsonFile(path) # root @@ -68,6 +75,6 @@ teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): - print each[0] + print(each[0]) sc.stop() diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index a33bdc475a06d..49b7902185aaa 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import time import threading import Queue @@ -52,15 +54,15 @@ def run(): ids = status.getJobIdsForGroup() for id in ids: job = status.getJobInfo(id) - print "Job", id, "status: ", job.status + print("Job", id, "status: ", job.status) for sid in job.stageIds: info = status.getStageInfo(sid) if info: - print "Stage %d: %d tasks total (%d active, %d complete)" % \ - (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks) + print("Stage %d: %d tasks total (%d active, %d complete)" % + (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks)) time.sleep(1) - print "Job results are:", result.get() + print("Job results are:", result.get()) sc.stop() if __name__ == "__main__": diff --git a/examples/src/main/python/streaming/direct_kafka_wordcount.py b/examples/src/main/python/streaming/direct_kafka_wordcount.py new file mode 100644 index 0000000000000..6ef188a220c51 --- /dev/null +++ b/examples/src/main/python/streaming/direct_kafka_wordcount.py @@ -0,0 +1,55 @@ +# +# 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. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text directly received from Kafka in every 2 seconds. + Usage: direct_kafka_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first, see + http://kafka.apache.org/documentation.html#quickstart + + and then run the example + `$ bin/spark-submit --jars external/kafka-assembly/target/scala-*/\ + spark-streaming-kafka-assembly-*.jar \ + examples/src/main/python/streaming/direct_kafka_wordcount.py \ + localhost:9092 test` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: direct_kafka_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingDirectKafkaWordCount") + ssc = StreamingContext(sc, 2) + + brokers, topic = sys.argv[1:] + kvs = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index f7ffb5379681e..f815dd26823d1 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -25,6 +25,7 @@ Then create a text file in `localdir` and the words in the file will get counted. """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: hdfs_wordcount.py " + print("Usage: hdfs_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingHDFSWordCount") diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index 51e1ff822fc55..b178e7899b5e1 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -27,6 +27,7 @@ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ +from __future__ import print_function import sys @@ -36,7 +37,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kafka_wordcount.py " + print("Usage: kafka_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingKafkaWordCount") diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py index cfa9c1ff5bfbc..2b48bcfd55db0 100644 --- a/examples/src/main/python/streaming/network_wordcount.py +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -25,6 +25,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: network_wordcount.py " + print("Usage: network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index fc6827c82bf9b..ac91f0a06b172 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -35,6 +35,7 @@ checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from the checkpoint data. """ +from __future__ import print_function import os import sys @@ -46,7 +47,7 @@ def createContext(host, port, outputPath): # If you do not see this printed, that means the StreamingContext has been loaded # from the new checkpoint - print "Creating new context" + print("Creating new context") if os.path.exists(outputPath): os.remove(outputPath) sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") @@ -60,8 +61,8 @@ def createContext(host, port, outputPath): def echo(time, rdd): counts = "Counts at time %s %s" % (time, rdd.collect()) - print counts - print "Appending to " + os.path.abspath(outputPath) + print(counts) + print("Appending to " + os.path.abspath(outputPath)) with open(outputPath, 'a') as f: f.write(counts + "\n") @@ -70,8 +71,8 @@ def echo(time, rdd): if __name__ == "__main__": if len(sys.argv) != 5: - print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ - " " + print("Usage: recoverable_network_wordcount.py " + " ", file=sys.stderr) exit(-1) host, port, checkpoint, output = sys.argv[1:] ssc = StreamingContext.getOrCreate(checkpoint, diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index f89bc562d856b..da90c07dbd82f 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -27,6 +27,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` """ +from __future__ import print_function import os import sys @@ -44,7 +45,7 @@ def getSqlContextInstance(sparkContext): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: sql_network_wordcount.py " + print("Usage: sql_network_wordcount.py ", file=sys.stderr) exit(-1) host, port = sys.argv[1:] sc = SparkContext(appName="PythonSqlNetworkWordCount") @@ -57,7 +58,7 @@ def getSqlContextInstance(sparkContext): # Convert RDDs of the words DStream to DataFrame and run SQL query def process(time, rdd): - print "========= %s =========" % str(time) + print("========= %s =========" % str(time)) try: # Get the singleton instance of SQLContext diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index 18a9a5a452ffb..16ef646b7c42e 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -29,6 +29,7 @@ `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ localhost 9999` """ +from __future__ import print_function import sys @@ -37,7 +38,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: stateful_network_wordcount.py " + print("Usage: stateful_network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingStatefulNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 00a281bfb6506..7bf5fb6ddfe29 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from random import Random @@ -49,20 +51,20 @@ def generateGraph(): # the graph to obtain the path (x, z). # Because join() joins on keys, the edges are stored in reversed order. - edges = tc.map(lambda (x, y): (y, x)) + edges = tc.map(lambda x_y: (x_y[1], x_y[0])) - oldCount = 0L + oldCount = 0 nextCount = tc.count() while True: oldCount = nextCount # Perform the join, obtaining an RDD of (y, (z, x)) pairs, # then project the result to obtain the new (x, z) paths. - new_edges = tc.join(edges).map(lambda (_, (a, b)): (b, a)) + new_edges = tc.join(edges).map(lambda __a_b: (__a_b[1][1], __a_b[1][0])) tc = tc.union(new_edges).distinct().cache() nextCount = tc.count() if nextCount == oldCount: break - print "TC has %i edges" % tc.count() + print("TC has %i edges" % tc.count()) sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index ae6cd13b83d92..7c0143607b61d 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from operator import add @@ -23,7 +25,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: wordcount " + print("Usage: wordcount ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonWordCount") lines = sc.textFile(sys.argv[1], 1) @@ -32,6 +34,6 @@ .reduceByKey(add) output = counts.collect() for (word, count) in output: - print "%s: %i" % (word, count) + print("%s: %i" % (word, count)) sc.stop() diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R new file mode 100644 index 0000000000000..6e6b5cb93789c --- /dev/null +++ b/examples/src/main/r/kmeans.R @@ -0,0 +1,93 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +# Logistic regression in Spark. +# Note: unlike the example in Scala, a point here is represented as a vector of +# doubles. + +parseVectors <- function(lines) { + lines <- strsplit(as.character(lines) , " ", fixed = TRUE) + list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) +} + +dist.fun <- function(P, C) { + apply( + C, + 1, + function(x) { + colSums((t(P) - x)^2) + } + ) +} + +closestPoint <- function(P, C) { + max.col(-dist.fun(P, C)) +} +# Main program + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: kmeans ") + q("no") +} + +sc <- sparkR.init(appName = "RKMeans") +K <- as.integer(args[[2]]) +convergeDist <- as.double(args[[3]]) + +lines <- textFile(sc, args[[1]]) +points <- cache(lapplyPartition(lines, parseVectors)) +# kPoints <- take(points, K) +kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) +tempDist <- 1.0 + +while (tempDist > convergeDist) { + closest <- lapplyPartition( + lapply(points, + function(p) { + cp <- closestPoint(p, kPoints); + mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) + }), + function(x) {do.call(c, x) + }) + + pointStats <- reduceByKey(closest, + function(p1, p2) { + t(colSums(rbind(p1, p2))) + }, + 2L) + + newPoints <- do.call( + rbind, + collect(lapply(pointStats, + function(tup) { + point.sum <- tup[[2]][, -1] + point.count <- tup[[2]][, 1] + point.sum/point.count + }))) + + D <- dist.fun(kPoints, newPoints) + tempDist <- sum(D[cbind(1:3, max.col(-D))]) + kPoints <- newPoints + cat("Finished iteration (delta = ", tempDist, ")\n") +} + +cat("Final centers:\n") +writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R new file mode 100644 index 0000000000000..c864a4232d010 --- /dev/null +++ b/examples/src/main/r/linear_solver_mnist.R @@ -0,0 +1,107 @@ +# +# 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. +# + +# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 + +library(SparkR) +library(Matrix) + +args <- commandArgs(trailing = TRUE) + +# number of random features; default to 1100 +D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) +# number of partitions for training dataset +trainParts <- 12 +# dimension of digits +d <- 784 +# number of test examples +NTrain <- 60000 +# number of training examples +NTest <- 10000 +# scale of features +gamma <- 4e-4 + +sc <- sparkR.init(appName = "SparkR-LinearSolver") + +# You can also use HDFS path to speed things up: +# hdfs:///train-mnist-dense-with-labels.data +file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) + +W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) +b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) +broadcastW <- broadcast(sc, W) +broadcastB <- broadcast(sc, b) + +includePackage(sc, Matrix) +numericLines <- lapplyPartitionsWithIndex(file, + function(split, part) { + matList <- sapply(part, function(line) { + as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) + }, simplify=FALSE) + mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), + sparse=T, byrow=T) + mat + }) + +featureLabels <- cache(lapplyPartition( + numericLines, + function(part) { + label <- part[,1] + mat <- part[,-1] + ones <- rep(1, nrow(mat)) + features <- cos( + mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) + onesMat <- Matrix(ones) + featuresPlus <- cBind(features, onesMat) + labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) + for (i in 1:nrow(mat)) { + labels[i, label[i]] <- 1 + } + list(label=labels, features=featuresPlus) + })) + +FTF <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$features + }), flatten=F)) + +FTY <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$label + }), flatten=F)) + +# solve for the coefficient matrix +C <- solve(FTF, FTY) + +test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", + header=F), sparse=T)) +testData <- test[,-1] +testLabels <- matrix(ncol=1, test[,1]) + +err <- 0 + +# contstruct the feature maps for all examples from this digit +featuresTest <- cos(testData %*% t(value(broadcastW)) + + (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) +featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) + +# extract the one vs. all assignment +results <- featuresTest %*% C +labelsGot <- apply(results, 1, which.max) +err <- sum(testLabels != labelsGot) / nrow(testLabels) + +cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R new file mode 100644 index 0000000000000..2a86aa98160d3 --- /dev/null +++ b/examples/src/main/r/logistic_regression.R @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: logistic_regression ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "LogisticRegressionR") +iterations <- as.integer(args[[2]]) +D <- as.integer(args[[3]]) + +readPartition <- function(part){ + part = strsplit(part, " ", fixed = T) + list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) +} + +# Read data points and convert each partition to a matrix +points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) + +# Initialize w to a random value +w <- runif(n=D, min = -1, max = 1) +cat("Initial w: ", w, "\n") + +# Compute logistic regression gradient for a matrix of data points +gradient <- function(partition) { + partition = partition[[1]] + Y <- partition[, 1] # point labels (first column of input file) + X <- partition[, -1] # point coordinates + + # For each point (x, y), compute gradient function + dot <- X %*% w + logit <- 1 / (1 + exp(-Y * dot)) + grad <- t(X) %*% ((logit - 1) * Y) + list(grad) +} + +for (i in 1:iterations) { + cat("On iteration ", i, "\n") + w <- w - reduce(lapplyPartition(points, gradient), "+") +} + +cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R new file mode 100644 index 0000000000000..aa7a833e147a0 --- /dev/null +++ b/examples/src/main/r/pi.R @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +sc <- sparkR.init(appName = "PiR") + +slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) + +n <- 100000 * slices + +piFunc <- function(elem) { + rands <- runif(n = 2, min = -1, max = 1) + val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) + val +} + + +piFuncVec <- function(elems) { + message(length(elems)) + rands1 <- runif(n = length(elems), min = -1, max = 1) + rands2 <- runif(n = length(elems), min = -1, max = 1) + val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) + sum(val) +} + +rdd <- parallelize(sc, 1:n, slices) +count <- reduce(lapplyPartition(rdd, piFuncVec), sum) +cat("Pi is roughly", 4.0 * count / n, "\n") +cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R new file mode 100644 index 0000000000000..b734cb0ecf55b --- /dev/null +++ b/examples/src/main/r/wordcount.R @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 1) { + print("Usage: wordcount ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "RwordCount") +lines <- textFile(sc, args[[1]]) + +words <- flatMap(lines, + function(line) { + strsplit(line, " ")[[1]] + }) +wordCount <- lapply(words, function(word) { list(word, 1L) }) + +counts <- reduceByKey(wordCount, "+", 2L) +output <- collect(counts) + +for (wordcount in output) { + cat(wordcount[[1]], ": ", wordcount[[2]], "\n") +} diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index f4684b42b5d41..849887d23c9cf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -28,7 +28,19 @@ object HBaseTest { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HBaseTest") val sc = new SparkContext(sparkConf) + + // please ensure HBASE_CONF_DIR is on classpath of spark driver + // e.g: set it through spark.driver.extraClassPath property + // in spark-defaults.conf or through --driver-class-path + // command line option of spark-submit + val conf = HBaseConfiguration.create() + + if (args.length < 1) { + System.err.println("Usage: HBaseTest ") + System.exit(1) + } + // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(0)) 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 17624c20cff3d..04fc0a033014a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -40,8 +40,8 @@ object LocalKMeans { val convergeDist = 0.001 val rand = new Random(42) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DenseVector[Double]] = { + def generatePoint(i: Int): DenseVector[Double] = { DenseVector.fill(D){rand.nextDouble * R} } Array.tabulate(N)(generatePoint) @@ -99,7 +99,7 @@ object LocalKMeans { var pointStats = mappings.map { pair => pair._2.reduceLeft [(Int, (Vector[Double], Int))] { - case ((id1, (x1, y1)), (id2, (x2, y2))) => (id1, (x1 + x2, y1 + y2)) + case ((id1, (p1, c1)), (id2, (p2, c2))) => (id1, (p1 + p2, c1 + c2)) } } 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 92a683ad57ea1..a55e0dc8d36c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -37,8 +37,8 @@ object LocalLR { case class DataPoint(x: Vector[Double], y: Double) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DataPoint] = { + def generatePoint(i: Int): DataPoint = { val y = if(i % 2 == 0) -1 else 1 val x = DenseVector.fill(D){rand.nextGaussian + y * R} DataPoint(x, y) diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 74620ad007d83..32e02eab8b031 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -54,8 +54,8 @@ object LogQuery { // scalastyle:on /** Tracks the total query count and number of aggregate bytes for a particular group. */ class Stats(val count: Int, val numBytes: Int) extends Serializable { - def merge(other: Stats) = new Stats(count + other.count, numBytes + other.numBytes) - override def toString = "bytes=%s\tn=%s".format(numBytes, count) + def merge(other: Stats): Stats = new Stats(count + other.count, numBytes + other.numBytes) + override def toString: String = "bytes=%s\tn=%s".format(numBytes, count) } def extractKey(line: String): (String, String, String) = { 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 48e8d11cdf95b..b514d9123f5e7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -79,7 +79,7 @@ object SparkKMeans { while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - val pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)} + val pointStats = closest.reduceByKey{case ((p1, c1), (p2, c2)) => (p1 + p2, c1 + c2)} val newPoints = pointStats.map {pair => (pair._1, pair._2._1 * (1.0 / pair._2._2))}.collectAsMap() 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 257a7d29f922a..8c01a60844620 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -42,8 +42,8 @@ object SparkLR { case class DataPoint(x: Vector[Double], y: Double) - def generateData = { - def generatePoint(i: Int) = { + def generateData: Array[DataPoint] = { + def generatePoint(i: Int): DataPoint = { val y = if(i % 2 == 0) -1 else 1 val x = DenseVector.fill(D){rand.nextGaussian + y * R} DataPoint(x, y) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index f7f83086df3db..772cd897f5140 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -31,7 +31,7 @@ object SparkTC { val numVertices = 100 val rand = new Random(42) - def generateGraph = { + def generateGraph: Seq[(Int, Int)] = { val edges: mutable.Set[(Int, Int)] = mutable.Set.empty while (edges.size < numEdges) { val from = rand.nextInt(numVertices) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index e322d4ce5a745..ab6e63deb3c95 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -90,7 +90,7 @@ class PRMessage() extends Message[String] with Serializable { } class CustomPartitioner(partitions: Int) extends Partitioner { - def numPartitions = partitions + def numPartitions: Int = partitions def getPartition(key: Any): Int = { val hash = key match { 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 new file mode 100644 index 0000000000000..8340d91101ab3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -0,0 +1,357 @@ +/* + * 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 + +import scala.collection.mutable +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.{VectorIndexer, StringIndexer} +import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.evaluation.{RegressionMetrics, MulticlassMetrics} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.{SQLContext, DataFrame} + + +/** + * 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 + * fails, try running via spark-submit and specifying the amount of memory as at least 1g. + * For local mode, run + * {{{ + * ./bin/spark-submit --class org.apache.spark.examples.ml.DecisionTreeExample --driver-memory 1g + * [examples JAR path] [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DecisionTreeExample { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "Classification", + maxDepth: Int = 5, + maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, + fracTest: Double = 0.2, + cacheNodeIds: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DecisionTreeExample") { + head("DecisionTreeExample: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (classification, regression), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .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, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("cacheNodeIds") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.cacheNodeIds}") + .action((x, c) => c.copy(cacheNodeIds = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + }}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + 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." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = 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) + } + } + + /** Load a dataset from the given path, using the given format */ + private[ml] def loadData( + sc: SparkContext, + path: String, + format: String, + expectedNumFeatures: Option[Int] = None): RDD[LabeledPoint] = { + format match { + case "dense" => MLUtils.loadLabeledPoints(sc, path) + case "libsvm" => expectedNumFeatures match { + case Some(numFeatures) => MLUtils.loadLibSVMFile(sc, path, numFeatures) + case None => MLUtils.loadLibSVMFile(sc, path) + } + case _ => throw new IllegalArgumentException(s"Bad data format: $format") + } + } + + /** + * Load training and test data from files. + * @param input Path to input dataset. + * @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. + * @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) + import sqlContext.implicits._ + + // Load training data + val origExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat) + + // Load or create test set + val splits: Array[RDD[LabeledPoint]] = if (testInput != "") { + // Load testInput. + val numFeatures = origExamples.take(1)(0).features.size + val origTestExamples: RDD[LabeledPoint] = + loadData(sc, testInput, dataFormat, Some(numFeatures)) + Array(origExamples, origTestExamples) + } else { + // Split input into training, test. + origExamples.randomSplit(Array(1.0 - fracTest, fracTest), seed = 12345) + } + + // For classification, convert labels to Strings since we will index them later with + // StringIndexer. + def labelsToStrings(data: DataFrame): DataFrame = { + algo.toLowerCase match { + case "classification" => + data.withColumn("labelString", data("label").cast(StringType)) + case "regression" => + data + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + } + val dataframes = splits.map(_.toDF()).map(labelsToStrings) + val training = dataframes(0).cache() + val test = dataframes(1).cache() + + val numTraining = training.count() + val numTest = test.count() + val numFeatures = training.select("features").first().getAs[Vector](0).size + println("Loaded data:") + println(s" numTraining = $numTraining, numTest = $numTest") + println(s" numFeatures = $numFeatures") + + (training, test) + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"DecisionTreeExample with $params") + val sc = new SparkContext(conf) + params.checkpointDir.foreach(sc.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) + + // Set up Pipeline + val stages = new mutable.ArrayBuffer[PipelineStage]() + // (1) For classification, re-index classes. + val labelColName = if (algo == "classification") "indexedLabel" else "label" + if (algo == "classification") { + val labelIndexer = new StringIndexer() + .setInputCol("labelString") + .setOutputCol(labelColName) + stages += labelIndexer + } + // (2) Identify categorical features using VectorIndexer. + // Features with more than maxCategories values will be treated as continuous. + val featuresIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(10) + stages += featuresIndexer + // (3) Learn Decision Tree + val dt = algo match { + case "classification" => + new DecisionTreeClassifier() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case "regression" => + new DecisionTreeRegressor() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + stages += dt + val pipeline = new Pipeline().setStages(stages.toArray) + + // 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 + algo match { + case "classification" => + val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeClassificationModel] + if (treeModel.numNodes < 20) { + println(treeModel.toDebugString) // Print full model. + } else { + println(treeModel) // Print model summary. + } + case "regression" => + val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeRegressionModel] + if (treeModel.numNodes < 20) { + println(treeModel.toDebugString) // Print full model. + } else { + println(treeModel) // Print model summary. + } + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + // Evaluate model on training, test data + algo match { + case "classification" => + println("Training data results:") + evaluateClassificationModel(pipelineModel, training, labelColName) + println("Test data results:") + evaluateClassificationModel(pipelineModel, test, labelColName) + case "regression" => + println("Training data results:") + evaluateRegressionModel(pipelineModel, training, labelColName) + println("Test data results:") + evaluateRegressionModel(pipelineModel, test, labelColName) + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + sc.stop() + } + + /** + * 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 + * + * TODO: Change model type to ClassificationModel once that API is public. SPARK-5995 + */ + private[ml] def evaluateClassificationModel( + model: Transformer, + data: DataFrame, + labelColName: String): Unit = { + val fullPredictions = model.transform(data).cache() + val predictions = fullPredictions.select("prediction").map(_.getDouble(0)) + val labels = fullPredictions.select(labelColName).map(_.getDouble(0)) + // Print number of classes for reference + val numClasses = MetadataUtils.getNumClasses(fullPredictions.schema(labelColName)) match { + case Some(n) => n + case None => throw new RuntimeException( + "Unknown failure when indexing labels for classification.") + } + val accuracy = new MulticlassMetrics(predictions.zip(labels)).precision + println(s" Accuracy ($numClasses classes): $accuracy") + } + + /** + * 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 + * + * TODO: Change model type to RegressionModel once that API is public. SPARK-5995 + */ + private[ml] def evaluateRegressionModel( + model: Transformer, + data: DataFrame, + labelColName: String): Unit = { + val fullPredictions = model.transform(data).cache() + val predictions = fullPredictions.select("prediction").map(_.getDouble(0)) + val labels = fullPredictions.select(labelColName).map(_.getDouble(0)) + val RMSE = new RegressionMetrics(predictions.zip(labels)).rootMeanSquaredError + println(s" Root mean squared error (RMSE): $RMSE") + } +} 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 df26798e41b7b..2a2d0677272a0 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 @@ package org.apache.spark.examples.ml import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel} -import org.apache.spark.ml.param.{Params, IntParam, ParamMap} +import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} +import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql.{DataFrame, Row, SQLContext} - /** * A simple example demonstrating how to write your own learning algorithm using Estimator, * Transformer, and other abstractions. @@ -99,7 +98,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * 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") - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = $(maxIter) } /** @@ -117,18 +116,16 @@ private class MyLogisticRegression def setMaxIter(value: Int): this.type = set(maxIter, value) // This method is used by fit() - override protected def train( - dataset: DataFrame, - paramMap: ParamMap): MyLogisticRegressionModel = { + override protected def train(dataset: DataFrame): MyLogisticRegressionModel = { // Extract columns from data using helper method. - val oldDataset = extractLabeledPoints(dataset, paramMap) + val oldDataset = extractLabeledPoints(dataset) // Do learning to estimate the weight vector. val numFeatures = oldDataset.take(1)(0).features.size val weights = Vectors.zeros(numFeatures) // Learning would happen here. // Create a model, and return it. - new MyLogisticRegressionModel(this, paramMap, weights) + new MyLogisticRegressionModel(this, weights) } } @@ -139,7 +136,6 @@ private class MyLogisticRegression */ private class MyLogisticRegressionModel( override val parent: MyLogisticRegression, - override val fittingParamMap: ParamMap, val weights: Vector) extends ClassificationModel[Vector, MyLogisticRegressionModel] with MyLogisticRegressionParams { @@ -174,11 +170,9 @@ private class MyLogisticRegressionModel( * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. * - * This is used for the defaul implementation of [[transform()]]. + * This is used for the default implementation of [[transform()]]. */ - override protected def copy(): MyLogisticRegressionModel = { - val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) - Params.inheritValues(this.paramMap, this, m) - m + override def copy(extra: ParamMap): MyLogisticRegressionModel = { + copyValues(new MyLogisticRegressionModel(parent, weights), extra) } } 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 new file mode 100644 index 0000000000000..c5899b6683c79 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -0,0 +1,238 @@ +/* + * 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 + +import scala.collection.mutable +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 + + +/** + * An example runner for decision trees. Run with + * {{{ + * ./bin/run-example ml.GBTExample [options] + * }}} + * 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 + * {{{ + * ./bin/spark-submit --class org.apache.spark.examples.ml.GBTExample --driver-memory 1g + * [examples JAR path] [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object GBTExample { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "classification", + maxDepth: Int = 5, + maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, + maxIter: Int = 10, + fracTest: Double = 0.2, + cacheNodeIds: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("GBTExample") { + head("GBTExample: an example Gradient-Boosted Trees app.") + opt[String]("algo") + .text(s"algorithm (classification, regression), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) + opt[Int]("maxIter") + .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, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("cacheNodeIds") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.cacheNodeIds}") + .action((x, c) => c.copy(cacheNodeIds = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${ + defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + } + }") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + 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." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = 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) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"GBTExample with $params") + val sc = new SparkContext(conf) + params.checkpointDir.foreach(sc.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, + params.dataFormat, params.testInput, algo, params.fracTest) + + // Set up Pipeline + val stages = new mutable.ArrayBuffer[PipelineStage]() + // (1) For classification, re-index classes. + val labelColName = if (algo == "classification") "indexedLabel" else "label" + if (algo == "classification") { + val labelIndexer = new StringIndexer() + .setInputCol("labelString") + .setOutputCol(labelColName) + stages += labelIndexer + } + // (2) Identify categorical features using VectorIndexer. + // Features with more than maxCategories values will be treated as continuous. + val featuresIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(10) + stages += featuresIndexer + // (3) Learn GBT + val dt = algo match { + case "classification" => + new GBTClassifier() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + .setMaxIter(params.maxIter) + case "regression" => + new GBTRegressor() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + .setMaxIter(params.maxIter) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + stages += dt + val pipeline = new Pipeline().setStages(stages.toArray) + + // 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 + algo match { + case "classification" => + val rfModel = pipelineModel.stages.last.asInstanceOf[GBTClassificationModel] + if (rfModel.totalNumNodes < 30) { + println(rfModel.toDebugString) // Print full model. + } else { + println(rfModel) // Print model summary. + } + case "regression" => + val rfModel = pipelineModel.stages.last.asInstanceOf[GBTRegressionModel] + if (rfModel.totalNumNodes < 30) { + println(rfModel.toDebugString) // Print full model. + } else { + println(rfModel) // Print model summary. + } + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + // Evaluate model on training, test data + algo match { + case "classification" => + println("Training data results:") + DecisionTreeExample.evaluateClassificationModel(pipelineModel, training, labelColName) + println("Test data results:") + DecisionTreeExample.evaluateClassificationModel(pipelineModel, test, labelColName) + case "regression" => + println("Training data results:") + DecisionTreeExample.evaluateRegressionModel(pipelineModel, training, labelColName) + println("Test data results:") + DecisionTreeExample.evaluateRegressionModel(pipelineModel, test, labelColName) + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + sc.stop() + } +} 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 new file mode 100644 index 0000000000000..7f88d2681bcaa --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -0,0 +1,246 @@ +/* + * 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 + +import scala.collection.mutable +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 + + +/** + * An example runner for decision trees. Run with + * {{{ + * ./bin/run-example ml.RandomForestExample [options] + * }}} + * 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 + * {{{ + * ./bin/spark-submit --class org.apache.spark.examples.ml.RandomForestExample --driver-memory 1g + * [examples JAR path] [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object RandomForestExample { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "classification", + maxDepth: Int = 5, + maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, + numTrees: Int = 10, + featureSubsetStrategy: String = "auto", + fracTest: Double = 0.2, + cacheNodeIds: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("RandomForestExample") { + head("RandomForestExample: an example random forest app.") + opt[String]("algo") + .text(s"algorithm (classification, regression), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("maxBins") + .text(s"max number of bins, default: ${defaultParams.maxBins}") + .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) + opt[Int]("numTrees") + .text(s"number of trees in ensemble, default: ${defaultParams.numTrees}") + .action((x, c) => c.copy(numTrees = x)) + opt[String]("featureSubsetStrategy") + .text(s"number of features to use per node (supported:" + + s" ${RandomForestClassifier.supportedFeatureSubsetStrategies.mkString(",")})," + + 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, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("cacheNodeIds") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.cacheNodeIds}") + .action((x, c) => c.copy(cacheNodeIds = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${ + defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + } + }") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + 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." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = 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) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"RandomForestExample with $params") + val sc = new SparkContext(conf) + params.checkpointDir.foreach(sc.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, + params.dataFormat, params.testInput, algo, params.fracTest) + + // Set up Pipeline + val stages = new mutable.ArrayBuffer[PipelineStage]() + // (1) For classification, re-index classes. + val labelColName = if (algo == "classification") "indexedLabel" else "label" + if (algo == "classification") { + val labelIndexer = new StringIndexer() + .setInputCol("labelString") + .setOutputCol(labelColName) + stages += labelIndexer + } + // (2) Identify categorical features using VectorIndexer. + // Features with more than maxCategories values will be treated as continuous. + val featuresIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(10) + stages += featuresIndexer + // (3) Learn Random Forest + val dt = algo match { + case "classification" => + new RandomForestClassifier() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + .setFeatureSubsetStrategy(params.featureSubsetStrategy) + .setNumTrees(params.numTrees) + case "regression" => + new RandomForestRegressor() + .setFeaturesCol("indexedFeatures") + .setLabelCol(labelColName) + .setMaxDepth(params.maxDepth) + .setMaxBins(params.maxBins) + .setMinInstancesPerNode(params.minInstancesPerNode) + .setMinInfoGain(params.minInfoGain) + .setCacheNodeIds(params.cacheNodeIds) + .setCheckpointInterval(params.checkpointInterval) + .setFeatureSubsetStrategy(params.featureSubsetStrategy) + .setNumTrees(params.numTrees) + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + stages += dt + val pipeline = new Pipeline().setStages(stages.toArray) + + // 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 + algo match { + case "classification" => + val rfModel = pipelineModel.stages.last.asInstanceOf[RandomForestClassificationModel] + if (rfModel.totalNumNodes < 30) { + println(rfModel.toDebugString) // Print full model. + } else { + println(rfModel) // Print model summary. + } + case "regression" => + val rfModel = pipelineModel.stages.last.asInstanceOf[RandomForestRegressionModel] + if (rfModel.totalNumNodes < 30) { + println(rfModel.toDebugString) // Print full model. + } else { + println(rfModel) // Print model summary. + } + case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + // Evaluate model on training, test data + algo match { + case "classification" => + println("Training data results:") + DecisionTreeExample.evaluateClassificationModel(pipelineModel, training, labelColName) + println("Test data results:") + DecisionTreeExample.evaluateClassificationModel(pipelineModel, test, labelColName) + case "regression" => + println("Training data results:") + DecisionTreeExample.evaluateRegressionModel(pipelineModel, training, labelColName) + println("Test data results:") + DecisionTreeExample.evaluateRegressionModel(pipelineModel, test, labelColName) + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } + + sc.stop() + } +} 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 bf805149d0af6..e8a991f50e338 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 @@ -63,7 +63,7 @@ object SimpleParamsExample { // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this // LogisticRegression instance. - println("Model 1 was fit using parameters: " + model1.fittingParamMap) + println("Model 1 was fit using parameters: " + model1.parent.extractParamMap()) // We may alternatively specify parameters using a ParamMap, // which supports several methods for specifying parameters. @@ -78,7 +78,7 @@ object SimpleParamsExample { // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. val model2 = lr.fit(training.toDF(), paramMapCombined) - println("Model 2 was fit using parameters: " + model2.fittingParamMap) + println("Model 2 was fit using parameters: " + model2.parent.extractParamMap()) // Prepare test data. val test = sc.parallelize(Seq( diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 431ead8c0c165..0763a7736305a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -25,6 +25,7 @@ import org.apache.spark.mllib.tree.GradientBoostedTrees import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo} import org.apache.spark.util.Utils + /** * An example runner for Gradient Boosting using decision trees as weak learners. Run with * {{{ 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 08a93595a2e17..a1850390c0a86 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 @@ -26,7 +26,7 @@ import scopt.OptionParser import org.apache.log4j.{Level, Logger} import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.mllib.clustering.LDA +import org.apache.spark.mllib.clustering.{DistributedLDAModel, LDA} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD @@ -137,7 +137,7 @@ object LDAExample { sc.setCheckpointDir(params.checkpointDir.get) } val startTime = System.nanoTime() - val ldaModel = lda.run(corpus) + val ldaModel = lda.run(corpus).asInstanceOf[DistributedLDAModel] val elapsed = (System.nanoTime() - startTime) / 1e9 println(s"Finished training LDA model. Summary:") diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 1f4ca4fbe7778..99588b0984ab2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -100,7 +100,7 @@ object MovieLensALS { val conf = new SparkConf().setAppName(s"MovieLensALS with $params") if (params.kryo) { conf.registerKryoClasses(Array(classOf[mutable.BitSet], classOf[Rating])) - .set("spark.kryoserializer.buffer.mb", "8") + .set("spark.kryoserializer.buffer", "8m") } val sc = new SparkContext(conf) @@ -178,7 +178,9 @@ object MovieLensALS { def computeRmse(model: MatrixFactorizationModel, data: RDD[Rating], implicitPrefs: Boolean) : Double = { - def mapPredictedRating(r: Double) = if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r + def mapPredictedRating(r: Double): Double = { + if (implicitPrefs) math.max(math.min(r, 1.0), 0.0) else r + } val predictions: RDD[Rating] = model.predict(data.map(x => (x.user, x.product))) val predictionsAndRatings = predictions.map{ x => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 9f22d40c15f3f..6d8b806569dfd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -65,7 +65,7 @@ object PowerIterationClusteringExample { def main(args: Array[String]) { val defaultParams = Params() - val parser = new OptionParser[Params]("PIC Circles") { + val parser = new OptionParser[Params]("PowerIterationClusteringExample") { head("PowerIterationClusteringExample: an example PIC app using concentric circles.") opt[Int]('k', "k") .text(s"number of circles (/clusters), default: ${defaultParams.k}") @@ -76,9 +76,9 @@ object PowerIterationClusteringExample { opt[Int]("maxIterations") .text(s"number of iterations, default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Int]('r', "r") + opt[Double]('r', "r") .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") - .action((x, c) => c.copy(numPoints = x)) + .action((x, c) => c.copy(outerRadius = x)) } parser.parse(args, defaultParams).map { params => @@ -154,3 +154,4 @@ object PowerIterationClusteringExample { coeff * math.exp(expCoeff * ssquares) } } + diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index b433082dce1a2..92867b44be138 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -85,13 +85,13 @@ extends Actor with ActorHelper { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) - override def preStart = remotePublisher ! SubscribeReceiver(context.self) + override def preStart(): Unit = remotePublisher ! SubscribeReceiver(context.self) - def receive = { + def receive: PartialFunction[Any, Unit] = { case msg => store(msg.asInstanceOf[T]) } - override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) + override def postStop(): Unit = remotePublisher ! UnsubscribeReceiver(context.self) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index f40caad322f59..85b9a54b40baf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -56,7 +56,7 @@ object MQTTPublisher { while (true) { try { msgtopic.publish(message) - println(s"Published data. topic: {msgtopic.getName()}; Message: {message}") + println(s"Published data. topic: ${msgtopic.getName()}; Message: $message") } catch { case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => Thread.sleep(10) 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 c3a05c89d817e..751b30ea15782 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 @@ -55,7 +55,8 @@ import org.apache.spark.util.IntParam */ object RecoverableNetworkWordCount { - def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) = { + def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) + : StreamingContext = { // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index 62f49530edb12..c10de84a80ffe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -18,6 +18,7 @@ package org.apache.spark.examples.streaming import com.twitter.algebird._ +import com.twitter.algebird.CMSHasherImplicits._ import org.apache.spark.SparkConf import org.apache.spark.SparkContext._ @@ -67,7 +68,8 @@ object TwitterAlgebirdCMS { val users = stream.map(status => status.getUser.getId) - val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC) + // val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC) + val cms = TopPctCMS.monoid[Long](EPS, DELTA, SEED, PERC) var globalCMS = cms.zero val mm = new MapMonoid[Long, Int]() var globalExact = Map[Long, Int]() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 6510c70bd1866..e99d1baa72b9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -35,7 +35,7 @@ import org.apache.spark.SparkConf */ object SimpleZeroMQPublisher { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { if (args.length < 2) { System.err.println("Usage: SimpleZeroMQPublisher ") System.exit(1) @@ -45,7 +45,7 @@ object SimpleZeroMQPublisher { val acs: ActorSystem = ActorSystem() val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - implicit def stringToByteString(x: String) = ByteString(x) + implicit def stringToByteString(x: String): ByteString = ByteString(x) val messages: List[ByteString] = List("words ", "may ", "count ") while (true) { Thread.sleep(1000) @@ -86,7 +86,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(sparkConf, Seconds(2)) - def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator + def bytesToStringIterator(x: Seq[ByteString]): Iterator[String] = x.map(_.utf8String).iterator // For this stream, a zeroMQ publisher should be running. val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 8402491b62671..54d996b8ac990 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -94,7 +94,7 @@ object PageViewGenerator { while (true) { val socket = listener.accept() new Thread() { - override def run = { + override def run(): Unit = { println("Got client connected from: " + socket.getInetAddress) val out = new PrintWriter(socket.getOutputStream(), true) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 67907bbfb6d1b..1f3e619d97a24 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -35,6 +35,10 @@ http://spark.apache.org/ + + org.apache.commons + commons-lang3 + org.apache.flume flume-ng-sdk diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index 4373be443e67d..fd01807fc3ac4 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable -import org.apache.flume.Channel -import org.apache.commons.lang.RandomStringUtils import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.flume.Channel +import org.apache.commons.lang3.RandomStringUtils /** * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 2a58e99817224..42df8792f147f 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 2de2a7926bfd1..60e2994431b38 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -37,8 +37,7 @@ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver -import org.jboss.netty.channel.ChannelPipelineFactory -import org.jboss.netty.channel.Channels +import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.compression._ @@ -187,8 +186,8 @@ class FlumeReceiver( logInfo("Flume receiver stopped") } - override def preferredLocation = Some(host) - + override def preferredLocation: Option[String] = Option(host) + /** A Netty Pipeline factory that will decompress incoming data from * and the Netty client and compress data going back to the client. * @@ -198,13 +197,12 @@ class FlumeReceiver( */ private[streaming] class CompressionChannelPipelineFactory extends ChannelPipelineFactory { - - def getPipeline() = { + def getPipeline(): ChannelPipeline = { val pipeline = Channels.pipeline() val encoder = new ZlibEncoder(6) pipeline.addFirst("deflater", encoder) pipeline.addFirst("inflater", new ZlibDecoder()) pipeline + } } } -} diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN 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 e04d4088df7dc..43c1b865b64a1 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 @@ -1,21 +1,20 @@ /* - * 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 + * 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 + * 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. + * 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.streaming.flume import java.net.InetSocketAddress @@ -58,11 +57,11 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging before(beforeFunction()) - test("flume polling test") { + ignore("flume polling test") { testMultipleTimes(testFlumePolling) } - test("flume polling test multiple hosts") { + ignore("flume polling test multiple hosts") { testMultipleTimes(testFlumePollingMultipleHost) } @@ -213,7 +212,7 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging assert(counter === totalEventsPerChannel * channels.size) } - def assertChannelIsEmpty(channel: MemoryChannel) = { + def assertChannelIsEmpty(channel: MemoryChannel): Unit = { val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 322de7bf2fed8..39e6754c81dbf 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -28,6 +28,7 @@ import scala.language.postfixOps import com.google.common.base.Charsets import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.commons.lang3.RandomUtils import org.apache.flume.source.avro import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.jboss.netty.channel.ChannelPipeline @@ -40,7 +41,6 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} -import org.apache.spark.streaming.scheduler.{StreamingListener, StreamingListenerReceiverStarted} import org.apache.spark.util.Utils class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { @@ -76,7 +76,8 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L /** Find a free port */ private def findFreePort(): Int = { - Utils.startServiceOnPort(23456, (trialPort: Int) => { + val candidatePort = RandomUtils.nextInt(1024, 65536) + Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) @@ -150,7 +151,9 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L } /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { + private class CompressionChannelFactory(compressionLevel: Int) + extends NioClientSocketChannelFactory { + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { val encoder = new ZlibEncoder(compressionLevel) pipeline.addFirst("deflater", encoder) diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index f695cff410a18..243ce6eaca658 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -44,7 +44,7 @@ org.apache.kafka kafka_${scala.binary.version} - 0.8.1.1 + 0.8.2.1 com.sun.jmx diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 04e65cb3d708c..6715aede7928a 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming.kafka - import scala.annotation.tailrec import scala.collection.mutable import scala.reflect.{classTag, ClassTag} @@ -27,10 +26,10 @@ import kafka.message.MessageAndMetadata import kafka.serializer.Decoder import org.apache.spark.{Logging, SparkException} -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler.InputInfo /** * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where @@ -117,6 +116,11 @@ class DirectKafkaInputDStream[ val rdd = KafkaRDD[K, V, U, T, R]( context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + // Report the record number of this batch interval to InputInfoTracker. + val numRecords = rdd.offsetRanges.map(r => r.untilOffset - r.fromOffset).sum + val inputInfo = InputInfo(id, numRecords) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) Some(rdd) } @@ -129,8 +133,9 @@ class DirectKafkaInputDStream[ private[streaming] class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def batchForTime = data.asInstanceOf[mutable.HashMap[ - Time, Array[OffsetRange.OffsetRangeTuple]]] + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } override def update(time: Time) { batchForTime.clear() diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 2f7e0ab39fefd..6cf254a7b69cb 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -20,9 +20,10 @@ package org.apache.spark.streaming.kafka import scala.util.control.NonFatal import scala.util.Random import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ import java.util.Properties import kafka.api._ -import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} import kafka.consumer.{ConsumerConfig, SimpleConsumer} import org.apache.spark.SparkException @@ -123,9 +124,17 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { val errs = new Err withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => val resp: TopicMetadataResponse = consumer.send(req) - // error codes here indicate missing / just created topic, - // repeating on a different broker wont be useful - return Right(resp.topicsMetadata.toSet) + val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) + + if (respErrs.isEmpty) { + return Right(resp.topicsMetadata.toSet) + } else { + respErrs.foreach { m => + val cause = ErrorMapping.exceptionFor(m.errorCode) + val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" + errs.append(new SparkException(msg, cause)) + } + } } Left(errs) } @@ -212,12 +221,22 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI // scalastyle:on + // this 0 here indicates api version, in this case the original ZK backed api. + private def defaultConsumerApiVersion: Short = 0 + /** Requires Kafka >= 0.8.1.1 */ def getConsumerOffsets( groupId: String, topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, Long]] = + getConsumerOffsets(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short ): Either[Err, Map[TopicAndPartition, Long]] = { - getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r => + getConsumerOffsetMetadata(groupId, topicAndPartitions, consumerApiVersion).right.map { r => r.map { kv => kv._1 -> kv._2.offset } @@ -228,9 +247,16 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { def getConsumerOffsetMetadata( groupId: String, topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = + getConsumerOffsetMetadata(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { var result = Map[TopicAndPartition, OffsetMetadataAndError]() - val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq) + val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq, consumerApiVersion) val errs = new Err withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => val resp = consumer.fetchOffsets(req) @@ -258,24 +284,39 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { def setConsumerOffsets( groupId: String, offsets: Map[TopicAndPartition, Long] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsets(groupId, offsets, defaultConsumerApiVersion) + + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long], + consumerApiVersion: Short ): Either[Err, Map[TopicAndPartition, Short]] = { - setConsumerOffsetMetadata(groupId, offsets.map { kv => - kv._1 -> OffsetMetadataAndError(kv._2) - }) + val meta = offsets.map { kv => + kv._1 -> OffsetAndMetadata(kv._2) + } + setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) } /** Requires Kafka >= 0.8.1.1 */ def setConsumerOffsetMetadata( groupId: String, - metadata: Map[TopicAndPartition, OffsetMetadataAndError] + metadata: Map[TopicAndPartition, OffsetAndMetadata] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsetMetadata(groupId, metadata, defaultConsumerApiVersion) + + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetAndMetadata], + consumerApiVersion: Short ): Either[Err, Map[TopicAndPartition, Short]] = { var result = Map[TopicAndPartition, Short]() - val req = OffsetCommitRequest(groupId, metadata) + val req = OffsetCommitRequest(groupId, metadata, consumerApiVersion) val errs = new Err val topicAndPartitions = metadata.keySet withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => val resp = consumer.commitOffsets(req) - val respMap = resp.requestInfo + val respMap = resp.commitStatus val needed = topicAndPartitions.diff(result.keySet) needed.foreach { tp: TopicAndPartition => respMap.get(tp).foreach { err: Short => diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 4d26b640e8d74..cca0fac0234e1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils /** * Input stream that pulls messages from a Kafka Broker. @@ -111,7 +111,8 @@ class KafkaReceiver[ val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") + val executorPool = + ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") try { // Start the messages handler for each partition topicMessageStreams.values.foreach { streams => diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 6d465bcb6bfc0..a1b4a12e5d6a0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -23,10 +23,9 @@ import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskC import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator -import java.util.Properties import kafka.api.{FetchRequestBuilder, FetchResponse} import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import kafka.consumer.SimpleConsumer import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties @@ -86,7 +85,7 @@ class KafkaRDD[ val part = thePart.asInstanceOf[KafkaRDDPartition] assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) if (part.fromOffset == part.untilOffset) { - log.warn(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + log.info(s"Beginning offset ${part.fromOffset} is the same as ending offset " + s"skipping ${part.topic} ${part.partition}") Iterator.empty } else { @@ -155,7 +154,7 @@ class KafkaRDD[ .dropWhile(_.offset < requestOffset) } - override def close() = consumer.close() + override def close(): Unit = consumer.close() override def getNext(): R = { if (iter == null || !iter.hasNext) { @@ -207,7 +206,7 @@ object KafkaRDD { fromOffsets: Map[TopicAndPartition, Long], untilOffsets: Map[TopicAndPartition, LeaderOffset], messageHandler: MessageAndMetadata[K, V] => R - ): KafkaRDD[K, V, U, T, R] = { + ): KafkaRDD[K, V, U, T, R] = { val leaders = untilOffsets.map { case (tp, lo) => tp -> (lo.host, lo.port) }.toMap diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 0000000000000..6dc4e9517d5a4 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,286 @@ +/* + * 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.streaming.kafka + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap} +import java.util.Properties +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.language.postfixOps +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.common.TopicAndPartition +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.{ZKStringSerializer, ZkUtils} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.I0Itec.zkclient.ZkClient + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 6000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 9092 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, port) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + AdminUtils.createTopic(zkClient, topic, 1, 1) + // wait until metadata is propagated + waitUntilMetadataIsPropagated(topic, 0) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + import scala.collection.JavaConversions._ + sendMessages(topic, Map(messageToFreq.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + /** Wait until the leader offset for the given topic/partition equals the specified offset */ + def waitUntilLeaderOffset( + topic: String, + partition: Int, + offset: Long): Unit = { + eventually(Time(10000), Time(100)) { + val kc = new KafkaCluster(Map("metadata.broker.list" -> brokerAddress)) + val tp = TopicAndPartition(topic, partition) + val llo = kc.getLatestLeaderOffsets(Set(tp)).right.get.apply(tp).offset + assert( + llo == offset, + s"$topic $partition $offset not reached after timeout") + } + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(Time(10000), Time(100)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 5a9bd4214cf51..d7cf500577c2a 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -21,6 +21,7 @@ import java.lang.{Integer => JInt} import java.lang.{Long => JLong} import java.util.{Map => JMap} import java.util.{Set => JSet} +import java.util.{List => JList} import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -30,6 +31,7 @@ import kafka.message.MessageAndMetadata import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.streaming.util.WriteAheadLogUtils import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD @@ -79,7 +81,7 @@ object KafkaUtils { topics: Map[String, Int], storageLevel: StorageLevel ): ReceiverInputDStream[(K, V)] = { - val walEnabled = ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false) + val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) } @@ -234,7 +236,6 @@ object KafkaUtils { new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler) } - /** * Create a RDD from Kafka using offset ranges for each topic and partition. * @@ -558,4 +559,94 @@ private class KafkaUtilsPythonHelper { topics, storageLevel) } + + def createRDD( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaPairRDD[Array[Byte], Array[Byte]] = { + val messageHandler = new JFunction[MessageAndMetadata[Array[Byte], Array[Byte]], + (Array[Byte], Array[Byte])] { + def call(t1: MessageAndMetadata[Array[Byte], Array[Byte]]): (Array[Byte], Array[Byte]) = + (t1.key(), t1.message()) + } + + val jrdd = KafkaUtils.createRDD[ + Array[Byte], + Array[Byte], + DefaultDecoder, + DefaultDecoder, + (Array[Byte], Array[Byte])]( + jsc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + classOf[(Array[Byte], Array[Byte])], + kafkaParams, + offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), + leaders, + messageHandler + ) + new JavaPairRDD(jrdd.rdd) + } + + def createDirectStream( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JLong] + ): JavaPairInputDStream[Array[Byte], Array[Byte]] = { + + if (!fromOffsets.isEmpty) { + import scala.collection.JavaConversions._ + val topicsFromOffsets = fromOffsets.keySet().map(_.topic) + if (topicsFromOffsets != topics.toSet) { + throw new IllegalStateException(s"The specified topics: ${topics.toSet.mkString(" ")} " + + s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") + } + } + + if (fromOffsets.isEmpty) { + KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( + jssc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + kafkaParams, + topics) + } else { + val messageHandler = new JFunction[MessageAndMetadata[Array[Byte], Array[Byte]], + (Array[Byte], Array[Byte])] { + def call(t1: MessageAndMetadata[Array[Byte], Array[Byte]]): (Array[Byte], Array[Byte]) = + (t1.key(), t1.message()) + } + + val jstream = KafkaUtils.createDirectStream[ + Array[Byte], + Array[Byte], + DefaultDecoder, + DefaultDecoder, + (Array[Byte], Array[Byte])]( + jssc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + classOf[(Array[Byte], Array[Byte])], + kafkaParams, + fromOffsets, + messageHandler) + new JavaPairInputDStream(jstream.inputDStream) + } + } + + def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + + def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = + TopicAndPartition(topic, partition) + + def createBroker(host: String, port: JInt): Broker = Broker(host, port) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index c4a44c1822c39..ea87e960379f1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -33,7 +33,7 @@ import org.I0Itec.zkclient.ZkClient import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} -import org.apache.spark.util.Utils +import org.apache.spark.util.ThreadUtils /** * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. @@ -121,7 +121,7 @@ class ReliableKafkaReceiver[ zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) - messageHandlerThreadPool = Utils.newDaemonFixedThreadPool( + messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( topics.values.sum, "KafkaMessageHandler") blockGenerator.start() diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index d6ca6d58b5665..4c1d6a03eb2b8 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -41,24 +41,28 @@ public class JavaDirectKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); } @After public void tearDown() { + if (ssc != null) { ssc.stop(); ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -74,7 +78,7 @@ public void testKafkaStream() throws InterruptedException { sent.addAll(Arrays.asList(topic2data)); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); kafkaParams.put("auto.offset.reset", "smallest"); JavaDStream stream1 = KafkaUtils.createDirectStream( @@ -147,8 +151,8 @@ private HashMap topicOffsetToMap(String topic, Long off private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 4477b81827c70..5cf379635354f 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -37,13 +37,12 @@ public class JavaKafkaRDDSuite implements Serializable { private transient JavaSparkContext sc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); sc = new JavaSparkContext(sparkConf); @@ -51,10 +50,15 @@ public void setUp() { @After public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -66,7 +70,10 @@ public void testKafkaRDD() throws InterruptedException { String[] topic2data = createTopicAndSendData(topic2); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + + kafkaTestUtils.waitUntilLeaderOffset(topic1, 0, topic1data.length); + kafkaTestUtils.waitUntilLeaderOffset(topic2, 0, topic2data.length); OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), @@ -75,7 +82,7 @@ public void testKafkaRDD() throws InterruptedException { HashMap emptyLeaders = new HashMap(); HashMap leaders = new HashMap(); - String[] hostAndPort = suiteBase.brokerAddress().split(":"); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); leaders.put(new TopicAndPartition(topic1, 0), broker); leaders.put(new TopicAndPartition(topic2, 0), broker); @@ -144,8 +151,8 @@ public String call(MessageAndMetadata msgAndMd) throws Exception private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index bad0a93eb2e84..540f4ceabab47 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,9 +22,7 @@ import java.util.List; import java.util.Random; -import scala.Predef; import scala.Tuple2; -import scala.collection.JavaConverters; import kafka.serializer.StringDecoder; import org.junit.After; @@ -44,13 +42,12 @@ public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; private transient Random random = new Random(); - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); ssc = new JavaStreamingContext(sparkConf, new Duration(500)); @@ -58,10 +55,15 @@ public void setUp() { @After public void tearDown() { - ssc.stop(); - ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -75,15 +77,11 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - suiteBase.createTopic(topic); - HashMap tmp = new HashMap(sent); - suiteBase.sendMessages(topic, - JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms()) - ); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, sent); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); + kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress()); kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); @@ -126,6 +124,7 @@ public Void call(JavaPairRDD rdd) throws Exception { ); ssc.start(); + long startTime = System.currentTimeMillis(); boolean sizeMatches = false; while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { @@ -136,6 +135,5 @@ public Void call(JavaPairRDD rdd) throws Exception { for (String k : sent.keySet()) { Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); } - ssc.stop(); } } diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 17ca9d145d665..b6d314dfc7783 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.kafka import java.io.File +import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -27,31 +28,42 @@ import scala.language.postfixOps import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.StringDecoder -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.Utils -class DirectKafkaStreamSuite extends KafkaStreamSuiteBase - with BeforeAndAfter with BeforeAndAfterAll with Eventually { +class DirectKafkaStreamSuite + extends FunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ - var ssc: StreamingContext = _ - var testDir: File = _ + private var sc: SparkContext = _ + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } after { @@ -72,12 +84,12 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topics = Set("basic1", "basic2", "basic3") val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } val totalSent = data.values.sum * topics.size val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) @@ -121,9 +133,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "largest" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -132,7 +144,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() > 3) } @@ -154,7 +166,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -166,9 +178,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "offset" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -177,7 +189,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() >= 10) } @@ -200,7 +212,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -210,18 +222,18 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase // Test to verify the offset ranges can be recovered from the checkpoints test("offset recovery") { val topic = "recovery" - createTopic(topic) + kafkaTestUtils.createTopic(topic) testDir = Utils.createTempDir() val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { val strings = data.map { _.toString} - sendMessages(topic, strings.map { _ -> 1}.toMap) + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) eventually(timeout(10 seconds), interval(50 milliseconds)) { assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) } @@ -280,7 +292,6 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase }, "Recovered ranges are not the same as the ones generated" ) - // 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 ssc.start() @@ -291,6 +302,44 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase ssc.stop() } + test("Direct Kafka stream report input information") { + val topic = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + + val allReceived = new ArrayBuffer[(String, String)] + + stream.foreachRDD { rdd => allReceived ++= rdd.collect() } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + /** Get the generated offset ranges from the DirectKafkaStream */ private def getOffsetRanges[K, V]( kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { @@ -303,4 +352,22 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase object DirectKafkaStreamSuite { val collectedData = new mutable.ArrayBuffer[String]() var total = -1L + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index fc9275b7207be..7fb841b79cb65 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -20,31 +20,41 @@ package org.apache.spark.streaming.kafka import scala.util.Random import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} -class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val topic = "kcsuitetopic" + Random.nextInt(10000) - val topicAndPartition = TopicAndPartition(topic, 0) - var kc: KafkaCluster = null +class KafkaClusterSuite extends FunSuite with BeforeAndAfterAll { + private val topic = "kcsuitetopic" + Random.nextInt(10000) + private val topicAndPartition = TopicAndPartition(topic, 0) + private var kc: KafkaCluster = null + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll() { - setupKafka() - createTopic(topic) - sendMessages(topic, Map("a" -> 1)) - kc = new KafkaCluster(Map("metadata.broker.list" -> s"$brokerAddress")) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) } override def afterAll() { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("metadata apis") { val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) val leaderAddress = s"${leader._1}:${leader._2}" - assert(leaderAddress === brokerAddress, "didn't get leader") + assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") + + val err = kc.getPartitions(Set(topic + "BAD")) + assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") } test("leader offset apis") { diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index a223da70b043f..39c3fb448ff57 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -22,18 +22,22 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext._ -class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ +class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + override def beforeAll { sc = new SparkContext(sparkConf) - - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { @@ -41,18 +45,23 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { sc.stop sc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("basic usage") { - val topic = "topicbasic" - createTopic(topic) + val topic = s"topicbasic-${Random.nextInt}" + kafkaTestUtils.createTopic(topic) val messages = Set("the", "quick", "brown", "fox") - sendMessages(topic, messages.toArray) + kafkaTestUtils.sendMessages(topic, messages.toArray) + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}") - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt(10000)}") + kafkaTestUtils.waitUntilLeaderOffset(topic, 0, messages.size) val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) @@ -65,41 +74,54 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { test("iterator boundary conditions") { // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd - val topic = "topic1" + val topic = s"topicboundary-${Random.nextInt}" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt(10000)}") + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}") val kc = new KafkaCluster(kafkaParams) // this is the "lots of messages" case - sendMessages(topic, sent) + kafkaTestUtils.sendMessages(topic, sent) + val sentCount = sent.values.sum + kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount) + // rdd defined from leaders after sending messages, should get the number sent val rdd = getRdd(kc, Set(topic)) assert(rdd.isDefined) - assert(rdd.get.count === sent.values.sum, "didn't get all sent messages") - val ranges = rdd.get.asInstanceOf[HasOffsetRanges] - .offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap + val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + assert(rangeCount === sentCount, "offset range didn't include all sent messages") + assert(rdd.get.count === sentCount, "didn't get all sent messages") - kc.setConsumerOffsets(kafkaParams("group.id"), ranges) + val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap + + // make sure consumer offsets are committed before the next getRdd call + kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( + err => throw new Exception(err.mkString("\n")), + _ => () + ) // this is the "0 messages" case val rdd2 = getRdd(kc, Set(topic)) // shouldn't get anything, since message is sent after rdd was defined val sentOnlyOne = Map("d" -> 1) - sendMessages(topic, sentOnlyOne) + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount + 1) + assert(rdd2.isDefined) assert(rdd2.get.count === 0, "got messages when there shouldn't be any") // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above val rdd3 = getRdd(kc, Set(topic)) // send lots of messages after rdd was defined, they shouldn't show up - sendMessages(topic, Map("extra" -> 22)) + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) assert(rdd3.isDefined) assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index e4966eebb9b34..24699dfc33adb 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -17,209 +17,38 @@ package org.apache.spark.streaming.kafka -import java.io.File -import java.net.InetSocketAddress -import java.util.Properties - import scala.collection.mutable import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import kafka.admin.AdminUtils -import kafka.common.{KafkaException, TopicAndPartition} -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.{StringDecoder, StringEncoder} -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient.ZkClient -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} -import org.scalatest.{BeforeAndAfter, FunSuite} +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils - -/** - * This is an abstract base class for Kafka testsuites. This has the functionality to set up - * and tear down local Kafka servers, and to push data using Kafka producers. - */ -abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging { - - private val zkHost = "localhost" - private var zkPort: Int = 0 - private val zkConnectionTimeout = 6000 - private val zkSessionTimeout = 6000 - private var zookeeper: EmbeddedZookeeper = _ - private val brokerHost = "localhost" - private var brokerPort = 9092 - private var brokerConf: KafkaConfig = _ - private var server: KafkaServer = _ - private var producer: Producer[String, String] = _ - private var zkReady = false - private var brokerReady = false - - protected var zkClient: ZkClient = _ - - def zkAddress: String = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") - s"$zkHost:$zkPort" - } - def brokerAddress: String = { - assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") - s"$brokerHost:$brokerPort" - } - - def setupKafka() { - // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") - // Get the actual zookeeper binding port - zkPort = zookeeper.actualPort - zkReady = true - logInfo("==================== Zookeeper Started ====================") +class KafkaStreamSuite extends FunSuite with Eventually with BeforeAndAfterAll { + private var ssc: StreamingContext = _ + private var kafkaTestUtils: KafkaTestUtils = _ - zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) - logInfo("==================== Zookeeper Client Created ====================") - - // Kafka broker startup - var bindSuccess: Boolean = false - while(!bindSuccess) { - try { - val brokerProps = getBrokerConfig() - brokerConf = new KafkaConfig(brokerProps) - server = new KafkaServer(brokerConf) - server.startup() - logInfo("==================== Kafka Broker Started ====================") - bindSuccess = true - } catch { - case e: KafkaException => - if (e.getMessage != null && e.getMessage.contains("Socket server failed to bind to")) { - brokerPort += 1 - } - case e: Exception => throw new Exception("Kafka server create failed", e) - } - } - - Thread.sleep(2000) - logInfo("==================== Kafka + Zookeeper Ready ====================") - brokerReady = true + override def beforeAll(): Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } - def tearDownKafka() { - brokerReady = false - zkReady = false - if (producer != null) { - producer.close() - producer = null - } - - if (server != null) { - server.shutdown() - server = null - } - - brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (zookeeper != null) { - zookeeper.shutdown() - zookeeper = null - } - } - - def createTopic(topic: String) { - AdminUtils.createTopic(zkClient, topic, 1, 1) - // wait until metadata is propagated - waitUntilMetadataIsPropagated(topic, 0) - logInfo(s"==================== Topic $topic Created ====================") - } - - def sendMessages(topic: String, messageToFreq: Map[String, Int]) { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - def sendMessages(topic: String, messages: Array[String]) { - producer = new Producer[String, String](new ProducerConfig(getProducerConfig())) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) - producer.close() - logInfo(s"==================== Sent Messages: ${messages.mkString(", ")} ====================") - } - - private def getBrokerConfig(): Properties = { - val props = new Properties() - props.put("broker.id", "0") - props.put("host.name", "localhost") - props.put("port", brokerPort.toString) - props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkAddress) - props.put("log.flush.interval.messages", "1") - props.put("replica.socket.timeout.ms", "1500") - props - } - - private def getProducerConfig(): Properties = { - val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - val props = new Properties() - props.put("metadata.broker.list", brokerAddr) - props.put("serializer.class", classOf[StringEncoder].getName) - props - } - - private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert( - server.apis.metadataCache.containsTopicAndPartition(topic, partition), - s"Partition [$topic, $partition] metadata not propagated after timeout" - ) - } - } - - class EmbeddedZookeeper(val zkConnect: String) { - val random = new Random() - val snapshotDir = Utils.createTempDir() - val logDir = Utils.createTempDir() - - val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) - val (ip, port) = { - val splits = zkConnect.split(":") - (splits(0), splits(1).toInt) - } - val factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress(ip, port), 16) - factory.startup(zookeeper) - - val actualPort = factory.getLocalPort - - def shutdown() { - factory.shutdown() - Utils.deleteRecursively(snapshotDir) - Utils.deleteRecursively(logDir) - } - } -} - - -class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { - var ssc: StreamingContext = _ - - before { - setupKafka() - } - - after { + override def afterAll(): Unit = { if (ssc != null) { ssc.stop() ssc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("Kafka input stream") { @@ -227,10 +56,10 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { ssc = new StreamingContext(sparkConf, Milliseconds(500)) val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) - sendMessages(topic, sent) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> zkAddress, + val kafkaParams = Map("zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}", "auto.offset.reset" -> "smallest") @@ -244,14 +73,14 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { result.put(kv._1, count) } } + ssc.start() + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert(sent.size === result.size) sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } } - ssc.stop() } } - diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 3cd960d1fd1d4..38548dd73b82c 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming.kafka - import java.io.File import scala.collection.mutable @@ -27,7 +26,7 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkConf @@ -35,47 +34,61 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils -class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { +class ReliableKafkaStreamSuite extends FunSuite + with BeforeAndAfterAll with BeforeAndAfter with Eventually { - val sparkConf = new SparkConf() + private val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private var kafkaTestUtils: KafkaTestUtils = _ - var groupId: String = _ - var kafkaParams: Map[String, String] = _ - var ssc: StreamingContext = _ - var tempDirectory: File = null + private var groupId: String = _ + private var kafkaParams: Map[String, String] = _ + private var ssc: StreamingContext = _ + private var tempDirectory: File = null + + override def beforeAll() : Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() - before { - setupKafka() groupId = s"test-consumer-${Random.nextInt(10000)}" kafkaParams = Map( - "zookeeper.connect" -> zkAddress, + "zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> groupId, "auto.offset.reset" -> "smallest" ) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) tempDirectory = Utils.createTempDir() + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDirectory) + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + before { + ssc = new StreamingContext(sparkConf, Milliseconds(500)) ssc.checkpoint(tempDirectory.getAbsolutePath) } after { if (ssc != null) { ssc.stop() + ssc = null } - Utils.deleteRecursively(tempDirectory) - tearDownKafka() } - test("Reliable Kafka input stream with single topic") { - var topic = "test-topic" - createTopic(topic) - sendMessages(topic, data) + val topic = "test-topic" + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === None) @@ -91,6 +104,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } } ssc.start() + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { // A basic process verification for ReliableKafkaReceiver. // Verify whether received message number is equal to the sent message number. @@ -100,14 +114,13 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter // Verify the offset number whether it is equal to the total message number. assert(getCommitOffset(groupId, topic, 0) === Some(29L)) } - ssc.stop() } test("Reliable Kafka input stream with multiple topics") { val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } // Before started, verify all the group/topic/partition offsets are 0. @@ -118,19 +131,18 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() + eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { // Verify the offset for each group/topic to see whether they are equal to the expected one. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } } - ssc.stop() } /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { - assert(zkClient != null, "Zookeeper client is not initialized") val topicDirs = new ZKGroupTopicDirs(groupId, topic) val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) + ZkUtils.readDataMaybeNull(kafkaTestUtils.zookeeperClient, zkPath)._1.map(_.toLong) } } diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 0f3298af6234a..a19a72c58a705 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import org.apache.activemq.broker.{TransportConnector, BrokerService} +import org.apache.commons.lang3.RandomUtils import org.eclipse.paho.client.mqttv3._ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence @@ -113,7 +114,8 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { } private def findFreePort(): Int = { - Utils.startServiceOnPort(23456, (trialPort: Int) => { + val candidatePort = RandomUtils.nextInt(1024, 65536) + Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) @@ -137,7 +139,8 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { msgTopic.publish(message) } catch { case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - Thread.sleep(50) // wait for Spark streaming to consume something from the message queue + // wait for Spark streaming to consume something from the message queue + Thread.sleep(50) } } } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 4eacc47da5699..7cf02d85d73d3 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -70,7 +70,7 @@ class TwitterReceiver( try { val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) newTwitterStream.addListener(new StatusListener { - def onStatus(status: Status) = { + def onStatus(status: Status): Unit = { store(status) } // Unimplemented diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 64bfc5745088f..9a3569789d2e0 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 554705878ee78..588e6bac7b14a 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -29,13 +29,16 @@ import org.apache.spark.streaming.receiver.ActorHelper /** * A receiver to subscribe to ZeroMQ stream. */ -private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T]) +private[streaming] class ZeroMQReceiver[T: ClassTag]( + publisherUrl: String, + subscribe: Subscribe, + bytesToObjects: Seq[ByteString] => Iterator[T]) extends Actor with ActorHelper with Logging { - override def preStart() = ZeroMQExtension(context.system) - .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) + override def preStart(): Unit = { + ZeroMQExtension(context.system) + .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) + } def receive: Receive = { diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties index 97348fb5b6123..6cdc9286c5d76 100644 --- a/extras/kinesis-asl/src/main/resources/log4j.properties +++ b/extras/kinesis-asl/src/main/resources/log4j.properties @@ -31,7 +31,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 1bd1f324298e7..a7fe4476cacb8 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -23,6 +23,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.Duration import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils import com.amazonaws.auth.AWSCredentialsProvider import com.amazonaws.auth.DefaultAWSCredentialsProviderChain @@ -118,7 +119,7 @@ private[kinesis] class KinesisReceiver( * method. */ override def onStart() { - workerId = InetAddress.getLocalHost.getHostAddress() + ":" + UUID.randomUUID() + workerId = Utils.localHostName() + ":" + UUID.randomUUID() credentialsProvider = new DefaultAWSCredentialsProviderChain() kinesisClientLibConfiguration = new KinesisClientLibConfiguration(appName, streamName, credentialsProvider, workerId).withKinesisEndpoint(endpointUrl) diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index d8be02e2023d5..23430179f12ec 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -62,7 +62,6 @@ object EdgeContext { * , _ + _) * }}} */ - def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]) = + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] = Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) } - diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 6f03eb1439773..058c8c8aa1b24 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -34,12 +34,12 @@ class EdgeDirection private (private val name: String) extends Serializable { override def toString: String = "EdgeDirection." + name - override def equals(o: Any) = o match { + override def equals(o: Any): Boolean = o match { case other: EdgeDirection => other.name == name case _ => false } - override def hashCode = name.hashCode + override def hashCode: Int = name.hashCode } @@ -48,14 +48,14 @@ class EdgeDirection private (private val name: String) extends Serializable { */ object EdgeDirection { /** Edges arriving at a vertex. */ - final val In = new EdgeDirection("In") + final val In: EdgeDirection = new EdgeDirection("In") /** Edges originating from a vertex. */ - final val Out = new EdgeDirection("Out") + final val Out: EdgeDirection = new EdgeDirection("Out") /** Edges originating from *or* arriving at a vertex of interest. */ - final val Either = new EdgeDirection("Either") + final val Either: EdgeDirection = new EdgeDirection("Either") /** Edges originating from *and* arriving at a vertex of interest. */ - final val Both = new EdgeDirection("Both") + final val Both: EdgeDirection = new EdgeDirection("Both") } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index 9d473d5ebda44..c8790cac3d8a0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -62,7 +62,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { def vertexAttr(vid: VertexId): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } - override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + override def toString: String = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 8494d06b1cdb7..36dc7b0f86c89 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -409,7 +409,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * {{{ * val rawGraph: Graph[_, _] = Graph.textFile("twittergraph") * val inDeg: RDD[(VertexId, Int)] = - * aggregateMessages[Int](ctx => ctx.sendToDst(1), _ + _) + * rawGraph.aggregateMessages[Int](ctx => ctx.sendToDst(1), _ + _) * }}} * * @note By expressing computation at the edge level we achieve diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index dc8b4789c4b61..7edd627b20918 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -113,7 +113,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Collect the neighbor vertex attributes for each vertex. * * @note This function could be highly inefficient on power-law - * graphs where high degree vertices may force a large ammount of + * graphs where high degree vertices may force a large amount of * information to be collected to a single location. * * @param edgeDirection the direction along which to collect @@ -187,7 +187,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali /** * Join the vertices with an RDD and then apply a function from the - * the vertex and RDD entry to a new vertex value. The input table + * vertex and RDD entry to a new vertex value. The input table * should contain at most one entry for each vertex. If no entry is * provided the map function is skipped and the old value is used. * @@ -372,6 +372,31 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali PageRank.runUntilConvergence(graph, tol, resetProb) } + + /** + * Run personalized PageRank for a given vertex, such that all random walks + * are started relative to the source node. + * + * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergenceWithOptions]] + */ + def personalizedPageRank(src: VertexId, tol: Double, + resetProb: Double = 0.15) : Graph[Double, Double] = { + PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) + } + + /** + * Run Personalized PageRank for a fixed number of iterations with + * with all iterations originating at the source node + * returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see [[org.apache.spark.graphx.lib.PageRank$#runWithOptions]] + */ + def staticPersonalizedPageRank(src: VertexId, numIter: Int, + resetProb: Double = 0.15) : Graph[Double, Double] = { + PageRank.runWithOptions(graph, numIter, resetProb, Some(src)) + } + /** * Run PageRank for a fixed number of iterations returning a graph with vertex attributes * containing the PageRank and edge attributes the normalized edge weight. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 5e55620147df8..01b013ff716fc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -78,8 +78,8 @@ object Pregel extends Logging { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the on - * the first iteration + * @param initialMsg the message each vertex will receive at the first + * iteration * * @param maxIterations the maximum number of iterations to run for * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index ad4bfe077293a..a9f04b559c3d1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -121,6 +121,22 @@ abstract class VertexRDD[VD]( */ def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other an RDD to run the set operation against + */ + def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] + + /** + * For each VertexId present in both `this` and `other`, minus will act as a set difference + * operation returning only those unique VertexId's present in `this`. + * + * @param other a VertexRDD to run the set operation against + */ + def minus(other: VertexRDD[VD]): VertexRDD[VD] + /** * For each vertex present in both `this` and `other`, `diff` returns only those vertices with * differing values; for values that are different, keeps the values from `other`. This is diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 373af75448374..c561570809253 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -324,7 +324,7 @@ class EdgePartition[ * * @return an iterator over edges in the partition */ - def iterator = new Iterator[Edge[ED]] { + def iterator: Iterator[Edge[ED]] = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] private[this] var pos = 0 @@ -351,7 +351,7 @@ class EdgePartition[ override def hasNext: Boolean = pos < EdgePartition.this.size - override def next() = { + override def next(): EdgeTriplet[VD, ED] = { val triplet = new EdgeTriplet[VD, ED] val localSrcId = localSrcIds(pos) val localDstId = localDstIds(pos) @@ -518,11 +518,11 @@ private class AggregatingEdgeContext[VD, ED, A]( _attr = attr } - override def srcId = _srcId - override def dstId = _dstId - override def srcAttr = _srcAttr - override def dstAttr = _dstAttr - override def attr = _attr + override def srcId: VertexId = _srcId + override def dstId: VertexId = _dstId + override def srcAttr: VD = _srcAttr + override def dstAttr: VD = _dstAttr + override def attr: ED = _attr override def sendToSrc(msg: A) { send(_localSrcId, msg) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 43a3aea0f6196..c88b2f65a86cd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -70,9 +70,9 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } - override def getStorageLevel = partitionsRDD.getStorageLevel + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel - override def checkpoint() = { + override def checkpoint(): Unit = { partitionsRDD.checkpoint() } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 8ab255bd4038c..1df86449fa0c2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -50,7 +50,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped to * match. */ - def reverse() = { + def reverse(): ReplicatedVertexView[VD, ED] = { val newEdges = edges.mapEdgePartitions((pid, part) => part.reverse) new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 4fd2548b7faf6..b90f9fa327052 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -88,6 +88,21 @@ private[graphx] abstract class VertexPartitionBaseOps this.withMask(newMask) } + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Minus operations on two VertexPartitions with different indexes is slow.") + minus(createUsingIndex(other.iterator)) + } else { + self.withMask(self.mask.andNot(other.mask)) + } + } + + /** Hides the VertexId's that are the same between `this` and `other`. */ + def minus(other: Iterator[(VertexId, VD)]): Self[VD] = { + minus(createUsingIndex(other)) + } + /** * Hides vertices that are the same between this and other. For vertices that are different, keeps * the values from `other`. The indices of `this` and `other` must be the same. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 125692ddaad83..33ac7b0ed6095 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -71,9 +71,9 @@ class VertexRDDImpl[VD] private[graphx] ( this } - override def getStorageLevel = partitionsRDD.getStorageLevel + override def getStorageLevel: StorageLevel = partitionsRDD.getStorageLevel - override def checkpoint() = { + override def checkpoint(): Unit = { partitionsRDD.checkpoint() } @@ -103,6 +103,31 @@ class VertexRDDImpl[VD] private[graphx] ( override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + override def minus(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + minus(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) + } + + override def minus (other: VertexRDD[VD]): VertexRDD[VD] = { + other match { + case other: VertexRDD[_] if this.partitioner == other.partitioner => + this.withPartitionsRDD[VD]( + partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.minus(otherPart)) + }) + case _ => + this.withPartitionsRDD[VD]( + partitionsRDD.zipPartitions( + other.partitionBy(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.minus(msgs)) + } + ) + } + } + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { diff(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index e2f6cc138958e..859f896039047 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -37,7 +37,7 @@ object ConnectedComponents { */ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(edge: EdgeTriplet[VertexId, ED]) = { + def sendMessage(edge: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, VertexId)] = { if (edge.srcAttr < edge.dstAttr) { Iterator((edge.dstId, edge.srcAttr)) } else if (edge.srcAttr > edge.dstAttr) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala index 82e9e06515179..2bcf8684b8b8e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -43,7 +43,7 @@ object LabelPropagation { */ def run[VD, ED: ClassTag](graph: Graph[VD, ED], maxSteps: Int): Graph[VertexId, ED] = { val lpaGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(e: EdgeTriplet[VertexId, ED]) = { + def sendMessage(e: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, Map[VertexId, VertexId])] = { Iterator((e.srcId, Map(e.dstAttr -> 1L)), (e.dstId, Map(e.srcAttr -> 1L))) } def mergeMessage(count1: Map[VertexId, Long], count2: Map[VertexId, Long]) @@ -54,7 +54,7 @@ object LabelPropagation { i -> (count1Val + count2Val) }.toMap } - def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]) = { + def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]): VertexId = { if (message.isEmpty) attr else message.maxBy(_._2)._1 } val initialMessage = Map[VertexId, Long]() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index e139959c3f5c1..bc974b2f04e70 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx.lib import scala.reflect.ClassTag +import scala.language.postfixOps import org.apache.spark.Logging import org.apache.spark.graphx._ @@ -25,8 +26,8 @@ import org.apache.spark.graphx._ /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. * - * The first implementation uses the [[Pregel]] interface and runs PageRank for a fixed number - * of iterations: + * The first implementation uses the standalone [[Graph]] interface and runs PageRank + * for a fixed number of iterations: * {{{ * var PR = Array.fill(n)( 1.0 ) * val oldPR = Array.fill(n)( 1.0 ) @@ -38,7 +39,7 @@ import org.apache.spark.graphx._ * } * }}} * - * The second implementation uses the standalone [[Graph]] interface and runs PageRank until + * The second implementation uses the [[Pregel]] interface and runs PageRank until * convergence: * * {{{ @@ -60,6 +61,7 @@ import org.apache.spark.graphx._ */ object PageRank extends Logging { + /** * Run PageRank for a fixed number of iterations returning a graph * with vertex attributes containing the PageRank and edge @@ -74,10 +76,33 @@ object PageRank extends Logging { * * @return the graph containing with each vertex containing the PageRank and each edge * containing the normalized weight. + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int, + resetProb: Double = 0.15): Graph[Double, Double] = + { + runWithOptions(graph, numIter, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph + * with vertex attributes containing the PageRank and edge + * attributes the normalized edge weight. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * @param srcId the source vertex for a Personalized Page Rank (optional) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. * */ - def run[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = + def runWithOptions[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15, + srcId: Option[VertexId] = None): Graph[Double, Double] = { // Initialize the PageRank graph with each edge attribute having // weight 1/outDegree and each vertex with attribute 1.0. @@ -89,6 +114,10 @@ object PageRank extends Logging { // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => resetProb ) + val personalized = srcId isDefined + val src: VertexId = srcId.getOrElse(-1L) + def delta(u: VertexId, v: VertexId):Double = { if (u == v) 1.0 else 0.0 } + var iteration = 0 var prevRankGraph: Graph[Double, Double] = null while (iteration < numIter) { @@ -103,8 +132,14 @@ object PageRank extends Logging { // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the // edge partitions. prevRankGraph = rankGraph + val rPrb = if (personalized) { + (src: VertexId ,id: VertexId) => resetProb * delta(src,id) + } else { + (src: VertexId, id: VertexId) => resetProb + } + rankGraph = rankGraph.joinVertices(rankUpdates) { - (id, oldRank, msgSum) => resetProb + (1.0 - resetProb) * msgSum + (id, oldRank, msgSum) => rPrb(src,id) + (1.0 - resetProb) * msgSum }.cache() rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices @@ -133,7 +168,29 @@ object PageRank extends Logging { * containing the normalized weight. */ def runUntilConvergence[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = + { + runUntilConvergenceWithOptions(graph, tol, resetProb) + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param tol the tolerance allowed at convergence (smaller => more accurate). + * @param resetProb the random reset probability (alpha) + * @param srcId the source vertex for a Personalized Page Rank (optional) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + */ + def runUntilConvergenceWithOptions[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15, + srcId: Option[VertexId] = None): Graph[Double, Double] = { // Initialize the pagerankGraph with each edge attribute // having weight 1/outDegree and each vertex with attribute 1.0. @@ -148,6 +205,10 @@ object PageRank extends Logging { .mapVertices( (id, attr) => (0.0, 0.0) ) .cache() + val personalized = srcId.isDefined + val src: VertexId = srcId.getOrElse(-1L) + + // Define the three functions needed to implement PageRank in the GraphX // version of Pregel def vertexProgram(id: VertexId, attr: (Double, Double), msgSum: Double): (Double, Double) = { @@ -156,6 +217,17 @@ object PageRank extends Logging { (newPR, newPR - oldPR) } + def personalizedVertexProgram(id: VertexId, attr: (Double, Double), + msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + var teleport = oldPR + val delta = if (src==id) 1.0 else 0.0 + teleport = oldPR*delta + + val newPR = teleport + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) + } + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { if (edge.srcAttr._2 > tol) { Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) @@ -170,8 +242,17 @@ object PageRank extends Logging { val initialMessage = resetProb / (1.0 - resetProb) // Execute a dynamic version of Pregel. + val vp = if (personalized) { + (id: VertexId, attr: (Double, Double),msgSum: Double) => + personalizedVertexProgram(id, attr, msgSum) + } else { + (id: VertexId, attr: (Double, Double), msgSum: Double) => + vertexProgram(id, attr, msgSum) + } + Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)( - vertexProgram, sendMessage, messageCombiner) + vp, sendMessage, messageCombiner) .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank + } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 1a7178b82e3af..3b0e1628d86b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -93,7 +93,7 @@ object SVDPlusPlus { val gJoinT0 = g.outerJoinVertices(t0) { (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => - (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + (vd._1, vd._2, msg.get._2 / msg.get._1 - u, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() materialize(gJoinT0) g.unpersist() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 57b01b6f2e1fb..e2754ea699da9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -56,7 +56,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, private var _oldValues: Array[V] = null - override def size = keySet.size + override def size: Int = keySet.size /** Get the value for a given key */ def apply(k: K): V = { @@ -112,7 +112,7 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - override def iterator = new Iterator[(K, V)] { + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { var pos = 0 var nextPair: (K, V) = computeNextPair() @@ -128,9 +128,9 @@ class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, } } - def hasNext = nextPair != null + def hasNext: Boolean = nextPair != null - def next() = { + def next(): (K, V) = { val pair = nextPair nextPair = computeNextPair() pair diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 8d15150458d26..a570e4ed75fc3 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -38,12 +38,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { val doubleRing = ring ++ ring val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) assert(graph.edges.count() === doubleRing.size) - assert(graph.edges.collect.forall(e => e.attr == 1)) + assert(graph.edges.collect().forall(e => e.attr == 1)) // uniqueEdges option should uniquify edges and store duplicate count in edge attributes val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) assert(uniqueGraph.edges.count() === ring.size) - assert(uniqueGraph.edges.collect.forall(e => e.attr == 2)) + assert(uniqueGraph.edges.collect().forall(e => e.attr == 2)) } } @@ -64,7 +64,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.collect().map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } @@ -75,15 +75,17 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === - (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) + assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect().toSet + === (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) } } test("partitionBy") { withSpark { sc => - def mkGraph(edges: List[(Long, Long)]) = Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) - def nonemptyParts(graph: Graph[Int, Int]) = { + def mkGraph(edges: List[(Long, Long)]): Graph[Int, Int] = { + Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + } + def nonemptyParts(graph: Graph[Int, Int]): RDD[List[Edge[Int]]] = { graph.edges.partitionsRDD.mapPartitions { iter => Iterator(iter.next()._2.iterator.toList) }.filter(_.nonEmpty) @@ -102,7 +104,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1) // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into // the same partition - assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) + assert( + nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) // partitionBy(EdgePartition2D) puts identical edges in the same partition assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1) @@ -140,10 +143,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val g = Graph( sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))), sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2)) - assert(g.triplets.collect.map(_.toTuple).toSet === + assert(g.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) val gPart = g.partitionBy(EdgePartition2D) - assert(gPart.triplets.collect.map(_.toTuple).toSet === + assert(gPart.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) } } @@ -154,10 +157,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = starGraph(sc, n) // mapVertices preserving type val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") - assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) + assert(mappedVAttrs.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) // mapVertices changing type val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) - assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) + assert(mappedVAttrs2.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -177,12 +180,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { // Trigger initial vertex replication graph0.triplets.foreach(x => {}) // Change type of replicated vertices, but preserve erased type - val graph1 = graph0.mapVertices { - case (vid, integerOpt) => integerOpt.map((x: java.lang.Integer) => (x.toDouble): java.lang.Double) + val graph1 = graph0.mapVertices { case (vid, integerOpt) => + integerOpt.map((x: java.lang.Integer) => x.toDouble: java.lang.Double) } // Access replicated vertices, exposing the erased type val graph2 = graph1.mapTriplets(t => t.srcAttr.get) - assert(graph2.edges.map(_.attr).collect.toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) + assert(graph2.edges.map(_.attr).collect().toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) } } @@ -202,7 +205,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect.toSet === + assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect().toSet === (1L to n).map(x => Edge(0, x, "vv")).toSet) } } @@ -211,7 +214,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) + assert(star.reverse.outDegrees.collect().toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -221,7 +224,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) val graph = Graph(vertices, edges).reverse val result = graph.mapReduceTriplets[Int](et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(result.collect.toSet === Set((1L, 2))) + assert(result.collect().toSet === Set((1L, 2))) } } @@ -237,7 +240,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) // And 4 edges. - assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) + assert(subgraph.edges.map(_.copy()).collect().toSet === + (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } @@ -273,9 +277,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 to n).flatMap(x => List((0: VertexId, x: VertexId), (0: VertexId, x: VertexId))), 1), "v") val star2 = doubleStar.groupEdges { (a, b) => a} - assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === - star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) - assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) + assert(star2.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect().toSet === star.vertices.collect().toSet) } } @@ -300,21 +304,23 @@ class GraphSuite extends FunSuite with LocalSparkContext { throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) } Iterator((et.srcId, 1)) - }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect().toSet assert(numEvenNeighbors === (1 to n).map(x => (x: VertexId, n / 2)).toSet) // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x+1) % n: VertexId)), 3) + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x + 1) % n: VertexId)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() - val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => + newOpt.getOrElse(old) + } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set if (et.srcId % 2 != 1) { throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) } Iterator((et.dstId, 1)) - }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect().toSet assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexId, 1)).toSet) } @@ -340,17 +346,18 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type - val reverseStarDegrees = - reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { + (vid, a, bOpt) => bOpt.getOrElse(0) + } val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), - (a: Int, b: Int) => a + b).collect.toSet + (a: Int, b: Int) => a + b).collect().toSet assert(neighborDegreeSums === Set((0: VertexId, n)) ++ (1 to n).map(x => (x: VertexId, 0))) // outerJoinVertices preserving type val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } val newReverseStar = reverseStar.outerJoinVertices(messages) { (vid, a, bOpt) => a + bOpt.getOrElse("") } - assert(newReverseStar.vertices.map(_._2).collect.toSet === + assert(newReverseStar.vertices.map(_._2).collect().toSet === (0 to n).map(x => "v%d".format(x)).toSet) } } @@ -361,7 +368,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) val graph = Graph(verts, edges) val triplets = graph.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)) - .collect.toSet + .collect().toSet assert(triplets === Set((1: VertexId, 2: VertexId, "a", "b"), (2: VertexId, 1: VertexId, "b", "a"))) } @@ -417,7 +424,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val neighborAttrSums = graph.mapReduceTriplets[Int]( et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) + assert(neighborAttrSums.collect().toSet === Set((0: VertexId, n))) } finally { sc.stop() } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index a3e28efc75a98..d2ad9be555770 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkContext */ trait LocalSparkContext { /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ - def withSpark[T](f: SparkContext => T) = { + def withSpark[T](f: SparkContext => T): T = { val conf = new SparkConf() GraphXUtils.registerKryoClasses(conf) val sc = new SparkContext("local", "test", conf) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 4f7a442ab503d..d0a7198d691d7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { - def vertices(sc: SparkContext, n: Int) = { + private def vertices(sc: SparkContext, n: Int) = { VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) } @@ -47,6 +47,35 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("minus") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB = VertexRDD(sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1))).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) + } + } + + test("minus with RDD[(VertexId, VD)]") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() + val vertexB: RDD[(VertexId, Int)] = + sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1)).cache() + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) + } + } + + test("minus with non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 75, 5).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(50 until 100, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.minus(vertexB) + assert(vertexC.map(_._1).collect().toSet === (0 until 50).toSet) + } + } + test("diff") { withSpark { sc => val n = 100 @@ -71,13 +100,13 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } - test("diff vertices with the non-equal number of partitions") { + test("diff vertices with non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) assert(vertexA.partitions.size != vertexB.partitions.size) val vertexC = vertexA.diff(vertexB) - assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + assert(vertexC.map(_._1).collect().toSet === (8 until 16).toSet) } } @@ -87,16 +116,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD - assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) // leftJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) } } - test("leftJoin vertices with the non-equal number of partitions") { + test("leftJoin vertices with non-equal number of partitions") { withSpark { sc => val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) val vertexB = VertexRDD( @@ -105,7 +134,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => old - newOpt.getOrElse(0) } - assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect().toSet == (1 to 99 by 2).toSet) } } @@ -115,11 +144,11 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD - assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) // innerJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } @@ -132,7 +161,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => old - newVal } - assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect().toSet == (0 to 98 by 2).toSet) } } @@ -142,7 +171,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n) val messageTargets = (0 to n) ++ (0 to n by 2) val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) - assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect.toSet === + assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect().toSet === (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) } } @@ -154,7 +183,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) // test merge function - assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + assert(rdd.collect().toSet == Set((0L, 0), (1L, 3), (2L, 9))) } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 3915be15b3434..4cc30a96408f8 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -32,7 +32,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -42,7 +42,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -50,8 +50,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) val ccGraph = twoChains.connectedComponents() @@ -73,12 +73,12 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse val ccGraph = twoChains.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if (id < 10) { assert(cc === 0) @@ -120,9 +120,9 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { // Build the initial Graph val graph = Graph(users, relationships, defaultUser) val ccGraph = graph.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { - assert(cc == 0) + assert(cc === 0) } } } // end of toy connected components diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index fc491ae327c2a..3f3c9dfd7b3dd 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -19,15 +19,12 @@ package org.apache.spark.graphx.lib import org.scalatest.FunSuite -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ + object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double): Seq[(VertexId, Double)] = { val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) val outDegree = Array.fill(nRows * nCols)(0) // Convert row column address into vertex ids (row major order) @@ -35,13 +32,13 @@ object GridPageRank { // Make the grid graph for (r <- 0 until nRows; c <- 0 until nCols) { val ind = sub2ind(r,c) - if (r+1 < nRows) { + if (r + 1 < nRows) { outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind + inNbrs(sub2ind(r + 1,c)) += ind } - if (c+1 < nCols) { + if (c + 1 < nCols) { outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind + inNbrs(sub2ind(r,c + 1)) += ind } } // compute the pagerank @@ -64,7 +61,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum + .map { case (id, error) => error }.sum() } test("Star PageRank") { @@ -80,12 +77,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum + }.map { case (vid, test) => test }.sum() assert(notMatching === 0) val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + val p = math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) + val correct = (vid > 0 && pr == resetProb) || (vid == 0L && p < 1.0E-5) if (!correct) 1 else 0 } assert(staticErrors.sum === 0) @@ -95,7 +92,35 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } } // end of test Star PageRank + test("Star PersonalPageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + + val staticRanks1 = starGraph.staticPersonalizedPageRank(0,numIter = 1, resetProb).vertices + val staticRanks2 = starGraph.staticPersonalizedPageRank(0,numIter = 2, resetProb) + .vertices.cache() + + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 + }.map { case (vid, test) => test }.sum + assert(notMatching === 0) + + val staticErrors = staticRanks2.map { case (vid, pr) => + val correct = (vid > 0 && pr == resetProb) || + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * + (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 + } + assert(staticErrors.sum === 0) + val dynamicRanks = starGraph.personalizedPageRank(0,0, resetProb).vertices.cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + } + } // end of test Star PageRank test("Grid PageRank") { withSpark { sc => @@ -109,18 +134,18 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() + val referenceRanks = VertexRDD( + sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) } } // end of Grid PageRank - test("Chain PageRank") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() val resetProb = 0.15 val tol = 0.0001 @@ -133,4 +158,21 @@ class PageRankSuite extends FunSuite with LocalSparkContext { assert(compareRanks(staticRanks, dynamicRanks) < errorTol) } } + + test("Chain PersonalizedPageRank") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-1 + + val staticRanks = chain.staticPersonalizedPageRank(4, numIter, resetProb).vertices + val dynamicRanks = chain.personalizedPageRank(4, tol, resetProb).vertices + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala index df54aa37cad68..1f658c371ffcf 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -34,8 +34,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) val sccGraph = graph.stronglyConnectedComponents(5) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(id === scc) } } } @@ -45,8 +45,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(0L == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(0L === scc) } } } @@ -60,13 +60,14 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - if (id < 3) - assert(0L == scc) - else if (id < 6) - assert(3L == scc) - else - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + if (id < 3) { + assert(0L === scc) + } else if (id < 6) { + assert(3L === scc) + } else { + assert(id === scc) + } } } } diff --git a/launcher/pom.xml b/launcher/pom.xml index 0fe2814135d88..ebfa7685eaa18 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -52,11 +52,6 @@ mockito-all test - - org.scalatest - scalatest_${scala.binary.version} - test - org.slf4j slf4j-api @@ -73,6 +68,12 @@ org.apache.hadoop hadoop-client test + + + org.codehaus.jackson + jackson-mapper-asl + + diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index dc90e9e987234..b8f02b961113d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -86,10 +86,14 @@ public AbstractCommandBuilder() { */ List buildJavaCommand(String extraClassPath) throws IOException { List cmd = new ArrayList(); - if (javaHome == null) { - cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); - } else { + String envJavaHome; + + if (javaHome != null) { cmd.add(join(File.separator, javaHome, "bin", "java")); + } else if ((envJavaHome = System.getenv("JAVA_HOME")) != null) { + cmd.add(join(File.separator, envJavaHome, "bin", "java")); + } else { + cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java")); } // Load extra JAVA_OPTS from conf/java-opts, if it exists. @@ -147,7 +151,6 @@ void addOptionString(List cmd, String options) { */ List buildClassPath(String appClassPath) throws IOException { String sparkHome = getSparkHome(); - String scala = getScalaVersion(); List cp = new ArrayList(); addToClassPath(cp, getenv("SPARK_CLASSPATH")); @@ -158,6 +161,7 @@ List buildClassPath(String appClassPath) throws IOException { boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES")); boolean isTesting = "1".equals(getenv("SPARK_TESTING")); if (prependClasses || isTesting) { + String scala = getScalaVersion(); List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", "yarn", "launcher"); @@ -182,59 +186,37 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - String assembly = findAssembly(scala); - addToClassPath(cp, assembly); - - // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus - // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt - // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built - // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark - // assembly is built for Hive, before actually populating the CLASSPATH with the jars. + // We can't rely on the ENV_SPARK_ASSEMBLY variable to be set. Certain situations, such as + // when running unit tests, or user code that embeds Spark and creates a SparkContext + // with a local or local-cluster master, will cause this code to be called from an + // environment where that env variable is not guaranteed to exist. // - // This block also serves as a check for SPARK-1703, when the assembly jar is built with - // Java 7 and ends up with too many files, causing issues with other JDK versions. - boolean needsDataNucleus = false; - JarFile assemblyJar = null; - try { - assemblyJar = new JarFile(assembly); - needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null; - } catch (IOException ioe) { - if (ioe.getMessage().indexOf("invalid CEN header") >= 0) { - System.err.println( - "Loading Spark jar failed.\n" + - "This is likely because Spark was compiled with Java 7 and run\n" + - "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" + - "or build Spark with Java 6."); - System.exit(1); - } else { - throw ioe; - } - } finally { - if (assemblyJar != null) { - try { - assemblyJar.close(); - } catch (IOException e) { - // Ignore. - } - } + // For the testing case, we rely on the test code to set and propagate the test classpath + // appropriately. + // + // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. + // That duplicates some of the code in the shell scripts that look for the assembly, though. + String assembly = getenv(ENV_SPARK_ASSEMBLY); + if (assembly == null && isEmpty(getenv("SPARK_TESTING"))) { + assembly = findAssembly(); } + addToClassPath(cp, assembly); - if (needsDataNucleus) { - System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " + - "in classpath."); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - } else { - libdir = new File(sparkHome, "lib_managed/jars"); - } + // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only + // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate + // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + } else { + libdir = new File(sparkHome, "lib_managed/jars"); + } - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); - } + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); } } @@ -270,7 +252,6 @@ String getScalaVersion() { if (scala != null) { return scala; } - String sparkHome = getSparkHome(); File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); @@ -330,7 +311,7 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } - private String findAssembly(String scalaVersion) { + private String findAssembly() { String sparkHome = getSparkHome(); File libdir; if (new File(sparkHome, "RELEASE").isFile()) { @@ -338,7 +319,7 @@ private String findAssembly(String scalaVersion) { checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", libdir.getAbsolutePath()); } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion)); + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); } final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); 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 9b04732afee14..261402856ac5e 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -30,6 +30,7 @@ class CommandBuilderUtils { static final String DEFAULT_MEM = "512m"; static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; static final String ENV_SPARK_HOME = "SPARK_HOME"; + static final String ENV_SPARK_ASSEMBLY = "_SPARK_ASSEMBLY"; /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { @@ -243,7 +244,7 @@ static String quoteForBatchScript(String arg) { boolean needsQuotes = false; for (int i = 0; i < arg.length(); i++) { int c = arg.codePointAt(i); - if (Character.isWhitespace(c) || c == '"' || c == '=') { + if (Character.isWhitespace(c) || c == '"' || c == '=' || c == ',' || c == ';') { needsQuotes = true; break; } @@ -260,28 +261,27 @@ static String quoteForBatchScript(String arg) { quoted.append('"'); break; - case '=': - quoted.append('^'); - break; - default: break; } quoted.appendCodePoint(cp); } + if (arg.codePointAt(arg.length() - 1) == '\\') { + quoted.append("\\"); + } quoted.append("\""); return quoted.toString(); } /** - * Quotes a string so that it can be used in a command string and be parsed back into a single - * argument by python's "shlex.split()" function. - * + * Quotes a string so that it can be used in a command string. * Basically, just add simple escapes. E.g.: * original single argument : ab "cd" ef * after: "ab \"cd\" ef" + * + * This can be parsed back into a single argument by python's "shlex.split()" function. */ - static String quoteForPython(String s) { + static String quoteForCommandString(String s) { StringBuilder quoted = new StringBuilder().append('"'); for (int i = 0; i < s.length(); i++) { int cp = s.codePointAt(i); diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index 206acfb514d86..929b29a49ed70 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -101,12 +101,9 @@ public static void main(String[] argsArray) throws Exception { * The method quotes all arguments so that spaces are handled as expected. Quotes within arguments * are "double quoted" (which is batch for escaping a quote). This page has more details about * quoting and other batch script fun stuff: http://ss64.com/nt/syntax-esc.html - * - * The command is executed using "cmd /c" and formatted in single line, since that's the - * easiest way to consume this from a batch script (see spark-class2.cmd). */ private static String prepareWindowsCommand(List cmd, Map childEnv) { - StringBuilder cmdline = new StringBuilder("cmd /c \""); + StringBuilder cmdline = new StringBuilder(); for (Map.Entry e : childEnv.entrySet()) { cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue())); cmdline.append(" && "); @@ -115,7 +112,6 @@ private static String prepareWindowsCommand(List cmd, Map buildCommand(Map env) throws IOException { } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) { javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); memKey = "SPARK_EXECUTOR_MEMORY"; + } else if (className.equals("org.apache.spark.deploy.ExternalShuffleService")) { + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_SHUFFLE_OPTS"); + memKey = "SPARK_DAEMON_MEMORY"; } else if (className.startsWith("org.apache.spark.tools.")) { String sparkHome = getSparkHome(); File toolsDir = new File(join(File.separator, sparkHome, "tools", "target", 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 b566507ee6061..d4cfeacb6ef18 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -52,7 +52,7 @@ public class SparkLauncher { /** Configuration key for the executor VM options. */ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ - public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryPath"; /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; 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 91dcf70f105db..7d387d406edae 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -17,14 +17,9 @@ package org.apache.spark.launcher; +import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -53,6 +48,20 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { */ static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell"; + /** + * Name of the app resource used to identify the SparkR shell. The command line parser expects + * the resource name to be the very first argument to spark-submit in this case. + * + * NOTE: this cannot be "sparkr-shell" since that identifies the SparkR shell to SparkSubmit + * (see sparkR.R), and can cause this code to enter into an infinite loop. + */ + static final String SPARKR_SHELL = "sparkr-shell-main"; + + /** + * This is the actual resource name that identifies the SparkR shell to SparkSubmit. + */ + static final String SPARKR_SHELL_RESOURCE = "sparkr-shell"; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling @@ -87,6 +96,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { this.allowsMixedArguments = true; appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); + } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL_RESOURCE; + submitArgs = args.subList(1, args.size()); } else { this.allowsMixedArguments = false; } @@ -98,6 +111,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { public List buildCommand(Map env) throws IOException { if (PYSPARK_SHELL_RESOURCE.equals(appResource)) { return buildPySparkShellCommand(env); + } else if (SPARKR_SHELL_RESOURCE.equals(appResource)) { + return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); } @@ -175,6 +190,10 @@ private List buildSparkSubmitCommand(Map env) throws IOE firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null; List cmd = buildJavaCommand(extraClassPath); + // Take Thrift Server as daemon + if (isThriftServer(mainClass)) { + addOptionString(cmd, System.getenv("SPARK_DAEMON_JAVA_OPTS")); + } addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); @@ -186,7 +205,11 @@ private List buildSparkSubmitCommand(Map env) throws IOE // - SPARK_DRIVER_MEMORY env variable // - SPARK_MEM env variable // - default value (512m) - String memory = firstNonEmpty(firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props), + // Take Thrift Server as daemon + String tsMemory = + isThriftServer(mainClass) ? System.getenv("SPARK_DAEMON_MEMORY") : null; + String memory = firstNonEmpty(tsMemory, + firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props), System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); @@ -213,36 +236,62 @@ private List buildPySparkShellCommand(Map env) throws IO return buildCommand(env); } - // When launching the pyspark shell, the spark-submit arguments should be stored in the - // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable - // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. + constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS"); + + // The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script, + // followed by PYSPARK_DRIVER_PYTHON_OPTS. + List pyargs = new ArrayList(); + pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return pyargs; + } + + private List buildSparkRCommand(Map env) throws IOException { + if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { + appResource = appArgs.get(0); + appArgs.remove(0); + return buildCommand(env); + } + // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS + // env variable. + constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS"); + + // Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up. + String sparkHome = System.getenv("SPARK_HOME"); + env.put("R_PROFILE_USER", + join(File.separator, sparkHome, "R", "lib", "SparkR", "profile", "shell.R")); + + List args = new ArrayList(); + args.add(firstNonEmpty(System.getenv("SPARKR_DRIVER_R"), "R")); + return args; + } + + private void constructEnvVarArgs( + Map env, + String submitArgsEnvVariable) throws IOException { Properties props = loadPropertiesFile(); mergeEnvPathList(env, getLibPathEnvName(), firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); - // Store spark-submit arguments in an environment variable, since there's no way to pass - // them to shell.py on the comand line. StringBuilder submitArgs = new StringBuilder(); for (String arg : buildSparkSubmitArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(quoteForPython(arg)); - } - env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); - - List pyargs = new ArrayList(); - pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); - String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); - if (!isEmpty(pyOpts)) { - pyargs.addAll(parseOptionString(pyOpts)); + submitArgs.append(quoteForCommandString(arg)); } - - return pyargs; + env.put(submitArgsEnvVariable, submitArgs.toString()); } + private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); // Default master is "local[*]", so assume client mode in that case. @@ -251,6 +300,15 @@ private boolean isClientMode(Properties userProps) { (!userMaster.equals("yarn-cluster") && deployMode == null); } + /** + * Return whether the given main class represents a thrift server. + */ + private boolean isThriftServer(String mainClass) { + return (mainClass != null && + mainClass.equals("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2")); + } + + private class OptionParser extends SparkSubmitOptionParser { @Override diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 8526d2e7cfa3f..229000087688f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -69,8 +69,10 @@ class SparkSubmitOptionParser { // YARN-only options. protected final String ARCHIVES = "--archives"; protected final String EXECUTOR_CORES = "--executor-cores"; - protected final String QUEUE = "--queue"; + protected final String KEYTAB = "--keytab"; protected final String NUM_EXECUTORS = "--num-executors"; + protected final String PRINCIPAL = "--principal"; + protected final String QUEUE = "--queue"; /** * This is the canonical list of spark-submit options. Each entry in the array contains the @@ -96,11 +98,13 @@ class SparkSubmitOptionParser { { EXECUTOR_MEMORY }, { FILES }, { JARS }, + { KEYTAB }, { KILL_SUBMISSION }, { MASTER }, { NAME }, { NUM_EXECUTORS }, { PACKAGES }, + { PRINCIPAL }, { PROPERTIES_FILE }, { PROXY_USER }, { PY_FILES }, 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 dba0203867372..bc513ec9b3d10 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -74,14 +74,17 @@ public void testWindowsBatchQuoting() { assertEquals("\"a b c\"", quoteForBatchScript("a b c")); assertEquals("\"a \"\"b\"\" c\"", quoteForBatchScript("a \"b\" c")); assertEquals("\"a\"\"b\"\"c\"", quoteForBatchScript("a\"b\"c")); - assertEquals("\"ab^=\"\"cd\"\"\"", quoteForBatchScript("ab=\"cd\"")); + assertEquals("\"ab=\"\"cd\"\"\"", quoteForBatchScript("ab=\"cd\"")); + assertEquals("\"a,b,c\"", quoteForBatchScript("a,b,c")); + assertEquals("\"a;b;c\"", quoteForBatchScript("a;b;c")); + assertEquals("\"a,b,c\\\\\"", quoteForBatchScript("a,b,c\\")); } @Test public void testPythonArgQuoting() { - assertEquals("\"abc\"", quoteForPython("abc")); - assertEquals("\"a b c\"", quoteForPython("a b c")); - assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c")); + assertEquals("\"abc\"", quoteForCommandString("abc")); + assertEquals("\"a b c\"", quoteForCommandString("a b c")); + assertEquals("\"a \\\"b\\\" c\"", quoteForCommandString("a \"b\" c")); } private void testOpt(String opts, List expected) { 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 626116a9e7477..97043a76cc612 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -98,7 +98,7 @@ public void testShellCliParser() throws Exception { parser.NAME, "appName"); - List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List args = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); } @@ -110,7 +110,7 @@ public void testAlternateSyntaxParsing() throws Exception { parser.MASTER + "=foo", parser.DEPLOY_MODE + "=bar"); - List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); assertEquals("foo", findArgValue(cmd, parser.MASTER)); assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); @@ -153,7 +153,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - new SparkSubmitCommandBuilder(Collections.emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; @@ -273,10 +273,15 @@ private boolean findInStringList(String list, String sep, String needle) { return contains(needle, list.split(sep)); } - private List buildCommand(List args, Map env) throws Exception { + private SparkSubmitCommandBuilder newCommandBuilder(List args) { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); - return builder.buildCommand(env); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_ASSEMBLY, "dummy"); + return builder; + } + + private List buildCommand(List args, Map env) throws Exception { + return newCommandBuilder(args).buildCommand(env); } } diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties index 00c20ad69cd4d..67a6a98217118 100644 --- a/launcher/src/test/resources/log4j.properties +++ b/launcher/src/test/resources/log4j.properties @@ -27,5 +27,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/make-distribution.sh b/make-distribution.sh index 9ed1abfe8c598..1bfa9acb1fe6e 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -26,13 +26,14 @@ set -o pipefail set -e +set -x # Figure out where the Spark framework is installed SPARK_HOME="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$SPARK_HOME/dist" SPARK_TACHYON=false -TACHYON_VERSION="0.5.0" +TACHYON_VERSION="0.6.4" TACHYON_TGZ="tachyon-${TACHYON_VERSION}-bin.tar.gz" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/${TACHYON_TGZ}" @@ -57,7 +58,7 @@ while (( "$#" )); do --hadoop) echo "Error: '--hadoop' is no longer supported:" echo "Error: use Maven profiles and options -Dhadoop.version and -Dyarn.version instead." - echo "Error: Related profiles include hadoop-0.23, hdaoop-2.2, hadoop-2.3 and hadoop-2.4." + echo "Error: Related profiles include hadoop-2.2, hadoop-2.3 and hadoop-2.4." exit_with_usage ;; --with-yarn) @@ -126,7 +127,7 @@ if [ ! $(command -v "$MVN") ] ; then exit -1; fi -VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +VERSION=$("$MVN" help:evaluate -Dexpression=project.version $@ 2>/dev/null | grep -v "INFO" | tail -n 1) SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) @@ -199,7 +200,6 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" -cp "$SPARK_HOME"/launcher/target/spark-launcher_$SCALA_VERSION-$VERSION.jar "$DISTDIR/lib/" # This will fail if the -Pyarn profile is not provided # In this case, silence the error and ignore the return code of this command cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || : diff --git a/mllib/pom.xml b/mllib/pom.xml index 4c183543e3fa8..a3c57ae26000b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -64,7 +64,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.11.1 + 0.11.2 @@ -109,6 +109,21 @@ test-jar test + + org.jpmml + pmml-model + 1.1.15 + + + com.sun.xml.fastinfoset + FastInfoset + + + com.sun.istack + istack-commons-runtime + + + diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index eff7ef925dfbd..7f3f3262a644f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -34,13 +34,16 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Fits a single model to the input data with optional parameters. * * @param dataset input dataset - * @param paramPairs Optional list of param pairs. - * These values override any specified in this Estimator's embedded ParamMap. + * @param firstParamPair the first param pair, overrides embedded params + * @param otherParamPairs other param pairs. These values override any specified in this + * Estimator's embedded ParamMap. * @return fitted model */ @varargs - def fit(dataset: DataFrame, paramPairs: ParamPair[_]*): M = { - val map = new ParamMap().put(paramPairs: _*) + def fit(dataset: DataFrame, firstParamPair: ParamPair[_], otherParamPairs: ParamPair[_]*): M = { + val map = new ParamMap() + .put(firstParamPair) + .put(otherParamPairs: _*) fit(dataset, map) } @@ -52,12 +55,19 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ - def fit(dataset: DataFrame, paramMap: ParamMap): M + def fit(dataset: DataFrame, paramMap: ParamMap): M = { + copy(paramMap).fit(dataset) + } + + /** + * Fits a model to the input data. + */ + def fit(dataset: DataFrame): M /** * Fits multiple models to the input data with multiple sets of parameters. * The default implementation uses a for loop on each parameter map. - * Subclasses could overwrite this to optimize multi-model training. + * Subclasses could override this to optimize multi-model training. * * @param dataset input dataset * @param paramMaps An array of parameter maps. @@ -67,4 +77,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { def fit(dataset: DataFrame, paramMaps: Array[ParamMap]): Seq[M] = { paramMaps.map(fit(dataset, _)) } + + override def copy(extra: ParamMap): Estimator[M] = { + super.copy(extra).asInstanceOf[Estimator[M]] + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala index d2ca2e6871e6b..5f2f8c94e9ff7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Evaluator.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.{ParamMap, Params} import org.apache.spark.sql.DataFrame /** @@ -26,7 +26,7 @@ import org.apache.spark.sql.DataFrame * Abstract class for evaluators that compute metrics from predictions. */ @AlphaComponent -abstract class Evaluator extends Identifiable { +abstract class Evaluator extends Params { /** * Evaluates the output. @@ -35,5 +35,18 @@ abstract class Evaluator extends Identifiable { * @param paramMap parameter map that specifies the input columns and output metrics * @return metric */ - def evaluate(dataset: DataFrame, paramMap: ParamMap): Double + def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { + this.copy(paramMap).evaluate(dataset) + } + + /** + * Evaluates the output. + * @param dataset a dataset that contains labels/observations and predictions. + * @return metric + */ + def evaluate(dataset: DataFrame): Double + + override def copy(extra: ParamMap): Evaluator = { + super.copy(extra).asInstanceOf[Evaluator] + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Model.scala b/mllib/src/main/scala/org/apache/spark/ml/Model.scala index cae5082b51196..9974efe7b1d25 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Model.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Model.scala @@ -30,11 +30,12 @@ import org.apache.spark.ml.param.ParamMap abstract class Model[M <: Model[M]] extends Transformer { /** * The parent estimator that produced this model. + * Note: For ensembles' component Models, this value can be null. */ val parent: Estimator[M] - /** - * Fitting parameters, such that parent.fit(..., fittingParamMap) could reproduce the model. - */ - val fittingParamMap: ParamMap + override def copy(extra: ParamMap): M = { + // The default implementation of Params.copy doesn't work for models. + throw new NotImplementedError(s"${this.getClass} doesn't implement copy(extra: ParamMap)") + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index 5bbcd2e080e07..33d430f5671ee 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.Logging import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.ml.param.{Param, ParamMap, Params} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.StructType @@ -30,37 +30,41 @@ import org.apache.spark.sql.types.StructType * A stage in a pipeline, either an [[Estimator]] or a [[Transformer]]. */ @AlphaComponent -abstract class PipelineStage extends Serializable with Logging { +abstract class PipelineStage extends Params with Logging { /** - * :: DeveloperAPI :: + * :: DeveloperApi :: * - * Derives the output schema from the input schema and parameters. - * The schema describes the columns and types of the data. - * - * @param schema Input schema to this stage - * @param paramMap Parameters passed to this stage - * @return Output schema from this stage + * Derives the output schema from the input schema. */ @DeveloperApi - def transformSchema(schema: StructType, paramMap: ParamMap): StructType + def transformSchema(schema: StructType): StructType /** + * :: DeveloperApi :: + * * Derives the output schema from the input schema and parameters, optionally with logging. + * + * This should be optimistic. If it is unclear whether the schema will be valid, then it should + * be assumed valid until proven otherwise. */ + @DeveloperApi protected def transformSchema( schema: StructType, - paramMap: ParamMap, logging: Boolean): StructType = { if (logging) { logDebug(s"Input schema: ${schema.json}") } - val outputSchema = transformSchema(schema, paramMap) + val outputSchema = transformSchema(schema) if (logging) { logDebug(s"Expected output schema: ${outputSchema.json}") } outputSchema } + + override def copy(extra: ParamMap): PipelineStage = { + super.copy(extra).asInstanceOf[PipelineStage] + } } /** @@ -78,10 +82,25 @@ abstract class PipelineStage extends Serializable with Logging { @AlphaComponent class Pipeline extends Estimator[PipelineModel] { - /** param for pipeline stages */ + /** + * param for pipeline stages + * @group param + */ val stages: Param[Array[PipelineStage]] = new Param(this, "stages", "stages of the pipeline") + + /** @group setParam */ def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this } - def getStages: Array[PipelineStage] = get(stages) + + /** @group getParam */ + def getStages: Array[PipelineStage] = $(stages).clone() + + override def validateParams(paramMap: ParamMap): Unit = { + val map = extractParamMap(paramMap) + getStages.foreach { + case pStage: Params => pStage.validateParams(map) + case _ => + } + } /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an @@ -93,13 +112,11 @@ class Pipeline extends Estimator[PipelineModel] { * pipeline stages. If there are no stages, the output model acts as an identity transformer. * * @param dataset input dataset - * @param paramMap parameter map * @return fitted pipeline */ - override def fit(dataset: DataFrame, paramMap: ParamMap): PipelineModel = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val theStages = map(stages) + override def fit(dataset: DataFrame): PipelineModel = { + transformSchema(dataset.schema, logging = true) + val theStages = $(stages) // Search for the last estimator. var indexOfLastEstimator = -1 theStages.view.zipWithIndex.foreach { case (stage, index) => @@ -115,7 +132,7 @@ class Pipeline extends Estimator[PipelineModel] { if (index <= indexOfLastEstimator) { val transformer = stage match { case estimator: Estimator[_] => - estimator.fit(curDataset, paramMap) + estimator.fit(curDataset) case t: Transformer => t case _ => @@ -123,7 +140,7 @@ class Pipeline extends Estimator[PipelineModel] { s"Do not support stage $stage of type ${stage.getClass}") } if (index < indexOfLastEstimator) { - curDataset = transformer.transform(curDataset, paramMap) + curDataset = transformer.transform(curDataset) } transformers += transformer } else { @@ -131,15 +148,20 @@ class Pipeline extends Estimator[PipelineModel] { } } - new PipelineModel(this, map, transformers.toArray) + new PipelineModel(this, transformers.toArray) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - val theStages = map(stages) - require(theStages.toSet.size == theStages.size, + override def copy(extra: ParamMap): Pipeline = { + val map = extractParamMap(extra) + val newStages = map(stages).map(_.copy(extra)) + new Pipeline().setStages(newStages) + } + + override def transformSchema(schema: StructType): StructType = { + val theStages = $(stages) + require(theStages.toSet.size == theStages.length, "Cannot have duplicate components in a pipeline.") - theStages.foldLeft(schema)((cur, stage) => stage.transformSchema(cur, paramMap)) + theStages.foldLeft(schema)((cur, stage) => stage.transformSchema(cur)) } } @@ -150,38 +172,24 @@ class Pipeline extends Estimator[PipelineModel] { @AlphaComponent class PipelineModel private[ml] ( override val parent: Pipeline, - override val fittingParamMap: ParamMap, - private[ml] val stages: Array[Transformer]) + val stages: Array[Transformer]) extends Model[PipelineModel] with Logging { - /** - * Gets the model produced by the input estimator. Throws an NoSuchElementException is the input - * estimator does not exist in the pipeline. - */ - def getModel[M <: Model[M]](stage: Estimator[M]): M = { - val matched = stages.filter { - case m: Model[_] => m.parent.eq(stage) - case _ => false - } - if (matched.isEmpty) { - throw new NoSuchElementException(s"Cannot find stage $stage from the pipeline.") - } else if (matched.size > 1) { - throw new IllegalStateException(s"Cannot have duplicate estimators in the sample pipeline.") - } else { - matched.head.asInstanceOf[M] - } + override def validateParams(): Unit = { + super.validateParams() + stages.foreach(_.validateParams()) + } + + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur)) } - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap - transformSchema(dataset.schema, map, logging = true) - stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) + override def transformSchema(schema: StructType): StructType = { + stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur)) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap - stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) + override def copy(extra: ParamMap): PipelineModel = { + new PipelineModel(parent, stages) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 9a5848684b179..d96b54e511e9c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -22,6 +22,7 @@ import scala.annotation.varargs import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -36,13 +37,18 @@ abstract class Transformer extends PipelineStage with Params { /** * Transforms the dataset with optional parameters * @param dataset input dataset - * @param paramPairs optional list of param pairs, overwrite embedded params + * @param firstParamPair the first param pair, overwrite embedded params + * @param otherParamPairs other param pairs, overwrite embedded params * @return transformed dataset */ @varargs - def transform(dataset: DataFrame, paramPairs: ParamPair[_]*): DataFrame = { + def transform( + dataset: DataFrame, + firstParamPair: ParamPair[_], + otherParamPairs: ParamPair[_]*): DataFrame = { val map = new ParamMap() - paramPairs.foreach(map.put(_)) + .put(firstParamPair) + .put(otherParamPairs: _*) transform(dataset, map) } @@ -52,7 +58,18 @@ abstract class Transformer extends PipelineStage with Params { * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame + def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + this.copy(paramMap).transform(dataset) + } + + /** + * Transforms the input dataset. + */ + def transform(dataset: DataFrame): DataFrame + + override def copy(extra: ParamMap): Transformer = { + super.copy(extra).asInstanceOf[Transformer] + } } /** @@ -73,7 +90,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O * account of the embedded param map. So the param values should be determined solely by the input * param map. */ - protected def createTransformFunc(paramMap: ParamMap): IN => OUT + protected def createTransformFunc: IN => OUT /** * Returns the data type of the output column. @@ -85,22 +102,20 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O */ protected def validateInputType(inputType: DataType): Unit = {} - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - val inputType = schema(map(inputCol)).dataType + override def transformSchema(schema: StructType): StructType = { + val inputType = schema($(inputCol)).dataType validateInputType(inputType) - if (schema.fieldNames.contains(map(outputCol))) { - throw new IllegalArgumentException(s"Output column ${map(outputCol)} already exists.") + if (schema.fieldNames.contains($(outputCol))) { + throw new IllegalArgumentException(s"Output column ${$(outputCol)} already exists.") } val outputFields = schema.fields :+ - StructField(map(outputCol), outputDataType, !outputDataType.isPrimitive) + StructField($(outputCol), outputDataType, nullable = false) StructType(outputFields) } - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - dataset.withColumn(map(outputCol), - callUDF(this.createTransformFunc(map), outputDataType, dataset(map(inputCol)))) + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + dataset.withColumn($(outputCol), + callUDF(this.createTransformFunc, outputDataType, dataset($(inputCol)))) } } 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 970e6ad5514d1..d7dee8fed2a55 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 @@ -106,7 +106,7 @@ class AttributeGroup private ( def getAttr(attrIndex: Int): Attribute = this(attrIndex) /** Converts to metadata without name. */ - private[attribute] def toMetadata: Metadata = { + private[attribute] def toMetadataImpl: Metadata = { import AttributeKeys._ val bldr = new MetadataBuilder() if (attributes.isDefined) { @@ -117,12 +117,12 @@ class AttributeGroup private ( case numeric: NumericAttribute => // Skip default numeric attributes. if (numeric.withoutIndex != NumericAttribute.defaultAttr) { - numericMetadata += numeric.toMetadata(withType = false) + numericMetadata += numeric.toMetadataImpl(withType = false) } case nominal: NominalAttribute => - nominalMetadata += nominal.toMetadata(withType = false) + nominalMetadata += nominal.toMetadataImpl(withType = false) case binary: BinaryAttribute => - binaryMetadata += binary.toMetadata(withType = false) + binaryMetadata += binary.toMetadataImpl(withType = false) } val attrBldr = new MetadataBuilder if (numericMetadata.nonEmpty) { @@ -142,13 +142,20 @@ class AttributeGroup private ( bldr.build() } - /** Converts to a StructField with some existing metadata. */ - def toStructField(existingMetadata: Metadata): StructField = { - val newMetadata = new MetadataBuilder() + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, toMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl) .build() - StructField(name, new VectorUDT, nullable = false, newMetadata) + } + + /** Converts to ML metadata */ + def toMetadata(): Metadata = toMetadata(Metadata.empty) + + /** Converts to a StructField with some existing metadata. */ + def toStructField(existingMetadata: Metadata): StructField = { + StructField(name, new VectorUDT, nullable = false, toMetadata(existingMetadata)) } /** Converts to a StructField. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index 00b7566aab434..5717d6ec2eaec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -68,21 +68,32 @@ sealed abstract class Attribute extends Serializable { * Converts this attribute to [[Metadata]]. * @param withType whether to include the type info */ - private[attribute] def toMetadata(withType: Boolean): Metadata + private[attribute] def toMetadataImpl(withType: Boolean): Metadata /** * Converts this attribute to [[Metadata]]. For numeric attributes, the type info is excluded to * save space, because numeric type is the default attribute type. For nominal and binary * attributes, the type info is included. */ - private[attribute] def toMetadata(): Metadata = { + private[attribute] def toMetadataImpl(): Metadata = { if (attrType == AttributeType.Numeric) { - toMetadata(withType = false) + toMetadataImpl(withType = false) } else { - toMetadata(withType = true) + toMetadataImpl(withType = true) } } + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() + .withMetadata(existingMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl()) + .build() + } + + /** Converts to ML metadata */ + def toMetadata(): Metadata = toMetadata(Metadata.empty) + /** * Converts to a [[StructField]] with some existing metadata. * @param existingMetadata existing metadata to carry over @@ -90,7 +101,7 @@ sealed abstract class Attribute extends Serializable { def toStructField(existingMetadata: Metadata): StructField = { val newMetadata = new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadata()) + .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadataImpl()) .build() StructField(name.get, DoubleType, nullable = false, newMetadata) } @@ -98,7 +109,7 @@ sealed abstract class Attribute extends Serializable { /** Converts to a [[StructField]]. */ def toStructField(): StructField = toStructField(Metadata.empty) - override def toString: String = toMetadata(withType = true).toString + override def toString: String = toMetadataImpl(withType = true).toString } /** Trait for ML attribute factories. */ @@ -210,7 +221,7 @@ class NumericAttribute private[ml] ( override def isNominal: Boolean = false /** Convert this attribute to metadata. */ - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -353,6 +364,20 @@ class NominalAttribute private[ml] ( /** Copy without the `numValues`. */ def withoutNumValues: NominalAttribute = copy(numValues = None) + /** + * Get the number of values, either from `numValues` or from `values`. + * Return None if unknown. + */ + def getNumValues: Option[Int] = { + if (numValues.nonEmpty) { + numValues + } else if (values.nonEmpty) { + Some(values.get.length) + } else { + None + } + } + /** Creates a copy of this attribute with optional changes. */ private def copy( name: Option[String] = name, @@ -363,7 +388,7 @@ class NominalAttribute private[ml] ( new NominalAttribute(name, index, isOrdinal, numValues, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder() if (withType) bldr.putString(TYPE, attrType.name) @@ -465,7 +490,7 @@ class BinaryAttribute private[ml] ( new BinaryAttribute(name, index, values) } - private[attribute] override def toMetadata(withType: Boolean): Metadata = { + override private[attribute] def toMetadataImpl(withType: Boolean): Metadata = { import org.apache.spark.ml.attribute.AttributeKeys._ val bldr = new MetadataBuilder if (withType) bldr.putString(TYPE, attrType.name) 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 c5fc89f935432..d3361e24705c8 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 @@ -17,15 +17,15 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} +import org.apache.spark.ml.param.shared.HasRawPredictionCol +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.functions._ import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} - /** * :: DeveloperApi :: * Params for classification. @@ -38,12 +38,10 @@ private[spark] trait ClassifierParams extends PredictorParams override protected def validateAndTransformSchema( schema: StructType, - paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + val parentSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + SchemaUtils.appendColumn(parentSchema, $(rawPredictionCol), new VectorUDT) } } @@ -67,8 +65,7 @@ private[spark] abstract class Classifier[ with ClassifierParams { /** @group setParam */ - def setRawPredictionCol(value: String): E = - set(rawPredictionCol, value).asInstanceOf[E] + def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) } @@ -101,27 +98,16 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]]. * * @param dataset input dataset - * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + override def transform(dataset: DataFrame): DataFrame = { // This default implementation should be overridden as needed. // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - - // Prepare model - val tmpModel = if (paramMap.size != 0) { - val tmpModel = this.copy() - Params.inheritValues(paramMap, parent, tmpModel) - tmpModel - } else { - this - } + transformSchema(dataset.schema, logging = true) val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) if (numColsOutput == 0) { logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + " since no output columns were set.") @@ -157,7 +143,6 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur */ @DeveloperApi protected def predictRaw(features: FeaturesType): Vector - } private[ml] object ClassificationModel { @@ -166,38 +151,35 @@ private[ml] object ClassificationModel { * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. * @param dataset Input dataset - * @param map Parameter map. This will NOT be merged with the embedded paramMap; the merge - * should already be done. * @return (number of columns added, transformed dataset) */ def transformColumnsImpl[FeaturesType]( dataset: DataFrame, - model: ClassificationModel[FeaturesType, _], - map: ParamMap): (Int, DataFrame) = { + model: ClassificationModel[FeaturesType, _]): (Int, DataFrame) = { // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. var tmpData = dataset var numColsOutput = 0 - if (map(model.rawPredictionCol) != "") { + if (model.getRawPredictionCol != "") { // output raw prediction val features2raw: FeaturesType => Vector = model.predictRaw - tmpData = tmpData.withColumn(map(model.rawPredictionCol), - callUDF(features2raw, new VectorUDT, col(map(model.featuresCol)))) + tmpData = tmpData.withColumn(model.getRawPredictionCol, + callUDF(features2raw, new VectorUDT, col(model.getFeaturesCol))) numColsOutput += 1 - if (map(model.predictionCol) != "") { + if (model.getPredictionCol != "") { val raw2pred: Vector => Double = (rawPred) => { rawPred.toArray.zipWithIndex.maxBy(_._1)._2 } - tmpData = tmpData.withColumn(map(model.predictionCol), - callUDF(raw2pred, DoubleType, col(map(model.rawPredictionCol)))) + tmpData = tmpData.withColumn(model.getPredictionCol, + callUDF(raw2pred, DoubleType, col(model.getRawPredictionCol))) numColsOutput += 1 } - } else if (map(model.predictionCol) != "") { + } else if (model.getPredictionCol != "") { // output prediction val features2pred: FeaturesType => Double = model.predict - tmpData = tmpData.withColumn(map(model.predictionCol), - callUDF(features2pred, DoubleType, col(map(model.featuresCol)))) + tmpData = tmpData.withColumn(model.getPredictionCol, + callUDF(features2pred, DoubleType, col(model.getFeaturesCol))) numColsOutput += 1 } (numColsOutput, tmpData) 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 new file mode 100644 index 0000000000000..419e5ba05d38a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -0,0 +1,144 @@ +/* + * 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.classification + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +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 +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassifier + extends Predictor[Vector, DecisionTreeClassifier, DecisionTreeClassificationModel] + with DecisionTreeParams with TreeClassifierParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train(dataset: DataFrame): DecisionTreeClassificationModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => n + case None => throw new IllegalArgumentException("DecisionTreeClassifier was given input" + + s" with invalid label column ${$(labelCol)}, without the number of classes" + + " specified. See StringIndexer.") + // TODO: Automatically index labels: SPARK-7126 + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val strategy = getOldStrategy(categoricalFeatures, numClasses) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeClassificationModel.fromOld(oldModel, this, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int): OldStrategy = { + super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity, + subsamplingRate = 1.0) + } +} + +object DecisionTreeClassifier { + /** Accessor for supported impurities: entropy, gini */ + final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class DecisionTreeClassificationModel private[ml] ( + override val parent: DecisionTreeClassifier, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeClassificationModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override def copy(extra: ParamMap): DecisionTreeClassificationModel = { + copyValues(new DecisionTreeClassificationModel(parent, rootNode), extra) + } + + override def toString: String = { + s"DecisionTreeClassificationModel of depth $depth with $numNodes nodes" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Classification) + } +} + +private[ml] object DecisionTreeClassificationModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeClassifier, + categoricalFeatures: Map[Int, Int]): DecisionTreeClassificationModel = { + require(oldModel.algo == OldAlgo.Classification, + s"Cannot convert non-classification DecisionTreeModel (old API) to" + + s" DecisionTreeClassificationModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeClassificationModel(parent, rootNode) + } +} 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 new file mode 100644 index 0000000000000..534ea95b1c538 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -0,0 +1,216 @@ +/* + * 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.classification + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.Logging +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.ml.regression.DecisionTreeRegressionModel +import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{GradientBoostedTrees => OldGBT} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.loss.{LogLoss => OldLogLoss, Loss => OldLoss} +import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * learning algorithm for classification. + * It supports binary labels, as well as both continuous and categorical features. + * Note: Multiclass labels are not currently supported. + */ +@AlphaComponent +final class GBTClassifier + extends Predictor[Vector, GBTClassifier, GBTClassificationModel] + with GBTParams with TreeClassifierParams with Logging { + + // Override parameter setters from parent trait for Java API compatibility. + + // Parameters from TreeClassifierParams: + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + /** + * The impurity setting is ignored for GBT models. + * Individual trees are built using impurity "Variance." + */ + override def setImpurity(value: String): this.type = { + logWarning("GBTClassifier.setImpurity should NOT be used") + this + } + + // Parameters from TreeEnsembleParams: + + override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + + override def setSeed(value: Long): this.type = { + logWarning("The 'seed' parameter is currently ignored by Gradient Boosting.") + super.setSeed(value) + } + + // Parameters from GBTParams: + + override def setMaxIter(value: Int): this.type = super.setMaxIter(value) + + override def setStepSize(value: Double): this.type = super.setStepSize(value) + + // Parameters for GBTClassifier: + + /** + * Loss function which GBT tries to minimize. (case-insensitive) + * Supported: "logistic" + * (default = logistic) + * @group param + */ + val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + + " tries to minimize (case-insensitive). Supported options:" + + s" ${GBTClassifier.supportedLossTypes.mkString(", ")}", + (value: String) => GBTClassifier.supportedLossTypes.contains(value.toLowerCase)) + + setDefault(lossType -> "logistic") + + /** @group setParam */ + def setLossType(value: String): this.type = set(lossType, value) + + /** @group getParam */ + def getLossType: String = $(lossType).toLowerCase + + /** (private[ml]) Convert new loss to old loss. */ + override private[ml] def getOldLossType: OldLoss = { + getLossType match { + case "logistic" => OldLogLoss + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException(s"GBTClassifier was given bad loss type: $getLossType") + } + } + + override protected def train(dataset: DataFrame): GBTClassificationModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => n + case None => throw new IllegalArgumentException("GBTClassifier was given input" + + s" with invalid label column ${$(labelCol)}, without the number of classes" + + " specified. See StringIndexer.") + // TODO: Automatically index labels: SPARK-7126 + } + require(numClasses == 2, + s"GBTClassifier only supports binary classification but was given numClasses = $numClasses") + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) + val oldGBT = new OldGBT(boostingStrategy) + val oldModel = oldGBT.run(oldDataset) + GBTClassificationModel.fromOld(oldModel, this, categoricalFeatures) + } +} + +object GBTClassifier { + // The losses below should be lowercase. + /** Accessor for supported loss settings: logistic */ + final val supportedLossTypes: Array[String] = Array("logistic").map(_.toLowerCase) +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * model for classification. + * It supports binary labels, as well as both continuous and categorical features. + * Note: Multiclass labels are not currently supported. + * @param _trees Decision trees in the ensemble. + * @param _treeWeights Weights for the decision trees in the ensemble. + */ +@AlphaComponent +final class GBTClassificationModel( + override val parent: GBTClassifier, + private val _trees: Array[DecisionTreeRegressionModel], + private val _treeWeights: Array[Double]) + extends PredictionModel[Vector, GBTClassificationModel] + with TreeEnsembleModel with Serializable { + + require(numTrees > 0, "GBTClassificationModel requires at least 1 tree.") + require(_trees.length == _treeWeights.length, "GBTClassificationModel given trees, treeWeights" + + s" of non-matching lengths (${_trees.length}, ${_treeWeights.length}, respectively).") + + override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + + override def treeWeights: Array[Double] = _treeWeights + + override protected def predict(features: Vector): Double = { + // TODO: Override transform() to broadcast model: SPARK-7127 + // TODO: When we add a generic Boosting class, handle transform there? SPARK-7129 + // Classifies by thresholding sum of weighted tree predictions + val treePredictions = _trees.map(_.rootNode.predict(features)) + val prediction = blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + if (prediction > 0.0) 1.0 else 0.0 + } + + override def copy(extra: ParamMap): GBTClassificationModel = { + copyValues(new GBTClassificationModel(parent, _trees, _treeWeights), extra) + } + + override def toString: String = { + s"GBTClassificationModel with $numTrees trees" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldGBTModel = { + new OldGBTModel(OldAlgo.Classification, _trees.map(_.toOld), _treeWeights) + } +} + +private[ml] object GBTClassificationModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldGBTModel, + parent: GBTClassifier, + categoricalFeatures: Map[Int, Int]): GBTClassificationModel = { + require(oldModel.algo == OldAlgo.Classification, "Cannot convert GradientBoostedTreesModel" + + s" with algo=${oldModel.algo} (old API) to GBTClassificationModel (new API).") + val newTrees = oldModel.trees.map { tree => + // parent, fittingParamMap for each tree is null since there are no good ways to set these. + DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) + } + new GBTClassificationModel(parent, newTrees, oldModel.treeWeights) + } +} 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 21f61d80dd95a..b73be035e29b5 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 @@ -19,20 +19,21 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} +import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT, Vectors} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel - /** * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams - with HasRegParam with HasMaxIter with HasThreshold + with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold { + setDefault(regParam -> 0.1, maxIter -> 100, threshold -> 0.5) +} /** * :: AlphaComponent :: @@ -45,39 +46,39 @@ class LogisticRegression extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { - setRegParam(0.1) - setMaxIter(100) - setThreshold(0.5) - /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) /** @group setParam */ def setMaxIter(value: Int): this.type = set(maxIter, value) + /** @group setParam */ + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) - override protected def train(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { + override protected def train(dataset: DataFrame): LogisticRegressionModel = { // Extract columns from data. If dataset is persisted, do not persist oldDataset. - val oldDataset = extractLabeledPoints(dataset, paramMap) + val oldDataset = extractLabeledPoints(dataset) val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) { oldDataset.persist(StorageLevel.MEMORY_AND_DISK) } // Train model - val lr = new LogisticRegressionWithLBFGS + val lr = new LogisticRegressionWithLBFGS() + .setIntercept($(fitIntercept)) lr.optimizer - .setRegParam(paramMap(regParam)) - .setNumIterations(paramMap(maxIter)) + .setRegParam($(regParam)) + .setNumIterations($(maxIter)) val oldModel = lr.run(oldDataset) - val lrm = new LogisticRegressionModel(this, paramMap, oldModel.weights, oldModel.intercept) + val lrm = new LogisticRegressionModel(this, oldModel.weights, oldModel.intercept) if (handlePersistence) { oldDataset.unpersist() } - lrm + copyValues(lrm) } } @@ -90,14 +91,11 @@ class LogisticRegression @AlphaComponent class LogisticRegressionModel private[ml] ( override val parent: LogisticRegression, - override val fittingParamMap: ParamMap, val weights: Vector, val intercept: Double) extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams { - setThreshold(0.5) - /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) @@ -110,16 +108,14 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + override def transform(dataset: DataFrame): DataFrame = { // This is overridden (a) to be more efficient (avoiding re-computing values when creating // multiple output columns) and (b) to handle threshold, which the abstractions do not use. // TODO: We should abstract away the steps defined by UDFs below so that the abstractions // can call whichever UDFs are needed to create the output columns. // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - - val map = this.paramMap ++ paramMap + transformSchema(dataset.schema, logging = true) // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. @@ -128,41 +124,41 @@ class LogisticRegressionModel private[ml] ( // prediction (max margin) var tmpData = dataset var numColsOutput = 0 - if (map(rawPredictionCol) != "") { + if ($(rawPredictionCol) != "") { val features2raw: Vector => Vector = (features) => predictRaw(features) - tmpData = tmpData.withColumn(map(rawPredictionCol), - callUDF(features2raw, new VectorUDT, col(map(featuresCol)))) + tmpData = tmpData.withColumn($(rawPredictionCol), + callUDF(features2raw, new VectorUDT, col($(featuresCol)))) numColsOutput += 1 } - if (map(probabilityCol) != "") { - if (map(rawPredictionCol) != "") { + if ($(probabilityCol) != "") { + if ($(rawPredictionCol) != "") { val raw2prob = udf { (rawPreds: Vector) => val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) Vectors.dense(1.0 - prob1, prob1): Vector } - tmpData = tmpData.withColumn(map(probabilityCol), raw2prob(col(map(rawPredictionCol)))) + tmpData = tmpData.withColumn($(probabilityCol), raw2prob(col($(rawPredictionCol)))) } else { val features2prob = udf { (features: Vector) => predictProbabilities(features) : Vector } - tmpData = tmpData.withColumn(map(probabilityCol), features2prob(col(map(featuresCol)))) + tmpData = tmpData.withColumn($(probabilityCol), features2prob(col($(featuresCol)))) } numColsOutput += 1 } - if (map(predictionCol) != "") { - val t = map(threshold) - if (map(probabilityCol) != "") { + if ($(predictionCol) != "") { + val t = $(threshold) + if ($(probabilityCol) != "") { val predict = udf { probs: Vector => if (probs(1) > t) 1.0 else 0.0 } - tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(probabilityCol)))) - } else if (map(rawPredictionCol) != "") { + tmpData = tmpData.withColumn($(predictionCol), predict(col($(probabilityCol)))) + } else if ($(rawPredictionCol) != "") { val predict = udf { rawPreds: Vector => val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) if (prob1 > t) 1.0 else 0.0 } - tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(rawPredictionCol)))) + tmpData = tmpData.withColumn($(predictionCol), predict(col($(rawPredictionCol)))) } else { val predict = udf { features: Vector => this.predict(features) } - tmpData = tmpData.withColumn(map(predictionCol), predict(col(map(featuresCol)))) + tmpData = tmpData.withColumn($(predictionCol), predict(col($(featuresCol)))) } numColsOutput += 1 } @@ -180,8 +176,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { - println(s"LR.predict with threshold: ${paramMap(threshold)}") - if (score(features) > paramMap(threshold)) 1 else 0 + if (score(features) > getThreshold) 1 else 0 } override protected def predictProbabilities(features: Vector): Vector = { @@ -194,9 +189,7 @@ class LogisticRegressionModel private[ml] ( Vectors.dense(0.0, m) } - override protected def copy(): LogisticRegressionModel = { - val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) - m + override def copy(extra: ParamMap): LogisticRegressionModel = { + copyValues(new LogisticRegressionModel(parent, weights, intercept), extra) } } 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 bd8caac855981..8519841c5c26c 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,13 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} +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.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} - /** * Params for probabilistic classification. */ @@ -33,12 +33,10 @@ private[classification] trait ProbabilisticClassifierParams override protected def validateAndTransformSchema( schema: StructType, - paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + val parentSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + SchemaUtils.appendColumn(parentSchema, $(probabilityCol), new VectorUDT) } } @@ -94,36 +92,22 @@ private[spark] abstract class ProbabilisticClassificationModel[ * - probability of each class as [[probabilityCol]] of type [[Vector]]. * * @param dataset input dataset - * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + override def transform(dataset: DataFrame): DataFrame = { // This default implementation should be overridden as needed. // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - - // Prepare model - val tmpModel = if (paramMap.size != 0) { - val tmpModel = this.copy() - Params.inheritValues(paramMap, parent, tmpModel) - tmpModel - } else { - this - } + transformSchema(dataset.schema, logging = true) val (numColsOutput, outputData) = - ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, this) // Output selected columns only. - if (map(probabilityCol) != "") { + if ($(probabilityCol) != "") { // output probabilities - val features2probs: FeaturesType => Vector = (features) => { - tmpModel.predictProbabilities(features) - } - outputData.withColumn(map(probabilityCol), - callUDF(features2probs, new VectorUDT, col(map(featuresCol)))) + outputData.withColumn($(probabilityCol), + callUDF(predictProbabilities _, new VectorUDT, col($(featuresCol)))) } else { if (numColsOutput == 0) { this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + 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 new file mode 100644 index 0000000000000..17f59bb42e129 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -0,0 +1,178 @@ +/* + * 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.classification + +import scala.collection.mutable + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{RandomForest => OldRandomForest} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for + * classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + */ +@AlphaComponent +final class RandomForestClassifier + extends Predictor[Vector, RandomForestClassifier, RandomForestClassificationModel] + with RandomForestParams with TreeClassifierParams { + + // Override parameter setters from parent trait for Java API compatibility. + + // Parameters from TreeClassifierParams: + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + // Parameters from TreeEnsembleParams: + + override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + + override def setSeed(value: Long): this.type = super.setSeed(value) + + // Parameters from RandomForestParams: + + override def setNumTrees(value: Int): this.type = super.setNumTrees(value) + + override def setFeatureSubsetStrategy(value: String): this.type = + super.setFeatureSubsetStrategy(value) + + override protected def train(dataset: DataFrame): RandomForestClassificationModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => n + case None => throw new IllegalArgumentException("RandomForestClassifier was given input" + + s" with invalid label column ${$(labelCol)}, without the number of classes" + + " specified. See StringIndexer.") + // TODO: Automatically index labels: SPARK-7126 + } + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val strategy = + super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) + val oldModel = OldRandomForest.trainClassifier( + oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed.toInt) + RandomForestClassificationModel.fromOld(oldModel, this, categoricalFeatures) + } +} + +object RandomForestClassifier { + /** Accessor for supported impurity settings: entropy, gini */ + final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities + + /** Accessor for supported featureSubsetStrategy settings: auto, all, onethird, sqrt, log2 */ + final val supportedFeatureSubsetStrategies: Array[String] = + RandomForestParams.supportedFeatureSubsetStrategies +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for classification. + * It supports both binary and multiclass labels, as well as both continuous and categorical + * features. + * @param _trees Decision trees in the ensemble. + * Warning: These have null parents. + */ +@AlphaComponent +final class RandomForestClassificationModel private[ml] ( + override val parent: RandomForestClassifier, + private val _trees: Array[DecisionTreeClassificationModel]) + extends PredictionModel[Vector, RandomForestClassificationModel] + with TreeEnsembleModel with Serializable { + + require(numTrees > 0, "RandomForestClassificationModel requires at least 1 tree.") + + override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + + // Note: We may add support for weights (based on tree performance) later on. + private lazy val _treeWeights: Array[Double] = Array.fill[Double](numTrees)(1.0) + + override def treeWeights: Array[Double] = _treeWeights + + override protected def predict(features: Vector): Double = { + // TODO: Override transform() to broadcast model. SPARK-7127 + // TODO: When we add a generic Bagging class, handle transform there: SPARK-7128 + // Classifies using majority votes. + // Ignore the weights since all are 1.0 for now. + val votes = mutable.Map.empty[Int, Double] + _trees.view.foreach { tree => + val prediction = tree.rootNode.predict(features).toInt + votes(prediction) = votes.getOrElse(prediction, 0.0) + 1.0 // 1.0 = weight + } + votes.maxBy(_._2)._1 + } + + override def copy(extra: ParamMap): RandomForestClassificationModel = { + copyValues(new RandomForestClassificationModel(parent, _trees), extra) + } + + override def toString: String = { + s"RandomForestClassificationModel with $numTrees trees" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldRandomForestModel = { + new OldRandomForestModel(OldAlgo.Classification, _trees.map(_.toOld)) + } +} + +private[ml] object RandomForestClassificationModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldRandomForestModel, + parent: RandomForestClassifier, + categoricalFeatures: Map[Int, Int]): RandomForestClassificationModel = { + require(oldModel.algo == OldAlgo.Classification, "Cannot convert RandomForestModel" + + s" with algo=${oldModel.algo} (old API) to RandomForestClassificationModel (new API).") + val newTrees = oldModel.trees.map { tree => + // parent, fittingParamMap for each tree is null since there are no good ways to set these. + DecisionTreeClassificationModel.fromOld(tree, null, categoricalFeatures) + } + new RandomForestClassificationModel(parent, newTrees) + } +} 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 2360f4479f1c2..e5a73c6087a11 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 @@ -20,30 +20,30 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Evaluator 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.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType - /** * :: AlphaComponent :: * * Evaluator for binary classification, which expects two input columns: score and label. */ @AlphaComponent -class BinaryClassificationEvaluator extends Evaluator with Params - with HasRawPredictionCol with HasLabelCol { +class BinaryClassificationEvaluator extends Evaluator with HasRawPredictionCol with HasLabelCol { /** * param for metric name in evaluation * @group param */ val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (areaUnderROC|areaUnderPR)", Some("areaUnderROC")) + "metric name in evaluation (areaUnderROC|areaUnderPR)") /** @group getParam */ - def getMetricName: String = get(metricName) + def getMetricName: String = $(metricName) /** @group setParam */ def setMetricName(value: String): this.type = set(metricName, value) @@ -54,20 +54,20 @@ class BinaryClassificationEvaluator extends Evaluator with Params /** @group setParam */ def setLabelCol(value: String): this.type = set(labelCol, value) - override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { - val map = this.paramMap ++ paramMap + setDefault(metricName -> "areaUnderROC") + override def evaluate(dataset: DataFrame): Double = { val schema = dataset.schema - checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, $(rawPredictionCol), new VectorUDT) + SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. - val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) + val scoreAndLabels = dataset.select($(rawPredictionCol), $(labelCol)) .map { case Row(rawPrediction: Vector, label: Double) => (rawPrediction(1), label) } val metrics = new BinaryClassificationMetrics(scoreAndLabels) - val metric = map(metricName) match { + val metric = $(metricName) match { case "areaUnderROC" => metrics.areaUnderROC() case "areaUnderPR" => 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 new file mode 100644 index 0000000000000..6eb1db6971111 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -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.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DoubleType, StructType} + +/** + * :: AlphaComponent :: + * Binarize a column of continuous features given a threshold. + */ +@AlphaComponent +final class Binarizer extends Transformer with HasInputCol with HasOutputCol { + + /** + * Param for threshold used to binarize continuous features. + * The features greater than the threshold, will be binarized to 1.0. + * The features equal to or less than the threshold, will be binarized to 0.0. + * @group param + */ + val threshold: DoubleParam = + new DoubleParam(this, "threshold", "threshold used to binarize continuous features") + + /** @group getParam */ + def getThreshold: Double = $(threshold) + + /** @group setParam */ + def setThreshold(value: Double): this.type = set(threshold, value) + + setDefault(threshold -> 0.0) + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + val td = $(threshold) + val binarizer = udf { in: Double => if (in > td) 1.0 else 0.0 } + val outputColName = $(outputCol) + val metadata = BinaryAttribute.defaultAttr.withName(outputColName).toMetadata() + dataset.select(col("*"), + binarizer(col($(inputCol))).as(outputColName, metadata)) + } + + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) + + val inputFields = schema.fields + val outputColName = $(outputCol) + + require(inputFields.forall(_.name != outputColName), + s"Output column $outputColName already exists.") + + val attr = BinaryAttribute.defaultAttr.withName(outputColName) + val outputFields = inputFields :+ attr.toStructField() + StructType(outputFields) + } +} 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 fc4e12773c46d..c305a819a8966 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 @@ -19,9 +19,9 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.param.{IntParam, ParamMap} +import org.apache.spark.ml.param.{IntParam, ParamValidators} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{VectorUDT, Vector} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.types.DataType /** @@ -32,19 +32,23 @@ import org.apache.spark.sql.types.DataType class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { /** - * number of features + * Number of features. Should be > 0. + * (default = 2^18^) * @group param */ - val numFeatures = new IntParam(this, "numFeatures", "number of features", Some(1 << 18)) + val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", + ParamValidators.gt(0)) + + setDefault(numFeatures -> (1 << 18)) /** @group getParam */ - def getNumFeatures: Int = get(numFeatures) + def getNumFeatures: Int = $(numFeatures) /** @group setParam */ def setNumFeatures(value: Int): this.type = set(numFeatures, value) - override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { - val hashingTF = new feature.HashingTF(paramMap(numFeatures)) + override protected def createTransformFunc: Iterable[_] => Vector = { + val hashingTF = new feature.HashingTF($(numFeatures)) hashingTF.transform } 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 new file mode 100644 index 0000000000000..d901a20aed002 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -0,0 +1,110 @@ +/* + * 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 org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml._ +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.feature +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StructType + +/** + * Params for [[IDF]] and [[IDFModel]]. + */ +private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol { + + /** + * The minimum of documents in which a term should appear. + * @group param + */ + final val minDocFreq = new IntParam( + this, "minDocFreq", "minimum of documents in which a term should appear for filtering") + + setDefault(minDocFreq -> 0) + + /** @group getParam */ + def getMinDocFreq: Int = $(minDocFreq) + + /** @group setParam */ + def setMinDocFreq(value: Int): this.type = set(minDocFreq, value) + + /** + * Validate and transform the input schema. + */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) + } +} + +/** + * :: AlphaComponent :: + * Compute the Inverse Document Frequency (IDF) given a collection of documents. + */ +@AlphaComponent +final class IDF extends Estimator[IDFModel] with IDFBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def fit(dataset: DataFrame): IDFModel = { + transformSchema(dataset.schema, logging = true) + val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val idf = new feature.IDF($(minDocFreq)).fit(input) + copyValues(new IDFModel(this, idf)) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + +/** + * :: AlphaComponent :: + * Model fitted by [[IDF]]. + */ +@AlphaComponent +class IDFModel private[ml] ( + override val parent: IDF, + idfModel: feature.IDFModel) + extends Model[IDFModel] with IDFBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + val idf = udf { vec: Vector => idfModel.transform(vec) } + dataset.withColumn($(outputCol), idf(col($(inputCol)))) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} 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 new file mode 100644 index 0000000000000..755b46a64c7f1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -0,0 +1,55 @@ +/* + * 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 org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.{DoubleParam, ParamValidators} +import org.apache.spark.mllib.feature +import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Normalize a vector to have unit norm using the given p-norm. + */ +@AlphaComponent +class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { + + /** + * Normalization in L^p^ space. Must be >= 1. + * (default: p = 2) + * @group param + */ + val p = new DoubleParam(this, "p", "the p norm value", ParamValidators.gtEq(1)) + + setDefault(p -> 2.0) + + /** @group getParam */ + def getP: Double = $(p) + + /** @group setParam */ + def setP(value: Double): this.type = set(p, value) + + override protected def createTransformFunc: Vector => Vector = { + val normalizer = new feature.Normalizer($(p)) + normalizer.transform + } + + override protected def outputDataType: DataType = new VectorUDT() +} 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 new file mode 100644 index 0000000000000..63e190c8aae53 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -0,0 +1,173 @@ +/* + * 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.collection.mutable + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.{IntParam, ParamValidators} +import org.apache.spark.mllib.linalg._ +import org.apache.spark.sql.types.DataType + +/** + * :: AlphaComponent :: + * Perform feature expansion in a polynomial space. As said in wikipedia of Polynomial Expansion, + * which is available at [[http://en.wikipedia.org/wiki/Polynomial_expansion]], "In mathematics, an + * expansion of a product of sums expresses it as a sum of products by using the fact that + * multiplication distributes over addition". Take a 2-variable feature vector as an example: + * `(x, y)`, if we want to expand it with degree 2, then we get `(x, y, x * x, x * y, y * y)`. + */ +@AlphaComponent +class PolynomialExpansion extends UnaryTransformer[Vector, Vector, PolynomialExpansion] { + + /** + * The polynomial degree to expand, which should be >= 1. A value of 1 means no expansion. + * Default: 2 + * @group param + */ + val degree = new IntParam(this, "degree", "the polynomial degree to expand (>= 1)", + ParamValidators.gt(1)) + + setDefault(degree -> 2) + + /** @group getParam */ + def getDegree: Int = $(degree) + + /** @group setParam */ + def setDegree(value: Int): this.type = set(degree, value) + + override protected def createTransformFunc: Vector => Vector = { v => + PolynomialExpansion.expand(v, $(degree)) + } + + override protected def outputDataType: DataType = new VectorUDT() +} + +/** + * The expansion is done via recursion. Given n features and degree d, the size after expansion is + * (n + d choose d) (including 1 and first-order values). For example, let f([a, b, c], 3) be the + * function that expands [a, b, c] to their monomials of degree 3. We have the following recursion: + * + * {{{ + * f([a, b, c], 3) = f([a, b], 3) ++ f([a, b], 2) * c ++ f([a, b], 1) * c^2 ++ [c^3] + * }}} + * + * To handle sparsity, if c is zero, we can skip all monomials that contain it. We remember the + * current index and increment it properly for sparse input. + */ +object PolynomialExpansion { + + private def choose(n: Int, k: Int): Int = { + Range(n, n - k, -1).product / Range(k, 1, -1).product + } + + private def getPolySize(numFeatures: Int, degree: Int): Int = choose(numFeatures + degree, degree) + + private def expandDense( + values: Array[Double], + lastIdx: Int, + degree: Int, + multiplier: Double, + polyValues: Array[Double], + curPolyIdx: Int): Int = { + if (multiplier == 0.0) { + // do nothing + } else if (degree == 0 || lastIdx < 0) { + if (curPolyIdx >= 0) { // skip the very first 1 + polyValues(curPolyIdx) = multiplier + } + } else { + val v = values(lastIdx) + val lastIdx1 = lastIdx - 1 + var alpha = multiplier + var i = 0 + var curStart = curPolyIdx + while (i <= degree && alpha != 0.0) { + curStart = expandDense(values, lastIdx1, degree - i, alpha, polyValues, curStart) + i += 1 + alpha *= v + } + } + curPolyIdx + getPolySize(lastIdx + 1, degree) + } + + private def expandSparse( + indices: Array[Int], + values: Array[Double], + lastIdx: Int, + lastFeatureIdx: Int, + degree: Int, + multiplier: Double, + polyIndices: mutable.ArrayBuilder[Int], + polyValues: mutable.ArrayBuilder[Double], + curPolyIdx: Int): Int = { + if (multiplier == 0.0) { + // do nothing + } else if (degree == 0 || lastIdx < 0) { + if (curPolyIdx >= 0) { // skip the very first 1 + polyIndices += curPolyIdx + polyValues += multiplier + } + } else { + // Skip all zeros at the tail. + val v = values(lastIdx) + val lastIdx1 = lastIdx - 1 + val lastFeatureIdx1 = indices(lastIdx) - 1 + var alpha = multiplier + var curStart = curPolyIdx + var i = 0 + while (i <= degree && alpha != 0.0) { + curStart = expandSparse(indices, values, lastIdx1, lastFeatureIdx1, degree - i, alpha, + polyIndices, polyValues, curStart) + i += 1 + alpha *= v + } + } + curPolyIdx + getPolySize(lastFeatureIdx + 1, degree) + } + + private def expand(dv: DenseVector, degree: Int): DenseVector = { + val n = dv.size + val polySize = getPolySize(n, degree) + val polyValues = new Array[Double](polySize - 1) + expandDense(dv.values, n - 1, degree, 1.0, polyValues, -1) + new DenseVector(polyValues) + } + + private def expand(sv: SparseVector, degree: Int): SparseVector = { + val polySize = getPolySize(sv.size, degree) + val nnz = sv.values.length + val nnzPolySize = getPolySize(nnz, degree) + val polyIndices = mutable.ArrayBuilder.make[Int] + polyIndices.sizeHint(nnzPolySize - 1) + val polyValues = mutable.ArrayBuilder.make[Double] + polyValues.sizeHint(nnzPolySize - 1) + expandSparse( + sv.indices, sv.values, nnz - 1, sv.size - 1, degree, 1.0, polyIndices, polyValues, -1) + new SparseVector(polySize - 1, polyIndices.result(), polyValues.result()) + } + + def expand(v: Vector, degree: Int): Vector = { + v match { + case dv: DenseVector => expand(dv, degree) + case sv: SparseVector => expand(sv, degree) + case _ => throw new IllegalArgumentException + } + } +} 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 1142aa4f8e73d..7cad59ff3fa37 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 @@ -20,6 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ @@ -29,7 +30,24 @@ import org.apache.spark.sql.types.{StructField, StructType} /** * Params for [[StandardScaler]] and [[StandardScalerModel]]. */ -private[feature] trait StandardScalerParams extends Params with HasInputCol with HasOutputCol +private[feature] trait StandardScalerParams extends Params with HasInputCol with HasOutputCol { + + /** + * Centers the data with mean before scaling. + * It will build a dense output, so this does not work on sparse input + * and will raise an exception. + * Default: false + * @group param + */ + val withMean: BooleanParam = new BooleanParam(this, "withMean", "Center data with mean") + + /** + * Scales the data to unit standard deviation. + * Default: true + * @group param + */ + val withStd: BooleanParam = new BooleanParam(this, "withStd", "Scale to unit standard deviation") +} /** * :: AlphaComponent :: @@ -39,30 +57,35 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with @AlphaComponent class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerParams { + setDefault(withMean -> false, withStd -> true) + /** @group setParam */ def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) - - override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } - val scaler = new feature.StandardScaler().fit(input) - val model = new StandardScalerModel(this, map, scaler) - Params.inheritValues(map, this, model) - model + + /** @group setParam */ + def setWithMean(value: Boolean): this.type = set(withMean, value) + + /** @group setParam */ + def setWithStd(value: Boolean): this.type = set(withStd, value) + + override def fit(dataset: DataFrame): StandardScalerModel = { + transformSchema(dataset.schema, logging = true) + val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val scaler = new feature.StandardScaler(withMean = $(withMean), withStd = $(withStd)) + val scalerModel = scaler.fit(input) + copyValues(new StandardScalerModel(this, scalerModel)) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - val inputType = schema(map(inputCol)).dataType + override def transformSchema(schema: StructType): StructType = { + val inputType = schema($(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], - s"Input column ${map(inputCol)} must be a vector column") - require(!schema.fieldNames.contains(map(outputCol)), - s"Output column ${map(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField(map(outputCol), new VectorUDT, false) + s"Input column ${$(inputCol)} must be a vector column") + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) StructType(outputFields) } } @@ -74,7 +97,6 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP @AlphaComponent class StandardScalerModel private[ml] ( override val parent: StandardScaler, - override val fittingParamMap: ParamMap, scaler: feature.StandardScalerModel) extends Model[StandardScalerModel] with StandardScalerParams { @@ -84,21 +106,19 @@ class StandardScalerModel private[ml] ( /** @group setParam */ def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) - dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + val scale = udf { scaler.transform _ } + dataset.withColumn($(outputCol), scale(col($(inputCol)))) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - val inputType = schema(map(inputCol)).dataType + override def transformSchema(schema: StructType): StructType = { + val inputType = schema($(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], - s"Input column ${map(inputCol)} must be a vector column") - require(!schema.fieldNames.contains(map(outputCol)), - s"Output column ${map(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField(map(outputCol), new VectorUDT, false) + s"Input column ${$(inputCol)} must be a vector column") + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) StructType(outputFields) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala new file mode 100644 index 0000000000000..3d78537ad84cb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -0,0 +1,129 @@ +/* + * 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 org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{NumericType, StringType, StructType} +import org.apache.spark.util.collection.OpenHashMap + +/** + * Base trait for [[StringIndexer]] and [[StringIndexerModel]]. + */ +private[feature] trait StringIndexerBase extends Params with HasInputCol with HasOutputCol { + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + val inputColName = $(inputCol) + val inputDataType = schema(inputColName).dataType + require(inputDataType == StringType || inputDataType.isInstanceOf[NumericType], + s"The input column $inputColName must be either string type or numeric type, " + + s"but got $inputDataType.") + val inputFields = schema.fields + val outputColName = $(outputCol) + require(inputFields.forall(_.name != outputColName), + s"Output column $outputColName already exists.") + val attr = NominalAttribute.defaultAttr.withName($(outputCol)) + val outputFields = inputFields :+ attr.toStructField() + StructType(outputFields) + } +} + +/** + * :: AlphaComponent :: + * A label indexer that maps a string column of labels to an ML column of label indices. + * If the input column is numeric, we cast it to string and index the string values. + * The indices are in [0, numLabels), ordered by label frequencies. + * So the most frequent label gets index 0. + */ +@AlphaComponent +class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + // TODO: handle unseen labels + + override def fit(dataset: DataFrame): StringIndexerModel = { + val counts = dataset.select(col($(inputCol)).cast(StringType)) + .map(_.getString(0)) + .countByValue() + val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray + copyValues(new StringIndexerModel(this, labels)) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + +/** + * :: AlphaComponent :: + * Model fitted by [[StringIndexer]]. + */ +@AlphaComponent +class StringIndexerModel private[ml] ( + override val parent: StringIndexer, + labels: Array[String]) extends Model[StringIndexerModel] with StringIndexerBase { + + private val labelToIndex: OpenHashMap[String, Double] = { + val n = labels.length + val map = new OpenHashMap[String, Double](n) + var i = 0 + while (i < n) { + map.update(labels(i), i) + i += 1 + } + map + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + val indexer = udf { label: String => + if (labelToIndex.contains(label)) { + labelToIndex(label) + } else { + // TODO: handle unseen labels + throw new SparkException(s"Unseen label: $label.") + } + } + val outputColName = $(outputCol) + val metadata = NominalAttribute.defaultAttr + .withName(outputColName).withValues(labels).toMetadata() + dataset.select(col("*"), + indexer(dataset($(inputCol)).cast(StringType)).as(outputColName, metadata)) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 0b1f90daa7d8e..2863b7621526e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -19,8 +19,8 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.param.ParamMap -import org.apache.spark.sql.types.{DataType, StringType, ArrayType} +import org.apache.spark.ml.param._ +import org.apache.spark.sql.types.{ArrayType, DataType, StringType} /** * :: AlphaComponent :: @@ -29,7 +29,7 @@ import org.apache.spark.sql.types.{DataType, StringType, ArrayType} @AlphaComponent class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { - override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { + override protected def createTransformFunc: String => Seq[String] = { _.toLowerCase.split("\\s") } @@ -39,3 +39,69 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { override protected def outputDataType: DataType = new ArrayType(StringType, false) } + +/** + * :: AlphaComponent :: + * A regex based tokenizer that extracts tokens either by repeatedly matching the regex(default) + * or using it to split the text (set matching to false). Optional parameters also allow filtering + * tokens using a minimal length. + * It returns an array of strings that can be empty. + */ +@AlphaComponent +class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenizer] { + + /** + * Minimum token length, >= 0. + * Default: 1, to avoid returning empty strings + * @group param + */ + val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length (>= 0)", + ParamValidators.gtEq(0)) + + /** @group setParam */ + def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) + + /** @group getParam */ + def getMinTokenLength: Int = $(minTokenLength) + + /** + * Indicates whether regex splits on gaps (true) or matching tokens (false). + * Default: false + * @group param + */ + val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") + + /** @group setParam */ + def setGaps(value: Boolean): this.type = set(gaps, value) + + /** @group getParam */ + def getGaps: Boolean = $(gaps) + + /** + * Regex pattern used by tokenizer. + * Default: `"\\p{L}+|[^\\p{L}\\s]+"` + * @group param + */ + val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") + + /** @group setParam */ + def setPattern(value: String): this.type = set(pattern, value) + + /** @group getParam */ + def getPattern: String = $(pattern) + + setDefault(minTokenLength -> 1, gaps -> false, pattern -> "\\p{L}+|[^\\p{L}\\s]+") + + override protected def createTransformFunc: String => Seq[String] = { str => + val re = $(pattern).r + val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq + val minLength = $(minTokenLength) + tokens.filter(_.length >= minLength) + } + + override protected def validateInputType(inputType: DataType): Unit = { + require(inputType == StringType, s"Input type must be string type but got $inputType.") + } + + override protected def outputDataType: DataType = new ArrayType(StringType, false) +} 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 new file mode 100644 index 0000000000000..8f2e62a8e2081 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -0,0 +1,107 @@ +/* + * 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.collection.mutable.ArrayBuilder + +import org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.param.shared._ +import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * :: AlphaComponent :: + * A feature transformer than merge multiple columns into a vector column. + */ +@AlphaComponent +class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { + + /** @group setParam */ + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + val assembleFunc = udf { r: Row => + VectorAssembler.assemble(r.toSeq: _*) + } + val schema = dataset.schema + val inputColNames = $(inputCols) + val args = inputColNames.map { c => + schema(c).dataType match { + case DoubleType => dataset(c) + case _: VectorUDT => dataset(c) + case _: NumericType | BooleanType => dataset(c).cast(DoubleType).as(s"${c}_double_$uid") + } + } + dataset.select(col("*"), assembleFunc(struct(args : _*)).as($(outputCol))) + } + + override def transformSchema(schema: StructType): StructType = { + val inputColNames = $(inputCols) + val outputColName = $(outputCol) + val inputDataTypes = inputColNames.map(name => schema(name).dataType) + inputDataTypes.foreach { + case _: NumericType | BooleanType => + case t if t.isInstanceOf[VectorUDT] => + case other => + throw new IllegalArgumentException(s"Data type $other is not supported.") + } + if (schema.fieldNames.contains(outputColName)) { + throw new IllegalArgumentException(s"Output column $outputColName already exists.") + } + StructType(schema.fields :+ new StructField(outputColName, new VectorUDT, false)) + } +} + +@AlphaComponent +object VectorAssembler { + + private[feature] def assemble(vv: Any*): Vector = { + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + var cur = 0 + vv.foreach { + case v: Double => + if (v != 0.0) { + indices += cur + values += v + } + cur += 1 + case vec: Vector => + vec.foreachActive { case (i, v) => + if (v != 0.0) { + indices += cur + i + values += v + } + } + cur += vec.size + case null => + // TODO: output Double.NaN? + throw new SparkException("Values to assemble cannot be null.") + case o => + throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") + } + Vectors.sparse(cur, indices.result(), values.result()) + } +} 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 new file mode 100644 index 0000000000000..07ea579d69893 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -0,0 +1,385 @@ +/* + * 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 org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, BinaryAttribute, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.param.{IntParam, ParamValidators, Params} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, VectorUDT} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions.callUDF +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.collection.OpenHashSet + +/** Private trait for params for VectorIndexer and VectorIndexerModel */ +private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOutputCol { + + /** + * Threshold for the number of values a categorical feature can take. + * If a feature is found to have > maxCategories values, then it is declared continuous. + * Must be >= 2. + * + * (default = 20) + */ + val maxCategories = new IntParam(this, "maxCategories", + "Threshold for the number of values a categorical feature can take (>= 2)." + + " If a feature is found to have > maxCategories values, then it is declared continuous.", + ParamValidators.gtEq(2)) + + setDefault(maxCategories -> 20) + + /** @group getParam */ + def getMaxCategories: Int = $(maxCategories) +} + +/** + * :: AlphaComponent :: + * + * Class for indexing categorical feature columns in a dataset of [[Vector]]. + * + * This has 2 usage modes: + * - Automatically identify categorical features (default behavior) + * - This helps process a dataset of unknown vectors into a dataset with some continuous + * features and some categorical features. The choice between continuous and categorical + * is based upon a maxCategories parameter. + * - Set maxCategories to the maximum number of categorical any categorical feature should have. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories = 2, then feature 0 will be declared categorical and use indices {0, 1}, + * and feature 1 will be declared continuous. + * - Index all features, if all features are categorical + * - If maxCategories is set to be very large, then this will build an index of unique + * values for all features. + * - Warning: This can cause problems if features are continuous since this will collect ALL + * unique values to the driver. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories >= 3, then both features will be declared categorical. + * + * This returns a model which can transform categorical features to use 0-based indices. + * + * Index stability: + * - This is not guaranteed to choose the same category index across multiple runs. + * - If a categorical feature includes value 0, then this is guaranteed to map value 0 to index 0. + * This maintains vector sparsity. + * - More stability may be added in the future. + * + * TODO: Future extensions: The following functionality is planned for the future: + * - Preserve metadata in transform; if a feature's metadata is already present, do not recompute. + * - Specify certain features to not index, either via a parameter or via existing metadata. + * - Add warning if a categorical feature has only 1 category. + * - Add option for allowing unknown categories. + */ +@AlphaComponent +class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerParams { + + /** @group setParam */ + def setMaxCategories(value: Int): this.type = set(maxCategories, value) + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def fit(dataset: DataFrame): VectorIndexerModel = { + transformSchema(dataset.schema, logging = true) + val firstRow = dataset.select($(inputCol)).take(1) + require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") + val numFeatures = firstRow(0).getAs[Vector](0).size + val vectorDataset = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val maxCats = $(maxCategories) + val categoryStats: VectorIndexer.CategoryStats = vectorDataset.mapPartitions { iter => + val localCatStats = new VectorIndexer.CategoryStats(numFeatures, maxCats) + iter.foreach(localCatStats.addVector) + Iterator(localCatStats) + }.reduce((stats1, stats2) => stats1.merge(stats2)) + copyValues(new VectorIndexerModel(this, numFeatures, categoryStats.getCategoryMaps)) + } + + override def transformSchema(schema: StructType): StructType = { + // We do not transfer feature metadata since we do not know what types of features we will + // produce in transform(). + val dataType = new VectorUDT + require(isDefined(inputCol), s"VectorIndexer requires input column parameter: $inputCol") + require(isDefined(outputCol), s"VectorIndexer requires output column parameter: $outputCol") + SchemaUtils.checkColumnType(schema, $(inputCol), dataType) + SchemaUtils.appendColumn(schema, $(outputCol), dataType) + } +} + +private object VectorIndexer { + + /** + * Helper class for tracking unique values for each feature. + * + * TODO: Track which features are known to be continuous already; do not update counts for them. + * + * @param numFeatures This class fails if it encounters a Vector whose length is not numFeatures. + * @param maxCategories This class caps the number of unique values collected at maxCategories. + */ + class CategoryStats(private val numFeatures: Int, private val maxCategories: Int) + extends Serializable { + + /** featureValueSets[feature index] = set of unique values */ + private val featureValueSets = + Array.fill[OpenHashSet[Double]](numFeatures)(new OpenHashSet[Double]()) + + /** Merge with another instance, modifying this instance. */ + def merge(other: CategoryStats): CategoryStats = { + featureValueSets.zip(other.featureValueSets).foreach { case (thisValSet, otherValSet) => + otherValSet.iterator.foreach { x => + // Once we have found > maxCategories values, we know the feature is continuous + // and do not need to collect more values for it. + if (thisValSet.size <= maxCategories) thisValSet.add(x) + } + } + this + } + + /** Add a new vector to this index, updating sets of unique feature values */ + def addVector(v: Vector): Unit = { + require(v.size == numFeatures, s"VectorIndexer expected $numFeatures features but" + + s" found vector of size ${v.size}.") + v match { + case dv: DenseVector => addDenseVector(dv) + case sv: SparseVector => addSparseVector(sv) + } + } + + /** + * Based on stats collected, decide which features are categorical, + * and choose indices for categories. + * + * Sparsity: This tries to maintain sparsity by treating value 0.0 specially. + * If a categorical feature takes value 0.0, then value 0.0 is given index 0. + * + * @return Feature value index. Keys are categorical feature indices (column indices). + * Values are mappings from original features values to 0-based category indices. + */ + def getCategoryMaps: Map[Int, Map[Double, Int]] = { + // Filter out features which are declared continuous. + featureValueSets.zipWithIndex.filter(_._1.size <= maxCategories).map { + case (featureValues: OpenHashSet[Double], featureIndex: Int) => + var sortedFeatureValues = featureValues.iterator.filter(_ != 0.0).toArray.sorted + val zeroExists = sortedFeatureValues.length + 1 == featureValues.size + if (zeroExists) { + sortedFeatureValues = 0.0 +: sortedFeatureValues + } + val categoryMap: Map[Double, Int] = sortedFeatureValues.zipWithIndex.toMap + (featureIndex, categoryMap) + }.toMap + } + + private def addDenseVector(dv: DenseVector): Unit = { + var i = 0 + while (i < dv.size) { + if (featureValueSets(i).size <= maxCategories) { + featureValueSets(i).add(dv(i)) + } + i += 1 + } + } + + private def addSparseVector(sv: SparseVector): Unit = { + // TODO: This might be able to handle 0's more efficiently. + var vecIndex = 0 // index into vector + var k = 0 // index into non-zero elements + while (vecIndex < sv.size) { + val featureValue = if (k < sv.indices.length && vecIndex == sv.indices(k)) { + k += 1 + sv.values(k - 1) + } else { + 0.0 + } + if (featureValueSets(vecIndex).size <= maxCategories) { + featureValueSets(vecIndex).add(featureValue) + } + vecIndex += 1 + } + } + } +} + +/** + * :: AlphaComponent :: + * + * 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), + * Nominal (categorical), or Binary (either continuous or categorical). + * Non-ML metadata is not carried over from the input to the output column. + * + * This maintains vector sparsity. + * + * @param numFeatures Number of features, i.e., length of Vectors which this transforms + * @param categoryMaps Feature value index. Keys are categorical feature indices (column indices). + * Values are maps from original features values to 0-based category indices. + * If a feature is not in this map, it is treated as continuous. + */ +@AlphaComponent +class VectorIndexerModel private[ml] ( + override val parent: VectorIndexer, + val numFeatures: Int, + val categoryMaps: Map[Int, Map[Double, Int]]) + extends Model[VectorIndexerModel] with VectorIndexerParams { + + /** + * Pre-computed feature attributes, with some missing info. + * In transform(), set attribute name and other info, if available. + */ + private val partialFeatureAttributes: Array[Attribute] = { + val attrs = new Array[Attribute](numFeatures) + var categoricalFeatureCount = 0 // validity check for numFeatures, categoryMaps + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (categoryMaps.contains(featureIndex)) { + // categorical feature + val featureValues: Array[String] = + categoryMaps(featureIndex).toArray.sortBy(_._1).map(_._1).map(_.toString) + if (featureValues.length == 2) { + attrs(featureIndex) = new BinaryAttribute(index = Some(featureIndex), + values = Some(featureValues)) + } else { + attrs(featureIndex) = new NominalAttribute(index = Some(featureIndex), + isOrdinal = Some(false), values = Some(featureValues)) + } + categoricalFeatureCount += 1 + } else { + // continuous feature + attrs(featureIndex) = new NumericAttribute(index = Some(featureIndex)) + } + featureIndex += 1 + } + require(categoricalFeatureCount == categoryMaps.size, "VectorIndexerModel given categoryMaps" + + s" with keys outside expected range [0,...,numFeatures), where numFeatures=$numFeatures") + attrs + } + + // TODO: Check more carefully about whether this whole class will be included in a closure. + + /** Per-vector transform function */ + private val transformFunc: Vector => Vector = { + val sortedCatFeatureIndices = categoryMaps.keys.toArray.sorted + val localVectorMap = categoryMaps + val localNumFeatures = numFeatures + val f: Vector => Vector = { (v: Vector) => + assert(v.size == localNumFeatures, "VectorIndexerModel expected vector of length" + + s" $numFeatures but found length ${v.size}") + v match { + case dv: DenseVector => + val tmpv = dv.copy + localVectorMap.foreach { case (featureIndex: Int, categoryMap: Map[Double, Int]) => + tmpv.values(featureIndex) = categoryMap(tmpv(featureIndex)) + } + tmpv + case sv: SparseVector => + // We use the fact that categorical value 0 is always mapped to index 0. + val tmpv = sv.copy + var catFeatureIdx = 0 // index into sortedCatFeatureIndices + var k = 0 // index into non-zero elements of sparse vector + while (catFeatureIdx < sortedCatFeatureIndices.length && k < tmpv.indices.length) { + val featureIndex = sortedCatFeatureIndices(catFeatureIdx) + if (featureIndex < tmpv.indices(k)) { + catFeatureIdx += 1 + } else if (featureIndex > tmpv.indices(k)) { + k += 1 + } else { + tmpv.values(k) = localVectorMap(featureIndex)(tmpv.values(k)) + catFeatureIdx += 1 + k += 1 + } + } + tmpv + } + } + f + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + val newField = prepOutputField(dataset.schema) + val newCol = callUDF(transformFunc, new VectorUDT, dataset($(inputCol))) + dataset.withColumn($(outputCol), newCol.as($(outputCol), newField.metadata)) + } + + override def transformSchema(schema: StructType): StructType = { + val dataType = new VectorUDT + require(isDefined(inputCol), + s"VectorIndexerModel requires input column parameter: $inputCol") + require(isDefined(outputCol), + s"VectorIndexerModel requires output column parameter: $outputCol") + SchemaUtils.checkColumnType(schema, $(inputCol), dataType) + + // If the input metadata specifies numFeatures, compare with expected numFeatures. + val origAttrGroup = AttributeGroup.fromStructField(schema($(inputCol))) + val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { + Some(origAttrGroup.attributes.get.length) + } else { + origAttrGroup.numAttributes + } + require(origNumFeatures.forall(_ == numFeatures), "VectorIndexerModel expected" + + s" $numFeatures features, but input column ${$(inputCol)} had metadata specifying" + + s" ${origAttrGroup.numAttributes.get} features.") + + val newField = prepOutputField(schema) + val outputFields = schema.fields :+ newField + StructType(outputFields) + } + + /** + * Prepare the output column field, including per-feature metadata. + * @param schema Input schema + * @return Output column field. This field does not contain non-ML metadata. + */ + private def prepOutputField(schema: StructType): StructField = { + val origAttrGroup = AttributeGroup.fromStructField(schema($(inputCol))) + val featureAttributes: Array[Attribute] = if (origAttrGroup.attributes.nonEmpty) { + // Convert original attributes to modified attributes + val origAttrs: Array[Attribute] = origAttrGroup.attributes.get + origAttrs.zip(partialFeatureAttributes).map { + case (origAttr: Attribute, featAttr: BinaryAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NominalAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NumericAttribute) => + origAttr.withIndex(featAttr.index.get) + } + } else { + partialFeatureAttributes + } + val newAttributeGroup = new AttributeGroup($(outputCol), featureAttributes) + newAttributeGroup.toStructField() + } +} 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 new file mode 100644 index 0000000000000..34ff92970129f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -0,0 +1,179 @@ +/* + * 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 org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +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.feature +import org.apache.spark.mllib.linalg.{VectorUDT, Vectors} +import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * Params for [[Word2Vec]] and [[Word2VecModel]]. + */ +private[feature] trait Word2VecBase extends Params + with HasInputCol with HasOutputCol with HasMaxIter with HasStepSize with HasSeed { + + /** + * The dimension of the code that you want to transform from words. + */ + final val vectorSize = new IntParam( + this, "vectorSize", "the dimension of codes after transforming from words") + setDefault(vectorSize -> 100) + + /** @group getParam */ + def getVectorSize: Int = $(vectorSize) + + /** + * Number of partitions for sentences of words. + */ + final val numPartitions = new IntParam( + this, "numPartitions", "number of partitions for sentences of words") + setDefault(numPartitions -> 1) + + /** @group getParam */ + def getNumPartitions: Int = $(numPartitions) + + /** + * The minimum number of times a token must appear to be included in the word2vec model's + * vocabulary. + */ + final val minCount = new IntParam(this, "minCount", "the minimum number of times a token must " + + "appear to be included in the word2vec model's vocabulary") + setDefault(minCount -> 5) + + /** @group getParam */ + def getMinCount: Int = $(minCount) + + setDefault(stepSize -> 0.025) + setDefault(maxIter -> 1) + setDefault(seed -> 42L) + + /** + * Validate and transform the input schema. + */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new ArrayType(StringType, true)) + SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) + } +} + +/** + * :: AlphaComponent :: + * Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further + * natural language processing or machine learning process. + */ +@AlphaComponent +final class Word2Vec extends Estimator[Word2VecModel] with Word2VecBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + /** @group setParam */ + def setVectorSize(value: Int): this.type = set(vectorSize, value) + + /** @group setParam */ + def setStepSize(value: Double): this.type = set(stepSize, value) + + /** @group setParam */ + def setNumPartitions(value: Int): this.type = set(numPartitions, value) + + /** @group setParam */ + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + def setSeed(value: Long): this.type = set(seed, value) + + /** @group setParam */ + def setMinCount(value: Int): this.type = set(minCount, value) + + override def fit(dataset: DataFrame): Word2VecModel = { + transformSchema(dataset.schema, logging = true) + val input = dataset.select($(inputCol)).map(_.getAs[Seq[String]](0)) + val wordVectors = new feature.Word2Vec() + .setLearningRate($(stepSize)) + .setMinCount($(minCount)) + .setNumIterations($(maxIter)) + .setNumPartitions($(numPartitions)) + .setSeed($(seed)) + .setVectorSize($(vectorSize)) + .fit(input) + copyValues(new Word2VecModel(this, wordVectors)) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + +/** + * :: AlphaComponent :: + * Model fitted by [[Word2Vec]]. + */ +@AlphaComponent +class Word2VecModel private[ml] ( + override val parent: Word2Vec, + wordVectors: feature.Word2VecModel) + extends Model[Word2VecModel] with Word2VecBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + /** + * Transform a sentence column to a vector column to represent the whole sentence. The transform + * is performed by averaging all word vectors it contains. + */ + override def transform(dataset: DataFrame): DataFrame = { + transformSchema(dataset.schema, logging = true) + val bWordVectors = dataset.sqlContext.sparkContext.broadcast(wordVectors) + val word2Vec = udf { sentence: Seq[String] => + if (sentence.size == 0) { + Vectors.sparse($(vectorSize), Array.empty[Int], Array.empty[Double]) + } else { + val cum = Vectors.zeros($(vectorSize)) + val model = bWordVectors.value.getVectors + for (word <- sentence) { + if (model.contains(word)) { + axpy(1.0, bWordVectors.value.transform(word), cum) + } else { + // pass words which not belong to model + } + } + scal(1.0 / sentence.size, cum) + cum + } + } + dataset.withColumn($(outputCol), word2Vec(col($(inputCol)))) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index dfb89cc8d4af3..e8b3628140e99 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -20,14 +20,15 @@ package org.apache.spark.ml.impl.estimator import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ -import org.apache.spark.mllib.linalg.{VectorUDT, Vector} +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, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} - /** * :: DeveloperApi :: * @@ -42,7 +43,6 @@ private[spark] trait PredictorParams extends Params /** * Validates and transforms the input schema with the provided param map. * @param schema input schema - * @param paramMap additional parameters * @param fitting whether this is in fitting * @param featuresDataType SQL DataType for FeaturesType. * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. @@ -50,17 +50,15 @@ private[spark] trait PredictorParams extends Params */ protected def validateAndTransformSchema( schema: StructType, - paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val map = this.paramMap ++ paramMap // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector - checkInputColumn(schema, map(featuresCol), featuresDataType) + SchemaUtils.checkColumnType(schema, $(featuresCol), featuresDataType) if (fitting) { // TODO: Allow other numeric types - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) } - addOutputColumn(schema, map(predictionCol), DoubleType) + SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) } } @@ -94,14 +92,15 @@ private[spark] abstract class Predictor[ /** @group setParam */ def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] - override def fit(dataset: DataFrame, paramMap: ParamMap): M = { + override def fit(dataset: DataFrame): M = { // This handles a few items such as schema validation. // Developers only need to implement train(). - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val model = train(dataset, map) - Params.inheritValues(map, this, model) // copy params to model - model + transformSchema(dataset.schema, logging = true) + copyValues(train(dataset)) + } + + override def copy(extra: ParamMap): Learner = { + super.copy(extra).asInstanceOf[Learner] } /** @@ -112,12 +111,10 @@ private[spark] abstract class Predictor[ * and copying parameters into the model. * * @param dataset Training dataset - * @param paramMap Parameter map. Unlike [[fit()]]'s paramMap, this paramMap has already - * been combined with the embedded ParamMap. * @return Fitted model */ @DeveloperApi - protected def train(dataset: DataFrame, paramMap: ParamMap): M + protected def train(dataset: DataFrame): M /** * :: DeveloperApi :: @@ -132,17 +129,16 @@ private[spark] abstract class Predictor[ @DeveloperApi protected def featuresDataType: DataType = new VectorUDT - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = true, featuresDataType) + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema, fitting = true, featuresDataType) } /** * Extract [[labelCol]] and [[featuresCol]] from the given dataset, * and put it in an RDD with strong types. */ - protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { - val map = this.paramMap ++ paramMap - dataset.select(map(labelCol), map(featuresCol)) + protected def extractLabeledPoints(dataset: DataFrame): RDD[LabeledPoint] = { + dataset.select($(labelCol), $(featuresCol)) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } @@ -184,8 +180,8 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel @DeveloperApi protected def featuresDataType: DataType = new VectorUDT - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = false, featuresDataType) + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema, fitting = false, featuresDataType) } /** @@ -193,30 +189,16 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel * the predictions as a new column [[predictionCol]]. * * @param dataset input dataset - * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset with [[predictionCol]] of type [[Double]] */ - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + override def transform(dataset: DataFrame): DataFrame = { // This default implementation should be overridden as needed. // Check schema - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - - // Prepare model - val tmpModel = if (paramMap.size != 0) { - val tmpModel = this.copy() - Params.inheritValues(paramMap, parent, tmpModel) - tmpModel - } else { - this - } + transformSchema(dataset.schema, logging = true) - if (map(predictionCol) != "") { - val pred: FeaturesType => Double = (features) => { - tmpModel.predict(features) - } - dataset.withColumn(map(predictionCol), callUDF(pred, DoubleType, col(map(featuresCol)))) + if ($(predictionCol) != "") { + dataset.withColumn($(predictionCol), callUDF(predict _, DoubleType, col($(featuresCol)))) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + " since no output columns were set.") @@ -232,10 +214,4 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel */ @DeveloperApi protected def predict(features: FeaturesType): Double - - /** - * Create a copy of the model. - * The copy is shallow, except for the embedded paramMap, which gets a deep copy. - */ - protected def copy(): M } diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala new file mode 100644 index 0000000000000..0e225627d4ee3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/tree/treeParams.scala @@ -0,0 +1,431 @@ +/* + * 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.impl.tree + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.impl.estimator.PredictorParams +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.{HasMaxIter, HasSeed} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.impurity.{Entropy => OldEntropy, Gini => OldGini, Impurity => OldImpurity, Variance => OldVariance} +import org.apache.spark.mllib.tree.loss.{Loss => OldLoss} + +/** + * :: DeveloperApi :: + * Parameters for Decision Tree-based algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait DecisionTreeParams extends PredictorParams { + + /** + * Maximum depth of the tree (>= 0). + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (default = 5) + * @group param + */ + final val maxDepth: IntParam = + new IntParam(this, "maxDepth", "Maximum depth of the tree. (>= 0)" + + " E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.", + ParamValidators.gtEq(0)) + + /** + * Maximum number of bins used for discretizing continuous features and for choosing how to split + * on features at each node. More bins give higher granularity. + * Must be >= 2 and >= number of categories in any categorical feature. + * (default = 32) + * @group param + */ + final val maxBins: IntParam = new IntParam(this, "maxBins", "Max number of bins for" + + " discretizing continuous features. Must be >=2 and >= number of categories for any" + + " categorical feature.", ParamValidators.gtEq(2)) + + /** + * Minimum number of instances each child must have after split. + * If a split causes the left or right child to have fewer than minInstancesPerNode, + * the split will be discarded as invalid. + * Should be >= 1. + * (default = 1) + * @group param + */ + final val minInstancesPerNode: IntParam = new IntParam(this, "minInstancesPerNode", "Minimum" + + " number of instances each child must have after split. If a split causes the left or right" + + " child to have fewer than minInstancesPerNode, the split will be discarded as invalid." + + " Should be >= 1.", ParamValidators.gtEq(1)) + + /** + * Minimum information gain for a split to be considered at a tree node. + * (default = 0.0) + * @group param + */ + final val minInfoGain: DoubleParam = new DoubleParam(this, "minInfoGain", + "Minimum information gain for a split to be considered at a tree node.") + + /** + * Maximum memory in MB allocated to histogram aggregation. + * (default = 256 MB) + * @group expertParam + */ + final val maxMemoryInMB: IntParam = new IntParam(this, "maxMemoryInMB", + "Maximum memory in MB allocated to histogram aggregation.", + ParamValidators.gtEq(0)) + + /** + * If false, the algorithm will pass trees to executors to match instances with nodes. + * If true, the algorithm will cache node IDs for each instance. + * Caching can speed up training of deeper trees. + * (default = false) + * @group expertParam + */ + final val cacheNodeIds: BooleanParam = new BooleanParam(this, "cacheNodeIds", "If false, the" + + " algorithm will pass trees to executors to match instances with nodes. If true, the" + + " algorithm will cache node IDs for each instance. Caching can speed up training of deeper" + + " trees.") + + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be >= 1. + * (default = 10) + * @group expertParam + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "Specifies" + + " how often to checkpoint the cached node IDs. E.g. 10 means that the cache will get" + + " checkpointed every 10 iterations. This is only used if cacheNodeIds is true and if the" + + " checkpoint directory is set in the SparkContext. Must be >= 1.", + ParamValidators.gtEq(1)) + + setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, minInfoGain -> 0.0, + maxMemoryInMB -> 256, cacheNodeIds -> false, checkpointInterval -> 10) + + /** @group setParam */ + def setMaxDepth(value: Int): this.type = set(maxDepth, value) + + /** @group getParam */ + final def getMaxDepth: Int = $(maxDepth) + + /** @group setParam */ + def setMaxBins(value: Int): this.type = set(maxBins, value) + + /** @group getParam */ + final def getMaxBins: Int = $(maxBins) + + /** @group setParam */ + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + + /** @group getParam */ + final def getMinInstancesPerNode: Int = $(minInstancesPerNode) + + /** @group setParam */ + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + + /** @group getParam */ + final def getMinInfoGain: Double = $(minInfoGain) + + /** @group expertSetParam */ + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + + /** @group expertGetParam */ + final def getMaxMemoryInMB: Int = $(maxMemoryInMB) + + /** @group expertSetParam */ + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + + /** @group expertGetParam */ + final def getCacheNodeIds: Boolean = $(cacheNodeIds) + + /** @group expertSetParam */ + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + + /** @group expertGetParam */ + final def getCheckpointInterval: Int = $(checkpointInterval) + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int, + oldAlgo: OldAlgo.Algo, + oldImpurity: OldImpurity, + subsamplingRate: Double): OldStrategy = { + val strategy = OldStrategy.defaultStategy(oldAlgo) + strategy.impurity = oldImpurity + strategy.checkpointInterval = getCheckpointInterval + strategy.maxBins = getMaxBins + strategy.maxDepth = getMaxDepth + strategy.maxMemoryInMB = getMaxMemoryInMB + strategy.minInfoGain = getMinInfoGain + strategy.minInstancesPerNode = getMinInstancesPerNode + strategy.useNodeIdCache = getCacheNodeIds + strategy.numClasses = numClasses + strategy.categoricalFeaturesInfo = categoricalFeatures + strategy.subsamplingRate = subsamplingRate + strategy + } +} + +/** + * Parameters for Decision Tree-based classification algorithms. + */ +private[ml] trait TreeClassifierParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "entropy" and "gini". + * (default = gini) + * @group param + */ + final val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeClassifierParams.supportedImpurities.mkString(", ")}", + (value: String) => TreeClassifierParams.supportedImpurities.contains(value.toLowerCase)) + + setDefault(impurity -> "gini") + + /** @group setParam */ + def setImpurity(value: String): this.type = set(impurity, value) + + /** @group getParam */ + final def getImpurity: String = $(impurity).toLowerCase + + /** Convert new impurity to old impurity. */ + private[ml] def getOldImpurity: OldImpurity = { + getImpurity match { + case "entropy" => OldEntropy + case "gini" => OldGini + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeClassifierParams was given unrecognized impurity: $impurity.") + } + } +} + +private[ml] object TreeClassifierParams { + // These options should be lowercase. + final val supportedImpurities: Array[String] = Array("entropy", "gini").map(_.toLowerCase) +} + +/** + * Parameters for Decision Tree-based regression algorithms. + */ +private[ml] trait TreeRegressorParams extends Params { + + /** + * Criterion used for information gain calculation (case-insensitive). + * Supported: "variance". + * (default = variance) + * @group param + */ + final val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + + " information gain calculation (case-insensitive). Supported options:" + + s" ${TreeRegressorParams.supportedImpurities.mkString(", ")}", + (value: String) => TreeRegressorParams.supportedImpurities.contains(value.toLowerCase)) + + setDefault(impurity -> "variance") + + /** @group setParam */ + def setImpurity(value: String): this.type = set(impurity, value) + + /** @group getParam */ + final def getImpurity: String = $(impurity).toLowerCase + + /** Convert new impurity to old impurity. */ + private[ml] def getOldImpurity: OldImpurity = { + getImpurity match { + case "variance" => OldVariance + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException( + s"TreeRegressorParams was given unrecognized impurity: $impurity") + } + } +} + +private[ml] object TreeRegressorParams { + // These options should be lowercase. + final val supportedImpurities: Array[String] = Array("variance").map(_.toLowerCase) +} + +/** + * :: DeveloperApi :: + * Parameters for Decision Tree-based ensemble algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait TreeEnsembleParams extends DecisionTreeParams with HasSeed { + + /** + * Fraction of the training data used for learning each decision tree, in range (0, 1]. + * (default = 1.0) + * @group param + */ + final val subsamplingRate: DoubleParam = new DoubleParam(this, "subsamplingRate", + "Fraction of the training data used for learning each decision tree, in range (0, 1].", + ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) + + setDefault(subsamplingRate -> 1.0) + + /** @group setParam */ + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + + /** @group getParam */ + final def getSubsamplingRate: Double = $(subsamplingRate) + + /** @group setParam */ + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Create a Strategy instance to use with the old API. + * NOTE: The caller should set impurity and seed. + */ + private[ml] def getOldStrategy( + categoricalFeatures: Map[Int, Int], + numClasses: Int, + oldAlgo: OldAlgo.Algo, + oldImpurity: OldImpurity): OldStrategy = { + super.getOldStrategy(categoricalFeatures, numClasses, oldAlgo, oldImpurity, getSubsamplingRate) + } +} + +/** + * :: DeveloperApi :: + * Parameters for Random Forest algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait RandomForestParams extends TreeEnsembleParams { + + /** + * Number of trees to train (>= 1). + * If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. + * TODO: Change to always do bootstrapping (simpler). SPARK-7130 + * (default = 20) + * @group param + */ + final val numTrees: IntParam = new IntParam(this, "numTrees", "Number of trees to train (>= 1)", + ParamValidators.gtEq(1)) + + /** + * The number of features to consider for splits at each tree node. + * Supported options: + * - "auto": Choose automatically for task: + * If numTrees == 1, set to "all." + * If numTrees > 1 (forest), set to "sqrt" for classification and + * to "onethird" for regression. + * - "all": use all features + * - "onethird": use 1/3 of the features + * - "sqrt": use sqrt(number of features) + * - "log2": use log2(number of features) + * (default = "auto") + * + * These various settings are based on the following references: + * - log2: tested in Breiman (2001) + * - sqrt: recommended by Breiman manual for random forests + * - The defaults of sqrt (classification) and onethird (regression) match the R randomForest + * package. + * @see [[http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf Breiman (2001)]] + * @see [[http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf Breiman manual for + * random forests]] + * + * @group param + */ + final val featureSubsetStrategy: Param[String] = new Param[String](this, "featureSubsetStrategy", + "The number of features to consider for splits at each tree node." + + s" Supported options: ${RandomForestParams.supportedFeatureSubsetStrategies.mkString(", ")}", + (value: String) => + RandomForestParams.supportedFeatureSubsetStrategies.contains(value.toLowerCase)) + + setDefault(numTrees -> 20, featureSubsetStrategy -> "auto") + + /** @group setParam */ + def setNumTrees(value: Int): this.type = set(numTrees, value) + + /** @group getParam */ + final def getNumTrees: Int = $(numTrees) + + /** @group setParam */ + def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) + + /** @group getParam */ + final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase +} + +private[ml] object RandomForestParams { + // These options should be lowercase. + final val supportedFeatureSubsetStrategies: Array[String] = + Array("auto", "all", "onethird", "sqrt", "log2").map(_.toLowerCase) +} + +/** + * :: DeveloperApi :: + * Parameters for Gradient-Boosted Tree algorithms. + * + * Note: Marked as private and DeveloperApi since this may be made public in the future. + */ +@DeveloperApi +private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter { + + /** + * Step size (a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each + * estimator. + * (default = 0.1) + * @group param + */ + final val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size (a.k.a." + + " learning rate) in interval (0, 1] for shrinking the contribution of each estimator", + ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) + + /* TODO: Add this doc when we add this param. SPARK-7132 + * Threshold for stopping early when runWithValidation is used. + * If the error rate on the validation input changes by less than the validationTol, + * then learning will stop early (before [[numIterations]]). + * This parameter is ignored when run is used. + * (default = 1e-5) + * @group param + */ + // final val validationTol: DoubleParam = new DoubleParam(this, "validationTol", "") + // validationTol -> 1e-5 + + setDefault(maxIter -> 20, stepSize -> 0.1) + + /** @group setParam */ + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + def setStepSize(value: Double): this.type = set(stepSize, value) + + /** @group getParam */ + final def getStepSize: Double = $(stepSize) + + /** (private[ml]) Create a BoostingStrategy instance to use with the old API. */ + private[ml] def getOldBoostingStrategy( + categoricalFeatures: Map[Int, Int], + oldAlgo: OldAlgo.Algo): OldBoostingStrategy = { + val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 2, oldAlgo, OldVariance) + // NOTE: The old API does not support "seed" so we ignore it. + new OldBoostingStrategy(strategy, getOldLossType, getMaxIter, getStepSize) + } + + /** Get old Gradient Boosting Loss type */ + private[ml] def getOldLossType: OldLoss +} 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 b45bd1499b72e..ac75e9de1a8f2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -32,6 +32,18 @@ package org.apache.spark * @groupname getParam Parameter getters * @groupprio getParam 6 * + * @groupname expertParam (expert-only) Parameters + * @groupdesc expertParam A list of advanced, expert-only (hyper-)parameter keys this algorithm can + * take. Users can set and get the parameter values through setters and getters, + * respectively. + * @groupprio expertParam 7 + * + * @groupname expertSetParam (expert-only) Parameter setters + * @groupprio expertSetParam 8 + * + * @groupname expertGetParam (expert-only) Parameter getters + * @groupprio expertGetParam 9 + * * @groupname Ungrouped Members * @groupprio Ungrouped 0 */ 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 17ece897a6c55..51ce19d29cd29 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 @@ -17,15 +17,14 @@ package org.apache.spark.ml.param -import scala.annotation.varargs -import scala.collection.mutable - import java.lang.reflect.Modifier +import java.util.NoSuchElementException -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.Identifiable -import org.apache.spark.sql.types.{DataType, StructField, StructType} +import scala.annotation.varargs +import scala.collection.mutable +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.util.Identifiable /** * :: AlphaComponent :: @@ -35,16 +34,36 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * @param parent parent object * @param name param name * @param doc documentation + * @param isValid optional validation method which indicates if a value is valid. + * See [[ParamValidators]] for factory methods for common validation functions. * @tparam T param value type */ @AlphaComponent -class Param[T] ( - val parent: Params, - val name: String, - val doc: String, - val defaultValue: Option[T] = None) +class Param[T] (val parent: Params, val name: String, val doc: String, val isValid: T => Boolean) extends Serializable { + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue[T]) + + /** + * Assert that the given value is valid for this parameter. + * + * Note: Parameter checks involving interactions between multiple parameters should be + * implemented in [[Params.validateParams()]]. Checks for input/output columns should be + * implemented in [[org.apache.spark.ml.PipelineStage.transformSchema()]]. + * + * DEVELOPERS: This method is only called by [[ParamPair]], which means that all parameters + * should be specified via [[ParamPair]]. + * + * @throws IllegalArgumentException if the value is invalid + */ + private[param] def validate(value: T): Unit = { + if (!isValid(value)) { + throw new IllegalArgumentException(s"$parent parameter $name given invalid value $value." + + s" Parameter description: $toString") + } + } + /** * Creates a param pair with the given value (for Java). */ @@ -55,58 +74,146 @@ class Param[T] ( */ def ->(value: T): ParamPair[T] = ParamPair(this, value) + /** + * Converts this param's name, doc, and optionally its default value and the user-supplied + * value in its parent to string. + */ override def toString: String = { - if (defaultValue.isDefined) { - s"$name: $doc (default: ${defaultValue.get})" + val valueStr = if (parent.isDefined(this)) { + val defaultValueStr = parent.getDefault(this).map("default: " + _) + val currentValueStr = parent.get(this).map("current: " + _) + (defaultValueStr ++ currentValueStr).mkString("(", ", ", ")") } else { - s"$name: $doc" + "(undefined)" } + s"$name: $doc $valueStr" + } +} + +/** + * Factory methods for common validation functions for [[Param.isValid]]. + * The numerical methods only support Int, Long, Float, and Double. + */ +object ParamValidators { + + /** (private[param]) Default validation always return true */ + private[param] def alwaysTrue[T]: T => Boolean = (_: T) => true + + /** + * Private method for checking numerical types and converting to Double. + * This is mainly for the sake of compilation; type checks are really handled + * by [[Params]] setters and the [[ParamPair]] constructor. + */ + private def getDouble[T](value: T): Double = value match { + case x: Int => x.toDouble + case x: Long => x.toDouble + case x: Float => x.toDouble + case x: Double => x.toDouble + case _ => + // The type should be checked before this is ever called. + throw new IllegalArgumentException("Numerical Param validation failed because" + + s" of unexpected input type: ${value.getClass}") + } + + /** Check if value > lowerBound */ + def gt[T](lowerBound: Double): T => Boolean = { (value: T) => + getDouble(value) > lowerBound + } + + /** Check if value >= lowerBound */ + def gtEq[T](lowerBound: Double): T => Boolean = { (value: T) => + getDouble(value) >= lowerBound + } + + /** Check if value < upperBound */ + def lt[T](upperBound: Double): T => Boolean = { (value: T) => + getDouble(value) < upperBound + } + + /** Check if value <= upperBound */ + def ltEq[T](upperBound: Double): T => Boolean = { (value: T) => + getDouble(value) <= upperBound + } + + /** + * Check for value in range lowerBound to upperBound. + * @param lowerInclusive If true, check for value >= lowerBound. + * If false, check for value > lowerBound. + * @param upperInclusive If true, check for value <= upperBound. + * If false, check for value < upperBound. + */ + def inRange[T]( + lowerBound: Double, + upperBound: Double, + lowerInclusive: Boolean, + upperInclusive: Boolean): T => Boolean = { (value: T) => + val x: Double = getDouble(value) + val lowerValid = if (lowerInclusive) x >= lowerBound else x > lowerBound + val upperValid = if (upperInclusive) x <= upperBound else x < upperBound + lowerValid && upperValid + } + + /** Version of [[inRange()]] which uses inclusive be default: [lowerBound, upperBound] */ + def inRange[T](lowerBound: Double, upperBound: Double): T => Boolean = { + inRange[T](lowerBound, upperBound, lowerInclusive = true, upperInclusive = true) + } + + /** Check for value in an allowed set of values. */ + def inArray[T](allowed: Array[T]): T => Boolean = { (value: T) => + allowed.contains(value) + } + + /** Check for value in an allowed set of values. */ + def inArray[T](allowed: java.util.List[T]): T => Boolean = { (value: T) => + allowed.contains(value) } } // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** Specialized version of [[Param[Double]]] for Java. */ -class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double]) - extends Param[Double](parent, name, doc, defaultValue) { +class DoubleParam(parent: Params, name: String, doc: String, isValid: Double => Boolean) + extends Param[Double](parent, name, doc, isValid) { - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue) override def w(value: Double): ParamPair[Double] = super.w(value) } /** Specialized version of [[Param[Int]]] for Java. */ -class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int]) - extends Param[Int](parent, name, doc, defaultValue) { +class IntParam(parent: Params, name: String, doc: String, isValid: Int => Boolean) + extends Param[Int](parent, name, doc, isValid) { - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue) override def w(value: Int): ParamPair[Int] = super.w(value) } /** Specialized version of [[Param[Float]]] for Java. */ -class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float]) - extends Param[Float](parent, name, doc, defaultValue) { +class FloatParam(parent: Params, name: String, doc: String, isValid: Float => Boolean) + extends Param[Float](parent, name, doc, isValid) { - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue) override def w(value: Float): ParamPair[Float] = super.w(value) } /** Specialized version of [[Param[Long]]] for Java. */ -class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long]) - extends Param[Long](parent, name, doc, defaultValue) { +class LongParam(parent: Params, name: String, doc: String, isValid: Long => Boolean) + extends Param[Long](parent, name, doc, isValid) { - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) + def this(parent: Params, name: String, doc: String) = + this(parent, name, doc, ParamValidators.alwaysTrue) override def w(value: Long): ParamPair[Long] = super.w(value) } /** Specialized version of [[Param[Boolean]]] for Java. */ -class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean]) - extends Param[Boolean](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class BooleanParam(parent: Params, name: String, doc: String) // No need for isValid + extends Param[Boolean](parent, name, doc) { override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } @@ -114,7 +221,11 @@ class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Opti /** * A param amd its value. */ -case class ParamPair[T](param: Param[T], value: T) +case class ParamPair[T](param: Param[T], value: T) { + // This is *the* place Param.validate is called. Whenever a parameter is specified, we should + // always construct a ParamPair so that validate is called. + param.validate(value) +} /** * :: AlphaComponent :: @@ -124,8 +235,11 @@ case class ParamPair[T](param: Param[T], value: T) @AlphaComponent trait Params extends Identifiable with Serializable { - /** Returns all params. */ - def params: Array[Param[_]] = { + /** + * Returns all params sorted by their names. The default implementation uses Java reflection to + * list all public methods that have no arguments and return [[Param]]. + */ + lazy val params: Array[Param[_]] = { val methods = this.getClass.getMethods methods.filter { m => Modifier.isPublic(m.getModifiers) && @@ -138,14 +252,30 @@ trait Params extends Identifiable with Serializable { /** * Validates parameter values stored internally plus the input parameter map. * Raises an exception if any parameter is invalid. + * + * This only needs to check for interactions between parameters. + * Parameter value checks which do not depend on other parameters are handled by + * [[Param.validate()]]. This method does not handle input/output column parameters; + * those are checked during schema validation. */ - def validate(paramMap: ParamMap): Unit = {} + def validateParams(paramMap: ParamMap): Unit = { + copy(paramMap).validateParams() + } /** * Validates parameter values stored internally. * Raise an exception if any parameter value is invalid. + * + * This only needs to check for interactions between parameters. + * Parameter value checks which do not depend on other parameters are handled by + * [[Param.validate()]]. This method does not handle input/output column parameters; + * those are checked during schema validation. */ - def validate(): Unit = validate(ParamMap.empty) + def validateParams(): Unit = { + params.filter(isDefined _).foreach { param => + param.asInstanceOf[Param[Any]].validate($(param)) + } + } /** * Returns the documentation of all params. @@ -153,25 +283,34 @@ trait Params extends Identifiable with Serializable { def explainParams(): String = params.mkString("\n") /** Checks whether a param is explicitly set. */ - def isSet(param: Param[_]): Boolean = { - require(param.parent.eq(this)) + final def isSet(param: Param[_]): Boolean = { + shouldOwn(param) paramMap.contains(param) } + /** Checks whether a param is explicitly set or has a default value. */ + final def isDefined(param: Param[_]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) || paramMap.contains(param) + } + + /** Tests whether this instance contains a param with a given name. */ + def hasParam(paramName: String): Boolean = { + params.exists(_.name == paramName) + } + /** Gets a param by its name. */ - private[ml] def getParam(paramName: String): Param[Any] = { - val m = this.getClass.getMethod(paramName) - assert(Modifier.isPublic(m.getModifiers) && - classOf[Param[_]].isAssignableFrom(m.getReturnType) && - m.getParameterTypes.isEmpty) - m.invoke(this).asInstanceOf[Param[Any]] + def getParam(paramName: String): Param[Any] = { + params.find(_.name == paramName).getOrElse { + throw new NoSuchElementException(s"Param $paramName does not exist.") + }.asInstanceOf[Param[Any]] } /** * Sets a parameter in the embedded param map. */ - protected def set[T](param: Param[T], value: T): this.type = { - require(param.parent.eq(this)) + protected final def set[T](param: Param[T], value: T): this.type = { + shouldOwn(param) paramMap.put(param.asInstanceOf[Param[Any]], value) this } @@ -179,94 +318,171 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter (by name) in the embedded param map. */ - private[ml] def set(param: String, value: Any): this.type = { + protected final def set(param: String, value: Any): this.type = { set(getParam(param), value) } /** - * Gets the value of a parameter in the embedded param map. + * Optionally returns the user-supplied value of a param. + */ + final def get[T](param: Param[T]): Option[T] = { + shouldOwn(param) + paramMap.get(param) + } + + /** + * Clears the user-supplied value for the input param. + */ + protected final def clear(param: Param[_]): this.type = { + shouldOwn(param) + paramMap.remove(param) + this + } + + /** + * Gets the value of a param in the embedded param map or its default value. Throws an exception + * if neither is set. */ - protected def get[T](param: Param[T]): T = { - require(param.parent.eq(this)) - paramMap(param) + final def getOrDefault[T](param: Param[T]): T = { + shouldOwn(param) + get(param).orElse(getDefault(param)).get } + /** An alias for [[getOrDefault()]]. */ + protected final def $[T](param: Param[T]): T = getOrDefault(param) + /** - * Internal param map. + * Sets a default value for a param. + * @param param param to set the default value. Make sure that this param is initialized before + * this method gets called. + * @param value the default value */ - protected val paramMap: ParamMap = ParamMap.empty + protected final def setDefault[T](param: Param[T], value: T): this.type = { + shouldOwn(param) + defaultParamMap.put(param, value) + this + } /** - * Check whether the given schema contains an input column. - * @param colName Parameter name for the input column. - * @param dataType SQL DataType of the input column. + * Sets default values for a list of params. + * + * Note: Java developers should use the single-parameter [[setDefault()]]. + * Annotating this with varargs causes compilation failures. + * + * @param paramPairs a list of param pairs that specify params and their default values to set + * respectively. Make sure that the params are initialized before this method + * gets called. */ - protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { - val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), - s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + protected final def setDefault(paramPairs: ParamPair[_]*): this.type = { + paramPairs.foreach { p => + setDefault(p.param.asInstanceOf[Param[Any]], p.value) + } + this } - protected def addOutputColumn( - schema: StructType, - colName: String, - dataType: DataType): StructType = { - if (colName.length == 0) return schema - val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Prediction column $colName already exists.") - val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) - StructType(outputFields) + /** + * Gets the default value of a parameter. + */ + final def getDefault[T](param: Param[T]): Option[T] = { + shouldOwn(param) + defaultParamMap.get(param) } -} -/** - * :: DeveloperApi :: - * - * Helper functionality for developers. - * - * NOTE: This is currently private[spark] but will be made public later once it is stabilized. - */ -@DeveloperApi -private[spark] object Params { - - /** - * Copies parameter values from the parent estimator to the child model it produced. - * @param paramMap the param map that holds parameters of the parent - * @param parent the parent estimator - * @param child the child model - */ - def inheritValues[E <: Params, M <: E]( - paramMap: ParamMap, - parent: E, - child: M): Unit = { - parent.params.foreach { param => - if (paramMap.contains(param)) { - child.set(child.getParam(param.name), paramMap(param)) + /** + * Tests whether the input param has a default value set. + */ + final def hasDefault[T](param: Param[T]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) + } + + /** + * Creates a copy of this instance with a randomly generated uid and some extra params. + * The default implementation calls the default constructor to create a new instance, then + * copies the embedded and extra parameters over and returns the new instance. + * Subclasses should override this method if the default approach is not sufficient. + */ + def copy(extra: ParamMap): Params = { + val that = this.getClass.newInstance() + copyValues(that, extra) + that + } + + /** + * Extracts the embedded default param values and user-supplied values, and then merges them with + * extra values from input into a flat param map, where the latter value is used if there exist + * conflicts, i.e., with ordering: default param values < user-supplied values < extraParamMap. + */ + final def extractParamMap(extraParamMap: ParamMap): ParamMap = { + defaultParamMap ++ paramMap ++ extraParamMap + } + + /** + * [[extractParamMap]] with no extra values. + */ + final def extractParamMap(): ParamMap = { + extractParamMap(ParamMap.empty) + } + + /** Internal param map for user-supplied values. */ + private val paramMap: ParamMap = ParamMap.empty + + /** Internal param map for default values. */ + private val defaultParamMap: ParamMap = ParamMap.empty + + /** Validates that the input param belongs to this instance. */ + private def shouldOwn(param: Param[_]): Unit = { + require(param.parent.eq(this), s"Param $param does not belong to $this.") + } + + /** + * Copies param values from this instance to another instance for params shared by them. + * @param to the target instance + * @param extra extra params to be copied + * @return the target instance with param values copied + */ + protected def copyValues[T <: Params](to: T, extra: ParamMap = ParamMap.empty): T = { + val map = extractParamMap(extra) + params.foreach { param => + if (map.contains(param) && to.hasParam(param.name)) { + to.set(param.name, map(param)) } } + to } } +/** + * Java-friendly wrapper for [[Params]]. + * Java developers who need to extend [[Params]] should use this class instead. + * If you need to extend a abstract class which already extends [[Params]], then that abstract + * class should be Java-friendly as well. + */ +abstract class JavaParams extends Params + /** * :: AlphaComponent :: * A param to value map. */ @AlphaComponent -class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { +final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) + extends Serializable { + + /* DEVELOPERS: About validating parameter values + * This and ParamPair are the only two collections of parameters. + * This class should always create ParamPairs when + * specifying new parameter values. ParamPair will then call Param.validate(). + */ /** * Creates an empty param map. */ - def this() = this(mutable.Map.empty[Param[Any], Any]) + def this() = this(mutable.Map.empty) /** * Puts a (param, value) pair (overwrites if the input param exists). */ - def put[T](param: Param[T], value: T): this.type = { - map(param.asInstanceOf[Param[Any]]) = value - this - } + def put[T](param: Param[T], value: T): this.type = put(ParamPair(param, value)) /** * Puts a list of param pairs (overwrites if the input params exists). @@ -274,18 +490,23 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten @varargs def put(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => - put(p.param.asInstanceOf[Param[Any]], p.value) + map(p.param.asInstanceOf[Param[Any]]) = p.value } this } /** - * Optionally returns the value associated with a param or its default. + * Optionally returns the value associated with a param. */ def get[T](param: Param[T]): Option[T] = { - map.get(param.asInstanceOf[Param[Any]]) - .orElse(param.defaultValue) - .asInstanceOf[Option[T]] + map.get(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + + /** + * Returns the value associated with a param or a default value. + */ + def getOrElse[T](param: Param[T], default: T): T = { + get(param).getOrElse(default) } /** @@ -293,10 +514,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Raises a NoSuchElementException if there is no value associated with the input param. */ def apply[T](param: Param[T]): T = { - val value = get(param) - if (value.isDefined) { - value.get - } else { + get(param).getOrElse { throw new NoSuchElementException(s"Cannot find param ${param.name}.") } } @@ -308,6 +526,13 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten map.contains(param.asInstanceOf[Param[Any]]) } + /** + * Removes a key from this map and returns its value associated previously as an option. + */ + def remove[T](param: Param[T]): Option[T] = { + map.remove(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + /** * Filters this param map for the given parent. */ @@ -317,7 +542,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Make a copy of this param map. + * Creates a copy of this param map. */ def copy: ParamMap = new ParamMap(map.clone()) @@ -329,7 +554,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Returns a new param map that contains parameters in this map and the given map, - * where the latter overwrites this if there exists conflicts. + * where the latter overwrites this if there exist conflicts. */ def ++(other: ParamMap): ParamMap = { // TODO: Provide a better method name for Java users. @@ -355,7 +580,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Number of param pairs in this set. + * Number of param pairs in this map. */ def size: Int = map.size } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala new file mode 100644 index 0000000000000..d379172e0bf53 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -0,0 +1,184 @@ +/* + * 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.param.shared + +import java.io.PrintWriter + +import scala.reflect.ClassTag + +/** + * Code generator for shared params (sharedParams.scala). Run under the Spark folder with + * {{{ + * build/sbt "mllib/runMain org.apache.spark.ml.param.shared.SharedParamsCodeGen" + * }}} + */ +private[shared] object SharedParamsCodeGen { + + def main(args: Array[String]): Unit = { + val params = Seq( + ParamDesc[Double]("regParam", "regularization parameter (>= 0)", + isValid = "ParamValidators.gtEq(0)"), + ParamDesc[Int]("maxIter", "max number of iterations (>= 0)", + isValid = "ParamValidators.gtEq(0)"), + ParamDesc[String]("featuresCol", "features column name", Some("\"features\"")), + ParamDesc[String]("labelCol", "label column name", Some("\"label\"")), + ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), + ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("\"rawPrediction\"")), + ParamDesc[String]("probabilityCol", + "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[Double]("threshold", + "threshold in binary classification prediction, in range [0, 1]", + isValid = "ParamValidators.inRange(0, 1)"), + ParamDesc[String]("inputCol", "input column name"), + ParamDesc[Array[String]]("inputCols", "input column names"), + ParamDesc[String]("outputCol", "output column name"), + ParamDesc[Int]("checkpointInterval", "checkpoint interval (>= 1)", + isValid = "ParamValidators.gtEq(1)"), + ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true")), + ParamDesc[Long]("seed", "random seed", Some("Utils.random.nextLong()")), + ParamDesc[Double]("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]." + + " For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", + isValid = "ParamValidators.inRange(0, 1)"), + ParamDesc[Double]("tol", "the convergence tolerance for iterative algorithms"), + ParamDesc[Double]("stepSize", "Step size to be used for each iteration of optimization.")) + + val code = genSharedParams(params) + val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" + val writer = new PrintWriter(file) + writer.write(code) + writer.close() + } + + /** Description of a param. */ + private case class ParamDesc[T: ClassTag]( + name: String, + doc: String, + defaultValueStr: Option[String] = None, + isValid: String = "") { + + require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") + require(doc.nonEmpty) // TODO: more rigorous on doc + + def paramTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + c match { + case _ if c == classOf[Int] => "IntParam" + case _ if c == classOf[Long] => "LongParam" + case _ if c == classOf[Float] => "FloatParam" + case _ if c == classOf[Double] => "DoubleParam" + case _ if c == classOf[Boolean] => "BooleanParam" + case _ => s"Param[${getTypeString(c)}]" + } + } + + def valueTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + getTypeString(c) + } + + private def getTypeString(c: Class[_]): String = { + c match { + case _ if c == classOf[Int] => "Int" + case _ if c == classOf[Long] => "Long" + case _ if c == classOf[Float] => "Float" + case _ if c == classOf[Double] => "Double" + case _ if c == classOf[Boolean] => "Boolean" + case _ if c == classOf[String] => "String" + case _ if c.isArray => s"Array[${getTypeString(c.getComponentType)}]" + } + } + } + + /** Generates the HasParam trait code for the input param. */ + private def genHasParamTrait(param: ParamDesc[_]): String = { + val name = param.name + val Name = name(0).toUpper +: name.substring(1) + val Param = param.paramTypeName + val T = param.valueTypeName + val doc = param.doc + val defaultValue = param.defaultValueStr + val defaultValueDoc = defaultValue.map { v => + s" (default: $v)" + }.getOrElse("") + val setDefault = defaultValue.map { v => + s""" + | setDefault($name, $v) + |""".stripMargin + }.getOrElse("") + val isValid = if (param.isValid != "") { + ", " + param.isValid + } else { + "" + } + + s""" + |/** + | * (private[ml]) Trait for shared param $name$defaultValueDoc. + | */ + |private[ml] trait Has$Name extends Params { + | + | /** + | * Param for $doc. + | * @group param + | */ + | final val $name: $Param = new $Param(this, "$name", "$doc"$isValid) + |$setDefault + | /** @group getParam */ + | final def get$Name: $T = $$($name) + |} + |""".stripMargin + } + + /** Generates Scala source code for the input params with header. */ + private def genSharedParams(params: Seq[ParamDesc[_]]): String = { + val header = + """/* + | * 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.param.shared + | + |import org.apache.spark.ml.param._ + |import org.apache.spark.util.Utils + | + |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + | + |// scalastyle:off + |""".stripMargin + + val footer = "// scalastyle:on\n" + + val traits = params.map(genHasParamTrait).mkString + + header + traits + footer + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala new file mode 100644 index 0000000000000..fb1874ccfc8dc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -0,0 +1,295 @@ +/* + * 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.param.shared + +import org.apache.spark.ml.param._ +import org.apache.spark.util.Utils + +// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + +// scalastyle:off + +/** + * (private[ml]) Trait for shared param regParam. + */ +private[ml] trait HasRegParam extends Params { + + /** + * Param for regularization parameter (>= 0). + * @group param + */ + final val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter (>= 0)", ParamValidators.gtEq(0)) + + /** @group getParam */ + final def getRegParam: Double = $(regParam) +} + +/** + * (private[ml]) Trait for shared param maxIter. + */ +private[ml] trait HasMaxIter extends Params { + + /** + * Param for max number of iterations (>= 0). + * @group param + */ + final val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations (>= 0)", ParamValidators.gtEq(0)) + + /** @group getParam */ + final def getMaxIter: Int = $(maxIter) +} + +/** + * (private[ml]) Trait for shared param featuresCol (default: "features"). + */ +private[ml] trait HasFeaturesCol extends Params { + + /** + * Param for features column name. + * @group param + */ + final val featuresCol: Param[String] = new Param[String](this, "featuresCol", "features column name") + + setDefault(featuresCol, "features") + + /** @group getParam */ + final def getFeaturesCol: String = $(featuresCol) +} + +/** + * (private[ml]) Trait for shared param labelCol (default: "label"). + */ +private[ml] trait HasLabelCol extends Params { + + /** + * Param for label column name. + * @group param + */ + final val labelCol: Param[String] = new Param[String](this, "labelCol", "label column name") + + setDefault(labelCol, "label") + + /** @group getParam */ + final def getLabelCol: String = $(labelCol) +} + +/** + * (private[ml]) Trait for shared param predictionCol (default: "prediction"). + */ +private[ml] trait HasPredictionCol extends Params { + + /** + * Param for prediction column name. + * @group param + */ + final val predictionCol: Param[String] = new Param[String](this, "predictionCol", "prediction column name") + + setDefault(predictionCol, "prediction") + + /** @group getParam */ + final def getPredictionCol: String = $(predictionCol) +} + +/** + * (private[ml]) Trait for shared param rawPredictionCol (default: "rawPrediction"). + */ +private[ml] trait HasRawPredictionCol extends Params { + + /** + * Param for raw prediction (a.k.a. confidence) column name. + * @group param + */ + final val rawPredictionCol: Param[String] = new Param[String](this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") + + setDefault(rawPredictionCol, "rawPrediction") + + /** @group getParam */ + final def getRawPredictionCol: String = $(rawPredictionCol) +} + +/** + * (private[ml]) Trait for shared param probabilityCol (default: "probability"). + */ +private[ml] trait HasProbabilityCol extends Params { + + /** + * Param for column name for predicted class conditional probabilities. + * @group param + */ + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + + setDefault(probabilityCol, "probability") + + /** @group getParam */ + final def getProbabilityCol: String = $(probabilityCol) +} + +/** + * (private[ml]) Trait for shared param threshold. + */ +private[ml] trait HasThreshold extends Params { + + /** + * Param for threshold in binary classification prediction, in range [0, 1]. + * @group param + */ + final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction, in range [0, 1]", ParamValidators.inRange(0, 1)) + + /** @group getParam */ + final def getThreshold: Double = $(threshold) +} + +/** + * (private[ml]) Trait for shared param inputCol. + */ +private[ml] trait HasInputCol extends Params { + + /** + * Param for input column name. + * @group param + */ + final val inputCol: Param[String] = new Param[String](this, "inputCol", "input column name") + + /** @group getParam */ + final def getInputCol: String = $(inputCol) +} + +/** + * (private[ml]) Trait for shared param inputCols. + */ +private[ml] trait HasInputCols extends Params { + + /** + * Param for input column names. + * @group param + */ + final val inputCols: Param[Array[String]] = new Param[Array[String]](this, "inputCols", "input column names") + + /** @group getParam */ + final def getInputCols: Array[String] = $(inputCols) +} + +/** + * (private[ml]) Trait for shared param outputCol. + */ +private[ml] trait HasOutputCol extends Params { + + /** + * Param for output column name. + * @group param + */ + final val outputCol: Param[String] = new Param[String](this, "outputCol", "output column name") + + /** @group getParam */ + final def getOutputCol: String = $(outputCol) +} + +/** + * (private[ml]) Trait for shared param checkpointInterval. + */ +private[ml] trait HasCheckpointInterval extends Params { + + /** + * Param for checkpoint interval (>= 1). + * @group param + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval (>= 1)", ParamValidators.gtEq(1)) + + /** @group getParam */ + final def getCheckpointInterval: Int = $(checkpointInterval) +} + +/** + * (private[ml]) Trait for shared param fitIntercept (default: true). + */ +private[ml] trait HasFitIntercept extends Params { + + /** + * Param for whether to fit an intercept term. + * @group param + */ + final val fitIntercept: BooleanParam = new BooleanParam(this, "fitIntercept", "whether to fit an intercept term") + + setDefault(fitIntercept, true) + + /** @group getParam */ + final def getFitIntercept: Boolean = $(fitIntercept) +} + +/** + * (private[ml]) Trait for shared param seed (default: Utils.random.nextLong()). + */ +private[ml] trait HasSeed extends Params { + + /** + * Param for random seed. + * @group param + */ + final val seed: LongParam = new LongParam(this, "seed", "random seed") + + setDefault(seed, Utils.random.nextLong()) + + /** @group getParam */ + final def getSeed: Long = $(seed) +} + +/** + * (private[ml]) Trait for shared param elasticNetParam. + */ +private[ml] trait HasElasticNetParam extends Params { + + /** + * Param for the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.. + * @group param + */ + final val elasticNetParam: DoubleParam = new DoubleParam(this, "elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]. For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.", ParamValidators.inRange(0, 1)) + + /** @group getParam */ + final def getElasticNetParam: Double = $(elasticNetParam) +} + +/** + * (private[ml]) Trait for shared param tol. + */ +private[ml] trait HasTol extends Params { + + /** + * Param for the convergence tolerance for iterative algorithms. + * @group param + */ + final val tol: DoubleParam = new DoubleParam(this, "tol", "the convergence tolerance for iterative algorithms") + + /** @group getParam */ + final def getTol: Double = $(tol) +} + +/** + * (private[ml]) Trait for shared param stepSize. + */ +private[ml] trait HasStepSize extends Params { + + /** + * Param for Step size to be used for each iteration of optimization.. + * @group param + */ + final val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size to be used for each iteration of optimization.") + + /** @group getParam */ + final def getStepSize: Double = $(stepSize) +} +// scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala deleted file mode 100644 index 5d660d1e151a7..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ /dev/null @@ -1,151 +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.ml.param - -/* NOTE TO DEVELOPERS: - * If you mix these parameter traits into your algorithm, please add a setter method as well - * so that users may use a builder pattern: - * val myLearner = new MyLearner().setParam1(x).setParam2(y)... - */ - -private[ml] trait HasRegParam extends Params { - /** - * param for regularization parameter - * @group param - */ - val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") - - /** @group getParam */ - def getRegParam: Double = get(regParam) -} - -private[ml] trait HasMaxIter extends Params { - /** - * param for max number of iterations - * @group param - */ - val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - - /** @group getParam */ - def getMaxIter: Int = get(maxIter) -} - -private[ml] trait HasFeaturesCol extends Params { - /** - * param for features column name - * @group param - */ - val featuresCol: Param[String] = - new Param(this, "featuresCol", "features column name", Some("features")) - - /** @group getParam */ - def getFeaturesCol: String = get(featuresCol) -} - -private[ml] trait HasLabelCol extends Params { - /** - * param for label column name - * @group param - */ - val labelCol: Param[String] = new Param(this, "labelCol", "label column name", Some("label")) - - /** @group getParam */ - def getLabelCol: String = get(labelCol) -} - -private[ml] trait HasPredictionCol extends Params { - /** - * param for prediction column name - * @group param - */ - val predictionCol: Param[String] = - new Param(this, "predictionCol", "prediction column name", Some("prediction")) - - /** @group getParam */ - def getPredictionCol: String = get(predictionCol) -} - -private[ml] trait HasRawPredictionCol extends Params { - /** - * param for raw prediction column name - * @group param - */ - val rawPredictionCol: Param[String] = - new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", - Some("rawPrediction")) - - /** @group getParam */ - def getRawPredictionCol: String = get(rawPredictionCol) -} - -private[ml] trait HasProbabilityCol extends Params { - /** - * param for predicted class conditional probabilities column name - * @group param - */ - val probabilityCol: Param[String] = - new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", - Some("probability")) - - /** @group getParam */ - def getProbabilityCol: String = get(probabilityCol) -} - -private[ml] trait HasThreshold extends Params { - /** - * param for threshold in (binary) prediction - * @group param - */ - val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") - - /** @group getParam */ - def getThreshold: Double = get(threshold) -} - -private[ml] trait HasInputCol extends Params { - /** - * param for input column name - * @group param - */ - val inputCol: Param[String] = new Param(this, "inputCol", "input column name") - - /** @group getParam */ - def getInputCol: String = get(inputCol) -} - -private[ml] trait HasOutputCol extends Params { - /** - * param for output column name - * @group param - */ - val outputCol: Param[String] = new Param(this, "outputCol", "output column name") - - /** @group getParam */ - def getOutputCol: String = get(outputCol) -} - -private[ml] trait HasCheckpointInterval extends Params { - /** - * param for checkpoint interval - * @group param - */ - val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") - - /** @group getParam */ - def getCheckpointInterval: Int = get(checkpointInterval) -} 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 514b4ef98dc5b..6cf4b40075281 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 @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -51,106 +52,117 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR with HasPredictionCol with HasCheckpointInterval { /** - * Param for rank of the matrix factorization. + * Param for rank of the matrix factorization (>= 1). + * Default: 10 * @group param */ - val rank = new IntParam(this, "rank", "rank of the factorization", Some(10)) + val rank = new IntParam(this, "rank", "rank of the factorization", ParamValidators.gtEq(1)) /** @group getParam */ - def getRank: Int = get(rank) + def getRank: Int = $(rank) /** - * Param for number of user blocks. + * Param for number of user blocks (>= 1). + * Default: 10 * @group param */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", Some(10)) + val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", + ParamValidators.gtEq(1)) /** @group getParam */ - def getNumUserBlocks: Int = get(numUserBlocks) + def getNumUserBlocks: Int = $(numUserBlocks) /** - * Param for number of item blocks. + * Param for number of item blocks (>= 1). + * Default: 10 * @group param */ - val numItemBlocks = - new IntParam(this, "numItemBlocks", "number of item blocks", Some(10)) + val numItemBlocks = new IntParam(this, "numItemBlocks", "number of item blocks", + ParamValidators.gtEq(1)) /** @group getParam */ - def getNumItemBlocks: Int = get(numItemBlocks) + def getNumItemBlocks: Int = $(numItemBlocks) /** * Param to decide whether to use implicit preference. + * Default: false * @group param */ - val implicitPrefs = - new BooleanParam(this, "implicitPrefs", "whether to use implicit preference", Some(false)) + val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") /** @group getParam */ - def getImplicitPrefs: Boolean = get(implicitPrefs) + def getImplicitPrefs: Boolean = $(implicitPrefs) /** - * Param for the alpha parameter in the implicit preference formulation. + * Param for the alpha parameter in the implicit preference formulation (>= 0). + * Default: 1.0 * @group param */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", Some(1.0)) + val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", + ParamValidators.gtEq(0)) /** @group getParam */ - def getAlpha: Double = get(alpha) + def getAlpha: Double = $(alpha) /** * Param for the column name for user ids. + * Default: "user" * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids", Some("user")) + val userCol = new Param[String](this, "userCol", "column name for user ids") /** @group getParam */ - def getUserCol: String = get(userCol) + def getUserCol: String = $(userCol) /** * Param for the column name for item ids. + * Default: "item" * @group param */ - val itemCol = - new Param[String](this, "itemCol", "column name for item ids", Some("item")) + val itemCol = new Param[String](this, "itemCol", "column name for item ids") /** @group getParam */ - def getItemCol: String = get(itemCol) + def getItemCol: String = $(itemCol) /** * Param for the column name for ratings. + * Default: "rating" * @group param */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) + val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") /** @group getParam */ - def getRatingCol: String = get(ratingCol) + def getRatingCol: String = $(ratingCol) /** * Param for whether to apply nonnegativity constraints. + * Default: false * @group param */ val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + this, "nonnegative", "whether to use nonnegative constraint for least squares") /** @group getParam */ - val getNonnegative: Boolean = get(nonnegative) + def getNonnegative: Boolean = $(nonnegative) + + setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, + implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", + ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10) /** * Validates and transforms the input schema. * @param schema input schema - * @param paramMap extra params * @return output schema */ - protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - assert(schema(map(userCol)).dataType == IntegerType) - assert(schema(map(itemCol)).dataType== IntegerType) - val ratingType = schema(map(ratingCol)).dataType - assert(ratingType == FloatType || ratingType == DoubleType) - val predictionColName = map(predictionCol) - assert(!schema.fieldNames.contains(predictionColName), + protected def validateAndTransformSchema(schema: StructType): StructType = { + require(schema($(userCol)).dataType == IntegerType) + require(schema($(itemCol)).dataType== IntegerType) + val ratingType = schema($(ratingCol)).dataType + require(ratingType == FloatType || ratingType == DoubleType) + val predictionColName = $(predictionCol) + require(!schema.fieldNames.contains(predictionColName), s"Prediction column $predictionColName already exists.") - val newFields = schema.fields :+ StructField(map(predictionCol), FloatType, nullable = false) + val newFields = schema.fields :+ StructField($(predictionCol), FloatType, nullable = false) StructType(newFields) } } @@ -160,7 +172,6 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR */ class ALSModel private[ml] ( override val parent: ALS, - override val fittingParamMap: ParamMap, k: Int, userFactors: RDD[(Int, Array[Float])], itemFactors: RDD[(Int, Array[Float])]) @@ -169,9 +180,8 @@ class ALSModel private[ml] ( /** @group setParam */ def setPredictionCol(value: String): this.type = set(predictionCol, value) - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + override def transform(dataset: DataFrame): DataFrame = { import dataset.sqlContext.implicits._ - val map = this.paramMap ++ paramMap val users = userFactors.toDF("id", "features") val items = itemFactors.toDF("id", "features") @@ -185,13 +195,13 @@ class ALSModel private[ml] ( } } dataset - .join(users, dataset(map(userCol)) === users("id"), "left") - .join(items, dataset(map(itemCol)) === items("id"), "left") - .select(dataset("*"), predict(users("features"), items("features")).as(map(predictionCol))) + .join(users, dataset($(userCol)) === users("id"), "left") + .join(items, dataset($(itemCol)) === items("id"), "left") + .select(dataset("*"), predict(users("features"), items("features")).as($(predictionCol))) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap) + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) } } @@ -278,29 +288,22 @@ class ALS extends Estimator[ALSModel] with ALSParams { this } - setMaxIter(20) - setRegParam(1.0) - setCheckpointInterval(10) - - override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = { - val map = this.paramMap ++ paramMap + override def fit(dataset: DataFrame): ALSModel = { val ratings = dataset - .select(col(map(userCol)), col(map(itemCol)), col(map(ratingCol)).cast(FloatType)) + .select(col($(userCol)), col($(itemCol)), col($(ratingCol)).cast(FloatType)) .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) } - val (userFactors, itemFactors) = ALS.train(ratings, rank = map(rank), - numUserBlocks = map(numUserBlocks), numItemBlocks = map(numItemBlocks), - maxIter = map(maxIter), regParam = map(regParam), implicitPrefs = map(implicitPrefs), - alpha = map(alpha), nonnegative = map(nonnegative), - checkpointInterval = map(checkpointInterval)) - val model = new ALSModel(this, map, map(rank), userFactors, itemFactors) - Params.inheritValues(map, this, model) - model + val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), + numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), + maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), + alpha = $(alpha), nonnegative = $(nonnegative), + checkpointInterval = $(checkpointInterval)) + copyValues(new ALSModel(this, $(rank), userFactors, itemFactors)) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap) + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) } } @@ -320,7 +323,7 @@ object ALS extends Logging { /** Trait for least squares solvers applied to the normal equation. */ private[recommendation] trait LeastSquaresNESolver extends Serializable { - /** Solves a least squares problem (possibly with other constraints). */ + /** Solves a least squares problem with regularization (possibly with other constraints). */ def solve(ne: NormalEquation, lambda: Double): Array[Float] } @@ -332,20 +335,19 @@ object ALS extends Logging { /** * Solves a least squares problem with L2 regularization: * - * min norm(A x - b)^2^ + lambda * n * norm(x)^2^ + * min norm(A x - b)^2^ + lambda * norm(x)^2^ * * @param ne a [[NormalEquation]] instance that contains AtA, Atb, and n (number of instances) - * @param lambda regularization constant, which will be scaled by n + * @param lambda regularization constant * @return the solution x */ override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { val k = ne.k // Add scaled lambda to the diagonals of AtA. - val scaledlambda = lambda * ne.n var i = 0 var j = 2 while (i < ne.triK) { - ne.ata(i) += scaledlambda + ne.ata(i) += lambda i += j j += 1 } @@ -391,7 +393,7 @@ object ALS extends Logging { override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { val rank = ne.k initialize(rank) - fillAtA(ne.ata, lambda * ne.n) + fillAtA(ne.ata, lambda) val x = NNLS.solve(ata, ne.atb, workspace) ne.reset() x.map(x => x.toFloat) @@ -420,7 +422,15 @@ object ALS extends Logging { } } - /** Representing a normal equation (ALS' subproblem). */ + /** + * Representing a normal equation to solve the following weighted least squares problem: + * + * minimize \sum,,i,, c,,i,, (a,,i,,^T^ x - b,,i,,)^2^ + lambda * x^T^ x. + * + * Its normal equation is given by + * + * \sum,,i,, c,,i,, (a,,i,, a,,i,,^T^ x - b,,i,, a,,i,,) + lambda * x = 0. + */ private[recommendation] class NormalEquation(val k: Int) extends Serializable { /** Number of entries in the upper triangular part of a k-by-k matrix. */ @@ -429,8 +439,6 @@ object ALS extends Logging { val ata = new Array[Double](triK) /** A^T^ * b */ val atb = new Array[Double](k) - /** Number of observations. */ - var n = 0 private val da = new Array[Double](k) private val upper = "U" @@ -444,28 +452,13 @@ object ALS extends Logging { } /** Adds an observation. */ - def add(a: Array[Float], b: Float): this.type = { - require(a.length == k) - copyToDouble(a) - blas.dspr(upper, k, 1.0, da, 1, ata) - blas.daxpy(k, b.toDouble, da, 1, atb, 1) - n += 1 - this - } - - /** - * Adds an observation with implicit feedback. Note that this does not increment the counter. - */ - def addImplicit(a: Array[Float], b: Float, alpha: Double): this.type = { + def add(a: Array[Float], b: Double, c: Double = 1.0): this.type = { + require(c >= 0.0) require(a.length == k) - // Extension to the original paper to handle b < 0. confidence is a function of |b| instead - // so that it is never negative. - val confidence = 1.0 + alpha * math.abs(b) copyToDouble(a) - blas.dspr(upper, k, confidence - 1.0, da, 1, ata) - // For b <= 0, the corresponding preference is 0. So the term below is only added for b > 0. - if (b > 0) { - blas.daxpy(k, confidence, da, 1, atb, 1) + blas.dspr(upper, k, c, da, 1, ata) + if (b != 0.0) { + blas.daxpy(k, c * b, da, 1, atb, 1) } this } @@ -475,7 +468,6 @@ object ALS extends Logging { require(other.k == k) blas.daxpy(ata.length, 1.0, other.ata, 1, ata, 1) blas.daxpy(atb.length, 1.0, other.atb, 1, atb, 1) - n += other.n this } @@ -483,7 +475,6 @@ object ALS extends Logging { def reset(): Unit = { ju.Arrays.fill(ata, 0.0) ju.Arrays.fill(atb, 0.0) - n = 0 } } @@ -1114,6 +1105,7 @@ object ALS extends Logging { ls.merge(YtY.get) } var i = srcPtrs(j) + var numExplicits = 0 while (i < srcPtrs(j + 1)) { val encoded = srcEncodedIndices(i) val blockId = srcEncoder.blockId(encoded) @@ -1121,13 +1113,23 @@ object ALS extends Logging { val srcFactor = sortedSrcFactors(blockId)(localIndex) val rating = ratings(i) if (implicitPrefs) { - ls.addImplicit(srcFactor, rating, alpha) + // Extension to the original paper to handle b < 0. confidence is a function of |b| + // instead so that it is never negative. c1 is confidence - 1.0. + val c1 = alpha * math.abs(rating) + // For rating <= 0, the corresponding preference is 0. So the term below is only added + // for rating > 0. Because YtY is already added, we need to adjust the scaling here. + if (rating > 0) { + numExplicits += 1 + ls.add(srcFactor, (c1 + 1.0) / c1, c1) + } } else { ls.add(srcFactor, rating) + numExplicits += 1 } i += 1 } - dstFactors(j) = solver.solve(ls, regParam) + // Weight lambda by the number of explicit ratings based on the ALS-WR paper. + dstFactors(j) = solver.solve(ls, numExplicits * regParam) j += 1 } dstFactors @@ -1141,7 +1143,7 @@ object ALS extends Logging { private def computeYtY(factorBlocks: RDD[(Int, FactorBlock)], rank: Int): NormalEquation = { factorBlocks.values.aggregate(new NormalEquation(rank))( seqOp = (ne, factors) => { - factors.foreach(ne.add(_, 0.0f)) + factors.foreach(ne.add(_, 0.0)) ne }, combOp = (ne1, ne2) => ne1.merge(ne2)) 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 new file mode 100644 index 0000000000000..b07c26fe79b36 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -0,0 +1,134 @@ +/* + * 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.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.tree.{DecisionTreeModel, Node} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree} +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 +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * for regression. + * It supports both continuous and categorical features. + */ +@AlphaComponent +final class DecisionTreeRegressor + extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] + with DecisionTreeParams with TreeRegressorParams { + + // Override parameter setters from parent trait for Java API compatibility. + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + override protected def train(dataset: DataFrame): DecisionTreeRegressionModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val strategy = getOldStrategy(categoricalFeatures) + val oldModel = OldDecisionTree.train(oldDataset, strategy) + DecisionTreeRegressionModel.fromOld(oldModel, this, categoricalFeatures) + } + + /** (private[ml]) Create a Strategy instance to use with the old API. */ + private[ml] def getOldStrategy(categoricalFeatures: Map[Int, Int]): OldStrategy = { + super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity, + subsamplingRate = 1.0) + } +} + +object DecisionTreeRegressor { + /** Accessor for supported impurities: variance */ + final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * It supports both continuous and categorical features. + * @param rootNode Root of the decision tree + */ +@AlphaComponent +final class DecisionTreeRegressionModel private[ml] ( + override val parent: DecisionTreeRegressor, + override val rootNode: Node) + extends PredictionModel[Vector, DecisionTreeRegressionModel] + with DecisionTreeModel with Serializable { + + require(rootNode != null, + "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + + override protected def predict(features: Vector): Double = { + rootNode.predict(features) + } + + override def copy(extra: ParamMap): DecisionTreeRegressionModel = { + copyValues(new DecisionTreeRegressionModel(parent, rootNode), extra) + } + + override def toString: String = { + s"DecisionTreeRegressionModel of depth $depth with $numNodes nodes" + } + + /** Convert to a model in the old API */ + private[ml] def toOld: OldDecisionTreeModel = { + new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Regression) + } +} + +private[ml] object DecisionTreeRegressionModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldDecisionTreeModel, + parent: DecisionTreeRegressor, + categoricalFeatures: Map[Int, Int]): DecisionTreeRegressionModel = { + require(oldModel.algo == OldAlgo.Regression, + s"Cannot convert non-regression DecisionTreeModel (old API) to" + + s" DecisionTreeRegressionModel (new API). Algo is: ${oldModel.algo}") + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) + new DecisionTreeRegressionModel(parent, rootNode) + } +} 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 new file mode 100644 index 0000000000000..bc796958e4545 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -0,0 +1,205 @@ +/* + * 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.regression + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.Logging +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree._ +import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{GradientBoostedTrees => OldGBT} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.loss.{AbsoluteError => OldAbsoluteError, Loss => OldLoss, SquaredError => OldSquaredError} +import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * learning algorithm for regression. + * It supports both continuous and categorical features. + */ +@AlphaComponent +final class GBTRegressor + extends Predictor[Vector, GBTRegressor, GBTRegressionModel] + with GBTParams with TreeRegressorParams with Logging { + + // Override parameter setters from parent trait for Java API compatibility. + + // Parameters from TreeRegressorParams: + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + /** + * The impurity setting is ignored for GBT models. + * Individual trees are built using impurity "Variance." + */ + override def setImpurity(value: String): this.type = { + logWarning("GBTRegressor.setImpurity should NOT be used") + this + } + + // Parameters from TreeEnsembleParams: + + override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + + override def setSeed(value: Long): this.type = { + logWarning("The 'seed' parameter is currently ignored by Gradient Boosting.") + super.setSeed(value) + } + + // Parameters from GBTParams: + + override def setMaxIter(value: Int): this.type = super.setMaxIter(value) + + override def setStepSize(value: Double): this.type = super.setStepSize(value) + + // Parameters for GBTRegressor: + + /** + * Loss function which GBT tries to minimize. (case-insensitive) + * Supported: "squared" (L2) and "absolute" (L1) + * (default = squared) + * @group param + */ + val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + + " tries to minimize (case-insensitive). Supported options:" + + s" ${GBTRegressor.supportedLossTypes.mkString(", ")}", + (value: String) => GBTRegressor.supportedLossTypes.contains(value.toLowerCase)) + + setDefault(lossType -> "squared") + + /** @group setParam */ + def setLossType(value: String): this.type = set(lossType, value) + + /** @group getParam */ + def getLossType: String = $(lossType).toLowerCase + + /** (private[ml]) Convert new loss to old loss. */ + override private[ml] def getOldLossType: OldLoss = { + getLossType match { + case "squared" => OldSquaredError + case "absolute" => OldAbsoluteError + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException(s"GBTRegressorParams was given bad loss type: $getLossType") + } + } + + override protected def train(dataset: DataFrame): GBTRegressionModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) + val oldGBT = new OldGBT(boostingStrategy) + val oldModel = oldGBT.run(oldDataset) + GBTRegressionModel.fromOld(oldModel, this, categoricalFeatures) + } +} + +object GBTRegressor { + // The losses below should be lowercase. + /** Accessor for supported loss settings: squared (L2), absolute (L1) */ + final val supportedLossTypes: Array[String] = Array("squared", "absolute").map(_.toLowerCase) +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * model for regression. + * It supports both continuous and categorical features. + * @param _trees Decision trees in the ensemble. + * @param _treeWeights Weights for the decision trees in the ensemble. + */ +@AlphaComponent +final class GBTRegressionModel( + override val parent: GBTRegressor, + private val _trees: Array[DecisionTreeRegressionModel], + private val _treeWeights: Array[Double]) + extends PredictionModel[Vector, GBTRegressionModel] + with TreeEnsembleModel with Serializable { + + require(numTrees > 0, "GBTRegressionModel requires at least 1 tree.") + require(_trees.length == _treeWeights.length, "GBTRegressionModel given trees, treeWeights of" + + s" non-matching lengths (${_trees.length}, ${_treeWeights.length}, respectively).") + + override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + + override def treeWeights: Array[Double] = _treeWeights + + override protected def predict(features: Vector): Double = { + // TODO: Override transform() to broadcast model. SPARK-7127 + // TODO: When we add a generic Boosting class, handle transform there? SPARK-7129 + // Classifies by thresholding sum of weighted tree predictions + val treePredictions = _trees.map(_.rootNode.predict(features)) + val prediction = blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + if (prediction > 0.0) 1.0 else 0.0 + } + + override def copy(extra: ParamMap): GBTRegressionModel = { + copyValues(new GBTRegressionModel(parent, _trees, _treeWeights), extra) + } + + override def toString: String = { + s"GBTRegressionModel with $numTrees trees" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldGBTModel = { + new OldGBTModel(OldAlgo.Regression, _trees.map(_.toOld), _treeWeights) + } +} + +private[ml] object GBTRegressionModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldGBTModel, + parent: GBTRegressor, + categoricalFeatures: Map[Int, Int]): GBTRegressionModel = { + require(oldModel.algo == OldAlgo.Regression, "Cannot convert GradientBoostedTreesModel" + + s" with algo=${oldModel.algo} (old API) to GBTRegressionModel (new API).") + val newTrees = oldModel.trees.map { tree => + // parent, fittingParamMap for each tree is null since there are no good ways to set these. + DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) + } + new GBTRegressionModel(parent, newTrees, oldModel.treeWeights) + } +} 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 65f6627a0c351..66c475f2d9840 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 @@ -17,59 +17,168 @@ package org.apache.spark.ml.regression +import scala.collection.mutable + +import breeze.linalg.{DenseVector => BDV, norm => brzNorm} +import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, + OWLQN => BreezeOWLQN} + +import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} -import org.apache.spark.mllib.linalg.{BLAS, Vector} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared.{HasElasticNetParam, HasMaxIter, HasRegParam, HasTol} +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.storage.StorageLevel - +import org.apache.spark.util.StatCounter /** * Params for linear regression. */ private[regression] trait LinearRegressionParams extends RegressorParams - with HasRegParam with HasMaxIter - + with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol /** * :: AlphaComponent :: * * Linear regression. + * + * The learning objective is to minimize the squared error, with regularization. + * The specific squared error loss function used is: + * L = 1/2n ||A weights - y||^2^ + * + * This support multiple types of regularization: + * - none (a.k.a. ordinary least squares) + * - L2 (ridge regression) + * - L1 (Lasso) + * - L2 + L1 (elastic net) */ @AlphaComponent class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams { + with LinearRegressionParams with Logging { - setRegParam(0.1) - setMaxIter(100) - - /** @group setParam */ + /** + * Set the regularization parameter. + * Default is 0.0. + * @group setParam + */ def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Set the ElasticNet mixing parameter. + * For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty. + * For 0 < alpha < 1, the penalty is a combination of L1 and L2. + * Default is 0.0 which is an L2 penalty. + * @group setParam + */ + def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) + setDefault(elasticNetParam -> 0.0) - /** @group setParam */ + /** + * Set the maximal number of iterations. + * Default is 100. + * @group setParam + */ def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 100) - override protected def train(dataset: DataFrame, paramMap: ParamMap): LinearRegressionModel = { - // Extract columns from data. If dataset is persisted, do not persist oldDataset. - val oldDataset = extractLabeledPoints(dataset, paramMap) + /** + * Set the convergence tolerance of iterations. + * Smaller value will lead to higher accuracy with the cost of more iterations. + * Default is 1E-6. + * @group setParam + */ + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + override protected def train(dataset: DataFrame): LinearRegressionModel = { + // Extract columns from data. If dataset is persisted, do not persist instances. + val instances = extractLabeledPoints(dataset).map { + case LabeledPoint(label: Double, features: Vector) => (label, features) + } val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE - if (handlePersistence) { - oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + + val (summarizer, statCounter) = instances.treeAggregate( + (new MultivariateOnlineSummarizer, new StatCounter))( { + case ((summarizer: MultivariateOnlineSummarizer, statCounter: StatCounter), + (label: Double, features: Vector)) => + (summarizer.add(features), statCounter.merge(label)) + }, { + case ((summarizer1: MultivariateOnlineSummarizer, statCounter1: StatCounter), + (summarizer2: MultivariateOnlineSummarizer, statCounter2: StatCounter)) => + (summarizer1.merge(summarizer2), statCounter1.merge(statCounter2)) + }) + + val numFeatures = summarizer.mean.size + val yMean = statCounter.mean + val yStd = math.sqrt(statCounter.variance) + + // If the yStd is zero, then the intercept is yMean with zero weights; + // as a result, training is not needed. + if (yStd == 0.0) { + logWarning(s"The standard deviation of the label is zero, so the weights will be zeros " + + s"and the intercept will be the mean of the label; as a result, training is not needed.") + if (handlePersistence) instances.unpersist() + return new LinearRegressionModel(this, Vectors.sparse(numFeatures, Seq()), yMean) } - // Train model - val lr = new LinearRegressionWithSGD() - lr.optimizer - .setRegParam(paramMap(regParam)) - .setNumIterations(paramMap(maxIter)) - val model = lr.run(oldDataset) - val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) + val featuresMean = summarizer.mean.toArray + val featuresStd = summarizer.variance.toArray.map(math.sqrt) - if (handlePersistence) { - oldDataset.unpersist() + // Since we implicitly do the feature scaling when we compute the cost function + // to improve the convergence, the effective regParam will be changed. + val effectiveRegParam = $(regParam) / yStd + val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam + val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam + + val costFun = new LeastSquaresCostFun(instances, yStd, yMean, + featuresStd, featuresMean, effectiveL2RegParam) + + val optimizer = if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { + new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) + } else { + new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, effectiveL1RegParam, $(tol)) } - lrm + + val initialWeights = Vectors.zeros(numFeatures) + val states = + optimizer.iterations(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector) + + var state = states.next() + val lossHistory = mutable.ArrayBuilder.make[Double] + + while (states.hasNext) { + lossHistory += state.value + state = states.next() + } + lossHistory += state.value + + // The weights are trained in the scaled space; we're converting them back to + // the original space. + val weights = { + val rawWeights = state.x.toArray.clone() + var i = 0 + while (i < rawWeights.length) { + rawWeights(i) *= { if (featuresStd(i) != 0.0) yStd / featuresStd(i) else 0.0 } + i += 1 + } + Vectors.dense(rawWeights) + } + + // The intercept in R's GLMNET is computed using closed form after the coefficients are + // converged. See the following discussion for detail. + // http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet + val intercept = yMean - dot(weights, Vectors.dense(featuresMean)) + if (handlePersistence) instances.unpersist() + + // TODO: Converts to sparse format based on the storage, but may base on the scoring speed. + new LinearRegressionModel(this, weights.compressed, intercept) } } @@ -81,19 +190,236 @@ class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegress @AlphaComponent class LinearRegressionModel private[ml] ( override val parent: LinearRegression, - override val fittingParamMap: ParamMap, val weights: Vector, val intercept: Double) extends RegressionModel[Vector, LinearRegressionModel] with LinearRegressionParams { override protected def predict(features: Vector): Double = { - BLAS.dot(features, weights) + intercept + dot(features, weights) + intercept + } + + override def copy(extra: ParamMap): LinearRegressionModel = { + copyValues(new LinearRegressionModel(parent, weights, intercept), extra) + } +} + +/** + * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, + * as used in linear regression for samples in sparse or dense vector in a online fashion. + * + * Two LeastSquaresAggregator can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * For improving the convergence rate during the optimization process, and also preventing against + * features with very large variances exerting an overly large influence during model training, + * package like R's GLMNET performs the scaling to unit variance and removing the mean to reduce + * the condition number, and then trains the model in scaled space but returns the weights in + * the original scale. See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * However, we don't want to apply the `StandardScaler` on the training dataset, and then cache + * the standardized dataset since it will create a lot of overhead. As a result, we perform the + * scaling implicitly when we compute the objective function. The following is the mathematical + * derivation. + * + * Note that we don't deal with intercept by adding bias here, because the intercept + * can be computed using closed form after the coefficients are converged. + * See this discussion for detail. + * http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet + * + * The objective function in the scaled space is given by + * {{{ + * L = 1/2n ||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2, + * }}} + * where \bar{x_i} is the mean of x_i, \hat{x_i} is the standard deviation of x_i, + * \bar{y} is the mean of label, and \hat{y} is the standard deviation of label. + * + * This can be rewritten as + * {{{ + * L = 1/2n ||\sum_i (w_i/\hat{x_i})x_i - \sum_i (w_i/\hat{x_i})\bar{x_i} - y / \hat{y} + * + \bar{y} / \hat{y}||^2 + * = 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 + * }}} + * where w_i^\prime is the effective weights defined by w_i/\hat{x_i}, offset is + * {{{ + * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. + * }}}, and diff is + * {{{ + * \sum_i w_i^\prime x_i - y / \hat{y} + offset + * }}} + * + * Note that the effective weights and offset don't depend on training dataset, + * so they can be precomputed. + * + * Now, the first derivative of the objective function in scaled space is + * {{{ + * \frac{\partial L}{\partial\w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} + * }}} + * However, ($x_i - \bar{x_i}$) will densify the computation, so it's not + * an ideal formula when the training dataset is sparse format. + * + * This can be addressed by adding the dense \bar{x_i} / \har{x_i} terms + * in the end by keeping the sum of diff. The first derivative of total + * objective function from all the samples is + * {{{ + * \frac{\partial L}{\partial\w_i} = + * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} + * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i}) / \hat{x_i}) + * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) + * }}}, + * where correction_i = - diffSum \bar{x_i}) / \hat{x_i} + * + * A simple math can show that diffSum is actually zero, so we don't even + * need to add the correction terms in the end. From the definition of diff, + * {{{ + * diffSum = \sum_j (\sum_i w_i(x_{ij} - \bar{x_i}) / \hat{x_i} - (y_j - \bar{y}) / \hat{y}) + * = N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y_j} - \bar{y}) / \hat{y}) + * = 0 + * }}} + * + * As a result, the first derivative of the total objective function only depends on + * the training dataset, which can be easily computed in distributed fashion, and is + * sparse format friendly. + * {{{ + * \frac{\partial L}{\partial\w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + * }}}, + * + * @param weights The weights/coefficients corresponding to the features. + * @param labelStd The standard deviation value of the label. + * @param labelMean The mean value of the label. + * @param featuresStd The standard deviation values of the features. + * @param featuresMean The mean values of the features. + */ +private class LeastSquaresAggregator( + weights: Vector, + labelStd: Double, + labelMean: Double, + featuresStd: Array[Double], + featuresMean: Array[Double]) extends Serializable { + + private var totalCnt: Long = 0L + private var lossSum = 0.0 + + private val (effectiveWeightsArray: Array[Double], offset: Double, dim: Int) = { + val weightsArray = weights.toArray.clone() + var sum = 0.0 + var i = 0 + while (i < weightsArray.length) { + if (featuresStd(i) != 0.0) { + weightsArray(i) /= featuresStd(i) + sum += weightsArray(i) * featuresMean(i) + } else { + weightsArray(i) = 0.0 + } + i += 1 + } + (weightsArray, -sum + labelMean / labelStd, weightsArray.length) + } + + private val effectiveWeightsVector = Vectors.dense(effectiveWeightsArray) + + private val gradientSumArray = Array.ofDim[Double](dim) + + /** + * Add a new training data to this LeastSquaresAggregator, and update the loss and gradient + * of the objective function. + * + * @param label The label for this data point. + * @param data The features for one data point in dense/sparse vector format to be added + * into this aggregator. + * @return This LeastSquaresAggregator object. + */ + def add(label: Double, data: Vector): this.type = { + require(dim == data.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $dim but got ${data.size}.") + + val diff = dot(data, effectiveWeightsVector) - label / labelStd + offset + + if (diff != 0) { + val localGradientSumArray = gradientSumArray + data.foreachActive { (index, value) => + if (featuresStd(index) != 0.0 && value != 0.0) { + localGradientSumArray(index) += diff * value / featuresStd(index) + } + } + lossSum += diff * diff / 2.0 + } + + totalCnt += 1 + this } - override protected def copy(): LinearRegressionModel = { - val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) - m + /** + * Merge another LeastSquaresAggregator, and update the loss and gradient + * of the objective function. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other LeastSquaresAggregator to be merged. + * @return This LeastSquaresAggregator object. + */ + def merge(other: LeastSquaresAggregator): this.type = { + require(dim == other.dim, s"Dimensions mismatch when merging with another " + + s"LeastSquaresAggregator. Expecting $dim but got ${other.dim}.") + + if (other.totalCnt != 0) { + totalCnt += other.totalCnt + lossSum += other.lossSum + + var i = 0 + val localThisGradientSumArray = this.gradientSumArray + val localOtherGradientSumArray = other.gradientSumArray + while (i < dim) { + localThisGradientSumArray(i) += localOtherGradientSumArray(i) + i += 1 + } + } + this + } + + def count: Long = totalCnt + + def loss: Double = lossSum / totalCnt + + def gradient: Vector = { + val result = Vectors.dense(gradientSumArray.clone()) + scal(1.0 / totalCnt, result) + result + } +} + +/** + * LeastSquaresCostFun implements Breeze's DiffFunction[T] for Least Squares cost. + * It returns the loss and gradient with L2 regularization at a particular point (weights). + * It's used in Breeze's convex optimization routines. + */ +private class LeastSquaresCostFun( + data: RDD[(Double, Vector)], + labelStd: Double, + labelMean: Double, + featuresStd: Array[Double], + featuresMean: Array[Double], + effectiveL2regParam: Double) extends DiffFunction[BDV[Double]] { + + override def calculate(weights: BDV[Double]): (Double, BDV[Double]) = { + val w = Vectors.fromBreeze(weights) + + val leastSquaresAggregator = data.treeAggregate(new LeastSquaresAggregator(w, labelStd, + labelMean, featuresStd, featuresMean))( + seqOp = (c, v) => (c, v) match { + case (aggregator, (label, features)) => aggregator.add(label, features) + }, + combOp = (c1, c2) => (c1, c2) match { + case (aggregator1, aggregator2) => aggregator1.merge(aggregator2) + }) + + // regVal is the sum of weight squares for L2 regularization + val norm = brzNorm(weights, 2.0) + val regVal = 0.5 * effectiveL2regParam * norm * norm + + val loss = leastSquaresAggregator.loss + regVal + val gradient = leastSquaresAggregator.gradient + axpy(effectiveL2regParam, w, gradient) + + (loss, gradient.toBreeze.asInstanceOf[BDV[Double]]) } } 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 new file mode 100644 index 0000000000000..0468a1be1ba74 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -0,0 +1,160 @@ +/* + * 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.regression + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor} +import org.apache.spark.ml.impl.tree.{RandomForestParams, TreeRegressorParams} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.tree.{DecisionTreeModel, TreeEnsembleModel} +import org.apache.spark.ml.util.MetadataUtils +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{RandomForest => OldRandomForest} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for regression. + * It supports both continuous and categorical features. + */ +@AlphaComponent +final class RandomForestRegressor + extends Predictor[Vector, RandomForestRegressor, RandomForestRegressionModel] + with RandomForestParams with TreeRegressorParams { + + // Override parameter setters from parent trait for Java API compatibility. + + // Parameters from TreeRegressorParams: + + override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + + override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + + override def setMinInstancesPerNode(value: Int): this.type = + super.setMinInstancesPerNode(value) + + override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + + override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + + override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + + override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + + override def setImpurity(value: String): this.type = super.setImpurity(value) + + // Parameters from TreeEnsembleParams: + + override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + + override def setSeed(value: Long): this.type = super.setSeed(value) + + // Parameters from RandomForestParams: + + override def setNumTrees(value: Int): this.type = super.setNumTrees(value) + + override def setFeatureSubsetStrategy(value: String): this.type = + super.setFeatureSubsetStrategy(value) + + override protected def train(dataset: DataFrame): RandomForestRegressionModel = { + val categoricalFeatures: Map[Int, Int] = + MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val strategy = + super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) + val oldModel = OldRandomForest.trainRegressor( + oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed.toInt) + RandomForestRegressionModel.fromOld(oldModel, this, categoricalFeatures) + } +} + +object RandomForestRegressor { + /** Accessor for supported impurity settings: variance */ + final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities + + /** Accessor for supported featureSubsetStrategy settings: auto, all, onethird, sqrt, log2 */ + final val supportedFeatureSubsetStrategies: Array[String] = + RandomForestParams.supportedFeatureSubsetStrategies +} + +/** + * :: AlphaComponent :: + * + * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for regression. + * It supports both continuous and categorical features. + * @param _trees Decision trees in the ensemble. + */ +@AlphaComponent +final class RandomForestRegressionModel private[ml] ( + override val parent: RandomForestRegressor, + private val _trees: Array[DecisionTreeRegressionModel]) + extends PredictionModel[Vector, RandomForestRegressionModel] + with TreeEnsembleModel with Serializable { + + require(numTrees > 0, "RandomForestRegressionModel requires at least 1 tree.") + + override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + + // Note: We may add support for weights (based on tree performance) later on. + private lazy val _treeWeights: Array[Double] = Array.fill[Double](numTrees)(1.0) + + override def treeWeights: Array[Double] = _treeWeights + + override protected def predict(features: Vector): Double = { + // TODO: Override transform() to broadcast model. SPARK-7127 + // TODO: When we add a generic Bagging class, handle transform there. SPARK-7128 + // Predict average of tree predictions. + // Ignore the weights since all are 1.0 for now. + _trees.map(_.rootNode.predict(features)).sum / numTrees + } + + override def copy(extra: ParamMap): RandomForestRegressionModel = { + copyValues(new RandomForestRegressionModel(parent, _trees), extra) + } + + override def toString: String = { + s"RandomForestRegressionModel with $numTrees trees" + } + + /** (private[ml]) Convert to a model in the old API */ + private[ml] def toOld: OldRandomForestModel = { + new OldRandomForestModel(OldAlgo.Regression, _trees.map(_.toOld)) + } +} + +private[ml] object RandomForestRegressionModel { + + /** (private[ml]) Convert a model from the old API */ + def fromOld( + oldModel: OldRandomForestModel, + parent: RandomForestRegressor, + categoricalFeatures: Map[Int, Int]): RandomForestRegressionModel = { + require(oldModel.algo == OldAlgo.Regression, "Cannot convert RandomForestModel" + + s" with algo=${oldModel.algo} (old API) to RandomForestRegressionModel (new API).") + val newTrees = oldModel.trees.map { tree => + // parent, fittingParamMap for each tree is null since there are no good ways to set these. + DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) + } + new RandomForestRegressionModel(parent, newTrees) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index d679085eeafe1..c6b3327db6ad3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} /** 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 new file mode 100644 index 0000000000000..d2dec0c76cb12 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -0,0 +1,205 @@ +/* + * 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.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.model.{InformationGainStats => OldInformationGainStats, + Node => OldNode, Predict => OldPredict} + + +/** + * Decision tree node interface. + */ +sealed abstract class Node extends Serializable { + + // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree + // code into the new API and deprecate the old API. SPARK-3727 + + /** Prediction a leaf node makes, or which an internal node would make if it were a leaf node */ + def prediction: Double + + /** Impurity measure at this node (for training data) */ + def impurity: Double + + /** Recursive prediction helper method */ + private[ml] def predict(features: Vector): Double = prediction + + /** + * Get the number of nodes in tree below this node, including leaf nodes. + * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. + */ + private[tree] def numDescendants: Int + + /** + * Recursive print function. + * @param indentFactor The number of spaces to add to each level of indentation. + */ + private[tree] def subtreeToString(indentFactor: Int = 0): String + + /** + * Get depth of tree from this node. + * E.g.: Depth 0 means this is a leaf node. Depth 1 means 1 internal and 2 leaf nodes. + */ + private[tree] def subtreeDepth: Int + + /** + * Create a copy of this node in the old Node format, recursively creating child nodes as needed. + * @param id Node ID using old format IDs + */ + private[ml] def toOld(id: Int): OldNode +} + +private[ml] object Node { + + /** + * Create a new Node from the old Node format, recursively creating child nodes as needed. + */ + def fromOld(oldNode: OldNode, categoricalFeatures: Map[Int, Int]): Node = { + if (oldNode.isLeaf) { + // TODO: Once the implementation has been moved to this API, then include sufficient + // statistics here. + new LeafNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity) + } else { + val gain = if (oldNode.stats.nonEmpty) { + oldNode.stats.get.gain + } else { + 0.0 + } + new InternalNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity, + gain = gain, leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures), + rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures), + split = Split.fromOld(oldNode.split.get, categoricalFeatures)) + } + } +} + +/** + * Decision tree leaf node. + * @param prediction Prediction this node makes + * @param impurity Impurity measure at this node (for training data) + */ +final class LeafNode private[ml] ( + override val prediction: Double, + override val impurity: Double) extends Node { + + override def toString: String = s"LeafNode(prediction = $prediction, impurity = $impurity)" + + override private[ml] def predict(features: Vector): Double = prediction + + override private[tree] def numDescendants: Int = 0 + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"Predict: $prediction\n" + } + + override private[tree] def subtreeDepth: Int = 0 + + override private[ml] def toOld(id: Int): OldNode = { + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = true, + None, None, None, None) + } +} + +/** + * Internal Decision Tree node. + * @param prediction Prediction this node would make if it were a leaf node + * @param impurity Impurity measure at this node (for training data) + * @param gain Information gain value. + * Values < 0 indicate missing values; this quirk will be removed with future updates. + * @param leftChild Left-hand child node + * @param rightChild Right-hand child node + * @param split Information about the test used to split to the left or right child. + */ +final class InternalNode private[ml] ( + override val prediction: Double, + override val impurity: Double, + val gain: Double, + val leftChild: Node, + val rightChild: Node, + val split: Split) extends Node { + + override def toString: String = { + s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" + } + + override private[ml] def predict(features: Vector): Double = { + if (split.shouldGoLeft(features)) { + leftChild.predict(features) + } else { + rightChild.predict(features) + } + } + + override private[tree] def numDescendants: Int = { + 2 + leftChild.numDescendants + rightChild.numDescendants + } + + override private[tree] def subtreeToString(indentFactor: Int = 0): String = { + val prefix: String = " " * indentFactor + prefix + s"If (${InternalNode.splitToString(split, left=true)})\n" + + leftChild.subtreeToString(indentFactor + 1) + + prefix + s"Else (${InternalNode.splitToString(split, left=false)})\n" + + rightChild.subtreeToString(indentFactor + 1) + } + + override private[tree] def subtreeDepth: Int = { + 1 + math.max(leftChild.subtreeDepth, rightChild.subtreeDepth) + } + + override private[ml] def toOld(id: Int): OldNode = { + assert(id.toLong * 2 < Int.MaxValue, "Decision Tree could not be converted from new to old API" + + " since the old API does not support deep trees.") + // NOTE: We do NOT store 'prob' in the new API currently. + new OldNode(id, new OldPredict(prediction, prob = 0.0), impurity, isLeaf = false, + Some(split.toOld), Some(leftChild.toOld(OldNode.leftChildIndex(id))), + Some(rightChild.toOld(OldNode.rightChildIndex(id))), + Some(new OldInformationGainStats(gain, impurity, leftChild.impurity, rightChild.impurity, + new OldPredict(leftChild.prediction, prob = 0.0), + new OldPredict(rightChild.prediction, prob = 0.0)))) + } +} + +private object InternalNode { + + /** + * Helper method for [[Node.subtreeToString()]]. + * @param split Split to print + * @param left Indicates whether this is the part of the split going to the left, + * or that going to the right. + */ + private def splitToString(split: Split, left: Boolean): String = { + val featureStr = s"feature ${split.featureIndex}" + split match { + case contSplit: ContinuousSplit => + if (left) { + s"$featureStr <= ${contSplit.threshold}" + } else { + s"$featureStr > ${contSplit.threshold}" + } + case catSplit: CategoricalSplit => + val categoriesStr = catSplit.leftCategories.mkString("{", ",", "}") + if (left) { + s"$featureStr in $categoriesStr" + } else { + s"$featureStr not in $categoriesStr" + } + } + } +} 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 new file mode 100644 index 0000000000000..90f1d052764d3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -0,0 +1,152 @@ +/* + * 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.tree + +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} +import org.apache.spark.mllib.tree.model.{Split => OldSplit} + + +/** + * Interface for a "Split," which specifies a test made at a decision tree node + * to choose the left or right path. + */ +sealed trait Split extends Serializable { + + /** Index of feature which this split tests */ + def featureIndex: Int + + /** Return true (split to left) or false (split to right) */ + private[ml] def shouldGoLeft(features: Vector): Boolean + + /** Convert to old Split format */ + private[tree] def toOld: OldSplit +} + +private[tree] object Split { + + def fromOld(oldSplit: OldSplit, categoricalFeatures: Map[Int, Int]): Split = { + oldSplit.featureType match { + case OldFeatureType.Categorical => + new CategoricalSplit(featureIndex = oldSplit.feature, + _leftCategories = oldSplit.categories.toArray, categoricalFeatures(oldSplit.feature)) + case OldFeatureType.Continuous => + new ContinuousSplit(featureIndex = oldSplit.feature, threshold = oldSplit.threshold) + } + } +} + +/** + * Split which tests a categorical feature. + * @param featureIndex Index of the feature to test + * @param _leftCategories If the feature value is in this set of categories, then the split goes + * left. Otherwise, it goes right. + * @param numCategories Number of categories for this feature. + */ +final class CategoricalSplit private[ml] ( + override val featureIndex: Int, + _leftCategories: Array[Double], + private val numCategories: Int) + extends Split { + + require(_leftCategories.forall(cat => 0 <= cat && cat < numCategories), "Invalid leftCategories" + + s" (should be in range [0, $numCategories)): ${_leftCategories.mkString(",")}") + + /** + * If true, then "categories" is the set of categories for splitting to the left, and vice versa. + */ + private val isLeft: Boolean = _leftCategories.length <= numCategories / 2 + + /** Set of categories determining the splitting rule, along with [[isLeft]]. */ + private val categories: Set[Double] = { + if (isLeft) { + _leftCategories.toSet + } else { + setComplement(_leftCategories.toSet) + } + } + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + if (isLeft) { + categories.contains(features(featureIndex)) + } else { + !categories.contains(features(featureIndex)) + } + } + + override def equals(o: Any): Boolean = { + o match { + case other: CategoricalSplit => featureIndex == other.featureIndex && + isLeft == other.isLeft && categories == other.categories + case _ => false + } + } + + override private[tree] def toOld: OldSplit = { + val oldCats = if (isLeft) { + categories + } else { + setComplement(categories) + } + OldSplit(featureIndex, threshold = 0.0, OldFeatureType.Categorical, oldCats.toList) + } + + /** Get sorted categories which split to the left */ + def leftCategories: Array[Double] = { + val cats = if (isLeft) categories else setComplement(categories) + cats.toArray.sorted + } + + /** Get sorted categories which split to the right */ + def rightCategories: Array[Double] = { + val cats = if (isLeft) setComplement(categories) else categories + cats.toArray.sorted + } + + /** [0, numCategories) \ cats */ + private def setComplement(cats: Set[Double]): Set[Double] = { + Range(0, numCategories).map(_.toDouble).filter(cat => !cats.contains(cat)).toSet + } +} + +/** + * Split which tests a continuous feature. + * @param featureIndex Index of the feature to test + * @param threshold If the feature value is <= this threshold, then the split goes left. + * Otherwise, it goes right. + */ +final class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) + extends Split { + + override private[ml] def shouldGoLeft(features: Vector): Boolean = { + features(featureIndex) <= threshold + } + + override def equals(o: Any): Boolean = { + o match { + case other: ContinuousSplit => + featureIndex == other.featureIndex && threshold == other.threshold + case _ => + false + } + } + + override private[tree] def toOld: OldSplit = { + OldSplit(featureIndex, threshold, OldFeatureType.Continuous, List.empty[Double]) + } +} 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 new file mode 100644 index 0000000000000..1929f9d02156e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -0,0 +1,92 @@ +/* + * 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.tree + + +/** + * Abstraction for Decision Tree models. + * + * TODO: Add support for predicting probabilities and raw predictions SPARK-3727 + */ +private[ml] trait DecisionTreeModel { + + /** Root of the decision tree */ + def rootNode: Node + + /** Number of nodes in tree, including leaf nodes. */ + def numNodes: Int = { + 1 + rootNode.numDescendants + } + + /** + * Depth of the tree. + * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. + */ + lazy val depth: Int = { + rootNode.subtreeDepth + } + + /** Summary of the model */ + override def toString: String = { + // Implementing classes should generally override this method to be more descriptive. + s"DecisionTreeModel of depth $depth with $numNodes nodes" + } + + /** Full description of model */ + def toDebugString: String = { + val header = toString + "\n" + header + rootNode.subtreeToString(2) + } +} + +/** + * Abstraction for models which are ensembles of decision trees + * + * TODO: Add support for predicting probabilities and raw predictions SPARK-3727 + */ +private[ml] trait TreeEnsembleModel { + + // Note: We use getTrees since subclasses of TreeEnsembleModel will store subclasses of + // DecisionTreeModel. + + /** Trees in this ensemble. Warning: These have null parent Estimators. */ + def trees: Array[DecisionTreeModel] + + /** Weights for each tree, zippable with [[trees]] */ + def treeWeights: Array[Double] + + /** Summary of the model */ + override def toString: String = { + // Implementing classes should generally override this method to be more descriptive. + s"TreeEnsembleModel with $numTrees trees" + } + + /** Full description of model */ + def toDebugString: String = { + val header = toString + "\n" + header + trees.zip(treeWeights).zipWithIndex.map { case ((tree, weight), treeIndex) => + s" Tree $treeIndex (weight $weight):\n" + tree.rootNode.subtreeToString(4) + }.fold("")(_ + _) + } + + /** Number of trees in ensemble */ + val numTrees: Int = trees.length + + /** Total number of nodes, summed over all trees in the ensemble. */ + lazy val totalNumNodes: Int = trees.map(_.numNodes).sum +} 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 2eb1dac56f1e9..cee2aa6e85523 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 @@ -22,7 +22,7 @@ import com.github.fommil.netlib.F2jBLAS import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ -import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} +import org.apache.spark.ml.param._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.StructType @@ -31,6 +31,7 @@ import org.apache.spark.sql.types.StructType * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends Params { + /** * param for the estimator to be cross-validated * @group param @@ -38,7 +39,7 @@ private[ml] trait CrossValidatorParams extends Params { val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") /** @group getParam */ - def getEstimator: Estimator[_] = get(estimator) + def getEstimator: Estimator[_] = $(estimator) /** * param for estimator param maps @@ -48,7 +49,7 @@ private[ml] trait CrossValidatorParams extends Params { new Param(this, "estimatorParamMaps", "param maps for the estimator") /** @group getParam */ - def getEstimatorParamMaps: Array[ParamMap] = get(estimatorParamMaps) + def getEstimatorParamMaps: Array[ParamMap] = $(estimatorParamMaps) /** * param for the evaluator for selection @@ -57,17 +58,20 @@ private[ml] trait CrossValidatorParams extends Params { val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") /** @group getParam */ - def getEvaluator: Evaluator = get(evaluator) + def getEvaluator: Evaluator = $(evaluator) /** - * param for number of folds for cross validation + * Param for number of folds for cross validation. Must be >= 2. + * Default: 3 * @group param */ - val numFolds: IntParam = - new IntParam(this, "numFolds", "number of folds for cross validation", Some(3)) + val numFolds: IntParam = new IntParam(this, "numFolds", + "number of folds for cross validation (>= 2)", ParamValidators.gtEq(2)) /** @group getParam */ - def getNumFolds: Int = get(numFolds) + def getNumFolds: Int = $(numFolds) + + setDefault(numFolds -> 3) } /** @@ -91,17 +95,22 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP /** @group setParam */ def setNumFolds(value: Int): this.type = set(numFolds, value) - override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = { - val map = this.paramMap ++ paramMap + override def validateParams(paramMap: ParamMap): Unit = { + getEstimatorParamMaps.foreach { eMap => + getEstimator.validateParams(eMap ++ paramMap) + } + } + + override def fit(dataset: DataFrame): CrossValidatorModel = { val schema = dataset.schema - transformSchema(dataset.schema, paramMap, logging = true) + transformSchema(dataset.schema, logging = true) val sqlCtx = dataset.sqlContext - val est = map(estimator) - val eval = map(evaluator) - val epm = map(estimatorParamMaps) - val numModels = epm.size - val metrics = new Array[Double](epm.size) - val splits = MLUtils.kFold(dataset.rdd, map(numFolds), 0) + val est = $(estimator) + val eval = $(evaluator) + val epm = $(estimatorParamMaps) + val numModels = epm.length + val metrics = new Array[Double](epm.length) + val splits = MLUtils.kFold(dataset.rdd, $(numFolds), 0) splits.zipWithIndex.foreach { case ((training, validation), splitIndex) => val trainingDataset = sqlCtx.createDataFrame(training, schema).cache() val validationDataset = sqlCtx.createDataFrame(validation, schema).cache() @@ -111,27 +120,24 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP trainingDataset.unpersist() var i = 0 while (i < numModels) { - val metric = eval.evaluate(models(i).transform(validationDataset, epm(i)), map) + val metric = eval.evaluate(models(i).transform(validationDataset, epm(i))) logDebug(s"Got metric $metric for model trained with ${epm(i)}.") metrics(i) += metric i += 1 } validationDataset.unpersist() } - f2jBLAS.dscal(numModels, 1.0 / map(numFolds), metrics, 1) + f2jBLAS.dscal(numModels, 1.0 / $(numFolds), metrics, 1) logInfo(s"Average cross-validation metrics: ${metrics.toSeq}") val (bestMetric, bestIndex) = metrics.zipWithIndex.maxBy(_._1) logInfo(s"Best set of parameters:\n${epm(bestIndex)}") logInfo(s"Best cross-validation metric: $bestMetric.") val bestModel = est.fit(dataset, epm(bestIndex)).asInstanceOf[Model[_]] - val cvModel = new CrossValidatorModel(this, map, bestModel) - Params.inheritValues(map, this, cvModel) - cvModel + copyValues(new CrossValidatorModel(this, bestModel)) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - map(estimator).transformSchema(schema, paramMap) + override def transformSchema(schema: StructType): StructType = { + $(estimator).transformSchema(schema) } } @@ -142,15 +148,18 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP @AlphaComponent class CrossValidatorModel private[ml] ( override val parent: CrossValidator, - override val fittingParamMap: ParamMap, val bestModel: Model[_]) extends Model[CrossValidatorModel] with CrossValidatorParams { - override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - bestModel.transform(dataset, paramMap) + override def validateParams(paramMap: ParamMap): Unit = { + bestModel.validateParams(paramMap) + } + + override def transform(dataset: DataFrame): DataFrame = { + bestModel.transform(dataset) } - override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - bestModel.transformSchema(schema, paramMap) + override def transformSchema(schema: StructType): StructType = { + bestModel.transformSchema(schema) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala similarity index 88% rename from mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala rename to mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala index cd84b05bfb496..8a56748ab0a02 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Identifiable.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.ml +package org.apache.spark.ml.util import java.util.UUID @@ -25,9 +25,9 @@ import java.util.UUID private[ml] trait Identifiable extends Serializable { /** - * A unique id for the object. The default implementation concatenates the class name, "-", and 8 + * A unique id for the object. The default implementation concatenates the class name, "_", and 8 * random hex chars. */ private[ml] val uid: String = - this.getClass.getSimpleName + "-" + UUID.randomUUID().toString.take(8) + this.getClass.getSimpleName + "_" + UUID.randomUUID().toString.take(8) } 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 new file mode 100644 index 0000000000000..c84c8b4eb744f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -0,0 +1,82 @@ +/* + * 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.util + +import scala.collection.immutable.HashMap + +import org.apache.spark.annotation.Experimental +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, BinaryAttribute, NominalAttribute, + NumericAttribute} +import org.apache.spark.sql.types.StructField + + +/** + * :: Experimental :: + * + * Helper utilities for tree-based algorithms + */ +@Experimental +object MetadataUtils { + + /** + * Examine a schema to identify the number of classes in a label column. + * Returns None if the number of labels is not specified, or if the label column is continuous. + */ + def getNumClasses(labelSchema: StructField): Option[Int] = { + Attribute.fromStructField(labelSchema) match { + case numAttr: NumericAttribute => None + case binAttr: BinaryAttribute => Some(2) + case nomAttr: NominalAttribute => nomAttr.getNumValues + } + } + + /** + * Examine a schema to identify categorical (Binary and Nominal) features. + * + * @param featuresSchema Schema of the features column. + * If a feature does not have metadata, it is assumed to be continuous. + * If a feature is Nominal, then it must have the number of values + * specified. + * @return Map: feature index --> number of categories. + * The map's set of keys will be the set of categorical feature indices. + */ + def getCategoricalFeatures(featuresSchema: StructField): Map[Int, Int] = { + val metadata = AttributeGroup.fromStructField(featuresSchema) + if (metadata.attributes.isEmpty) { + HashMap.empty[Int, Int] + } else { + metadata.attributes.get.zipWithIndex.flatMap { case (attr, idx) => + if (attr == null) { + Iterator() + } else { + attr match { + case numAttr: NumericAttribute => Iterator() + case binAttr: BinaryAttribute => Iterator(idx -> 2) + case nomAttr: NominalAttribute => + nomAttr.getNumValues match { + case Some(numValues: Int) => Iterator(idx -> numValues) + case None => throw new IllegalArgumentException(s"Feature $idx is marked as" + + " Nominal (categorical), but it does not have the number of values specified.") + } + } + } + }.toMap + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala new file mode 100644 index 0000000000000..0383bf0b382b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -0,0 +1,61 @@ +/* + * 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.util + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.types.{DataType, StructField, StructType} + +/** + * :: DeveloperApi :: + * Utils for handling schemas. + */ +@DeveloperApi +object SchemaUtils { + + // TODO: Move the utility methods to SQL. + + /** + * Check whether the given schema contains a column of the required data type. + * @param colName column name + * @param dataType required column data type + */ + def checkColumnType(schema: StructType, colName: String, dataType: DataType): Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Column $colName must be of type $dataType but was actually $actualDataType.") + } + + /** + * Appends a new column to the input schema. This fails if the given output column already exists. + * @param schema input schema + * @param colName new column name. If this column name is an empty string "", this method returns + * the input schema unchanged. This allows users to disable output columns. + * @param dataType new column data type + * @return new schema with the input column appended + */ + def appendColumn( + schema: StructType, + colName: String, + dataType: DataType): StructType = { + if (colName.isEmpty) return schema + val fieldNames = schema.fieldNames + require(!fieldNames.contains(colName), s"Column $colName already exists.") + val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) + StructType(outputFields) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala new file mode 100644 index 0000000000000..ee933f4cfcafd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api.python + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.rdd.RDD + +/** + * A Wrapper of FPGrowthModel to provide helper method for Python + */ +private[python] class FPGrowthModelWrapper(model: FPGrowthModel[Any]) + extends FPGrowthModel(model.freqItemsets) { + + def getFreqItemsets: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(model.freqItemsets.map(x => (x.javaItems, x.freq))) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala new file mode 100644 index 0000000000000..534edac56bc5a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.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.mllib.api.python + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.recommendation.{MatrixFactorizationModel, Rating} +import org.apache.spark.rdd.RDD + +/** + * A Wrapper of MatrixFactorizationModel to provide helper method for Python. + */ +private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) + extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { + + def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = + predict(SerDe.asTupleRDD(userAndProducts.rdd)) + + def getUserFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(userFeatures.map { + case (user, feature) => (user, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) + } + + def getProductFeatures: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(productFeatures.map { + case (product, feature) => (product, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) + } +} 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 15ca2547d56a8..b086cec083381 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 @@ -28,26 +28,26 @@ import scala.reflect.ClassTag import net.razorvine.pickle._ -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.feature._ +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.stat.test.ChiSqTestResult -import org.apache.spark.mllib.tree.{GradientBoostedTrees, RandomForest, DecisionTree} -import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo, Strategy} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel, RandomForestModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, RandomForestModel} +import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -58,7 +58,6 @@ import org.apache.spark.util.Utils */ private[python] class PythonMLLibAPI extends Serializable { - /** * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. * @param jsc Java SparkContext @@ -78,7 +77,13 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector): JList[Object] = { try { val model = learner.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), initialWeights) - List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + if (model.isInstanceOf[LogisticRegressionModel]) { + val lrModel = model.asInstanceOf[LogisticRegressionModel] + List(lrModel.weights, lrModel.intercept, lrModel.numFeatures, lrModel.numClasses) + .map(_.asInstanceOf[Object]).asJava + } else { + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + } } finally { data.rdd.unpersist(blocking = false) } @@ -111,9 +116,11 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val lrAlg = new LinearRegressionWithSGD() lrAlg.setIntercept(intercept) + .setValidateData(validateData) lrAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -135,8 +142,12 @@ private[python] class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector): JList[Object] = { + initialWeights: Vector, + intercept: Boolean, + validateData: Boolean): JList[Object] = { val lassoAlg = new LassoWithSGD() + lassoAlg.setIntercept(intercept) + .setValidateData(validateData) lassoAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -157,8 +168,12 @@ private[python] class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector): JList[Object] = { + initialWeights: Vector, + intercept: Boolean, + validateData: Boolean): JList[Object] = { val ridgeAlg = new RidgeRegressionWithSGD() + ridgeAlg.setIntercept(intercept) + .setValidateData(validateData) ridgeAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -181,9 +196,11 @@ private[python] class PythonMLLibAPI extends Serializable { miniBatchFraction: Double, initialWeights: Vector, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val SVMAlg = new SVMWithSGD() SVMAlg.setIntercept(intercept) + .setValidateData(validateData) SVMAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -207,9 +224,11 @@ private[python] class PythonMLLibAPI extends Serializable { initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): JList[Object] = { + intercept: Boolean, + validateData: Boolean): JList[Object] = { val LogRegAlg = new LogisticRegressionWithSGD() LogRegAlg.setIntercept(intercept) + .setValidateData(validateData) LogRegAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -233,9 +252,13 @@ private[python] class PythonMLLibAPI extends Serializable { regType: String, intercept: Boolean, corrections: Int, - tolerance: Double): JList[Object] = { + tolerance: Double, + validateData: Boolean, + numClasses: Int): JList[Object] = { val LogRegAlg = new LogisticRegressionWithLBFGS() LogRegAlg.setIntercept(intercept) + .setValidateData(validateData) + .setNumClasses(numClasses) LogRegAlg.optimizer .setNumIterations(numIterations) .setRegParam(regParam) @@ -255,7 +278,7 @@ private[python] class PythonMLLibAPI extends Serializable { data: JavaRDD[LabeledPoint], lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta.map(Vectors.dense)). map(_.asInstanceOf[Object]).asJava } @@ -268,12 +291,16 @@ private[python] class PythonMLLibAPI extends Serializable { maxIterations: Int, runs: Int, initializationMode: String, - seed: java.lang.Long): KMeansModel = { + seed: java.lang.Long, + initializationSteps: Int, + epsilon: Double): KMeansModel = { val kMeansAlg = new KMeans() .setK(k) .setMaxIterations(maxIterations) .setRuns(runs) .setInitializationMode(initializationMode) + .setInitializationSteps(initializationSteps) + .setEpsilon(epsilon) if (seed != null) kMeansAlg.setSeed(seed) @@ -284,6 +311,15 @@ private[python] class PythonMLLibAPI extends Serializable { } } + /** + * Java stub for Python mllib KMeansModel.computeCost() + */ + def computeCostKmeansModel( + data: JavaRDD[Vector], + centers: java.util.ArrayList[Vector]): Double = { + new KMeansModel(centers).computeCost(data) + } + /** * Java stub for Python mllib GaussianMixture.run() * Returns a list containing weights, mean and covariance of each mixture component. @@ -311,7 +347,7 @@ private[python] class PythonMLLibAPI extends Serializable { mu += model.gaussians(i).mu sigma += model.gaussians(i).sigma } - List(wt.toArray, mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + List(Vectors.dense(wt.toArray), mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava } finally { data.rdd.unpersist(blocking = false) } @@ -322,37 +358,18 @@ private[python] class PythonMLLibAPI extends Serializable { */ def predictSoftGMM( data: JavaRDD[Vector], - wt: Object, + wt: Vector, mu: Array[Object], - si: Array[Object]): RDD[Array[Double]] = { + si: Array[Object]): RDD[Vector] = { - val weight = wt.asInstanceOf[Array[Double]] + val weight = wt.toArray val mean = mu.map(_.asInstanceOf[DenseVector]) val sigma = si.map(_.asInstanceOf[DenseMatrix]) val gaussians = Array.tabulate(weight.length){ i => new MultivariateGaussian(mean(i), sigma(i)) } val model = new GaussianMixtureModel(weight, gaussians) - model.predictSoft(data) - } - - /** - * A Wrapper of MatrixFactorizationModel to provide helpfer method for Python - */ - private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) - extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { - - def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = - predict(SerDe.asTupleRDD(userAndProducts.rdd)) - - def getUserFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) - } - - def getProductFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) - } - + model.predictSoft(data).map(Vectors.dense) } /** @@ -414,6 +431,24 @@ private[python] class PythonMLLibAPI extends Serializable { new MatrixFactorizationModelWrapper(model) } + /** + * Java stub for Python mllib FPGrowth.train(). This stub returns a handle + * to the Java object instead of the content of the Java object. Extra care + * needs to be taken in the Python code to ensure it gets freed on exit; see + * the Py4J documentation. + */ + def trainFPGrowthModel( + data: JavaRDD[java.lang.Iterable[Any]], + minSupport: Double, + numPartitions: Int): FPGrowthModel[Any] = { + val fpg = new FPGrowth() + .setMinSupport(minSupport) + .setNumPartitions(numPartitions) + + val model = fpg.run(data.rdd.map(_.asScala.toArray)) + new FPGrowthModelWrapper(model) + } + /** * Java stub for Normalizer.transform() */ @@ -427,9 +462,9 @@ private[python] class PythonMLLibAPI extends Serializable { def normalizeVector(p: Double, rdd: JavaRDD[Vector]): JavaRDD[Vector] = { new Normalizer(p).transform(rdd) } - + /** - * Java stub for IDF.fit(). This stub returns a + * Java stub for StandardScaler.fit(). This stub returns a * handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. @@ -470,13 +505,15 @@ private[python] class PythonMLLibAPI extends Serializable { learningRate: Double, numPartitions: Int, numIterations: Int, - seed: Long): Word2VecModelWrapper = { + seed: Long, + minCount: Int): Word2VecModelWrapper = { val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) .setNumPartitions(numPartitions) .setNumIterations(numIterations) .setSeed(seed) + .setMinCount(minCount) try { val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) new Word2VecModelWrapper(model) @@ -510,6 +547,10 @@ private[python] class PythonMLLibAPI extends Serializable { val words = result.map(_._1) List(words, similarity).map(_.asInstanceOf[Object]).asJava } + + def getVectors: JMap[String, JList[Float]] = { + model.getVectors.map({case (k, v) => (k, v.toList.asJava)}).asJava + } } /** @@ -907,6 +948,14 @@ private[spark] object SerDe extends Serializable { out.write(code) } + protected def getBytes(obj: Object): Array[Byte] = { + if (obj.getClass.isArray) { + obj.asInstanceOf[Array[Byte]] + } else { + obj.asInstanceOf[String].getBytes(LATIN1) + } + } + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } @@ -932,7 +981,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(0)) val bb = ByteBuffer.wrap(bytes, 0, bytes.length) bb.order(ByteOrder.nativeOrder()) val db = bb.asDoubleBuffer() @@ -949,8 +998,10 @@ private[spark] object SerDe extends Serializable { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] val bytes = new Array[Byte](8 * m.values.size) 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) @@ -958,19 +1009,77 @@ private[spark] object SerDe extends Serializable { out.write(Opcodes.BINSTRING) out.write(PickleUtils.integer_to_bytes(bytes.length)) out.write(bytes) - out.write(Opcodes.TUPLE3) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) } def construct(args: Array[Object]): Object = { - if (args.length != 3) { - throw new PickleException("should be 3") + if (args.length != 4) { + throw new PickleException("should be 4") } - val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + 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) - new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for SparseMatrix + private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[SparseMatrix] + 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 SparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) } } @@ -1002,8 +1111,8 @@ private[spark] object SerDe extends Serializable { throw new PickleException("should be 3") } val size = args(0).asInstanceOf[Int] - val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) - val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + 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) @@ -1058,6 +1167,7 @@ private[spark] object SerDe extends Serializable { if (!initialized) { new DenseVectorPickler().register() new DenseMatrixPickler().register() + new SparseMatrixPickler().register() new SparseVectorPickler().register() new LabeledPointPickler().register() new RatingPickler().register() @@ -1107,7 +1217,10 @@ private[spark] object SerDe extends Serializable { iter.flatMap { row => val obj = unpickle.loads(row) if (batched) { - obj.asInstanceOf[JArrayList[_]].asScala + obj match { + case list: JArrayList[_] => list.asScala + case arr: Array[_] => arr + } } else { Seq(obj) } 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 e7c3599ff619c..bd2e9079ce1ae 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 @@ -23,6 +23,7 @@ import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} import org.apache.spark.rdd.RDD @@ -46,7 +47,7 @@ class LogisticRegressionModel ( val numFeatures: Int, val numClasses: Int) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable - with Saveable { + with Saveable with PMMLExportable { if (numClasses == 2) { require(weights.size == numFeatures, @@ -62,6 +63,15 @@ class LogisticRegressionModel ( s" but was given weights of length ${weights.size}") } + private val dataWithBiasSize: Int = weights.size / (numClasses - 1) + + private val weightsArray: Array[Double] = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.getClass}.") + } + /** * Constructs a [[LogisticRegressionModel]] with weights and intercept for binary classification. */ @@ -74,6 +84,7 @@ class LogisticRegressionModel ( * Sets the threshold that separates positive predictions from negative predictions * in Binary Logistic Regression. An example with prediction score greater than or equal to * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. + * It is only used for binary classification. */ @Experimental def setThreshold(threshold: Double): this.type = { @@ -84,6 +95,7 @@ class LogisticRegressionModel ( /** * :: Experimental :: * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. + * It is only used for binary classification. */ @Experimental def getThreshold: Option[Double] = threshold @@ -91,6 +103,7 @@ class LogisticRegressionModel ( /** * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. + * It is only used for binary classification. */ @Experimental def clearThreshold(): this.type = { @@ -106,7 +119,6 @@ class LogisticRegressionModel ( // If dataMatrix and weightMatrix have the same dimension, it's binary logistic regression. if (numClasses == 2) { - require(numFeatures == weightMatrix.size) val margin = dot(weightMatrix, dataMatrix) + intercept val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { @@ -114,30 +126,9 @@ class LogisticRegressionModel ( case None => score } } else { - val dataWithBiasSize = weightMatrix.size / (numClasses - 1) - - val weightsArray = weightMatrix match { - case dv: DenseVector => dv.values - case _ => - throw new IllegalArgumentException( - s"weights only supports dense vector but got type ${weightMatrix.getClass}.") - } - - val margins = (0 until numClasses - 1).map { i => - var margin = 0.0 - dataMatrix.foreachActive { (index, value) => - if (value != 0.0) margin += value * weightsArray((i * dataWithBiasSize) + index) - } - // Intercept is required to be added into margin. - if (dataMatrix.size + 1 == dataWithBiasSize) { - margin += weightsArray((i * dataWithBiasSize) + dataMatrix.size) - } - margin - } - /** - * Find the one with maximum margins. If the maxMargin is negative, then the prediction - * result will be the first class. + * Compute and find the one with maximum margins. If the maxMargin is negative, then the + * prediction result will be the first class. * * PS, if you want to compute the probabilities for each outcome instead of the outcome * with maximum probability, remember to subtract the maxMargin from margins if maxMargin @@ -145,13 +136,20 @@ class LogisticRegressionModel ( */ var bestClass = 0 var maxMargin = 0.0 - var i = 0 - while(i < margins.size) { - if (margins(i) > maxMargin) { - maxMargin = margins(i) + val withBias = dataMatrix.size + 1 == dataWithBiasSize + (0 until numClasses - 1).foreach { i => + var margin = 0.0 + dataMatrix.foreachActive { (index, value) => + if (value != 0.0) margin += value * weightsArray((i * dataWithBiasSize) + index) + } + // Intercept is required to be added into margin. + if (withBias) { + margin += weightsArray((i * dataWithBiasSize) + dataMatrix.size) + } + if (margin > maxMargin) { + maxMargin = margin bestClass = i + 1 } - i += 1 } bestClass.toDouble } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index d60e82c410979..c9b3ff0172e2e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -21,9 +21,12 @@ import java.lang.{Iterable => JIterable} import scala.collection.JavaConverters._ -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum, Axis} +import breeze.numerics.{exp => brzExp, log => brzLog} + import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.json4s.{DefaultFormats, JValue} import org.apache.spark.{Logging, SparkContext, SparkException} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector} @@ -32,6 +35,7 @@ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} + /** * Model for Naive Bayes Classifiers. * @@ -39,11 +43,17 @@ import org.apache.spark.sql.{DataFrame, SQLContext} * @param pi log of class priors, whose dimension is C, number of labels * @param theta log of class conditional probabilities, whose dimension is C-by-D, * where D is number of features + * @param modelType The type of NB model to fit can be "Multinomial" or "Bernoulli" */ class NaiveBayesModel private[mllib] ( val labels: Array[Double], val pi: Array[Double], - val theta: Array[Array[Double]]) extends ClassificationModel with Serializable with Saveable { + val theta: Array[Array[Double]], + val modelType: String) + extends ClassificationModel with Serializable with Saveable { + + private[mllib] def this(labels: Array[Double], pi: Array[Double], theta: Array[Array[Double]]) = + this(labels, pi, theta, "Multinomial") /** A Java-friendly constructor that takes three Iterable parameters. */ private[mllib] def this( @@ -53,19 +63,19 @@ class NaiveBayesModel private[mllib] ( this(labels.asScala.toArray, pi.asScala.toArray, theta.asScala.toArray.map(_.asScala.toArray)) private val brzPi = new BDV[Double](pi) - private val brzTheta = new BDM[Double](theta.length, theta(0).length) - - { - // Need to put an extra pair of braces to prevent Scala treating `i` as a member. - var i = 0 - while (i < theta.length) { - var j = 0 - while (j < theta(i).length) { - brzTheta(i, j) = theta(i)(j) - j += 1 - } - i += 1 - } + private val brzTheta = new BDM(theta(0).length, theta.length, theta.flatten).t + + // Bernoulli scoring requires log(condprob) if 1, log(1-condprob) if 0. + // This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra + // application of this condition (in predict function). + private val (brzNegTheta, brzNegThetaSum) = modelType match { + case "Multinomial" => (None, None) + case "Bernoulli" => + val negTheta = brzLog((brzExp(brzTheta.copy) :*= (-1.0)) :+= 1.0) // log(1.0 - exp(x)) + (Option(negTheta), Option(brzSum(negTheta, Axis._1))) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayesModel was created with an unknown ModelType: $modelType") } override def predict(testData: RDD[Vector]): RDD[Double] = { @@ -77,22 +87,78 @@ class NaiveBayesModel private[mllib] ( } override def predict(testData: Vector): Double = { - labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) + modelType match { + case "Multinomial" => + labels (brzArgmax (brzPi + brzTheta * testData.toBreeze) ) + case "Bernoulli" => + labels (brzArgmax (brzPi + + (brzTheta - brzNegTheta.get) * testData.toBreeze + brzNegThetaSum.get)) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayesModel was created with an unknown ModelType: $modelType") + } } override def save(sc: SparkContext, path: String): Unit = { - val data = NaiveBayesModel.SaveLoadV1_0.Data(labels, pi, theta) - NaiveBayesModel.SaveLoadV1_0.save(sc, path, data) + val data = NaiveBayesModel.SaveLoadV2_0.Data(labels, pi, theta, modelType) + NaiveBayesModel.SaveLoadV2_0.save(sc, path, data) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = "2.0" } object NaiveBayesModel extends Loader[NaiveBayesModel] { import org.apache.spark.mllib.util.Loader._ - private object SaveLoadV1_0 { + private[mllib] object SaveLoadV2_0 { + + def thisFormatVersion: String = "2.0" + + /** Hard-code class name string in case it changes in the future */ + def thisClassName: String = "org.apache.spark.mllib.classification.NaiveBayesModel" + + /** Model data for model import/export */ + case class Data( + labels: Array[Double], + pi: Array[Double], + theta: Array[Array[Double]], + modelType: String) + + def save(sc: SparkContext, path: String, data: Data): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("numFeatures" -> data.theta(0).length) ~ ("numClasses" -> data.pi.length))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) + + // Create Parquet data. + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() + dataRDD.saveAsParquetFile(dataPath(path)) + } + + def load(sc: SparkContext, path: String): NaiveBayesModel = { + val sqlContext = new SQLContext(sc) + // Load Parquet data. + val dataRDD = sqlContext.parquetFile(dataPath(path)) + // Check schema explicitly since erasure makes it hard to use match-case for checking. + checkSchema[Data](dataRDD.schema) + val dataArray = dataRDD.select("labels", "pi", "theta", "modelType").take(1) + assert(dataArray.size == 1, s"Unable to load NaiveBayesModel data from: ${dataPath(path)}") + val data = dataArray(0) + val labels = data.getAs[Seq[Double]](0).toArray + val pi = data.getAs[Seq[Double]](1).toArray + val theta = data.getAs[Seq[Seq[Double]]](2).map(_.toArray).toArray + val modelType = data.getString(3) + new NaiveBayesModel(labels, pi, theta, modelType) + } + + } + + private[mllib] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" @@ -100,7 +166,10 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { def thisClassName: String = "org.apache.spark.mllib.classification.NaiveBayesModel" /** Model data for model import/export */ - case class Data(labels: Array[Double], pi: Array[Double], theta: Array[Array[Double]]) + case class Data( + labels: Array[Double], + pi: Array[Double], + theta: Array[Array[Double]]) def save(sc: SparkContext, path: String, data: Data): Unit = { val sqlContext = new SQLContext(sc) @@ -136,26 +205,32 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { override def load(sc: SparkContext, path: String): NaiveBayesModel = { val (loadedClassName, version, metadata) = loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName - (loadedClassName, version) match { + val classNameV2_0 = SaveLoadV2_0.thisClassName + val (model, numFeatures, numClasses) = (loadedClassName, version) match { case (className, "1.0") if className == classNameV1_0 => val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) val model = SaveLoadV1_0.load(sc, path) - assert(model.pi.size == numClasses, - s"NaiveBayesModel.load expected $numClasses classes," + - s" but class priors vector pi had ${model.pi.size} elements") - assert(model.theta.size == numClasses, - s"NaiveBayesModel.load expected $numClasses classes," + - s" but class conditionals array theta had ${model.theta.size} elements") - assert(model.theta.forall(_.size == numFeatures), - s"NaiveBayesModel.load expected $numFeatures features," + - s" but class conditionals array theta had elements of size:" + - s" ${model.theta.map(_.size).mkString(",")}") - model + (model, numFeatures, numClasses) + case (className, "2.0") if className == classNameV2_0 => + val (numFeatures, numClasses) = ClassificationModel.getNumFeaturesClasses(metadata) + val model = SaveLoadV2_0.load(sc, path) + (model, numFeatures, numClasses) case _ => throw new Exception( s"NaiveBayesModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $version). Supported:\n" + s" ($classNameV1_0, 1.0)") } + assert(model.pi.size == numClasses, + s"NaiveBayesModel.load expected $numClasses classes," + + s" but class priors vector pi had ${model.pi.size} elements") + assert(model.theta.size == numClasses, + s"NaiveBayesModel.load expected $numClasses classes," + + s" but class conditionals array theta had ${model.theta.size} elements") + assert(model.theta.forall(_.size == numFeatures), + s"NaiveBayesModel.load expected $numFeatures features," + + s" but class conditionals array theta had elements of size:" + + s" ${model.theta.map(_.size).mkString(",")}") + model } } @@ -167,9 +242,14 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. */ -class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { - def this() = this(1.0) +class NaiveBayes private ( + private var lambda: Double, + private var modelType: String) extends Serializable with Logging { + + def this(lambda: Double) = this(lambda, "Multinomial") + + def this() = this(1.0, "Multinomial") /** Set the smoothing parameter. Default: 1.0. */ def setLambda(lambda: Double): NaiveBayes = { @@ -177,9 +257,24 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with this } - /** Get the smoothing parameter. Default: 1.0. */ + /** Get the smoothing parameter. */ def getLambda: Double = lambda + /** + * Set the model type using a string (case-sensitive). + * Supported options: "Multinomial" and "Bernoulli". + * (default: Multinomial) + */ + def setModelType(modelType:String): NaiveBayes = { + require(NaiveBayes.supportedModelTypes.contains(modelType), + s"NaiveBayes was created with an unknown ModelType: $modelType") + this.modelType = modelType + this + } + + /** Get the model type. */ + def getModelType: String = this.modelType + /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * @@ -213,21 +308,30 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => (c1._1 + c2._1, c1._2 += c2._2) ).collect() + val numLabels = aggregated.length var numDocuments = 0L aggregated.foreach { case (_, (n, _)) => numDocuments += n } val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } + val labels = new Array[Double](numLabels) val pi = new Array[Double](numLabels) val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) + val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 aggregated.foreach { case (label, (n, sumTermFreqs)) => labels(i) = label - val thetaLogDenom = math.log(brzSum(sumTermFreqs) + numFeatures * lambda) pi(i) = math.log(n + lambda) - piLogDenom + val thetaLogDenom = modelType match { + case "Multinomial" => math.log(brzSum(sumTermFreqs) + numFeatures * lambda) + case "Bernoulli" => math.log(n + 2.0 * lambda) + case _ => + // This should never happen. + throw new UnknownError(s"NaiveBayes was created with an unknown ModelType: $modelType") + } var j = 0 while (j < numFeatures) { theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom @@ -236,7 +340,7 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with i += 1 } - new NaiveBayesModel(labels, pi, theta) + new NaiveBayesModel(labels, pi, theta, modelType) } } @@ -244,13 +348,16 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with * Top-level methods for calling naive Bayes. */ object NaiveBayes { + + /* Set of modelTypes that NaiveBayes supports */ + private[mllib] val supportedModelTypes = Set("Multinomial", "Bernoulli") + /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of - * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for - * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all + * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it + * can be used for document classification. * * This version of the method uses a default smoothing parameter of 1.0. * @@ -264,16 +371,40 @@ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of - * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for - * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all + * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it + * can be used for document classification. * * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency * vector or a count vector. * @param lambda The smoothing parameter */ def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { - new NaiveBayes(lambda).run(input) + new NaiveBayes(lambda, "Multinomial").run(input) + } + + /** + * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. + * + * The model type can be set to either Multinomial NB ([[http://tinyurl.com/lsdw6p]]) + * or Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The Multinomial NB can handle + * discrete count data and can be called by setting the model type to "multinomial". + * For example, it can be used with word counts or TF_IDF vectors of documents. + * The Bernoulli model fits presence or absence (0-1) counts. By making every vector a + * 0-1 vector and setting the model type to "bernoulli", the fits and predicts as + * Bernoulli NB. + * + * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency + * vector or a count vector. + * @param lambda The smoothing parameter + * + * @param modelType The type of NB model to fit from the enumeration NaiveBayesModels, can be + * multinomial or bernoulli + */ + def train(input: RDD[LabeledPoint], lambda: Double, modelType: String): NaiveBayesModel = { + require(supportedModelTypes.contains(modelType), + s"NaiveBayes was created with an unknown ModelType: $modelType") + new NaiveBayes(lambda, modelType).run(input) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 52fb62dcff1b4..33104cf06c6ea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -22,6 +22,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{DataValidators, Loader, Saveable} import org.apache.spark.rdd.RDD @@ -36,7 +37,7 @@ class SVMModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable - with Saveable { + with Saveable with PMMLExportable { private var threshold: Option[Double] = Some(0.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala index b89f38cf5aba4..7d33df3221fbf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala @@ -63,6 +63,8 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( protected val algorithm = new LogisticRegressionWithSGD( stepSize, numIterations, regParam, miniBatchFraction) + protected var model: Option[LogisticRegressionModel] = None + /** Set the step size for gradient descent. Default: 0.1. */ def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index af6f83c74bb40..ec65a3da689de 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -19,11 +19,17 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BreezeVector} +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vector, Matrices, Matrix} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.{MLUtils, Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, Row} /** * :: Experimental :: @@ -41,10 +47,16 @@ import org.apache.spark.rdd.RDD @Experimental class GaussianMixtureModel( val weights: Array[Double], - val gaussians: Array[MultivariateGaussian]) extends Serializable { + val gaussians: Array[MultivariateGaussian]) extends Serializable with Saveable{ require(weights.length == gaussians.length, "Length of weight and Gaussian arrays must match") - + + override protected def formatVersion = "1.0" + + override def save(sc: SparkContext, path: String): Unit = { + GaussianMixtureModel.SaveLoadV1_0.save(sc, path, weights, gaussians) + } + /** Number of gaussians in mixture */ def k: Int = weights.length @@ -83,5 +95,79 @@ class GaussianMixtureModel( p(i) /= pSum } p - } + } +} + +@Experimental +object GaussianMixtureModel extends Loader[GaussianMixtureModel] { + + private object SaveLoadV1_0 { + + case class Data(weight: Double, mu: Vector, sigma: Matrix) + + val formatVersionV1_0 = "1.0" + + val classNameV1_0 = "org.apache.spark.mllib.clustering.GaussianMixtureModel" + + def save( + sc: SparkContext, + path: String, + weights: Array[Double], + gaussians: Array[MultivariateGaussian]): Unit = { + + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Create JSON metadata. + val metadata = compact(render + (("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ ("k" -> weights.length))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + // Create Parquet data. + val dataArray = Array.tabulate(weights.length) { i => + Data(weights(i), gaussians(i).mu, gaussians(i).sigma) + } + sc.parallelize(dataArray, 1).toDF().saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): GaussianMixtureModel = { + val dataPath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataFrame = sqlContext.parquetFile(dataPath) + val dataArray = dataFrame.select("weight", "mu", "sigma").collect() + + // Check schema explicitly since erasure makes it hard to use match-case for checking. + Loader.checkSchema[Data](dataFrame.schema) + + val (weights, gaussians) = dataArray.map { + case Row(weight: Double, mu: Vector, sigma: Matrix) => + (weight, new MultivariateGaussian(mu, sigma)) + }.unzip + + return new GaussianMixtureModel(weights.toArray, gaussians.toArray) + } + } + + override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + implicit val formats = DefaultFormats + val k = (metadata \ "k").extract[Int] + val classNameV1_0 = SaveLoadV1_0.classNameV1_0 + (loadedClassName, version) match { + case (classNameV1_0, "1.0") => { + val model = SaveLoadV1_0.load(sc, path) + require(model.weights.length == k, + s"GaussianMixtureModel requires weights of length $k " + + s"got weights of length ${model.weights.length}") + require(model.gaussians.length == k, + s"GaussianMixtureModel requires gaussians of length $k" + + s"got gaussians of length ${model.gaussians.length}") + model + } + case _ => throw new Exception( + s"GaussianMixtureModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index e4e411a3c8b42..ba228b11fcec3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -25,6 +25,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext @@ -34,7 +35,8 @@ import org.apache.spark.sql.Row /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { +class KMeansModel ( + val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { /** A Java-friendly constructor that takes an Iterable of Vectors. */ def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 5e17c8da61134..c8daa2388e868 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -17,16 +17,11 @@ package org.apache.spark.mllib.clustering -import java.util.Random - -import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} - +import breeze.linalg.{DenseVector => BDV} import org.apache.spark.Logging import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ -import org.apache.spark.graphx.impl.GraphImpl -import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -42,16 +37,9 @@ import org.apache.spark.util.Utils * - "token": instance of a term appearing in a document * - "topic": multinomial distribution over words representing some concept * - * Currently, the underlying implementation uses Expectation-Maximization (EM), implemented - * according to the Asuncion et al. (2009) paper referenced below. - * * References: * - Original LDA paper (journal version): * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. - * - This class implements their "smoothed" LDA model. - * - Paper which clearly explains several algorithms, including EM: - * Asuncion, Welling, Smyth, and Teh. - * "On Smoothing and Inference for Topic Models." UAI, 2009. * * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation * (Wikipedia)]] @@ -63,10 +51,11 @@ class LDA private ( private var docConcentration: Double, private var topicConcentration: Double, private var seed: Long, - private var checkpointInterval: Int) extends Logging { + private var checkpointInterval: Int, + private var ldaOptimizer: LDAOptimizer) extends Logging { def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, - seed = Utils.random.nextLong(), checkpointInterval = 10) + seed = Utils.random.nextLong(), checkpointInterval = 10, ldaOptimizer = new EMLDAOptimizer) /** * Number of topics to infer. I.e., the number of soft cluster centers. @@ -89,35 +78,29 @@ class LDA private ( * * This is the parameter to a symmetric Dirichlet distribution. */ - def getDocConcentration: Double = { - if (this.docConcentration == -1) { - (50.0 / k) + 1.0 - } else { - this.docConcentration - } - } + def getDocConcentration: Double = this.docConcentration /** * Concentration parameter (commonly named "alpha") for the prior placed on documents' * distributions over topics ("theta"). * - * This is the parameter to a symmetric Dirichlet distribution. + * This is the parameter to a symmetric Dirichlet distribution, where larger values + * mean more smoothing (more regularization). * - * This value should be > 1.0, where larger values mean more smoothing (more regularization). * If set to -1, then docConcentration is set automatically. * (default = -1 = automatic) * - * Automatic setting of parameter: - * - For EM: default = (50 / k) + 1. - * - The 50/k is common in LDA libraries. - * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. - * - * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), - * but values in (0,1) are not yet supported. + * Optimizer-specific parameter settings: + * - EM + * - Value should be > 1.0 + * - default = (50 / k) + 1, where 50/k is common in LDA libraries and +1 follows + * Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * - Online + * - Value should be >= 0 + * - default = (1.0 / k), following the implementation from + * [[https://github.com/Blei-Lab/onlineldavb]]. */ def setDocConcentration(docConcentration: Double): this.type = { - require(docConcentration > 1.0 || docConcentration == -1.0, - s"LDA docConcentration must be > 1.0 (or -1 for auto), but was set to $docConcentration") this.docConcentration = docConcentration this } @@ -137,13 +120,7 @@ class LDA private ( * Note: The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. */ - def getTopicConcentration: Double = { - if (this.topicConcentration == -1) { - 1.1 - } else { - this.topicConcentration - } - } + def getTopicConcentration: Double = this.topicConcentration /** * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics' @@ -154,21 +131,20 @@ class LDA private ( * Note: The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. * - * This value should be > 0.0. * If set to -1, then topicConcentration is set automatically. * (default = -1 = automatic) * - * Automatic setting of parameter: - * - For EM: default = 0.1 + 1. - * - The 0.1 gives a small amount of smoothing. - * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. - * - * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), - * but values in (0,1) are not yet supported. + * Optimizer-specific parameter settings: + * - EM + * - Value should be > 1.0 + * - default = 0.1 + 1, where 0.1 gives a small amount of smoothing and +1 follows + * Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * - Online + * - Value should be >= 0 + * - default = (1.0 / k), following the implementation from + * [[https://github.com/Blei-Lab/onlineldavb]]. */ def setTopicConcentration(topicConcentration: Double): this.type = { - require(topicConcentration > 1.0 || topicConcentration == -1.0, - s"LDA topicConcentration must be > 1.0 (or -1 for auto), but was set to $topicConcentration") this.topicConcentration = topicConcentration this } @@ -177,7 +153,7 @@ class LDA private ( def getBeta: Double = getTopicConcentration /** Alias for [[setTopicConcentration()]] */ - def setBeta(beta: Double): this.type = setBeta(beta) + def setBeta(beta: Double): this.type = setTopicConcentration(beta) /** * Maximum number of iterations for learning. @@ -220,6 +196,33 @@ class LDA private ( this } + + /** LDAOptimizer used to perform the actual calculation */ + def getOptimizer: LDAOptimizer = ldaOptimizer + + /** + * LDAOptimizer used to perform the actual calculation (default = EMLDAOptimizer) + */ + def setOptimizer(optimizer: LDAOptimizer): this.type = { + this.ldaOptimizer = optimizer + this + } + + /** + * Set the LDAOptimizer used to perform the actual calculation by algorithm name. + * Currently "em", "online" is supported. + */ + def setOptimizer(optimizerName: String): this.type = { + this.ldaOptimizer = + optimizerName.toLowerCase match { + case "em" => new EMLDAOptimizer + case "online" => new OnlineLDAOptimizer + case other => + throw new IllegalArgumentException(s"Only em, online are supported but got $other.") + } + this + } + /** * Learn an LDA model using the given dataset. * @@ -229,9 +232,8 @@ class LDA private ( * Document IDs must be unique and >= 0. * @return Inferred LDA model */ - def run(documents: RDD[(Long, Vector)]): DistributedLDAModel = { - val state = LDA.initialState(documents, k, getDocConcentration, getTopicConcentration, seed, - checkpointInterval) + def run(documents: RDD[(Long, Vector)]): LDAModel = { + val state = ldaOptimizer.initialize(documents, this) var iter = 0 val iterationTimes = Array.fill[Double](maxIterations)(0) while (iter < maxIterations) { @@ -241,12 +243,11 @@ class LDA private ( iterationTimes(iter) = elapsedSeconds iter += 1 } - state.graphCheckpointer.deleteAllCheckpoints() - new DistributedLDAModel(state, iterationTimes) + state.getLDAModel(iterationTimes) } /** Java-friendly version of [[run()]] */ - def run(documents: JavaPairRDD[java.lang.Long, Vector]): DistributedLDAModel = { + def run(documents: JavaPairRDD[java.lang.Long, Vector]): LDAModel = { run(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } } @@ -320,88 +321,10 @@ private[clustering] object LDA { private[clustering] def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 - /** - * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. - * - * @param graph EM graph, storing current parameter estimates in vertex descriptors and - * data (token counts) in edge descriptors. - * @param k Number of topics - * @param vocabSize Number of unique terms - * @param docConcentration "alpha" - * @param topicConcentration "beta" or "eta" - */ - private[clustering] class EMOptimizer( - var graph: Graph[TopicCounts, TokenCount], - val k: Int, - val vocabSize: Int, - val docConcentration: Double, - val topicConcentration: Double, - checkpointInterval: Int) { - - private[LDA] val graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount]( - graph, checkpointInterval) - - def next(): EMOptimizer = { - val eta = topicConcentration - val W = vocabSize - val alpha = docConcentration - - val N_k = globalTopicTotals - val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = - (edgeContext) => { - // Compute N_{wj} gamma_{wjk} - val N_wj = edgeContext.attr - // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count - // N_{wj}. - val scaledTopicDistribution: TopicCounts = - computePTopic(edgeContext.srcAttr, edgeContext.dstAttr, N_k, W, eta, alpha) *= N_wj - edgeContext.sendToDst((false, scaledTopicDistribution)) - edgeContext.sendToSrc((false, scaledTopicDistribution)) - } - // This is a hack to detect whether we could modify the values in-place. - // TODO: Add zero/seqOp/combOp option to aggregateMessages. (SPARK-5438) - val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = - (m0, m1) => { - val sum = - if (m0._1) { - m0._2 += m1._2 - } else if (m1._1) { - m1._2 += m0._2 - } else { - m0._2 + m1._2 - } - (true, sum) - } - // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. - val docTopicDistributions: VertexRDD[TopicCounts] = - graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) - .mapValues(_._2) - // Update the vertex descriptors with the new counts. - val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) - graph = newGraph - graphCheckpointer.updateGraph(newGraph) - globalTopicTotals = computeGlobalTopicTotals() - this - } - - /** - * Aggregate distributions over topics from all term vertices. - * - * Note: This executes an action on the graph RDDs. - */ - var globalTopicTotals: TopicCounts = computeGlobalTopicTotals() - - private def computeGlobalTopicTotals(): TopicCounts = { - val numTopics = k - graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) - } - - } - /** * Compute gamma_{wjk}, a distribution over topics k. */ - private def computePTopic( + private[clustering] def computePTopic( docTopicCounts: TopicCounts, termTopicCounts: TopicCounts, totalTopicCounts: TopicCounts, @@ -427,49 +350,4 @@ private[clustering] object LDA { // normalize BDV(gamma_wj) /= sum } - - /** - * Compute bipartite term/doc graph. - */ - private def initialState( - docs: RDD[(Long, Vector)], - k: Int, - docConcentration: Double, - topicConcentration: Double, - randomSeed: Long, - checkpointInterval: Int): EMOptimizer = { - // For each document, create an edge (Document -> Term) for each unique term in the document. - val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => - // Add edges for terms with non-zero counts. - termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => - Edge(docID, term2index(term), cnt) - } - } - - val vocabSize = docs.take(1).head._2.size - - // Create vertices. - // Initially, we use random soft assignments of tokens to topics (random gamma). - def createVertices(): RDD[(VertexId, TopicCounts)] = { - val verticesTMP: RDD[(VertexId, TopicCounts)] = - edges.mapPartitionsWithIndex { case (partIndex, partEdges) => - val random = new Random(partIndex + randomSeed) - partEdges.flatMap { edge => - val gamma = normalize(BDV.fill[Double](k)(random.nextDouble()), 1.0) - val sum = gamma * edge.attr - Seq((edge.srcId, sum), (edge.dstId, sum)) - } - } - verticesTMP.reduceByKey(_ + _) - } - - val docTermVertices = createVertices() - - // Partition such that edges are grouped by document - val graph = Graph(docTermVertices, edges) - .partitionBy(PartitionStrategy.EdgePartition1D) - - new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointInterval) - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 0a3f21ecee0dc..6cf26445f20a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -203,7 +203,7 @@ class DistributedLDAModel private ( import LDA._ - private[clustering] def this(state: LDA.EMOptimizer, iterationTimes: Array[Double]) = { + private[clustering] def this(state: EMLDAOptimizer, iterationTimes: Array[Double]) = { this(state.graph, state.globalTopicTotals, state.k, state.vocabSize, state.docConcentration, state.topicConcentration, iterationTimes) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala new file mode 100644 index 0000000000000..093aa0f315ab2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import java.util.Random + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, sum, normalize, kron} +import breeze.numerics.{digamma, exp, abs} +import breeze.stats.distributions.{Gamma, RandBasis} + +import org.apache.spark.annotation.Experimental +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.GraphImpl +import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer +import org.apache.spark.mllib.linalg.{Matrices, SparseVector, DenseVector, Vector} +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * + * An LDAOptimizer specifies which optimization/learning/inference algorithm to use, and it can + * hold optimizer-specific parameters for users to set. + */ +@Experimental +trait LDAOptimizer { + + /* + DEVELOPERS NOTE: + + An LDAOptimizer contains an algorithm for LDA and performs the actual computation, which + stores internal data structure (Graph or Matrix) and other parameters for the algorithm. + The interface is isolated to improve the extensibility of LDA. + */ + + /** + * Initializer for the optimizer. LDA passes the common parameters to the optimizer and + * the internal structure can be initialized properly. + */ + private[clustering] def initialize(docs: RDD[(Long, Vector)], lda: LDA): LDAOptimizer + + private[clustering] def next(): LDAOptimizer + + private[clustering] def getLDAModel(iterationTimes: Array[Double]): LDAModel +} + +/** + * :: Experimental :: + * + * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. + * + * Currently, the underlying implementation uses Expectation-Maximization (EM), implemented + * according to the Asuncion et al. (2009) paper referenced below. + * + * References: + * - Original LDA paper (journal version): + * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * - This class implements their "smoothed" LDA model. + * - Paper which clearly explains several algorithms, including EM: + * Asuncion, Welling, Smyth, and Teh. + * "On Smoothing and Inference for Topic Models." UAI, 2009. + * + */ +@Experimental +class EMLDAOptimizer extends LDAOptimizer { + + import LDA._ + + /** + * The following fields will only be initialized through the initialize() method + */ + private[clustering] var graph: Graph[TopicCounts, TokenCount] = null + private[clustering] var k: Int = 0 + private[clustering] var vocabSize: Int = 0 + private[clustering] var docConcentration: Double = 0 + private[clustering] var topicConcentration: Double = 0 + private[clustering] var checkpointInterval: Int = 10 + private var graphCheckpointer: PeriodicGraphCheckpointer[TopicCounts, TokenCount] = null + + /** + * Compute bipartite term/doc graph. + */ + override private[clustering] def initialize(docs: RDD[(Long, Vector)], lda: LDA): LDAOptimizer = { + + val docConcentration = lda.getDocConcentration + val topicConcentration = lda.getTopicConcentration + val k = lda.getK + + // Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), + // but values in (0,1) are not yet supported. + require(docConcentration > 1.0 || docConcentration == -1.0, s"LDA docConcentration must be" + + s" > 1.0 (or -1 for auto) for EM Optimizer, but was set to $docConcentration") + require(topicConcentration > 1.0 || topicConcentration == -1.0, s"LDA topicConcentration " + + s"must be > 1.0 (or -1 for auto) for EM Optimizer, but was set to $topicConcentration") + + this.docConcentration = if (docConcentration == -1) (50.0 / k) + 1.0 else docConcentration + this.topicConcentration = if (topicConcentration == -1) 1.1 else topicConcentration + val randomSeed = lda.getSeed + + // For each document, create an edge (Document -> Term) for each unique term in the document. + val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => + // Add edges for terms with non-zero counts. + termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + Edge(docID, term2index(term), cnt) + } + } + + // Create vertices. + // Initially, we use random soft assignments of tokens to topics (random gamma). + val docTermVertices: RDD[(VertexId, TopicCounts)] = { + val verticesTMP: RDD[(VertexId, TopicCounts)] = + edges.mapPartitionsWithIndex { case (partIndex, partEdges) => + val random = new Random(partIndex + randomSeed) + partEdges.flatMap { edge => + val gamma = normalize(BDV.fill[Double](k)(random.nextDouble()), 1.0) + val sum = gamma * edge.attr + Seq((edge.srcId, sum), (edge.dstId, sum)) + } + } + verticesTMP.reduceByKey(_ + _) + } + + // Partition such that edges are grouped by document + this.graph = Graph(docTermVertices, edges).partitionBy(PartitionStrategy.EdgePartition1D) + this.k = k + this.vocabSize = docs.take(1).head._2.size + this.checkpointInterval = lda.getCheckpointInterval + this.graphCheckpointer = new + PeriodicGraphCheckpointer[TopicCounts, TokenCount](graph, checkpointInterval) + this.globalTopicTotals = computeGlobalTopicTotals() + this + } + + override private[clustering] def next(): EMLDAOptimizer = { + require(graph != null, "graph is null, EMLDAOptimizer not initialized.") + + val eta = topicConcentration + val W = vocabSize + val alpha = docConcentration + + val N_k = globalTopicTotals + val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = + (edgeContext) => { + // Compute N_{wj} gamma_{wjk} + val N_wj = edgeContext.attr + // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count + // N_{wj}. + val scaledTopicDistribution: TopicCounts = + computePTopic(edgeContext.srcAttr, edgeContext.dstAttr, N_k, W, eta, alpha) *= N_wj + edgeContext.sendToDst((false, scaledTopicDistribution)) + edgeContext.sendToSrc((false, scaledTopicDistribution)) + } + // This is a hack to detect whether we could modify the values in-place. + // TODO: Add zero/seqOp/combOp option to aggregateMessages. (SPARK-5438) + val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = + (m0, m1) => { + val sum = + if (m0._1) { + m0._2 += m1._2 + } else if (m1._1) { + m1._2 += m0._2 + } else { + m0._2 + m1._2 + } + (true, sum) + } + // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. + val docTopicDistributions: VertexRDD[TopicCounts] = + graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) + .mapValues(_._2) + // Update the vertex descriptors with the new counts. + val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) + graph = newGraph + graphCheckpointer.updateGraph(newGraph) + globalTopicTotals = computeGlobalTopicTotals() + this + } + + /** + * Aggregate distributions over topics from all term vertices. + * + * Note: This executes an action on the graph RDDs. + */ + private[clustering] var globalTopicTotals: TopicCounts = null + + private def computeGlobalTopicTotals(): TopicCounts = { + val numTopics = k + graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) + } + + override private[clustering] def getLDAModel(iterationTimes: Array[Double]): LDAModel = { + require(graph != null, "graph is null, EMLDAOptimizer not initialized.") + this.graphCheckpointer.deleteAllCheckpoints() + new DistributedLDAModel(this, iterationTimes) + } +} + + +/** + * :: Experimental :: + * + * An online optimizer for LDA. The Optimizer implements the Online variational Bayes LDA + * algorithm, which processes a subset of the corpus on each iteration, and updates the term-topic + * distribution adaptively. + * + * Original Online LDA paper: + * Hoffman, Blei and Bach, "Online Learning for Latent Dirichlet Allocation." NIPS, 2010. + */ +@Experimental +class OnlineLDAOptimizer extends LDAOptimizer { + + // LDA common parameters + private var k: Int = 0 + private var corpusSize: Long = 0 + private var vocabSize: Int = 0 + + /** alias for docConcentration */ + private var alpha: Double = 0 + + /** (private[clustering] for debugging) Get docConcentration */ + private[clustering] def getAlpha: Double = alpha + + /** alias for topicConcentration */ + private var eta: Double = 0 + + /** (private[clustering] for debugging) Get topicConcentration */ + private[clustering] def getEta: Double = eta + + private var randomGenerator: java.util.Random = null + + // Online LDA specific parameters + // Learning rate is: (tau_0 + t)^{-kappa} + private var tau_0: Double = 1024 + private var kappa: Double = 0.51 + private var miniBatchFraction: Double = 0.05 + + // internal data structure + private var docs: RDD[(Long, Vector)] = null + + /** Dirichlet parameter for the posterior over topics */ + private var lambda: BDM[Double] = null + + /** (private[clustering] for debugging) Get parameter for topics */ + private[clustering] def getLambda: BDM[Double] = lambda + + /** Current iteration (count of invocations of [[next()]]) */ + private var iteration: Int = 0 + private var gammaShape: Double = 100 + + /** + * A (positive) learning parameter that downweights early iterations. Larger values make early + * iterations count less. + */ + def getTau_0: Double = this.tau_0 + + /** + * A (positive) learning parameter that downweights early iterations. Larger values make early + * iterations count less. + * Default: 1024, following the original Online LDA paper. + */ + def setTau_0(tau_0: Double): this.type = { + require(tau_0 > 0, s"LDA tau_0 must be positive, but was set to $tau_0") + this.tau_0 = tau_0 + this + } + + /** + * Learning rate: exponential decay rate + */ + def getKappa: Double = this.kappa + + /** + * Learning rate: exponential decay rate---should be between + * (0.5, 1.0] to guarantee asymptotic convergence. + * Default: 0.51, based on the original Online LDA paper. + */ + def setKappa(kappa: Double): this.type = { + require(kappa >= 0, s"Online LDA kappa must be nonnegative, but was set to $kappa") + this.kappa = kappa + this + } + + /** + * Mini-batch fraction, which sets the fraction of document sampled and used in each iteration + */ + def getMiniBatchFraction: Double = this.miniBatchFraction + + /** + * Mini-batch fraction in (0, 1], which sets the fraction of document sampled and used in + * each iteration. + * + * Note that this should be adjusted in synch with [[LDA.setMaxIterations()]] + * so the entire corpus is used. Specifically, set both so that + * maxIterations * miniBatchFraction >= 1. + * + * Default: 0.05, i.e., 5% of total documents. + */ + def setMiniBatchFraction(miniBatchFraction: Double): this.type = { + require(miniBatchFraction > 0.0 && miniBatchFraction <= 1.0, + s"Online LDA miniBatchFraction must be in range (0,1], but was set to $miniBatchFraction") + this.miniBatchFraction = miniBatchFraction + this + } + + /** + * (private[clustering]) + * Set the Dirichlet parameter for the posterior over topics. + * This is only used for testing now. In the future, it can help support training stop/resume. + */ + private[clustering] def setLambda(lambda: BDM[Double]): this.type = { + this.lambda = lambda + this + } + + /** + * (private[clustering]) + * Used for random initialization of the variational parameters. + * Larger value produces values closer to 1.0. + * This is only used for testing currently. + */ + private[clustering] def setGammaShape(shape: Double): this.type = { + this.gammaShape = shape + this + } + + override private[clustering] def initialize( + docs: RDD[(Long, Vector)], + lda: LDA): OnlineLDAOptimizer = { + this.k = lda.getK + this.corpusSize = docs.count() + this.vocabSize = docs.first()._2.size + this.alpha = if (lda.getDocConcentration == -1) 1.0 / k else lda.getDocConcentration + this.eta = if (lda.getTopicConcentration == -1) 1.0 / k else lda.getTopicConcentration + this.randomGenerator = new Random(lda.getSeed) + + this.docs = docs + + // Initialize the variational distribution q(beta|lambda) + this.lambda = getGammaMatrix(k, vocabSize) + this.iteration = 0 + this + } + + override private[clustering] def next(): OnlineLDAOptimizer = { + val batch = docs.sample(withReplacement = true, miniBatchFraction, randomGenerator.nextLong()) + if (batch.isEmpty()) return this + submitMiniBatch(batch) + } + + /** + * Submit a subset (like 1%, decide by the miniBatchFraction) of the corpus to the Online LDA + * model, and it will update the topic distribution adaptively for the terms appearing in the + * subset. + */ + private[clustering] def submitMiniBatch(batch: RDD[(Long, Vector)]): OnlineLDAOptimizer = { + iteration += 1 + val k = this.k + val vocabSize = this.vocabSize + val Elogbeta = dirichletExpectation(lambda) + val expElogbeta = exp(Elogbeta) + val alpha = this.alpha + val gammaShape = this.gammaShape + + val stats: RDD[BDM[Double]] = batch.mapPartitions { docs => + val stat = BDM.zeros[Double](k, vocabSize) + docs.foreach { doc => + val termCounts = doc._2 + val (ids: List[Int], cts: Array[Double]) = termCounts match { + case v: DenseVector => ((0 until v.size).toList, v.values) + case v: SparseVector => (v.indices.toList, v.values) + case v => throw new IllegalArgumentException("Online LDA does not support vector type " + + v.getClass) + } + + // Initialize the variational distribution q(theta|gamma) for the mini-batch + var gammad = new Gamma(gammaShape, 1.0 / gammaShape).samplesVector(k).t // 1 * K + var Elogthetad = digamma(gammad) - digamma(sum(gammad)) // 1 * K + var expElogthetad = exp(Elogthetad) // 1 * K + val expElogbetad = expElogbeta(::, ids).toDenseMatrix // K * ids + + var phinorm = expElogthetad * expElogbetad + 1e-100 // 1 * ids + var meanchange = 1D + val ctsVector = new BDV[Double](cts).t // 1 * ids + + // Iterate between gamma and phi until convergence + while (meanchange > 1e-3) { + val lastgamma = gammad + // 1*K 1 * ids ids * k + gammad = (expElogthetad :* ((ctsVector / phinorm) * expElogbetad.t)) + alpha + Elogthetad = digamma(gammad) - digamma(sum(gammad)) + expElogthetad = exp(Elogthetad) + phinorm = expElogthetad * expElogbetad + 1e-100 + meanchange = sum(abs(gammad - lastgamma)) / k + } + + val m1 = expElogthetad.t + val m2 = (ctsVector / phinorm).t.toDenseVector + var i = 0 + while (i < ids.size) { + stat(::, ids(i)) := stat(::, ids(i)) + m1 * m2(i) + i += 1 + } + } + Iterator(stat) + } + + val statsSum: BDM[Double] = stats.reduce(_ += _) + val batchResult = statsSum :* expElogbeta + + // Note that this is an optimization to avoid batch.count + update(batchResult, iteration, (miniBatchFraction * corpusSize).ceil.toInt) + this + } + + override private[clustering] def getLDAModel(iterationTimes: Array[Double]): LDAModel = { + new LocalLDAModel(Matrices.fromBreeze(lambda).transpose) + } + + /** + * Update lambda based on the batch submitted. batchSize can be different for each iteration. + */ + private[clustering] def update(stat: BDM[Double], iter: Int, batchSize: Int): Unit = { + val tau_0 = this.getTau_0 + val kappa = this.getKappa + + // weight of the mini-batch. + val weight = math.pow(tau_0 + iter, -kappa) + + // Update lambda based on documents. + lambda = lambda * (1 - weight) + + (stat * (corpusSize.toDouble / batchSize.toDouble) + eta) * weight + } + + /** + * Get a random matrix to initialize lambda + */ + private def getGammaMatrix(row: Int, col: Int): BDM[Double] = { + val randBasis = new RandBasis(new org.apache.commons.math3.random.MersenneTwister( + randomGenerator.nextLong())) + val gammaRandomGenerator = new Gamma(gammaShape, 1.0 / gammaShape)(randBasis) + val temp = gammaRandomGenerator.sample(row * col).toArray + new BDM[Double](col, row, temp).t + } + + /** + * For theta ~ Dir(alpha), computes E[log(theta)] given alpha. Currently the implementation + * uses digamma which is accurate but expensive. + */ + private def dirichletExpectation(alpha: BDM[Double]): BDM[Double] = { + val rowSum = sum(alpha(breeze.linalg.*, ::)) + val digAlpha = digamma(alpha) + val digRowSum = digamma(rowSum) + val result = digAlpha(::, breeze.linalg.*) - digRowSum + result + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 180023922a9b0..aa53e88d59856 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -17,15 +17,20 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.{Logging, SparkException} +import org.json4s.JsonDSL._ +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.{Logging, SparkContext, SparkException} /** * :: Experimental :: @@ -38,7 +43,60 @@ import org.apache.spark.util.random.XORShiftRandom @Experimental class PowerIterationClusteringModel( val k: Int, - val assignments: RDD[PowerIterationClustering.Assignment]) extends Serializable + val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { + + override def save(sc: SparkContext, path: String): Unit = { + PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) + } + + override protected def formatVersion: String = "1.0" +} + +object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { + override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) + } + + private[clustering] + object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" + + def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val dataRDD = model.assignments.toDF() + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + implicit val formats = DefaultFormats + val sqlContext = new SQLContext(sc) + + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + + val k = (metadata \ "k").extract[Int] + val assignments = sqlContext.parquetFile(Loader.dataPath(path)) + Loader.checkSchema[PowerIterationClustering.Assignment](assignments.schema) + + val assignmentsRDD = assignments.map { + case Row(id: Long, cluster: Int) => PowerIterationClustering.Assignment(id, cluster) + } + + new PowerIterationClusteringModel(k, assignmentsRDD) + } + } +} /** * :: Experimental :: @@ -135,7 +193,7 @@ class PowerIterationClustering private[clustering] ( val v = powerIter(w, maxIterations) val assignments = kMeans(v, k).mapPartitions({ iter => iter.map { case (id, cluster) => - new Assignment(id, cluster) + Assignment(id, cluster) } }, preservesPartitioning = true) new PowerIterationClusteringModel(k, assignments) @@ -152,7 +210,7 @@ object PowerIterationClustering extends Logging { * @param cluster assigned cluster id */ @Experimental - class Assignment(val id: Long, val cluster: Int) extends Serializable + case class Assignment(id: Long, cluster: Int) /** * Normalizes the affinity matrix (A) by row sums and returns the normalized affinity matrix (W). diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index f483fd1c7d2cf..812014a041719 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -20,8 +20,7 @@ package org.apache.spark.mllib.clustering import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.{Experimental, DeveloperApi} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.DStream @@ -165,7 +164,7 @@ class StreamingKMeansModel( class StreamingKMeans( var k: Int, var decayFactor: Double, - var timeUnit: String) extends Logging { + var timeUnit: String) extends Logging with Serializable { def this() = this(2, 1.0, StreamingKMeans.BATCHES) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 59a79e5c6a4ac..98e83112f52ae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -25,14 +25,21 @@ import scala.collection.mutable.ArrayBuilder import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.Logging +import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector, Vectors, DenseMatrix, BLAS, DenseVector} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.sql.{SQLContext, Row} /** * Entry in vocabulary @@ -422,7 +429,36 @@ class Word2Vec extends Serializable with Logging { */ @Experimental class Word2VecModel private[mllib] ( - private val model: Map[String, Array[Float]]) extends Serializable { + model: Map[String, Array[Float]]) extends Serializable with Saveable { + + // wordList: Ordered list of words obtained from model. + private val wordList: Array[String] = model.keys.toArray + + // wordIndex: Maps each word to an index, which can retrieve the corresponding + // vector from wordVectors (see below). + private val wordIndex: Map[String, Int] = wordList.zip(0 until model.size).toMap + + // vectorSize: Dimension of each word's vector. + private val vectorSize = model.head._2.size + private val numWords = wordIndex.size + + // wordVectors: Array of length numWords * vectorSize, vector corresponding to the word + // mapped with index i can be retrieved by the slice + // (ind * vectorSize, ind * vectorSize + vectorSize) + // wordVecNorms: Array of length numWords, each value being the Euclidean norm + // of the wordVector. + private val (wordVectors: Array[Float], wordVecNorms: Array[Double]) = { + val wordVectors = new Array[Float](vectorSize * numWords) + val wordVecNorms = new Array[Double](numWords) + var i = 0 + while (i < numWords) { + val vec = model.get(wordList(i)).get + Array.copy(vec, 0, wordVectors, i * vectorSize, vectorSize) + wordVecNorms(i) = blas.snrm2(vectorSize, vec, 1) + i += 1 + } + (wordVectors, wordVecNorms) + } private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { require(v1.length == v2.length, "Vectors should have the same length") @@ -432,7 +468,13 @@ class Word2VecModel private[mllib] ( if (norm1 == 0 || norm2 == 0) return 0.0 blas.sdot(n, v1, 1, v2,1) / norm1 / norm2 } - + + override protected def formatVersion = "1.0" + + def save(sc: SparkContext, path: String): Unit = { + Word2VecModel.SaveLoadV1_0.save(sc, path, getVectors) + } + /** * Transforms a word to its vector representation * @param word a word @@ -466,20 +508,104 @@ class Word2VecModel private[mllib] ( */ def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") - // TODO: optimize top-k + val fVector = vector.toArray.map(_.toFloat) - model.mapValues(vec => cosineSimilarity(fVector, vec)) + val cosineVec = Array.fill[Float](numWords)(0) + val alpha: Float = 1 + val beta: Float = 0 + + blas.sgemv( + "T", vectorSize, numWords, alpha, wordVectors, vectorSize, fVector, 1, beta, cosineVec, 1) + + // Need not divide with the norm of the given vector since it is constant. + val updatedCosines = new Array[Double](numWords) + var ind = 0 + while (ind < numWords) { + updatedCosines(ind) = cosineVec(ind) / wordVecNorms(ind) + ind += 1 + } + wordList.zip(updatedCosines) .toSeq .sortBy(- _._2) .take(num + 1) .tail .toArray } - + /** * Returns a map of words to their vector representations. */ def getVectors: Map[String, Array[Float]] = { - model + wordIndex.map { case (word, ind) => + (word, wordVectors.slice(vectorSize * ind, vectorSize * ind + vectorSize)) + } + } +} + +@Experimental +object Word2VecModel extends Loader[Word2VecModel] { + + private object SaveLoadV1_0 { + + val formatVersionV1_0 = "1.0" + + val classNameV1_0 = "org.apache.spark.mllib.feature.Word2VecModel" + + case class Data(word: String, vector: Array[Float]) + + def load(sc: SparkContext, path: String): Word2VecModel = { + val dataPath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataFrame = sqlContext.parquetFile(dataPath) + + val dataArray = dataFrame.select("word", "vector").collect() + + // Check schema explicitly since erasure makes it hard to use match-case for checking. + Loader.checkSchema[Data](dataFrame.schema) + + val word2VecMap = dataArray.map(i => (i.getString(0), i.getSeq[Float](1).toArray)).toMap + new Word2VecModel(word2VecMap) + } + + def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = { + + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val vectorSize = model.values.head.size + val numWords = model.size + val metadata = compact(render + (("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ + ("vectorSize" -> vectorSize) ~ ("numWords" -> numWords))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val dataArray = model.toSeq.map { case (w, v) => Data(w, v) } + sc.parallelize(dataArray.toSeq, 1).toDF().saveAsParquetFile(Loader.dataPath(path)) + } + } + + override def load(sc: SparkContext, path: String): Word2VecModel = { + + val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) + implicit val formats = DefaultFormats + val expectedVectorSize = (metadata \ "vectorSize").extract[Int] + val expectedNumWords = (metadata \ "numWords").extract[Int] + val classNameV1_0 = SaveLoadV1_0.classNameV1_0 + (loadedClassName, loadedVersion) match { + case (classNameV1_0, "1.0") => + val model = SaveLoadV1_0.load(sc, path) + val vectorSize = model.getVectors.values.head.size + val numWords = model.getVectors.size + require(expectedVectorSize == vectorSize, + s"Word2VecModel requires each word to be mapped to a vector of size " + + s"$expectedVectorSize, got vector of size $vectorSize") + require(expectedNumWords == numWords, + s"Word2VecModel requires $expectedNumWords words, but got $numWords") + model + case _ => throw new Exception( + s"Word2VecModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $loadedVersion). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } } } 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 849f44295f089..3fa5e068d16d4 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 @@ -87,6 +87,9 @@ sealed trait Matrix extends Serializable { /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() + /** A human readable representation of the matrix with maximum lines and width */ + def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + /** Map the values of this matrix using a function. Generates a new matrix. Performs the * function on only the backing array. For example, an operation such as addition or * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ @@ -187,6 +190,8 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { override def hashCode(): Int = 1994 + override def typeName: String = "matrix" + private[spark] override def asNullable: MatrixUDT = this } 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 2cda9b252ee06..188d1e542b5b5 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 @@ -52,7 +52,7 @@ sealed trait Vector extends Serializable { override def equals(other: Any): Boolean = { other match { - case v2: Vector => { + case v2: Vector => if (this.size != v2.size) return false (this, v2) match { case (s1: SparseVector, s2: SparseVector) => @@ -63,20 +63,28 @@ sealed trait Vector extends Serializable { Vectors.equals(0 until d1.size, d1.values, s1.indices, s1.values) case (_, _) => util.Arrays.equals(this.toArray, v2.toArray) } - } case _ => false } } + /** + * Returns a hash code value for the vector. The hash code is based on its size and its nonzeros + * in the first 16 entries, using a hash algorithm similar to [[java.util.Arrays.hashCode]]. + */ override def hashCode(): Int = { - var result: Int = size + 31 - this.foreachActive { case (index, value) => - // ignore explict 0 for comparison between sparse and dense - if (value != 0) { - result = 31 * result + index - // refer to {@link java.util.Arrays.equals} for hash algorithm - val bits = java.lang.Double.doubleToLongBits(value) - result = 31 * result + (bits ^ (bits >>> 32)).toInt + // This is a reference implementation. It calls return in foreachActive, which is slow. + // Subclasses should override it with optimized implementation. + var result: Int = 31 + size + this.foreachActive { (index, value) => + if (index < 16) { + // ignore explicit 0 for comparison between sparse and dense + if (value != 0) { + result = 31 * result + index + val bits = java.lang.Double.doubleToLongBits(value) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + } + } else { + return result } } result @@ -85,7 +93,7 @@ sealed trait Vector extends Serializable { /** * Converts the instance to a breeze vector. */ - private[mllib] def toBreeze: BV[Double] + private[spark] def toBreeze: BV[Double] /** * Gets the value of the ith element. @@ -108,6 +116,40 @@ sealed trait Vector extends Serializable { * with type `Double`. */ private[spark] def foreachActive(f: (Int, Double) => 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. + */ + def numActives: Int + + /** + * Number of nonzero elements. This scans all active values and count nonzeros. + */ + def numNonzeros: Int + + /** + * Converts this vector to a sparse vector with all explicit zeros removed. + */ + def toSparse: SparseVector + + /** + * Converts this vector to a dense vector. + */ + def toDense: DenseVector = new DenseVector(this.toArray) + + /** + * Returns a vector in either dense or sparse format, whichever uses less storage. + */ + def compressed: Vector = { + val nnz = numNonzeros + // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. + if (1.5 * (nnz + 1.0) < size) { + toSparse + } else { + toDense + } + } } /** @@ -185,6 +227,8 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def hashCode: Int = 7919 + override def typeName: String = "vector" + private[spark] override def asNullable: VectorUDT = this } @@ -225,7 +269,7 @@ object Vectors { * @param elements vector elements in (index, value) pairs. */ def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { - require(size > 0) + require(size > 0, "The size of the requested sparse vector must be greater than 0.") val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 @@ -233,7 +277,8 @@ object Vectors { require(prev < i, s"Found duplicate indices: $i.") prev = i } - require(prev < size) + require(prev < size, s"You may not write an element to index $prev because the declared " + + s"size of your vector is $size") new SparseVector(size, indices.toArray, values.toArray) } @@ -281,7 +326,7 @@ object Vectors { /** * Creates a vector instance from a breeze vector. */ - private[mllib] def fromBreeze(breezeVector: BV[Double]): Vector = { + private[spark] def fromBreeze(breezeVector: BV[Double]): Vector = { breezeVector match { case v: BDV[Double] => if (v.offset == 0 && v.stride == 1 && v.length == v.data.length) { @@ -307,13 +352,14 @@ object Vectors { * @return norm in L^p^ space. */ def norm(vector: Vector, p: Double): Double = { - require(p >= 1.0) + 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.") val values = vector match { case DenseVector(vs) => vs case SparseVector(n, ids, vs) => vs case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } - val size = values.size + val size = values.length if (p == 1) { var sum = 0.0 @@ -358,7 +404,8 @@ object Vectors { * @return squared distance between two Vectors. */ def sqdist(v1: Vector, v2: Vector): Double = { - require(v1.size == v2.size, "vector dimension mismatch") + require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + + s"=${v2.size}.") var squaredDistance = 0.0 (v1, v2) match { case (v1: SparseVector, v2: SparseVector) => @@ -366,8 +413,8 @@ object Vectors { val v1Indices = v1.indices val v2Values = v2.values val v2Indices = v2.indices - val nnzv1 = v1Indices.size - val nnzv2 = v2Indices.size + val nnzv1 = v1Indices.length + val nnzv2 = v2Indices.length var kv1 = 0 var kv2 = 0 @@ -396,7 +443,7 @@ object Vectors { case (DenseVector(vv1), DenseVector(vv2)) => var kv = 0 - val sz = vv1.size + val sz = vv1.length while (kv < sz) { val score = vv1(kv) - vv2(kv) squaredDistance += score * score @@ -417,7 +464,7 @@ object Vectors { var kv2 = 0 val indices = v1.indices var squaredDistance = 0.0 - val nnzv1 = indices.size + val nnzv1 = indices.length val nnzv2 = v2.size var iv1 = if (nnzv1 > 0) indices(kv1) else -1 @@ -446,8 +493,8 @@ object Vectors { v1Values: Array[Double], v2Indices: IndexedSeq[Int], v2Values: Array[Double]): Boolean = { - val v1Size = v1Values.size - val v2Size = v2Values.size + val v1Size = v1Values.length + val v2Size = v2Values.length var k1 = 0 var k2 = 0 var allEqual = true @@ -478,7 +525,7 @@ class DenseVector(val values: Array[Double]) extends Vector { override def toArray: Array[Double] = values - private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) + private[spark] override def toBreeze: BV[Double] = new BDV[Double](values) override def apply(i: Int): Double = values(i) @@ -488,7 +535,7 @@ class DenseVector(val values: Array[Double]) extends Vector { private[spark] override def foreachActive(f: (Int, Double) => Unit) = { var i = 0 - val localValuesSize = values.size + val localValuesSize = values.length val localValues = values while (i < localValuesSize) { @@ -496,6 +543,50 @@ class DenseVector(val values: Array[Double]) extends Vector { i += 1 } } + + override def hashCode(): Int = { + var result: Int = 31 + size + var i = 0 + val end = math.min(values.length, 16) + while (i < end) { + val v = values(i) + if (v != 0.0) { + result = 31 * result + i + val bits = java.lang.Double.doubleToLongBits(values(i)) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + } + i += 1 + } + result + } + + override def numActives: Int = size + + override def numNonzeros: Int = { + // same as values.count(_ != 0.0) but faster + var nnz = 0 + values.foreach { v => + if (v != 0.0) { + nnz += 1 + } + } + nnz + } + + override def toSparse: SparseVector = { + val nnz = numNonzeros + val ii = new Array[Int](nnz) + val vv = new Array[Double](nnz) + var k = 0 + foreachActive { (i, v) => + if (v != 0) { + ii(k) = i + vv(k) = v + k += 1 + } + } + new SparseVector(size, ii, vv) + } } object DenseVector { @@ -516,10 +607,12 @@ class SparseVector( val indices: Array[Int], val values: Array[Double]) extends Vector { - require(indices.length == values.length) + 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 " + + s" ${values.length} values.") override def toString: String = - "(%s,%s,%s)".format(size, indices.mkString("[", ",", "]"), values.mkString("[", ",", "]")) + s"($size,${indices.mkString("[", ",", "]")},${values.mkString("[", ",", "]")})" override def toArray: Array[Double] = { val data = new Array[Double](size) @@ -536,11 +629,11 @@ class SparseVector( new SparseVector(size, indices.clone(), values.clone()) } - private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + private[spark] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) private[spark] override def foreachActive(f: (Int, Double) => Unit) = { var i = 0 - val localValuesSize = values.size + val localValuesSize = values.length val localIndices = indices val localValues = values @@ -549,6 +642,59 @@ class SparseVector( i += 1 } } + + override def hashCode(): Int = { + var result: Int = 31 + size + val end = values.length + var continue = true + var k = 0 + while ((k < end) & continue) { + val i = indices(k) + if (i < 16) { + val v = values(k) + if (v != 0.0) { + result = 31 * result + i + val bits = java.lang.Double.doubleToLongBits(v) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + } + } else { + continue = false + } + k += 1 + } + result + } + + override def numActives: Int = values.length + + override def numNonzeros: Int = { + var nnz = 0 + values.foreach { v => + if (v != 0.0) { + nnz += 1 + } + } + nnz + } + + override def toSparse: SparseVector = { + val nnz = numNonzeros + if (nnz == numActives) { + this + } else { + val ii = new Array[Int](nnz) + val vv = new Array[Double](nnz) + var k = 0 + foreachActive { (i, v) => + if (v != 0.0) { + ii(k) = i + vv(k) = v + k += 1 + } + } + new SparseVector(size, ii, vv) + } + } } object SparseVector { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 961111507f2c2..9a89a6f3a515f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -531,7 +531,6 @@ class RowMatrix( val rand = new XORShiftRandom(indx) val scaled = new Array[Double](p.size) iter.flatMap { row => - val buf = new ListBuffer[((Int, Int), Double)]() row match { case SparseVector(size, indices, values) => val nnz = indices.size @@ -540,8 +539,9 @@ class RowMatrix( scaled(k) = values(k) / q(indices(k)) k += 1 } - k = 0 - while (k < nnz) { + + Iterator.tabulate (nnz) { k => + val buf = new ListBuffer[((Int, Int), Double)]() val i = indices(k) val iVal = scaled(k) if (iVal != 0 && rand.nextDouble() < p(i)) { @@ -555,8 +555,8 @@ class RowMatrix( l += 1 } } - k += 1 - } + buf + }.flatten case DenseVector(values) => val n = values.size var i = 0 @@ -564,8 +564,8 @@ class RowMatrix( scaled(i) = values(i) / q(i) i += 1 } - i = 0 - while (i < n) { + Iterator.tabulate (n) { i => + val buf = new ListBuffer[((Int, Int), Double)]() val iVal = scaled(i) if (iVal != 0 && rand.nextDouble() < p(i)) { var j = i + 1 @@ -577,10 +577,9 @@ class RowMatrix( j += 1 } } - i += 1 - } + buf + }.flatten } - buf } }.reduceByKey(_ + _).map { case ((i, j), sim) => MatrixEntry(i.toLong, j.toLong, sim) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 8bfa0d2b64995..240baeb5a158b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -37,7 +37,11 @@ abstract class Gradient extends Serializable { * * @return (gradient: Vector, loss: Double) */ - def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) + def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val gradient = Vectors.zeros(weights.size) + val loss = compute(data, label, weights, gradient) + (gradient, loss) + } /** * Compute the gradient and loss given the features of a single data point, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index ef6eccd90711a..efedc112d380e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.optimization +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV} @@ -164,7 +165,7 @@ object LBFGS extends Logging { regParam: Double, initialWeights: Vector): (Vector, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](maxNumIterations) + val lossHistory = mutable.ArrayBuilder.make[Double] val numExamples = data.count() @@ -181,17 +182,19 @@ object LBFGS extends Logging { * and regVal is the regularization value computed in the previous iteration as well. */ var state = states.next() - while(states.hasNext) { - lossHistory.append(state.value) + while (states.hasNext) { + lossHistory += state.value state = states.next() } - lossHistory.append(state.value) + lossHistory += state.value val weights = Vectors.fromBreeze(state.x) + val lossHistoryArray = lossHistory.result() + logInfo("LBFGS.runLBFGS finished. Last 10 losses %s".format( - lossHistory.takeRight(10).mkString(", "))) + lossHistoryArray.takeRight(10).mkString(", "))) - (weights, lossHistory.toArray) + (weights, lossHistoryArray) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala new file mode 100644 index 0000000000000..354e90f3eeaa6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml + +import java.io.{File, OutputStream, StringWriter} +import javax.xml.transform.stream.StreamResult + +import org.jpmml.model.JAXBUtil + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory + +/** + * Export model to the PMML format + * Predictive Model Markup Language (PMML) is an XML-based file format + * developed by the Data Mining Group (www.dmg.org). + */ +trait PMMLExportable { + + /** + * Export the model to the stream result in PMML format + */ + private def toPMML(streamResult: StreamResult): Unit = { + val pmmlModelExport = PMMLModelExportFactory.createPMMLModelExport(this) + JAXBUtil.marshalPMML(pmmlModelExport.getPmml, streamResult) + } + + /** + * Export the model to a local file in PMML format + */ + def toPMML(localPath: String): Unit = { + toPMML(new StreamResult(new File(localPath))) + } + + /** + * Export the model to a directory on a distributed file system in PMML format + */ + def toPMML(sc: SparkContext, path: String): Unit = { + val pmml = toPMML() + sc.parallelize(Array(pmml), 1).saveAsTextFile(path) + } + + /** + * Export the model to the OutputStream in PMML format + */ + def toPMML(outputStream: OutputStream): Unit = { + toPMML(new StreamResult(outputStream)) + } + + /** + * Export the model to a String in PMML format + */ + def toPMML(): String = { + val writer = new StringWriter + toPMML(new StreamResult(writer)) + writer.toString + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala new file mode 100644 index 0000000000000..34b447584e521 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import scala.{Array => SArray} + +import org.dmg.pmml._ + +import org.apache.spark.mllib.regression.GeneralizedLinearModel + +/** + * PMML Model Export for GeneralizedLinearModel class with binary ClassificationModel + */ +private[mllib] class BinaryClassificationPMMLModelExport( + model : GeneralizedLinearModel, + description : String, + normalizationMethod : RegressionNormalizationMethodType, + threshold: Double) + extends PMMLModelExport { + + populateBinaryClassificationPMML() + + /** + * Export the input LogisticRegressionModel or SVMModel to PMML format. + */ + private def populateBinaryClassificationPMML(): Unit = { + pmml.getHeader.setDescription(description) + + if (model.weights.size > 0) { + val fields = new SArray[FieldName](model.weights.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") + var interceptNO = threshold + if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { + if (threshold <= 0) { + interceptNO = Double.MinValue + } else if (threshold >= 1) { + interceptNO = Double.MaxValue + } else { + interceptNO = -math.log(1 / threshold - 1) + } + } + val regressionTableNO = new RegressionTable(interceptNO).withTargetCategory("0") + val regressionModel = new RegressionModel() + .withFunctionName(MiningFunctionType.CLASSIFICATION) + .withMiningSchema(miningSchema) + .withModelName(description) + .withNormalizationMethod(normalizationMethod) + .withRegressionTables(regressionTableYES, regressionTableNO) + + for (i <- 0 until model.weights.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTableYES.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + } + + // add target field + val targetField = FieldName.create("target") + dataDictionary + .withDataFields(new DataField(targetField, OpType.CATEGORICAL, DataType.STRING)) + miningSchema + .withMiningFields(new MiningField(targetField) + .withUsageType(FieldUsageType.TARGET)) + + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(regressionModel) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala new file mode 100644 index 0000000000000..1874786af0002 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import scala.{Array => SArray} + +import org.dmg.pmml._ + +import org.apache.spark.mllib.regression.GeneralizedLinearModel + +/** + * PMML Model Export for GeneralizedLinearModel abstract class + */ +private[mllib] class GeneralizedLinearPMMLModelExport( + model: GeneralizedLinearModel, + description: String) + extends PMMLModelExport { + + populateGeneralizedLinearPMML(model) + + /** + * Export the input GeneralizedLinearModel model to PMML format. + */ + private def populateGeneralizedLinearPMML(model: GeneralizedLinearModel): Unit = { + pmml.getHeader.setDescription(description) + + if (model.weights.size > 0) { + val fields = new SArray[FieldName](model.weights.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val regressionTable = new RegressionTable(model.intercept) + val regressionModel = new RegressionModel() + .withFunctionName(MiningFunctionType.REGRESSION) + .withMiningSchema(miningSchema) + .withModelName(description) + .withRegressionTables(regressionTable) + + for (i <- 0 until model.weights.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + } + + // for completeness add target field + val targetField = FieldName.create("target") + dataDictionary.withDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(targetField) + .withUsageType(FieldUsageType.TARGET)) + + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(regressionModel) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala new file mode 100644 index 0000000000000..069e7afc9fca0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -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.mllib.pmml.export + +import scala.{Array => SArray} + +import org.dmg.pmml._ + +import org.apache.spark.mllib.clustering.KMeansModel + +/** + * PMML Model Export for KMeansModel class + */ +private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ + + populateKMeansPMML(model) + + /** + * Export the input KMeansModel model to PMML format. + */ + private def populateKMeansPMML(model : KMeansModel): Unit = { + pmml.getHeader.setDescription("k-means clustering") + + if (model.clusterCenters.length > 0) { + val clusterCenter = model.clusterCenters(0) + val fields = new SArray[FieldName](clusterCenter.size) + val dataDictionary = new DataDictionary + val miningSchema = new MiningSchema + val comparisonMeasure = new ComparisonMeasure() + .withKind(ComparisonMeasure.Kind.DISTANCE) + .withMeasure(new SquaredEuclidean()) + val clusteringModel = new ClusteringModel() + .withModelName("k-means") + .withMiningSchema(miningSchema) + .withComparisonMeasure(comparisonMeasure) + .withFunctionName(MiningFunctionType.CLUSTERING) + .withModelClass(ClusteringModel.ModelClass.CENTER_BASED) + .withNumberOfClusters(model.clusterCenters.length) + + for (i <- 0 until clusterCenter.size) { + fields(i) = FieldName.create("field_" + i) + dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + miningSchema + .withMiningFields(new MiningField(fields(i)) + .withUsageType(FieldUsageType.ACTIVE)) + clusteringModel.withClusteringFields( + new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF)) + } + + dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + + for (i <- 0 until model.clusterCenters.length) { + val cluster = new Cluster() + .withName("cluster_" + i) + .withArray(new org.dmg.pmml.Array() + .withType(Array.Type.REAL) + .withN(clusterCenter.size) + .withValue(model.clusterCenters(i).toArray.mkString(" "))) + // we don't have the size of the single cluster but only the centroids (withValue) + // .withSize(value) + clusteringModel.withClusters(cluster) + } + + pmml.setDataDictionary(dataDictionary) + pmml.withModels(clusteringModel) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala new file mode 100644 index 0000000000000..ebdeae50bb32f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import java.text.SimpleDateFormat +import java.util.Date + +import scala.beans.BeanProperty + +import org.dmg.pmml.{Application, Header, PMML, Timestamp} + +private[mllib] trait PMMLModelExport { + + /** + * Holder of the exported model in PMML format + */ + @BeanProperty + val pmml: PMML = new PMML + + setHeader(pmml) + + private def setHeader(pmml: PMML): Unit = { + val version = getClass.getPackage.getImplementationVersion + val app = new Application().withName("Apache Spark MLlib").withVersion(version) + val timestamp = new Timestamp() + .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + val header = new Header() + .withApplication(app) + .withTimestamp(timestamp) + pmml.setHeader(header) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala new file mode 100644 index 0000000000000..c16e83d6a067d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -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.mllib.pmml.export + +import org.dmg.pmml.RegressionNormalizationMethodType + +import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.classification.SVMModel +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.regression.LassoModel +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.regression.RidgeRegressionModel + +private[mllib] object PMMLModelExportFactory { + + /** + * Factory object to help creating the necessary PMMLModelExport implementation + * taking as input the machine learning model (for example KMeansModel). + */ + def createPMMLModelExport(model: Any): PMMLModelExport = { + model match { + case kmeans: KMeansModel => + new KMeansPMMLModelExport(kmeans) + case linear: LinearRegressionModel => + new GeneralizedLinearPMMLModelExport(linear, "linear regression") + case ridge: RidgeRegressionModel => + new GeneralizedLinearPMMLModelExport(ridge, "ridge regression") + case lasso: LassoModel => + new GeneralizedLinearPMMLModelExport(lasso, "lasso regression") + case svm: SVMModel => + new BinaryClassificationPMMLModelExport( + svm, "linear SVM", RegressionNormalizationMethodType.NONE, + svm.getThreshold.getOrElse(0.0)) + case logistic: LogisticRegressionModel => + if (logistic.numClasses == 2) { + new BinaryClassificationPMMLModelExport( + logistic, "logistic regression", RegressionNormalizationMethodType.LOGIT, + logistic.getThreshold.getOrElse(0.5)) + } else { + throw new IllegalArgumentException( + "PMML Export not supported for Multinomial Logistic Regression") + } + case _ => + throw new IllegalArgumentException( + "PMML Export not supported for model: " + model.getClass.getName) + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index 9213fd3f595c3..5af55aaf84802 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -42,13 +42,11 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se self.aggregateByKey(new BoundedPriorityQueue[V](num)(ord))( seqOp = (queue, item) => { queue += item - queue }, combOp = (queue1, queue2) => { queue1 ++= queue2 - queue1 } - ).mapValues(_.toArray.sorted(ord.reverse)) + ).mapValues(_.toArray.reverse) // This is an min-heap, so we reverse the order. } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 36cbf060d9998..88c2148403313 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -20,14 +20,18 @@ package org.apache.spark.mllib.recommendation import java.io.IOException import java.lang.{Integer => JavaInteger} +import scala.collection.mutable + +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} @@ -57,7 +61,7 @@ class MatrixFactorizationModel( /** Validates factors and warns users if there are performance concerns. */ private def validateFeatures(name: String, features: RDD[(Int, Array[Double])]): Unit = { - require(features.first()._2.size == rank, + require(features.first()._2.length == rank, s"$name feature dimension does not match the rank $rank.") if (features.partitioner.isEmpty) { logWarning(s"$name factor does not have a partitioner. " @@ -72,19 +76,19 @@ class MatrixFactorizationModel( def predict(user: Int, product: Int): Double = { val userVector = userFeatures.lookup(user).head val productVector = productFeatures.lookup(product).head - blas.ddot(userVector.length, userVector, 1, productVector, 1) + blas.ddot(rank, userVector, 1, productVector, 1) } /** - * Predict the rating of many users for many products. - * The output RDD has an element per each element in the input RDD (including all duplicates) - * unless a user or product is missing in the training set. - * - * @param usersProducts RDD of (user, product) pairs. - * @return RDD of Ratings. - */ + * Predict the rating of many users for many products. + * The output RDD has an element per each element in the input RDD (including all duplicates) + * unless a user or product is missing in the training set. + * + * @param usersProducts RDD of (user, product) pairs. + * @return RDD of Ratings. + */ def predict(usersProducts: RDD[(Int, Int)]): RDD[Rating] = { - val users = userFeatures.join(usersProducts).map{ + val users = userFeatures.join(usersProducts).map { case (user, (uFeatures, product)) => (product, (user, uFeatures)) } users.join(productFeatures).map { @@ -112,7 +116,7 @@ class MatrixFactorizationModel( * recommended the product is. */ def recommendProducts(user: Int, num: Int): Array[Rating] = - recommend(userFeatures.lookup(user).head, productFeatures, num) + MatrixFactorizationModel.recommend(userFeatures.lookup(user).head, productFeatures, num) .map(t => Rating(user, t._1, t._2)) /** @@ -128,7 +132,7 @@ class MatrixFactorizationModel( * recommended the user is. */ def recommendUsers(product: Int, num: Int): Array[Rating] = - recommend(productFeatures.lookup(product).head, userFeatures, num) + MatrixFactorizationModel.recommend(productFeatures.lookup(product).head, userFeatures, num) .map(t => Rating(t._1, product, t._2)) protected override val formatVersion: String = "1.0" @@ -137,20 +141,113 @@ class MatrixFactorizationModel( MatrixFactorizationModel.SaveLoadV1_0.save(this, path) } + /** + * Recommends topK products for all users. + * + * @param num how many products to return for every user. + * @return [(Int, Array[Rating])] objects, where every tuple contains a userID and an array of + * rating objects which contains the same userId, recommended productID and a "score" in the + * rating field. Semantics of score is same as recommendProducts API + */ + def recommendProductsForUsers(num: Int): RDD[(Int, Array[Rating])] = { + MatrixFactorizationModel.recommendForAll(rank, userFeatures, productFeatures, num).map { + case (user, top) => + val ratings = top.map { case (product, rating) => Rating(user, product, rating) } + (user, ratings) + } + } + + + /** + * Recommends topK users for all products. + * + * @param num how many users to return for every product. + * @return [(Int, Array[Rating])] objects, where every tuple contains a productID and an array + * of rating objects which contains the recommended userId, same productID and a "score" in the + * rating field. Semantics of score is same as recommendUsers API + */ + def recommendUsersForProducts(num: Int): RDD[(Int, Array[Rating])] = { + MatrixFactorizationModel.recommendForAll(rank, productFeatures, userFeatures, num).map { + case (product, top) => + val ratings = top.map { case (user, rating) => Rating(user, product, rating) } + (product, ratings) + } + } +} + +object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { + + import org.apache.spark.mllib.util.Loader._ + + /** + * Makes recommendations for a single user (or product). + */ private def recommend( recommendToFeatures: Array[Double], recommendableFeatures: RDD[(Int, Array[Double])], num: Int): Array[(Int, Double)] = { - val scored = recommendableFeatures.map { case (id,features) => + val scored = recommendableFeatures.map { case (id, features) => (id, blas.ddot(features.length, recommendToFeatures, 1, features, 1)) } scored.top(num)(Ordering.by(_._2)) } -} -object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { + /** + * Makes recommendations for all users (or products). + * @param rank rank + * @param srcFeatures src features to receive recommendations + * @param dstFeatures dst features used to make recommendations + * @param num number of recommendations for each record + * @return an RDD of (srcId: Int, recommendations), where recommendations are stored as an array + * of (dstId, rating) pairs. + */ + private def recommendForAll( + rank: Int, + srcFeatures: RDD[(Int, Array[Double])], + dstFeatures: RDD[(Int, Array[Double])], + num: Int): RDD[(Int, Array[(Int, Double)])] = { + val srcBlocks = blockify(rank, srcFeatures) + val dstBlocks = blockify(rank, dstFeatures) + val ratings = srcBlocks.cartesian(dstBlocks).flatMap { + case ((srcIds, srcFactors), (dstIds, dstFactors)) => + val m = srcIds.length + val n = dstIds.length + val ratings = srcFactors.transpose.multiply(dstFactors) + val output = new Array[(Int, (Int, Double))](m * n) + var k = 0 + ratings.foreachActive { (i, j, r) => + output(k) = (srcIds(i), (dstIds(j), r)) + k += 1 + } + output.toSeq + } + ratings.topByKey(num)(Ordering.by(_._2)) + } - import org.apache.spark.mllib.util.Loader._ + /** + * Blockifies features to use Level-3 BLAS. + */ + private def blockify( + rank: Int, + features: RDD[(Int, Array[Double])]): RDD[(Array[Int], DenseMatrix)] = { + val blockSize = 4096 // TODO: tune the block size + val blockStorage = rank * blockSize + features.mapPartitions { iter => + iter.grouped(blockSize).map { grouped => + val ids = mutable.ArrayBuilder.make[Int] + ids.sizeHint(blockSize) + val factors = mutable.ArrayBuilder.make[Double] + factors.sizeHint(blockStorage) + var i = 0 + grouped.foreach { case (id, factor) => + ids += id + factors ++= factor + i += 1 + } + (ids.result(), new DenseMatrix(rank, i, factors.result())) + } + } + } override def load(sc: SparkContext, path: String): MatrixFactorizationModel = { val (loadedClassName, formatVersion, _) = loadMetadata(sc, path) @@ -214,4 +311,5 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { new Path(dataPath(path), "product").toUri.toString } } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 45b9ebb4cc0d6..26be30ff9d6fd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -211,6 +211,10 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { + if (numFeatures < 0) { + numFeatures = input.map(_.features.size).first() + } + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") @@ -221,7 +225,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] throw new SparkException("Input validation failed.") } - /* + /** * Scaling columns to unit variance as a heuristic to reduce the condition number: * * During the optimization process, the convergence (rate) depends on the condition number of diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index cb70852e3cc8d..1d7617046b6c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -23,9 +23,16 @@ import java.util.Arrays.binarySearch import scala.collection.mutable.ArrayBuffer +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.sql.{DataFrame, SQLContext} /** * :: Experimental :: @@ -42,7 +49,7 @@ import org.apache.spark.rdd.RDD class IsotonicRegressionModel ( val boundaries: Array[Double], val predictions: Array[Double], - val isotonic: Boolean) extends Serializable { + val isotonic: Boolean) extends Serializable with Saveable { private val predictionOrd = if (isotonic) Ordering[Double] else Ordering[Double].reverse @@ -124,6 +131,75 @@ class IsotonicRegressionModel ( predictions(foundIndex) } } + + override def save(sc: SparkContext, path: String): Unit = { + IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) + } + + override protected def formatVersion: String = "1.0" +} + +object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { + + import org.apache.spark.mllib.util.Loader._ + + private object SaveLoadV1_0 { + + def thisFormatVersion: String = "1.0" + + /** Hard-code class name string in case it changes in the future */ + def thisClassName: String = "org.apache.spark.mllib.regression.IsotonicRegressionModel" + + /** Model data for model import/export */ + case class Data(boundary: Double, prediction: Double) + + def save( + sc: SparkContext, + path: String, + boundaries: Array[Double], + predictions: Array[Double], + isotonic: Boolean): Unit = { + val sqlContext = new SQLContext(sc) + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ + ("isotonic" -> isotonic))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) + + sqlContext.createDataFrame( + boundaries.toSeq.zip(predictions).map { case (b, p) => Data(b, p) } + ).saveAsParquetFile(dataPath(path)) + } + + def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { + val sqlContext = new SQLContext(sc) + val dataRDD = sqlContext.parquetFile(dataPath(path)) + + checkSchema[Data](dataRDD.schema) + val dataArray = dataRDD.select("boundary", "prediction").collect() + val (boundaries, predictions) = dataArray.map { x => + (x.getDouble(0), x.getDouble(1)) + }.toList.sortBy(_._1).unzip + (boundaries.toArray, predictions.toArray) + } + } + + override def load(sc: SparkContext, path: String): IsotonicRegressionModel = { + implicit val formats = DefaultFormats + val (loadedClassName, version, metadata) = loadMetadata(sc, path) + val isotonic = (metadata \ "isotonic").extract[Boolean] + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, version) match { + case (className, "1.0") if className == classNameV1_0 => + val (boundaries, predictions) = SaveLoadV1_0.load(sc, path) + new IsotonicRegressionModel(boundaries, predictions, isotonic) + case _ => throw new Exception( + s"IsotonicRegressionModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)" + ) + } + } } /** 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 2067b36f246b3..d5fea822ad77b 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 @@ -32,7 +32,7 @@ import org.apache.spark.SparkException @BeanInfo case class LabeledPoint(label: Double, features: Vector) { override def toString: String = { - "(%s,%s)".format(label, features) + s"($label,$features)" } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index e8b03816573cf..4f482384f0f38 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD @@ -34,7 +35,7 @@ class LassoModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable with Saveable { + with RegressionModel with Serializable with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 6fa7ad52a5b33..9453c4f66c216 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD @@ -34,7 +35,7 @@ class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable - with Saveable { + with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 8838ca8c14718..e0c03d8180c7a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD @@ -35,7 +36,7 @@ class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable with Saveable { + with RegressionModel with Serializable with Saveable with PMMLExportable { override protected def predictPoint( dataMatrix: Vector, @@ -171,7 +172,7 @@ object RidgeRegressionWithSGD { numIterations: Int, stepSize: Double, regParam: Double): RidgeRegressionModel = { - train(input, numIterations, stepSize, regParam, 0.01) + train(input, numIterations, stepSize, regParam, 1.0) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index ce95c063db970..cea8f3f47307b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -60,7 +60,7 @@ abstract class StreamingLinearAlgorithm[ A <: GeneralizedLinearAlgorithm[M]] extends Logging { /** The model to be updated and used for prediction. */ - protected var model: Option[M] = None + protected var model: Option[M] /** The algorithm to use for updating. */ protected val algorithm: A @@ -114,7 +114,7 @@ abstract class StreamingLinearAlgorithm[ if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction.") } - data.map(model.get.predict) + data.map{x => model.get.predict(x)} } /** Java-friendly version of `predictOn`. */ @@ -132,7 +132,7 @@ abstract class StreamingLinearAlgorithm[ if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction") } - data.mapValues(model.get.predict) + data.mapValues{x => model.get.predict(x)} } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index e5e6301127a28..a49153bf73c0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -59,6 +59,8 @@ class StreamingLinearRegressionWithSGD private[mllib] ( val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + protected var model: Option[LinearRegressionModel] = None + /** Set the step size for gradient descent. Default: 0.1. */ def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala index 23b291eee070b..8a821d1b23bab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -101,7 +101,7 @@ private[stat] object PearsonCorrelation extends Correlation with Logging { Matrices.fromBreeze(cov) } - private def closeToZero(value: Double, threshhold: Double = 1e-12): Boolean = { - math.abs(value) <= threshhold + private def closeToZero(value: Double, threshold: Double = 1e-12): Boolean = { + math.abs(value) <= threshold } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b9d0c56dd1ea3..dfe3a0b6913ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -1147,7 +1147,10 @@ object DecisionTree extends Serializable with Logging { } } - assert(splits.length > 0) + // TODO: Do not fail; just ignore the useless feature. + assert(splits.length > 0, + s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + + " Please remove this feature and then try again.") // set number of splits accordingly metadata.setNumSplits(featureIndex, splits.length) 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 a9c93e181e3ce..1f779584dcffd 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 @@ -81,11 +81,11 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) /** * Method to validate a gradient boosting model * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @param validationInput Validation dataset: - RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - Should be different from and follow the same distribution as input. - e.g., these two datasets could be created from an original dataset - by using [[org.apache.spark.rdd.RDD.randomSplit()]] + * @param validationInput Validation dataset. + * This dataset should be different from the training dataset, + * but it should follow the same distribution. + * E.g., these two datasets could be created from an original dataset + * by using [[org.apache.spark.rdd.RDD.randomSplit()]] * @return a gradient boosted trees model that can be used for prediction */ def runWithValidation( @@ -157,7 +157,6 @@ object GradientBoostedTrees extends Logging { validationInput: RDD[LabeledPoint], boostingStrategy: BoostingStrategy, validate: Boolean): GradientBoostedTreesModel = { - val timer = new TimeTracker() timer.start("total") timer.start("init") @@ -178,8 +177,11 @@ object GradientBoostedTrees extends Logging { treeStrategy.assertValid() // Cache input - if (input.getStorageLevel == StorageLevel.NONE) { + val persistedInput = if (input.getStorageLevel == StorageLevel.NONE) { input.persist(StorageLevel.MEMORY_AND_DISK) + true + } else { + false } timer.stop("init") @@ -192,20 +194,27 @@ object GradientBoostedTrees extends Logging { // Initialize tree timer.start("building tree 0") val firstTreeModel = new DecisionTree(treeStrategy).run(data) + val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel - baseLearnerWeights(0) = 1.0 - val startingModel = new GradientBoostedTreesModel(Regression, Array(firstTreeModel), Array(1.0)) - logDebug("error of gbt = " + loss.computeError(startingModel, input)) + baseLearnerWeights(0) = firstTreeWeight + + var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + logDebug("error of gbt = " + predError.values.mean()) // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") - var bestValidateError = if (validate) loss.computeError(startingModel, validationInput) else 0.0 + var validatePredError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) + var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 var bestM = 1 - // psuedo-residual for second iteration - data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), - point.features)) + // pseudo-residual for second iteration + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } + var m = 1 while (m < numIterations) { timer.start(s"building tree $m") @@ -222,15 +231,22 @@ object GradientBoostedTrees extends Logging { baseLearnerWeights(m) = learningRate // Note: A model of type regression is used since we require raw prediction val partialModel = new GradientBoostedTreesModel( - Regression, baseLearners.slice(0, m + 1), baseLearnerWeights.slice(0, m + 1)) - logDebug("error of gbt = " + loss.computeError(partialModel, input)) + Regression, baseLearners.slice(0, m + 1), + baseLearnerWeights.slice(0, m + 1)) + + predError = GradientBoostedTreesModel.updatePredictionError( + input, predError, baseLearnerWeights(m), baseLearners(m), loss) + logDebug("error of gbt = " + predError.values.mean()) if (validate) { // Stop training early if // 1. Reduction in error is less than the validationTol or // 2. If the error increases, that is if the model is overfit. // We want the model returned corresponding to the best validation error. - val currentValidateError = loss.computeError(partialModel, validationInput) + + validatePredError = GradientBoostedTreesModel.updatePredictionError( + validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) + val currentValidateError = validatePredError.values.mean() if (bestValidateError - currentValidateError < validationTol) { return new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, @@ -242,8 +258,9 @@ object GradientBoostedTrees extends Logging { } } // Update data with pseudo-residuals - data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), - point.features)) + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } m += 1 } @@ -251,6 +268,9 @@ object GradientBoostedTrees extends Logging { logInfo("Internal timing for DecisionTree:") logInfo(s"$timer") + + if (persistedInput) input.unpersist() + if (validate) { new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, 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 db01f2e229e5a..055e60c7d9c95 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 @@ -249,7 +249,7 @@ private class RandomForest ( nodeIdCache.get.deleteAllCheckpoints() } catch { case e:IOException => - logWarning(s"delete all chackpoints failed. Error reason: ${e.getMessage}") + logWarning(s"delete all checkpoints failed. Error reason: ${e.getMessage}") } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 664c8df019233..2d6b01524ff3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -89,14 +89,14 @@ object BoostingStrategy { * @return Configuration for boosting algorithm */ def defaultParams(algo: Algo): BoostingStrategy = { - val treeStragtegy = Strategy.defaultStategy(algo) - treeStragtegy.maxDepth = 3 + val treeStrategy = Strategy.defaultStategy(algo) + treeStrategy.maxDepth = 3 algo match { case Algo.Classification => - treeStragtegy.numClasses = 2 - new BoostingStrategy(treeStragtegy, LogLoss) + treeStrategy.numClasses = 2 + new BoostingStrategy(treeStrategy, LogLoss) case Algo.Regression => - new BoostingStrategy(treeStragtegy, SquaredError) + new BoostingStrategy(treeStrategy, SquaredError) case _ => throw new IllegalArgumentException(s"$algo is not supported by boosting.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index f1a6ed230186e..f73896e37c05e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -107,7 +107,10 @@ private[tree] object DecisionTreeMetadata extends Logging { numTrees: Int, featureSubsetStrategy: String): DecisionTreeMetadata = { - val numFeatures = input.take(1)(0).features.size + val numFeatures = input.map(_.features.size).take(1).headOption.getOrElse { + throw new IllegalArgumentException(s"DecisionTree requires size of input RDD > 0, " + + s"but was given by empty one.") + } val numExamples = input.count() val numClasses = strategy.algo match { case Classification => strategy.numClasses diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index 793dd664c5d5a..2bdef73c4a8f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -37,19 +37,16 @@ object AbsoluteError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. * The gradient with respect to F(x) is: sign(F(x) - y) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 + override def gradient(prediction: Double, label: Double): Double = { + if (label - prediction < 0) 1.0 else -1.0 } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = label - prediction math.abs(err) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 51b1aed167b66..778c24526de70 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -39,21 +39,17 @@ object LogLoss extends Loss { * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification * The gradient with respect to F(x) is: - 4 y / (1 + exp(2 y F(x))) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - val prediction = model.predict(point.features) - - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) + override def gradient(prediction: Double, label: Double): Double = { + - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val margin = 2.0 * label * prediction // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. 2.0 * MLUtils.log1pExp(-margin) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 357869ff6b333..64ffccbce073f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -22,6 +22,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. @@ -31,13 +32,11 @@ trait Loss extends Serializable { /** * Method to calculate the gradients for the gradient boosting calculation. - * @param model Model of the weak learner. - * @param point Instance of the training dataset. + * @param prediction Predicted feature + * @param label true label. * @return Loss gradient. */ - def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double + def gradient(prediction: Double, label: Double): Double /** * Method to calculate error of the base learner for the gradient boosting calculation. @@ -59,6 +58,5 @@ trait Loss extends Serializable { * @param label True label. * @return Measure of model error on datapoint. */ - def computeError(prediction: Double, label: Double): Double - + private[mllib] def computeError(prediction: Double, label: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index b990707ca4525..a5582d3ef3324 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel -import org.apache.spark.rdd.RDD + /** * :: DeveloperApi :: @@ -37,19 +37,16 @@ object SquaredError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. * The gradient with respect to F(x) is: - 2 (y - F(x)) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - 2.0 * (model.predict(point.features) - point.label) + override def gradient(prediction: Double, label: Double): Double = { + 2.0 * (prediction - label) } - override def computeError(prediction: Double, label: Double): Double = { + override private[mllib] def computeError(prediction: Double, label: Double): Double = { val err = prediction - label err * err } - } 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 c9bafd60fba4d..331af428533de 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 @@ -113,11 +113,13 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = DecisionTreeModel.formatVersion } object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { + private[spark] def formatVersion: String = "1.0" + private[tree] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index f209fdafd3653..2d087c967f679 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -39,8 +39,8 @@ class InformationGainStats( val rightPredict: Predict) extends Serializable { override def toString: String = { - "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" - .format(gain, impurity, leftImpurity, rightImpurity) + s"gain = $gain, impurity = $impurity, left impurity = $leftImpurity, " + + s"right impurity = $rightImpurity" } override def equals(o: Any): Boolean = o match { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 4f72bb8014cc0..431a839817eac 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -51,8 +51,8 @@ class Node ( var stats: Option[InformationGainStats]) extends Serializable with Logging { override def toString: String = { - "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + - "impurity = " + impurity + "split = " + split + ", stats = " + stats + s"id = $id, isLeaf = $isLeaf, predict = $predict, impurity = $impurity, " + + s"split = $split, stats = $stats" } /** @@ -175,7 +175,7 @@ class Node ( } } -private[tree] object Node { +private[spark] object Node { /** * Return a node with the given node id (but nothing else set). diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 25990af7c6cf7..5cbe7c280dbee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -29,9 +29,7 @@ class Predict( val predict: Double, val prob: Double = 0.0) extends Serializable { - override def toString: String = { - "predict = %f, prob = %f".format(predict, prob) - } + override def toString: String = s"$predict (prob = $prob)" override def equals(other: Any): Boolean = { other match { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index fb35e70a8d077..be6c9b3de5479 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -39,8 +39,8 @@ case class Split( categories: List[Double]) { override def toString: String = { - "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + - ", categories = " + categories + s"Feature = $feature, threshold = $threshold, featureType = $featureType, " + + s"categories = $categories" } } @@ -68,4 +68,3 @@ private[tree] class DummyHighSplit(feature: Int, featureType: FeatureType) */ private[tree] class DummyCategoricalSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 1950254b2aa6d..8341219bfa71c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils + /** * :: Experimental :: * Represents a random forest model. @@ -47,7 +48,7 @@ import org.apache.spark.util.Utils */ @Experimental class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) - extends TreeEnsembleModel(algo, trees, Array.fill(trees.size)(1.0), + extends TreeEnsembleModel(algo, trees, Array.fill(trees.length)(1.0), combiningStrategy = if (algo == Classification) Vote else Average) with Saveable { @@ -58,11 +59,13 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis RandomForestModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override protected def formatVersion: String = RandomForestModel.formatVersion } object RandomForestModel extends Loader[RandomForestModel] { + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): RandomForestModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -102,15 +105,13 @@ class GradientBoostedTreesModel( extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) with Saveable { - require(trees.size == treeWeights.size) + require(trees.length == treeWeights.length) override def save(sc: SparkContext, path: String): Unit = { TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, GradientBoostedTreesModel.SaveLoadV1_0.thisClassName) } - override protected def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion - /** * Method to compute error or loss for every iteration of gradient boosting. * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] @@ -130,42 +131,88 @@ class GradientBoostedTreesModel( val numIterations = trees.length val evaluationArray = Array.fill(numIterations)(0.0) + val localTreeWeights = treeWeights + + var predictionAndError = GradientBoostedTreesModel.computeInitialPredictionAndError( + remappedData, localTreeWeights(0), trees(0), loss) - var predictionAndError: RDD[(Double, Double)] = remappedData.map { i => - val pred = treeWeights(0) * trees(0).predict(i.features) - val error = loss.computeError(pred, i.label) - (pred, error) - } evaluationArray(0) = predictionAndError.values.mean() - // Avoid the model being copied across numIterations. val broadcastTrees = sc.broadcast(trees) - val broadcastWeights = sc.broadcast(treeWeights) - - (1 until numIterations).map { nTree => + (1 until numIterations).foreach { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) - val currentTreeWeight = broadcastWeights.value(nTree) - iter.map { - case (point, (pred, error)) => { - val newPred = pred + currentTree.predict(point.features) * currentTreeWeight - val newError = loss.computeError(newPred, point.label) - (newPred, newError) - } + val currentTreeWeight = localTreeWeights(nTree) + iter.map { case (point, (pred, error)) => + val newPred = pred + currentTree.predict(point.features) * currentTreeWeight + val newError = loss.computeError(newPred, point.label) + (newPred, newError) } } evaluationArray(nTree) = predictionAndError.values.mean() } broadcastTrees.unpersist() - broadcastWeights.unpersist() evaluationArray } + override protected def formatVersion: String = GradientBoostedTreesModel.formatVersion } object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { + /** + * Compute the initial predictions and errors for a dataset for the first + * iteration of gradient boosting. + * @param data: training data. + * @param initTreeWeight: learning rate assigned to the first tree. + * @param initTree: first DecisionTreeModel. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to every sample. + */ + def computeInitialPredictionAndError( + data: RDD[LabeledPoint], + initTreeWeight: Double, + initTree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + data.map { lp => + val pred = initTreeWeight * initTree.predict(lp.features) + val error = loss.computeError(pred, lp.label) + (pred, error) + } + } + + /** + * Update a zipped predictionError RDD + * (as obtained with computeInitialPredictionAndError) + * @param data: training data. + * @param predictionAndError: predictionError RDD + * @param treeWeight: Learning rate. + * @param tree: Tree using which the prediction and error should be updated. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to each sample. + */ + def updatePredictionError( + data: RDD[LabeledPoint], + predictionAndError: RDD[(Double, Double)], + treeWeight: Double, + tree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + + val newPredError = data.zip(predictionAndError).mapPartitions { iter => + iter.map { case (lp, (pred, error)) => + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) + } + } + newPredError + } + + private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -295,12 +342,12 @@ private[tree] sealed class TreeEnsembleModel( } /** - * Get number of trees in forest. + * Get number of trees in ensemble. */ - def numTrees: Int = trees.size + def numTrees: Int = trees.length /** - * Get total number of nodes, summed over all trees in the forest. + * Get total number of nodes, summed over all trees in the ensemble. */ def totalNumNodes: Int = trees.map(_.numNodes).sum } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index c9d33787b0bb5..b1a4517344970 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -56,6 +56,10 @@ object LinearDataGenerator { } /** + * For compatibility, the generated data without specifying the mean and variance + * will have zero mean and variance of (1.0/3.0) since the original output range is + * [-1, 1] with uniform distribution, and the variance of uniform distribution + * is (b - a)^2^ / 12 which will be (1.0/3.0) * * @param intercept Data intercept * @param weights Weights to be applied. @@ -70,10 +74,45 @@ object LinearDataGenerator { nPoints: Int, seed: Int, eps: Double = 0.1): Seq[LabeledPoint] = { + generateLinearInput(intercept, weights, + Array.fill[Double](weights.length)(0.0), + Array.fill[Double](weights.length)(1.0 / 3.0), + nPoints, seed, eps)} + + /** + * + * @param intercept Data intercept + * @param weights Weights to be applied. + * @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 nPoints Number of points in sample. + * @param seed Random seed + * @param eps Epsilon scaling factor. + * @return Seq of input. + */ + def generateLinearInput( + intercept: Double, + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + nPoints: Int, + seed: Int, + eps: Double): Seq[LabeledPoint] = { val rnd = new Random(seed) val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) + Array.fill[Double](weights.length)(rnd.nextDouble())) + + x.foreach { v => + var i = 0 + while (i < v.length) { + v(i) = (v(i) - 0.5) * math.sqrt(12.0 * xVariance(i)) + xMean(i) + i += 1 + } + } + val y = x.map { xi => blas.ddot(weights.length, xi, 1, weights, 1) + intercept + eps * rnd.nextGaussian() } 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 new file mode 100644 index 0000000000000..60f25e5cce437 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -0,0 +1,96 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +public class JavaDecisionTreeClassifierSuite implements Serializable { + + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeClassifierSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeClassifier dt = new DecisionTreeClassifier() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String impurity: DecisionTreeClassifier.supportedImpurities()) { + dt.setImpurity(impurity); + } + DecisionTreeClassificationModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model3.save(sc.sc(), path); + DecisionTreeClassificationModel sameModel = + DecisionTreeClassificationModel.load(sc.sc(), path); + TreeTests.checkEqual(model3, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 new file mode 100644 index 0000000000000..3c69467fa119e --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -0,0 +1,100 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +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; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + GBTClassifier rf = new GBTClassifier() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setSubsamplingRate(1.0) + .setSeed(1234) + .setMaxIter(3) + .setStepSize(0.1) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String lossType: GBTClassifier.supportedLossTypes()) { + rf.setLossType(lossType); + } + GBTClassificationModel model = rf.fit(dataFrame); + + model.transform(dataFrame); + model.totalNumNodes(); + model.toDebugString(); + model.trees(); + model.treeWeights(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model3.save(sc.sc(), path); + GBTClassificationModel sameModel = GBTClassificationModel.load(sc.sc(), path); + TreeTests.checkEqual(model3, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 3f8e59de0f05c..7e7189a2b1d53 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 @@ -84,9 +84,10 @@ public void logisticRegressionWithSetters() { .setThreshold(0.6) .setProbabilityCol("myProbability"); LogisticRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().apply(lr.maxIter()).equals(10)); - assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); - assert(model.fittingParamMap().apply(lr.threshold()).equals(0.6)); + LogisticRegression parent = model.parent(); + assert(parent.getMaxIter() == 10); + assert(parent.getRegParam() == 1.0); + assert(parent.getThreshold() == 0.6); assert(model.getThreshold() == 0.6); // Modify model params, and check that the params worked. @@ -109,9 +110,10 @@ public void logisticRegressionWithSetters() { // 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")); - assert(model2.fittingParamMap().apply(lr.maxIter()).equals(5)); - assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); - assert(model2.fittingParamMap().apply(lr.threshold()).equals(0.4)); + LogisticRegression parent2 = model2.parent(); + assert(parent2.getMaxIter() == 5); + assert(parent2.getRegParam() == 0.1); + assert(parent2.getThreshold() == 0.4); assert(model2.getThreshold() == 0.4); assert(model2.getProbabilityCol().equals("theProb")); } 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 new file mode 100644 index 0000000000000..32d0b3856b7e2 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -0,0 +1,103 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +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; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + RandomForestClassifier rf = new RandomForestClassifier() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setSubsamplingRate(1.0) + .setSeed(1234) + .setNumTrees(3) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String impurity: RandomForestClassifier.supportedImpurities()) { + rf.setImpurity(impurity); + } + for (String featureSubsetStrategy: RandomForestClassifier.supportedFeatureSubsetStrategies()) { + rf.setFeatureSubsetStrategy(featureSubsetStrategy); + } + RandomForestClassificationModel model = rf.fit(dataFrame); + + model.transform(dataFrame); + model.totalNumNodes(); + model.toDebugString(); + model.trees(); + model.treeWeights(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model3.save(sc.sc(), path); + RandomForestClassificationModel sameModel = + RandomForestClassificationModel.load(sc.sc(), path); + TreeTests.checkEqual(model3, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 new file mode 100644 index 0000000000000..3806f650025b2 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -0,0 +1,71 @@ +/* + * 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 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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrame; +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; + } + + @Test + public void regexTokenizer() { + RegexTokenizer myRegExTokenizer = new RegexTokenizer() + .setInputCol("rawText") + .setOutputCol("tokens") + .setPattern("\\s") + .setGaps(true) + .setMinTokenLength(3); + + JavaRDD rdd = jsc.parallelize(Lists.newArrayList( + new TokenizerTestData("Test of tok.", new String[] {"Test", "tok."}), + new TokenizerTestData("Te,st. punct", new String[] {"Te,st.", "punct"}) + )); + DataFrame dataset = jsql.createDataFrame(rdd, TokenizerTestData.class); + + Row[] pairs = myRegExTokenizer.transform(dataset) + .select("tokens", "wantedTokens") + .collect(); + + for (Row r : pairs) { + Assert.assertEquals(r.get(0), r.get(1)); + } + } +} 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 new file mode 100644 index 0000000000000..161100134c92d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -0,0 +1,70 @@ +/* + * 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 java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.DataFrame; +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; + } + + @Test + public void vectorIndexerAPI() { + // The tests are to check Java compatibility. + List points = Lists.newArrayList( + new FeatureData(Vectors.dense(0.0, -2.0)), + new FeatureData(Vectors.dense(1.0, 3.0)), + new FeatureData(Vectors.dense(1.0, 4.0)) + ); + SQLContext sqlContext = new SQLContext(sc); + DataFrame data = sqlContext.createDataFrame(sc.parallelize(points, 2), FeatureData.class); + VectorIndexer indexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexed") + .setMaxCategories(2); + VectorIndexerModel model = indexer.fit(data); + Assert.assertEquals(model.numFeatures(), 2); + Assert.assertEquals(model.categoryMaps().size(), 1); + DataFrame indexedData = model.transform(data); + } +} 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 new file mode 100644 index 0000000000000..e7df10dfa63ac --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -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. + */ + +package org.apache.spark.ml.param; + +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.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(); + Assert.assertEquals(testParams.getMyIntParam(), 1); + testParams.setMyIntParam(2).setMyDoubleParam(0.4).setMyStringParam("a"); + Assert.assertEquals(testParams.getMyDoubleParam(), 0.4, 0.0); + Assert.assertEquals(testParams.getMyStringParam(), "a"); + } + + @Test + public void testParamValidate() { + ParamValidators.gt(1.0); + ParamValidators.gtEq(1.0); + ParamValidators.lt(1.0); + ParamValidators.ltEq(1.0); + ParamValidators.inRange(0, 1, true, false); + ParamValidators.inRange(0, 1); + ParamValidators.inArray(Lists.newArrayList(0, 1, 3)); + ParamValidators.inArray(Lists.newArrayList("a", "b")); + } +} 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 new file mode 100644 index 0000000000000..8abe575610d19 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -0,0 +1,63 @@ +/* + * 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.param; + +import java.util.List; + +import com.google.common.collect.Lists; + +/** + * A subclass of Params for testing. + */ +public class JavaTestParams extends JavaParams { + + public IntParam myIntParam; + + public int getMyIntParam() { return (Integer)getOrDefault(myIntParam); } + + public JavaTestParams setMyIntParam(int value) { + set(myIntParam, value); return this; + } + + public DoubleParam myDoubleParam; + + public double getMyDoubleParam() { return (Double)getOrDefault(myDoubleParam); } + + public JavaTestParams setMyDoubleParam(double value) { + set(myDoubleParam, value); return this; + } + + public Param myStringParam; + + public String getMyStringParam() { return (String)getOrDefault(myStringParam); } + + public JavaTestParams setMyStringParam(String value) { + set(myStringParam, value); return this; + } + + public JavaTestParams() { + myIntParam = new IntParam(this, "myIntParam", "this is an int param", ParamValidators.gt(0)); + myDoubleParam = new DoubleParam(this, "myDoubleParam", "this is a double param", + ParamValidators.inRange(0.0, 1.0)); + List validStrings = Lists.newArrayList("a", "b"); + myStringParam = new Param(this, "myStringParam", "this is a string param", + ParamValidators.inArray(validStrings)); + setDefault(myIntParam, 1); + setDefault(myDoubleParam, 0.5); + } +} 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 new file mode 100644 index 0000000000000..71b041818d7ee --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -0,0 +1,95 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +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; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + + // This tests setters. Training with various options is tested in Scala. + DecisionTreeRegressor dt = new DecisionTreeRegressor() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String impurity: DecisionTreeRegressor.supportedImpurities()) { + dt.setImpurity(impurity); + } + DecisionTreeRegressionModel model = dt.fit(dataFrame); + + model.transform(dataFrame); + model.numNodes(); + model.depth(); + model.toDebugString(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model2.save(sc.sc(), path); + DecisionTreeRegressionModel sameModel = DecisionTreeRegressionModel.load(sc.sc(), path); + TreeTests.checkEqual(model2, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 new file mode 100644 index 0000000000000..fc8c13db07e6f --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -0,0 +1,99 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +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; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); + + GBTRegressor rf = new GBTRegressor() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setSubsamplingRate(1.0) + .setSeed(1234) + .setMaxIter(3) + .setStepSize(0.1) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String lossType: GBTRegressor.supportedLossTypes()) { + rf.setLossType(lossType); + } + GBTRegressionModel model = rf.fit(dataFrame); + + model.transform(dataFrame); + model.totalNumNodes(); + model.toDebugString(); + model.trees(); + model.treeWeights(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model2.save(sc.sc(), path); + GBTRegressionModel sameModel = GBTRegressionModel.load(sc.sc(), path); + TreeTests.checkEqual(model2, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 0cc36c8d56d70..a82b86d560b6e 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 @@ -23,14 +23,15 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.assertEquals; 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.regression.LabeledPoint; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; +import static org.apache.spark.mllib.classification.LogisticRegressionSuite + .generateLogisticInputAsList; public class JavaLinearRegressionSuite implements Serializable { @@ -65,8 +66,8 @@ public void linearRegressionDefaultParams() { DataFrame predictions = jsql.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults - assert(model.getFeaturesCol().equals("features")); - assert(model.getPredictionCol().equals("prediction")); + assertEquals("features", model.getFeaturesCol()); + assertEquals("prediction", model.getPredictionCol()); } @Test @@ -76,14 +77,16 @@ public void linearRegressionWithSetters() { .setMaxIter(10) .setRegParam(1.0); LinearRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().apply(lr.maxIter()).equals(10)); - assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); + LinearRegression parent = model.parent(); + assertEquals(10, parent.getMaxIter()); + assertEquals(1.0, parent.getRegParam(), 0.0); // 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")); - assert(model2.fittingParamMap().apply(lr.maxIter()).equals(5)); - assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); - assert(model2.getPredictionCol().equals("thePred")); + LinearRegression parent2 = model2.parent(); + assertEquals(5, parent2.getMaxIter()); + assertEquals(0.1, parent2.getRegParam(), 0.0); + assertEquals("thePred", model2.getPredictionCol()); } } 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 new file mode 100644 index 0000000000000..e306ebadfe7cf --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -0,0 +1,102 @@ +/* + * 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.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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.classification.LogisticRegressionSuite; +import org.apache.spark.ml.impl.TreeTests; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; + + +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; + } + + @Test + public void runDT() { + int nPoints = 20; + double A = 2.0; + double B = -1.5; + + JavaRDD data = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + Map categoricalFeatures = new HashMap(); + DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); + + // This tests setters. Training with various options is tested in Scala. + RandomForestRegressor rf = new RandomForestRegressor() + .setMaxDepth(2) + .setMaxBins(10) + .setMinInstancesPerNode(5) + .setMinInfoGain(0.0) + .setMaxMemoryInMB(256) + .setCacheNodeIds(false) + .setCheckpointInterval(10) + .setSubsamplingRate(1.0) + .setSeed(1234) + .setNumTrees(3) + .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern + for (String impurity: RandomForestRegressor.supportedImpurities()) { + rf.setImpurity(impurity); + } + for (String featureSubsetStrategy: RandomForestRegressor.supportedFeatureSubsetStrategies()) { + rf.setFeatureSubsetStrategy(featureSubsetStrategy); + } + RandomForestRegressionModel model = rf.fit(dataFrame); + + model.transform(dataFrame); + model.totalNumNodes(); + model.toDebugString(); + model.trees(); + model.treeWeights(); + + /* + // TODO: Add test once save/load are implemented. SPARK-6725 + File tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + String path = tempDir.toURI().toString(); + try { + model2.save(sc.sc(), path); + RandomForestRegressionModel sameModel = RandomForestRegressionModel.load(sc.sc(), path); + TreeTests.checkEqual(model2, sameModel); + } finally { + Utils.deleteRecursively(tempDir); + } + */ + } +} 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 0bb6b489f2757..08eeca53f0721 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 @@ -68,8 +68,8 @@ public void crossValidationWithLogisticRegression() { .setEvaluator(eval) .setNumFolds(3); CrossValidatorModel cvModel = cv.fit(dataset); - ParamMap bestParamMap = cvModel.bestModel().fittingParamMap(); - Assert.assertEquals(0.001, bestParamMap.apply(lr.regParam())); - Assert.assertEquals(10, bestParamMap.apply(lr.maxIter())); + LogisticRegression parent = (LogisticRegression) cvModel.bestModel().parent(); + Assert.assertEquals(0.001, parent.getRegParam(), 0.0); + Assert.assertEquals(10, parent.getMaxIter()); } } 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 1c90522a0714a..71fb7f13c39c2 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,20 +17,22 @@ 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.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; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; public class JavaNaiveBayesSuite implements Serializable { private transient JavaSparkContext sc; @@ -102,4 +104,11 @@ public Vector call(LabeledPoint v) throws Exception { // Should be able to get the first prediction. predictions.first(); } + + @Test + public void testModelTypeSetters() { + NaiveBayes nb = new NaiveBayes() + .setModelType("Bernoulli") + .setModelType("Multinomial"); + } } 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 dc10aa67c7c1f..f394d903966de 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 @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.ArrayList; -import org.apache.spark.api.java.JavaRDD; import scala.Tuple2; import org.junit.After; @@ -30,6 +29,7 @@ import org.junit.Test; 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.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; @@ -88,7 +88,7 @@ public void distributedLDAModel() { .setMaxIterations(5) .setSeed(12345); - DistributedLDAModel model = lda.run(corpus); + DistributedLDAModel model = (DistributedLDAModel)lda.run(corpus); // Check: basic parameters LocalLDAModel localModel = model.toLocal(); @@ -109,11 +109,45 @@ public void distributedLDAModel() { assert(model.logPrior() < 0.0); } + @Test + public void OnlineOptimizerCompatibility() { + int k = 3; + double topicSmoothing = 1.2; + double termSmoothing = 1.2; + + // Train a model + OnlineLDAOptimizer op = new OnlineLDAOptimizer() + .setTau_0(1024) + .setKappa(0.51) + .setGammaShape(1e40) + .setMiniBatchFraction(0.5); + + LDA lda = new LDA(); + lda.setK(k) + .setDocConcentration(topicSmoothing) + .setTopicConcentration(termSmoothing) + .setMaxIterations(5) + .setSeed(12345) + .setOptimizer(op); + + LDAModel model = lda.run(corpus); + + // Check: basic parameters + assertEquals(model.k(), k); + assertEquals(model.vocabSize(), tinyVocabSize); + + // Check: topic summaries + Tuple2[] roundedTopicSummary = model.describeTopics(); + assertEquals(roundedTopicSummary.length, k); + Tuple2[] roundedLocalTopicSummary = model.describeTopics(); + assertEquals(roundedLocalTopicSummary.length, k); + } + private static int tinyK = LDASuite$.MODULE$.tinyK(); private static int tinyVocabSize = LDASuite$.MODULE$.tinyVocabSize(); private static Matrix tinyTopics = LDASuite$.MODULE$.tinyTopics(); private static Tuple2[] tinyTopicDescription = LDASuite$.MODULE$.tinyTopicDescription(); - JavaPairRDD corpus; + private JavaPairRDD corpus; } diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN 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 2f175fb117941..2b04a3034782e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -42,30 +42,32 @@ class PipelineSuite extends FunSuite { val dataset3 = mock[DataFrame] val dataset4 = mock[DataFrame] - when(estimator0.fit(meq(dataset0), any[ParamMap]())).thenReturn(model0) - when(model0.transform(meq(dataset0), any[ParamMap]())).thenReturn(dataset1) + when(estimator0.copy(any[ParamMap])).thenReturn(estimator0) + when(model0.copy(any[ParamMap])).thenReturn(model0) + when(transformer1.copy(any[ParamMap])).thenReturn(transformer1) + when(estimator2.copy(any[ParamMap])).thenReturn(estimator2) + when(model2.copy(any[ParamMap])).thenReturn(model2) + when(transformer3.copy(any[ParamMap])).thenReturn(transformer3) + + when(estimator0.fit(meq(dataset0))).thenReturn(model0) + when(model0.transform(meq(dataset0))).thenReturn(dataset1) when(model0.parent).thenReturn(estimator0) - when(transformer1.transform(meq(dataset1), any[ParamMap])).thenReturn(dataset2) - when(estimator2.fit(meq(dataset2), any[ParamMap]())).thenReturn(model2) - when(model2.transform(meq(dataset2), any[ParamMap]())).thenReturn(dataset3) + when(transformer1.transform(meq(dataset1))).thenReturn(dataset2) + when(estimator2.fit(meq(dataset2))).thenReturn(model2) + when(model2.transform(meq(dataset2))).thenReturn(dataset3) when(model2.parent).thenReturn(estimator2) - when(transformer3.transform(meq(dataset3), any[ParamMap]())).thenReturn(dataset4) + when(transformer3.transform(meq(dataset3))).thenReturn(dataset4) val pipeline = new Pipeline() .setStages(Array(estimator0, transformer1, estimator2, transformer3)) val pipelineModel = pipeline.fit(dataset0) - assert(pipelineModel.stages.size === 4) + assert(pipelineModel.stages.length === 4) assert(pipelineModel.stages(0).eq(model0)) assert(pipelineModel.stages(1).eq(transformer1)) assert(pipelineModel.stages(2).eq(model2)) assert(pipelineModel.stages(3).eq(transformer3)) - assert(pipelineModel.getModel(estimator0).eq(model0)) - assert(pipelineModel.getModel(estimator2).eq(model2)) - intercept[NoSuchElementException] { - pipelineModel.getModel(mock[Estimator[MyModel]]) - } val output = pipelineModel.transform(dataset0) assert(output.eq(dataset4)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala index 3fb6e2ec46468..17ddd335deb6d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -43,7 +43,7 @@ class AttributeGroupSuite extends FunSuite { intercept[NoSuchElementException] { group("abc") } - assert(group === AttributeGroup.fromMetadata(group.toMetadata, group.name)) + assert(group === AttributeGroup.fromMetadata(group.toMetadataImpl, group.name)) assert(group === AttributeGroup.fromStructField(group.toStructField())) } @@ -53,7 +53,7 @@ class AttributeGroupSuite extends FunSuite { assert(group0.numAttributes === Some(10)) assert(group0.size === 10) assert(group0.attributes.isEmpty) - assert(group0 === AttributeGroup.fromMetadata(group0.toMetadata, group0.name)) + assert(group0 === AttributeGroup.fromMetadata(group0.toMetadataImpl, group0.name)) assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) val group1 = new AttributeGroup("item") diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index 6ec35b03656f9..3e1a7196e37cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -36,9 +36,9 @@ class AttributeSuite extends FunSuite { assert(attr.max.isEmpty) assert(attr.std.isEmpty) assert(attr.sparsity.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) intercept[NoSuchElementException] { @@ -59,9 +59,9 @@ class AttributeSuite extends FunSuite { assert(!attr.isNominal) assert(attr.name === Some(name)) assert(attr.index === Some(index)) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = false) === metadata) - assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = false) === metadata) + assert(attr.toMetadataImpl(withType = true) === metadataWithType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === Attribute.fromMetadata(metadataWithType)) val field = attr.toStructField() @@ -81,7 +81,7 @@ class AttributeSuite extends FunSuite { assert(attr2.max === Some(1.0)) assert(attr2.std === Some(0.5)) assert(attr2.sparsity === Some(0.3)) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) } test("bad numeric attributes") { @@ -105,9 +105,9 @@ class AttributeSuite extends FunSuite { assert(attr.values.isEmpty) assert(attr.numValues.isEmpty) assert(attr.isOrdinal.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -135,9 +135,9 @@ class AttributeSuite extends FunSuite { assert(attr.values === Some(values)) assert(attr.indexOf("medium") === 1) assert(attr.getValue(1) === "medium") - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) @@ -147,8 +147,8 @@ class AttributeSuite extends FunSuite { assert(attr2.index.isEmpty) assert(attr2.values.get === Array("small", "medium", "large", "x-large")) assert(attr2.indexOf("x-large") === 3) - assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) - assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadata(withType = false))) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadataImpl())) + assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadataImpl(withType = false))) } test("bad nominal attributes") { @@ -168,9 +168,9 @@ class AttributeSuite extends FunSuite { assert(attr.name.isEmpty) assert(attr.index.isEmpty) assert(attr.values.isEmpty) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) intercept[NoSuchElementException] { @@ -196,9 +196,9 @@ class AttributeSuite extends FunSuite { assert(attr.name === Some(name)) assert(attr.index === Some(index)) assert(attr.values.get === values) - assert(attr.toMetadata() === metadata) - assert(attr.toMetadata(withType = true) === metadata) - assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr.toMetadataImpl() === metadata) + assert(attr.toMetadataImpl(withType = true) === metadata) + assert(attr.toMetadataImpl(withType = false) === metadataWithoutType) assert(attr === Attribute.fromMetadata(metadata)) assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) 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 new file mode 100644 index 0000000000000..03af4ecd7a7e0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -0,0 +1,274 @@ +/* + * 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.classification + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeClassifierSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeClassifierSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel0RDD: RDD[LabeledPoint] = _ + private var orderedLabeledPointsWithLabel1RDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var continuousDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ + private var categoricalDataPointsForMulticlassForOrderedFeaturesRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + orderedLabeledPointsWithLabel0RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + orderedLabeledPointsWithLabel1RDD = + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + categoricalDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + continuousDataPointsForMulticlassRDD = + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( + OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Binary classification stump with ordered categorical features") { + val dt = new DecisionTreeClassifier() + .setImpurity("gini") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + val numClasses = 2 + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with fixed labels 0,1 for Entropy,Gini") { + val dt = new DecisionTreeClassifier() + .setMaxDepth(3) + .setMaxBins(100) + val numClasses = 2 + Array(orderedLabeledPointsWithLabel0RDD, orderedLabeledPointsWithLabel1RDD).foreach { rdd => + DecisionTreeClassifier.supportedImpurities.foreach { impurity => + dt.setImpurity(impurity) + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + } + } + + test("Multiclass classification stump with 3-ary (unordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 3 + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Binary classification stump with 2 continuous features") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features + val rdd = categoricalDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(maxBins) + val categoricalFeatures = Map(0 -> 3, 1 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with continuous features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("Multiclass classification stump with continuous + unordered categorical features") { + val rdd = continuousDataPointsForMulticlassRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification stump with 10-ary (ordered) categorical features") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(4) + .setMaxBins(10) + val categoricalFeatures = Map(0 -> 10, 1 -> 10) + val numClasses = 3 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min instances per node requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + test("do not choose split that does not satisfy min instance per node requirements") { + // if a split does not satisfy min instances per node requirements, + // this split is invalid, even though the information gain of split is large. + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) + val rdd = sc.parallelize(arr) + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxBins(2) + .setMaxDepth(2) + .setMinInstancesPerNode(2) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures, numClasses) + } + + test("split must satisfy min info gain requirements") { + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) + val rdd = sc.parallelize(arr) + + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setMinInfoGain(1.0) + val numClasses = 2 + compareAPIs(rdd, dt, categoricalFeatures = Map.empty[Int, Int], numClasses) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented SPARK-6725 + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val oldModel = OldDecisionTreeSuite.createModel(OldAlgo.Classification) + val newModel = DecisionTreeClassificationModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = DecisionTreeClassificationModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private[ml] object DecisionTreeClassifierSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeClassifier, + categoricalFeatures: Map[Int, Int], + numClasses: Int): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeClassificationModel.fromOld( + oldTree, newTree.parent, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} 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 new file mode 100644 index 0000000000000..16c758b82c7cd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -0,0 +1,136 @@ +/* + * 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.classification + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +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 +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * Test suite for [[GBTClassifier]]. + */ +class GBTClassifierSuite extends FunSuite with MLlibTestSparkContext { + + import GBTClassifierSuite.compareAPIs + + // Combinations for estimators, learning rates and subsamplingRate + private val testCombinations = + Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) + + private var data: RDD[LabeledPoint] = _ + private var trainData: RDD[LabeledPoint] = _ + private var validationData: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + trainData = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + validationData = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + } + + test("Binary classification with continuous features: Log Loss") { + val categoricalFeatures = Map.empty[Int, Int] + testCombinations.foreach { + case (maxIter, learningRate, subsamplingRate) => + val gbt = new GBTClassifier() + .setMaxDepth(2) + .setSubsamplingRate(subsamplingRate) + .setLossType("logistic") + .setMaxIter(maxIter) + .setStepSize(learningRate) + compareAPIs(data, None, gbt, categoricalFeatures) + } + } + + // TODO: Reinstate test once runWithValidation is implemented SPARK-7132 + /* + test("runWithValidation stops early and performs better on a validation dataset") { + val categoricalFeatures = Map.empty[Int, Int] + // Set maxIter large enough so that it stops early. + val maxIter = 20 + GBTClassifier.supportedLossTypes.foreach { loss => + val gbt = new GBTClassifier() + .setMaxIter(maxIter) + .setMaxDepth(2) + .setLossType(loss) + .setValidationTol(0.0) + compareAPIs(trainData, None, gbt, categoricalFeatures) + compareAPIs(trainData, Some(validationData), gbt, categoricalFeatures) + } + } + */ + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented SPARK-6725 + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray + val treeWeights = Array(0.1, 0.3, 1.1) + val oldModel = new OldGBTModel(OldAlgo.Classification, trees, treeWeights) + val newModel = GBTClassificationModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = GBTClassificationModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private object GBTClassifierSuite { + + /** + * Train 2 models on the given dataset, one using the old API and one using the new API. + * Convert the old model to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + validationData: Option[RDD[LabeledPoint]], + gbt: GBTClassifier, + categoricalFeatures: Map[Int, Int]): Unit = { + val oldBoostingStrategy = + gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) + val oldGBT = new OldGBT(oldBoostingStrategy) + val oldModel = oldGBT.run(data) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) + val newModel = gbt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldModelAsNew = GBTClassificationModel.fromOld( + oldModel, newModel.parent, categoricalFeatures) + TreeTests.checkEqual(oldModelAsNew, newModel) + } +} 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 b3d1bfcfbee0f..6dd1fdf05514e 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 @@ -46,6 +46,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(lr.getPredictionCol == "prediction") assert(lr.getRawPredictionCol == "rawPrediction") assert(lr.getProbabilityCol == "probability") + assert(lr.getFitIntercept == true) val model = lr.fit(dataset) model.transform(dataset) .select("label", "probability", "prediction", "rawPrediction") @@ -55,6 +56,14 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(model.getPredictionCol == "prediction") assert(model.getRawPredictionCol == "rawPrediction") assert(model.getProbabilityCol == "probability") + assert(model.intercept !== 0.0) + } + + test("logistic regression doesn't fit intercept when fitIntercept is off") { + val lr = new LogisticRegression + lr.setFitIntercept(false) + val model = lr.fit(dataset) + assert(model.intercept === 0.0) } test("logistic regression with setters") { @@ -65,9 +74,10 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { .setThreshold(0.6) .setProbabilityCol("myProbability") val model = lr.fit(dataset) - assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) - assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) - assert(model.fittingParamMap.get(lr.threshold) === Some(0.6)) + val parent = model.parent + assert(parent.getMaxIter === 10) + assert(parent.getRegParam === 1.0) + assert(parent.getThreshold === 0.6) assert(model.getThreshold === 0.6) // Modify model params, and check that the params worked. @@ -90,9 +100,10 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { // Call fit() with new params, and check as many params as we can. val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.threshold -> 0.4, lr.probabilityCol -> "theProb") - assert(model2.fittingParamMap.get(lr.maxIter).get === 5) - assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) - assert(model2.fittingParamMap.get(lr.threshold).get === 0.4) + val parent2 = model2.parent + assert(parent2.getMaxIter === 5) + assert(parent2.getRegParam === 0.1) + assert(parent2.getThreshold === 0.4) assert(model2.getThreshold === 0.4) assert(model2.getProbabilityCol == "theProb") } @@ -108,7 +119,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { val results = model.transform(dataset) // Compare rawPrediction with probability - results.select("rawPrediction", "probability").collect().map { + results.select("rawPrediction", "probability").collect().foreach { case Row(raw: Vector, prob: Vector) => assert(raw.size === 2) assert(prob.size === 2) @@ -118,7 +129,7 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { } // Compare prediction with probability - results.select("prediction", "probability").collect().map { + results.select("prediction", "probability").collect().foreach { case Row(pred: Double, prob: Vector) => val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 assert(pred == predFromProb) 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 new file mode 100644 index 0000000000000..c41def9330504 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -0,0 +1,166 @@ +/* + * 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.classification + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +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 +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * Test suite for [[RandomForestClassifier]]. + */ +class RandomForestClassifierSuite extends FunSuite with MLlibTestSparkContext { + + import RandomForestClassifierSuite.compareAPIs + + private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ + private var orderedLabeledPoints5_20: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + orderedLabeledPoints50_1000 = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + orderedLabeledPoints5_20 = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + def binaryClassificationTestWithContinuousFeatures(rf: RandomForestClassifier) { + val categoricalFeatures = Map.empty[Int, Int] + val numClasses = 2 + val newRF = rf + .setImpurity("Gini") + .setMaxDepth(2) + .setNumTrees(1) + .setFeatureSubsetStrategy("auto") + .setSeed(123) + compareAPIs(orderedLabeledPoints50_1000, newRF, categoricalFeatures, numClasses) + } + + test("Binary classification with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val rf = new RandomForestClassifier() + binaryClassificationTestWithContinuousFeatures(rf) + } + + test("Binary classification with continuous features and node Id cache:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val rf = new RandomForestClassifier() + .setCacheNodeIds(true) + binaryClassificationTestWithContinuousFeatures(rf) + } + + test("alternating categorical and continuous features with multiclass labels to test indexing") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0, 3.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0, 1.0, 2.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0, 6.0, 3.0)), + LabeledPoint(2.0, Vectors.dense(0.0, 2.0, 1.0, 3.0, 2.0)) + ) + val rdd = sc.parallelize(arr) + val categoricalFeatures = Map(0 -> 3, 2 -> 2, 4 -> 4) + val numClasses = 3 + + val rf = new RandomForestClassifier() + .setImpurity("Gini") + .setMaxDepth(5) + .setNumTrees(2) + .setFeatureSubsetStrategy("sqrt") + .setSeed(12345) + compareAPIs(rdd, rf, categoricalFeatures, numClasses) + } + + test("subsampling rate in RandomForest"){ + val rdd = orderedLabeledPoints5_20 + val categoricalFeatures = Map.empty[Int, Int] + val numClasses = 2 + + val rf1 = new RandomForestClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + .setCacheNodeIds(true) + .setNumTrees(3) + .setFeatureSubsetStrategy("auto") + .setSeed(123) + compareAPIs(rdd, rf1, categoricalFeatures, numClasses) + + val rf2 = rf1.setSubsamplingRate(0.5) + compareAPIs(rdd, rf2, categoricalFeatures, numClasses) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented SPARK-6725 + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val trees = + Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Classification)).toArray + val oldModel = new OldRandomForestModel(OldAlgo.Classification, trees) + val newModel = RandomForestClassificationModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = RandomForestClassificationModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private object RandomForestClassifierSuite { + + /** + * Train 2 models on the given dataset, one using the old API and one using the new API. + * Convert the old model to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + rf: RandomForestClassifier, + categoricalFeatures: Map[Int, Int], + numClasses: Int): Unit = { + val oldStrategy = + rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) + val oldModel = OldRandomForest.trainClassifier( + data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) + val newModel = rf.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldModelAsNew = RandomForestClassificationModel.fromOld( + oldModel, newModel.parent, categoricalFeatures) + TreeTests.checkEqual(oldModelAsNew, newModel) + } +} 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 new file mode 100644 index 0000000000000..caf1b759593f3 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -0,0 +1,69 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + + +class BinarizerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var data: Array[Double] = _ + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + data = Array(0.1, -0.5, 0.2, -0.3, 0.8, 0.7, -0.1, -0.4) + } + + 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( + data.zip(defaultBinarized)).toDF("feature", "expected") + + val binarizer: Binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + + binarizer.transform(dataFrame).select("binarized_feature", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, "The feature value is not correct after binarization.") + } + } + + 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( + data.zip(thresholdBinarized)).toDF("feature", "expected") + + val binarizer: Binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(threshold) + + binarizer.transform(dataFrame).select("binarized_feature", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, "The feature value is not correct after binarization.") + } + } +} 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 new file mode 100644 index 0000000000000..eaee3443c1f23 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -0,0 +1,101 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{Row, SQLContext} + +class IDFSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + def scaleDataWithIDF(dataSet: Array[Vector], model: Vector): Array[Vector] = { + dataSet.map { + case data: DenseVector => + val res = data.toArray.zip(model.toArray).map { case (x, y) => x * y } + Vectors.dense(res) + case data: SparseVector => + val res = data.indices.zip(data.values).map { case (id, value) => + (id, value * model(id)) + } + Vectors.sparse(data.size, res) + } + } + + test("compute IDF with default parameter") { + val numOfFeatures = 4 + val data = Array( + Vectors.sparse(numOfFeatures, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(numOfFeatures, Array(1), Array(1.0)) + ) + val numOfData = data.size + val idf = Vectors.dense(Array(0, 3, 1, 2).map { x => + math.log((numOfData + 1.0) / (x + 1.0)) + }) + val expected = scaleDataWithIDF(data, idf) + + val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + + val idfModel = new IDF() + .setInputCol("features") + .setOutputCol("idfValue") + .fit(df) + + idfModel.transform(df).select("idfValue", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x ~== y absTol 1e-5, "Transformed vector is different with expected vector.") + } + } + + test("compute IDF with setter") { + val numOfFeatures = 4 + val data = Array( + Vectors.sparse(numOfFeatures, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(numOfFeatures, Array(1), Array(1.0)) + ) + val numOfData = data.size + val idf = Vectors.dense(Array(0, 3, 1, 2).map { x => + if (x > 0) math.log((numOfData + 1.0) / (x + 1.0)) else 0 + }) + val expected = scaleDataWithIDF(data, idf) + + val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + + val idfModel = new IDF() + .setInputCol("features") + .setOutputCol("idfValue") + .setMinDocFreq(1) + .fit(df) + + idfModel.transform(df).select("idfValue", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x ~== y absTol 1e-5, "Transformed vector is different with expected vector.") + } + } +} 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 new file mode 100644 index 0000000000000..9d09f24709e23 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -0,0 +1,112 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + + +class NormalizerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var data: Array[Vector] = _ + @transient var dataFrame: DataFrame = _ + @transient var normalizer: Normalizer = _ + @transient var l1Normalized: Array[Vector] = _ + @transient var l2Normalized: Array[Vector] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq((1, 0.91), (2, 3.2))), + Vectors.sparse(3, Seq((0, 5.7), (1, 0.72), (2, 2.7))), + Vectors.sparse(3, Seq()) + ) + l1Normalized = Array( + Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.12765957, -0.23404255, -0.63829787), + Vectors.sparse(3, Seq((1, 0.22141119), (2, 0.7785888))), + Vectors.dense(0.625, 0.07894737, 0.29605263), + Vectors.sparse(3, Seq()) + ) + l2Normalized = Array( + Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.184549876, -0.3383414, -0.922749378), + Vectors.sparse(3, Seq((1, 0.27352993), (2, 0.96186349))), + Vectors.dense(0.897906166, 0.113419726, 0.42532397), + Vectors.sparse(3, Seq()) + ) + + val sqlContext = new SQLContext(sc) + dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) + normalizer = new Normalizer() + .setInputCol("features") + .setOutputCol("normalized_features") + } + + def collectResult(result: DataFrame): Array[Vector] = { + result.select("normalized_features").collect().map { + case Row(features: Vector) => features + } + } + + def assertTypeOfVector(lhs: Array[Vector], rhs: Array[Vector]): Unit = { + assert((lhs, rhs).zipped.forall { + case (v1: DenseVector, v2: DenseVector) => true + case (v1: SparseVector, v2: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + } + + def assertValues(lhs: Array[Vector], rhs: Array[Vector]): Unit = { + assert((lhs, rhs).zipped.forall { (vector1, vector2) => + vector1 ~== vector2 absTol 1E-5 + }, "The vector value is not correct after normalization.") + } + + test("Normalization with default parameter") { + val result = collectResult(normalizer.transform(dataFrame)) + + assertTypeOfVector(data, result) + + assertValues(result, l2Normalized) + } + + test("Normalization with setter") { + normalizer.setP(1) + + val result = collectResult(normalizer.transform(dataFrame)) + + assertTypeOfVector(data, result) + + assertValues(result, l1Normalized) + } +} + +private object NormalizerSuite { + case class FeatureData(features: Vector) +} 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 new file mode 100644 index 0000000000000..c1d64fba0aa8f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -0,0 +1,104 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{Row, SQLContext} +import org.scalatest.exceptions.TestFailedException + +class PolynomialExpansionSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("Polynomial expansion with default parameter") { + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(-2.0, 2.3), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq()) + ) + + val twoDegreeExpansion: Array[Vector] = Array( + Vectors.sparse(9, Array(0, 1, 2, 3, 4), Array(-2.0, 4.0, 2.3, -4.6, 5.29)), + Vectors.dense(-2.0, 4.0, 2.3, -4.6, 5.29), + Vectors.dense(new Array[Double](9)), + Vectors.dense(0.6, 0.36, -1.1, -0.66, 1.21, -3.0, -1.8, 3.3, 9.0), + Vectors.sparse(9, Array.empty, Array.empty)) + + val df = sqlContext.createDataFrame(data.zip(twoDegreeExpansion)).toDF("features", "expected") + + val polynomialExpansion = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + + polynomialExpansion.transform(df).select("polyFeatures", "expected").collect().foreach { + case Row(expanded: DenseVector, expected: DenseVector) => + assert(expanded ~== expected absTol 1e-1) + case Row(expanded: SparseVector, expected: SparseVector) => + assert(expanded ~== expected absTol 1e-1) + case _ => + throw new TestFailedException("Unmatched data types after polynomial expansion", 0) + } + } + + test("Polynomial expansion with setter") { + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(-2.0, 2.3), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq()) + ) + + val threeDegreeExpansion: Array[Vector] = Array( + Vectors.sparse(19, Array(0, 1, 2, 3, 4, 5, 6, 7, 8), + Array(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17)), + Vectors.dense(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17), + Vectors.dense(new Array[Double](19)), + Vectors.dense(0.6, 0.36, 0.216, -1.1, -0.66, -0.396, 1.21, 0.726, -1.331, -3.0, -1.8, + -1.08, 3.3, 1.98, -3.63, 9.0, 5.4, -9.9, -27.0), + Vectors.sparse(19, Array.empty, Array.empty)) + + val df = sqlContext.createDataFrame(data.zip(threeDegreeExpansion)).toDF("features", "expected") + + val polynomialExpansion = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + .setDegree(3) + + polynomialExpansion.transform(df).select("polyFeatures", "expected").collect().foreach { + case Row(expanded: DenseVector, expected: DenseVector) => + assert(expanded ~== expected absTol 1e-1) + case Row(expanded: SparseVector, expected: SparseVector) => + assert(expanded ~== expected absTol 1e-1) + case _ => + throw new TestFailedException("Unmatched data types after polynomial expansion", 0) + } + } +} + 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 new file mode 100644 index 0000000000000..b6939e5870410 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -0,0 +1,71 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.SQLContext + +class StringIndexerSuite extends FunSuite with MLlibTestSparkContext { + private var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + 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 indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + .fit(df) + val transformed = indexer.transform(df) + val attr = Attribute.fromStructField(transformed.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attr.values.get === Array("a", "c", "b")) + val output = transformed.select("id", "labelIndex").map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 0, b -> 2, c -> 1 + val expected = Set((0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)) + assert(output === expected) + } + + 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 indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + .fit(df) + val transformed = indexer.transform(df) + val attr = Attribute.fromStructField(transformed.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attr.values.get === Array("100", "300", "200")) + val output = transformed.select("id", "labelIndex").map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // 100 -> 0, 200 -> 2, 300 -> 1 + val expected = Set((0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)) + assert(output === expected) + } +} 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 new file mode 100644 index 0000000000000..d186ead8f542f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +@BeanInfo +case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) + +class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { + import org.apache.spark.ml.feature.RegexTokenizerSuite._ + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("RegexTokenizer") { + val tokenizer = new RegexTokenizer() + .setInputCol("rawText") + .setOutputCol("tokens") + + val dataset0 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization", ".")), + TokenizerTestData("Te,st. punct", Array("Te", ",", "st", ".", "punct")) + )) + testRegexTokenizer(tokenizer, dataset0) + + val dataset1 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization")), + TokenizerTestData("Te,st. punct", Array("punct")) + )) + + tokenizer.setMinTokenLength(3) + testRegexTokenizer(tokenizer, dataset1) + + tokenizer + .setPattern("\\s") + .setGaps(true) + .setMinTokenLength(0) + val dataset2 = sqlContext.createDataFrame(Seq( + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization.")), + TokenizerTestData("Te,st. punct", Array("Te,st.", "", "punct")) + )) + testRegexTokenizer(tokenizer, dataset2) + } +} + +object RegexTokenizerSuite extends FunSuite { + + def testRegexTokenizer(t: RegexTokenizer, dataset: DataFrame): Unit = { + t.transform(dataset) + .select("tokens", "wantedTokens") + .collect() + .foreach { + case Row(tokens, wantedTokens) => + assert(tokens === wantedTokens) + } + } +} 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 new file mode 100644 index 0000000000000..57d0278e03639 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -0,0 +1,63 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Row, SQLContext} + +class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("assemble") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + assert(assemble(0.0) === Vectors.sparse(1, Array.empty, Array.empty)) + assert(assemble(0.0, 1.0) === Vectors.sparse(2, Array(1), Array(1.0))) + val dv = Vectors.dense(2.0, 0.0) + assert(assemble(0.0, dv, 1.0) === Vectors.sparse(4, Array(1, 3), Array(2.0, 1.0))) + val sv = Vectors.sparse(2, Array(0, 1), Array(3.0, 4.0)) + assert(assemble(0.0, dv, 1.0, sv) === + Vectors.sparse(6, Array(1, 3, 4, 5), Array(2.0, 1.0, 3.0, 4.0))) + for (v <- Seq(1, "a", null)) { + intercept[SparkException](assemble(v)) + intercept[SparkException](assemble(1.0, v)) + } + } + + test("VectorAssembler") { + val df = sqlContext.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() + .setInputCols(Array("x", "y", "z", "n")) + .setOutputCol("features") + assembler.transform(df).select("features").collect().foreach { + case Row(v: Vector) => + assert(v === Vectors.sparse(6, Array(1, 2, 4, 5), Array(1.0, 2.0, 3.0, 10.0))) + } + } +} 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 new file mode 100644 index 0000000000000..38dc83b1241cf --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -0,0 +1,253 @@ +/* + * 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, BeanProperty} + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.ml.attribute._ +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, SQLContext} + + +class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { + + import VectorIndexerSuite.FeatureData + + @transient var sqlContext: SQLContext = _ + + // identical, of length 3 + @transient var densePoints1: DataFrame = _ + @transient var sparsePoints1: DataFrame = _ + @transient var point1maxes: Array[Double] = _ + + // identical, of length 2 + @transient var densePoints2: DataFrame = _ + @transient var sparsePoints2: DataFrame = _ + + // different lengths + @transient var badPoints: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val densePoints1Seq = Seq( + Vectors.dense(1.0, 2.0, 0.0), + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(0.0, 0.0, -1.0), + Vectors.dense(1.0, 3.0, 2.0)) + val sparsePoints1Seq = Seq( + Vectors.sparse(3, Array(0, 1), Array(1.0, 2.0)), + Vectors.sparse(3, Array(1, 2), Array(1.0, 2.0)), + Vectors.sparse(3, Array(2), Array(-1.0)), + Vectors.sparse(3, Array(0, 1, 2), Array(1.0, 3.0, 2.0))) + point1maxes = Array(1.0, 3.0, 2.0) + + val densePoints2Seq = Seq( + Vectors.dense(1.0, 1.0, 0.0, 1.0), + Vectors.dense(0.0, 1.0, 1.0, 1.0), + Vectors.dense(-1.0, 1.0, 2.0, 0.0)) + val sparsePoints2Seq = Seq( + Vectors.sparse(4, Array(0, 1, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(1, 2, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(0, 1, 2), Array(-1.0, 1.0, 2.0))) + + val badPointsSeq = Seq( + Vectors.sparse(2, Array(0, 1), Array(1.0, 1.0)), + Vectors.sparse(3, Array(2), Array(-1.0))) + + // Sanity checks for assumptions made in tests + assert(densePoints1Seq.head.size == sparsePoints1Seq.head.size) + assert(densePoints2Seq.head.size == sparsePoints2Seq.head.size) + assert(densePoints1Seq.head.size != densePoints2Seq.head.size) + def checkPair(dvSeq: Seq[Vector], svSeq: Seq[Vector]): Unit = { + assert(dvSeq.zip(svSeq).forall { case (dv, sv) => dv.toArray === sv.toArray }, + "typo in unit test") + } + checkPair(densePoints1Seq, sparsePoints1Seq) + checkPair(densePoints2Seq, sparsePoints2Seq) + + sqlContext = new SQLContext(sc) + 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)) + } + + private def getIndexer: VectorIndexer = + new VectorIndexer().setInputCol("features").setOutputCol("indexed") + + test("Cannot fit an empty DataFrame") { + val rdd = sqlContext.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) + val vectorIndexer = getIndexer + intercept[IllegalArgumentException] { + vectorIndexer.fit(rdd) + } + } + + test("Throws error when given RDDs with different size vectors") { + val vectorIndexer = getIndexer + val model = vectorIndexer.fit(densePoints1) // vectors of length 3 + model.transform(densePoints1) // should work + model.transform(sparsePoints1) // should work + intercept[SparkException] { + model.transform(densePoints2).collect() + println("Did not throw error when fit, transform were called on vectors of different lengths") + } + intercept[SparkException] { + vectorIndexer.fit(badPoints) + println("Did not throw error when fitting vectors of different lengths in same RDD.") + } + } + + test("Same result with dense and sparse vectors") { + def testDenseSparse(densePoints: DataFrame, sparsePoints: DataFrame): Unit = { + val denseVectorIndexer = getIndexer.setMaxCategories(2) + val sparseVectorIndexer = getIndexer.setMaxCategories(2) + val denseModel = denseVectorIndexer.fit(densePoints) + val sparseModel = sparseVectorIndexer.fit(sparsePoints) + val denseMap = denseModel.categoryMaps + val sparseMap = sparseModel.categoryMaps + assert(denseMap.keys.toSet == sparseMap.keys.toSet, + "Categorical features chosen from dense vs. sparse vectors did not match.") + assert(denseMap == sparseMap, + "Categorical feature value indexes chosen from dense vs. sparse vectors did not match.") + } + testDenseSparse(densePoints1, sparsePoints1) + testDenseSparse(densePoints2, sparsePoints2) + } + + test("Builds valid categorical feature value index, transform correctly, check metadata") { + def checkCategoryMaps( + data: DataFrame, + maxCategories: Int, + categoricalFeatures: Set[Int]): Unit = { + val collectedData = data.collect().map(_.getAs[Vector](0)) + val errMsg = s"checkCategoryMaps failed for input with maxCategories=$maxCategories," + + s" categoricalFeatures=${categoricalFeatures.mkString(", ")}" + try { + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val categoryMaps = model.categoryMaps + // Chose correct categorical features + assert(categoryMaps.keys.toSet === categoricalFeatures) + val transformed = model.transform(data).select("indexed") + val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) + val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) + assert(featureAttrs.name === "indexed") + assert(featureAttrs.attributes.get.length === model.numFeatures) + categoricalFeatures.foreach { feature: Int => + val origValueSet = collectedData.map(_(feature)).toSet + val targetValueIndexSet = Range(0, origValueSet.size).toSet + val catMap = categoryMaps(feature) + assert(catMap.keys.toSet === origValueSet) // Correct categories + assert(catMap.values.toSet === targetValueIndexSet) // Correct category indices + if (origValueSet.contains(0.0)) { + assert(catMap(0.0) === 0) // value 0 gets index 0 + } + // Check transformed data + assert(indexedRDD.map(_(feature)).collect().toSet === targetValueIndexSet) + // Check metadata + val featureAttr = featureAttrs(feature) + assert(featureAttr.index.get === feature) + featureAttr match { + case attr: BinaryAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + case attr: NominalAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + assert(attr.isOrdinal.get === false) + case _ => + throw new RuntimeException(errMsg + s". Categorical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + // Check numerical feature metadata. + Range(0, model.numFeatures).filter(feature => !categoricalFeatures.contains(feature)) + .foreach { feature: Int => + val featureAttr = featureAttrs(feature) + featureAttr match { + case attr: NumericAttribute => + assert(featureAttr.index.get === feature) + case _ => + throw new RuntimeException(errMsg + s". Numerical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + } catch { + case e: org.scalatest.exceptions.TestFailedException => + println(errMsg) + throw e + } + } + checkCategoryMaps(densePoints1, maxCategories = 2, categoricalFeatures = Set(0)) + checkCategoryMaps(densePoints1, maxCategories = 3, categoricalFeatures = Set(0, 2)) + checkCategoryMaps(densePoints2, maxCategories = 2, categoricalFeatures = Set(1, 3)) + } + + test("Maintain sparsity for sparse vectors") { + def checkSparsity(data: DataFrame, maxCategories: Int): Unit = { + val points = data.collect().map(_.getAs[Vector](0)) + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val indexedPoints = model.transform(data).select("indexed").map(_.getAs[Vector](0)).collect() + points.zip(indexedPoints).foreach { + case (orig: SparseVector, indexed: SparseVector) => + assert(orig.indices.length == indexed.indices.length) + case _ => throw new UnknownError("Unit test has a bug in it.") // should never happen + } + } + checkSparsity(sparsePoints1, maxCategories = 2) + checkSparsity(sparsePoints2, maxCategories = 2) + } + + test("Preserve metadata") { + // For continuous features, preserve name and stats. + val featureAttributes: Array[Attribute] = point1maxes.zipWithIndex.map { case (maxVal, i) => + NumericAttribute.defaultAttr.withName(i.toString).withMax(maxVal) + } + val attrGroup = new AttributeGroup("features", featureAttributes) + val densePoints1WithMeta = + densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata())) + val vectorIndexer = getIndexer.setMaxCategories(2) + val model = vectorIndexer.fit(densePoints1WithMeta) + // Check that ML metadata are preserved. + val indexedPoints = model.transform(densePoints1WithMeta) + val transAttributes: Array[Attribute] = + AttributeGroup.fromStructField(indexedPoints.schema("indexed")).attributes.get + featureAttributes.zip(transAttributes).foreach { case (orig, trans) => + assert(orig.name === trans.name) + (orig, trans) match { + case (orig: NumericAttribute, trans: NumericAttribute) => + assert(orig.max.nonEmpty && orig.max === trans.max) + case _ => + // do nothing + // TODO: Once input features marked as categorical are handled correctly, check that here. + } + } + } +} + +private[feature] object VectorIndexerSuite { + @BeanInfo + case class FeatureData(@BeanProperty features: Vector) +} 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 new file mode 100644 index 0000000000000..03ba86670d453 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -0,0 +1,63 @@ +/* + * 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 org.scalatest.FunSuite + +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, SQLContext} + +class Word2VecSuite extends FunSuite with MLlibTestSparkContext { + + test("Word2Vec") { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val sentence = "a b " * 100 + "a c " * 10 + val numOfWords = sentence.split(" ").size + val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) + + val codes = Map( + "a" -> Array(-0.2811822295188904,-0.6356269121170044,-0.3020961284637451), + "b" -> Array(1.0309048891067505,-1.29472815990448,0.22276712954044342), + "c" -> Array(-0.08456747233867645,0.5137411952018738,0.11731560528278351) + ) + + val expected = doc.map { sentence => + Vectors.dense(sentence.map(codes.apply).reduce((word1, word2) => + word1.zip(word2).map { case (v1, v2) => v1 + v2 } + ).map(_ / numOfWords)) + } + + val docDF = doc.zip(expected).toDF("text", "expected") + + val model = new Word2Vec() + .setVectorSize(3) + .setInputCol("text") + .setOutputCol("result") + .fit(docDF) + + model.transform(docDF).select("result", "expected").collect().foreach { + case Row(vector1: Vector, vector2: Vector) => + assert(vector1 ~== vector2 absTol 1E-5, "Transformed vector is different with expected.") + } + } +} + diff --git a/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala new file mode 100644 index 0000000000000..1505ad872536b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala @@ -0,0 +1,128 @@ +/* + * 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.impl + +import scala.collection.JavaConverters._ + +import org.scalatest.FunSuite + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.tree._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, DataFrame} + + +private[ml] object TreeTests extends FunSuite { + + /** + * Convert the given data to a DataFrame, and set the features and label metadata. + * @param data Dataset. Categorical features and labels must already have 0-based indices. + * This must be non-empty. + * @param categoricalFeatures Map: categorical feature index -> number of distinct values + * @param numClasses Number of classes label can take. If 0, mark as continuous. + * @return DataFrame with metadata + */ + def setMetadata( + data: RDD[LabeledPoint], + categoricalFeatures: Map[Int, Int], + numClasses: Int): DataFrame = { + val sqlContext = new SQLContext(data.sparkContext) + import sqlContext.implicits._ + val df = data.toDF() + val numFeatures = data.first().features.size + val featuresAttributes = Range(0, numFeatures).map { feature => + if (categoricalFeatures.contains(feature)) { + NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) + } else { + NumericAttribute.defaultAttr.withIndex(feature) + } + }.toArray + val featuresMetadata = new AttributeGroup("features", featuresAttributes).toMetadata() + val labelAttribute = if (numClasses == 0) { + NumericAttribute.defaultAttr.withName("label") + } else { + NominalAttribute.defaultAttr.withName("label").withNumValues(numClasses) + } + val labelMetadata = labelAttribute.toMetadata() + df.select(df("features").as("features", featuresMetadata), + df("label").as("label", labelMetadata)) + } + + /** Java-friendly version of [[setMetadata()]] */ + def setMetadata( + data: JavaRDD[LabeledPoint], + categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer], + numClasses: Int): DataFrame = { + setMetadata(data.rdd, categoricalFeatures.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numClasses) + } + + /** + * Check if the two trees are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + * If the trees are not equal, this prints the two trees and throws an exception. + */ + def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + try { + checkEqual(a.rootNode, b.rootNode) + } catch { + case ex: Exception => + throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + "TREE A:\n" + a.toDebugString + "\n" + + "TREE B:\n" + b.toDebugString + "\n", ex) + } + } + + /** + * Return true iff the two nodes and their descendants are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + */ + private def checkEqual(a: Node, b: Node): Unit = { + assert(a.prediction === b.prediction) + assert(a.impurity === b.impurity) + (a, b) match { + case (aye: InternalNode, bee: InternalNode) => + assert(aye.split === bee.split) + checkEqual(aye.leftChild, bee.leftChild) + checkEqual(aye.rightChild, bee.rightChild) + case (aye: LeafNode, bee: LeafNode) => // do nothing + case _ => + throw new AssertionError("Found mismatched nodes") + } + } + + /** + * Check if the two models are exactly the same. + * If the models are not equal, this throws an exception. + */ + def checkEqual(a: TreeEnsembleModel, b: TreeEnsembleModel): Unit = { + try { + a.trees.zip(b.trees).foreach { case (treeA, treeB) => + TreeTests.checkEqual(treeA, treeB) + } + assert(a.treeWeights === b.treeWeights) + } catch { + case ex: Exception => throw new AssertionError( + "checkEqual failed since the two tree ensembles were not identical") + } + } +} 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 1ce2987612378..6056e7d3f6ff8 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 @@ -21,19 +21,33 @@ import org.scalatest.FunSuite class ParamsSuite extends FunSuite { - val solver = new TestParams() - import solver.{inputCol, maxIter} - test("param") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + assert(maxIter.name === "maxIter") - assert(maxIter.doc === "max number of iterations") - assert(maxIter.defaultValue.get === 100) + assert(maxIter.doc === "max number of iterations (>= 0)") assert(maxIter.parent.eq(solver)) - assert(maxIter.toString === "maxIter: max number of iterations (default: 100)") - assert(inputCol.defaultValue === None) + assert(maxIter.toString === "maxIter: max number of iterations (>= 0) (default: 10)") + assert(!maxIter.isValid(-1)) + assert(maxIter.isValid(0)) + assert(maxIter.isValid(1)) + + solver.setMaxIter(5) + assert(maxIter.toString === + "maxIter: max number of iterations (>= 0) (default: 10, current: 5)") + + assert(inputCol.toString === "inputCol: input column name (undefined)") + + intercept[IllegalArgumentException] { + solver.setMaxIter(-1) + } } test("param pair") { + val solver = new TestParams() + import solver.maxIter + val pair0 = maxIter -> 5 val pair1 = maxIter.w(5) val pair2 = ParamPair(maxIter, 5) @@ -41,16 +55,24 @@ class ParamsSuite extends FunSuite { assert(pair.param.eq(maxIter)) assert(pair.value === 5) } + intercept[IllegalArgumentException] { + val pair = maxIter -> -1 + } } test("param map") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val map0 = ParamMap.empty assert(!map0.contains(maxIter)) - assert(map0(maxIter) === maxIter.defaultValue.get) map0.put(maxIter, 10) assert(map0.contains(maxIter)) assert(map0(maxIter) === 10) + intercept[IllegalArgumentException] { + map0.put(maxIter, -1) + } assert(!map0.contains(inputCol)) intercept[NoSuchElementException] { @@ -78,31 +100,100 @@ class ParamsSuite extends FunSuite { } test("params") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val params = solver.params - assert(params.size === 2) + assert(params.length === 2) assert(params(0).eq(inputCol), "params must be ordered by name") assert(params(1).eq(maxIter)) + + assert(!solver.isSet(maxIter)) + assert(solver.isDefined(maxIter)) + assert(solver.getMaxIter === 10) + solver.setMaxIter(100) + assert(solver.isSet(maxIter)) + assert(solver.getMaxIter === 100) + assert(!solver.isSet(inputCol)) + assert(!solver.isDefined(inputCol)) + intercept[NoSuchElementException](solver.getInputCol) + assert(solver.explainParams() === Seq(inputCol, maxIter).mkString("\n")) + assert(solver.getParam("inputCol").eq(inputCol)) assert(solver.getParam("maxIter").eq(maxIter)) - intercept[NoSuchMethodException] { + assert(solver.hasParam("inputCol")) + assert(!solver.hasParam("abc")) + intercept[NoSuchElementException] { solver.getParam("abc") } - assert(!solver.isSet(inputCol)) + intercept[IllegalArgumentException] { - solver.validate() + solver.validateParams() } - solver.validate(ParamMap(inputCol -> "input")) + solver.validateParams(ParamMap(inputCol -> "input")) solver.setInputCol("input") assert(solver.isSet(inputCol)) + assert(solver.isDefined(inputCol)) assert(solver.getInputCol === "input") - solver.validate() + solver.validateParams() intercept[IllegalArgumentException] { - solver.validate(ParamMap(maxIter -> -10)) + ParamMap(maxIter -> -10) } - solver.setMaxIter(-10) intercept[IllegalArgumentException] { - solver.validate() + solver.setMaxIter(-10) } + + solver.clearMaxIter() + assert(!solver.isSet(maxIter)) + + val copied = solver.copy(ParamMap(solver.maxIter -> 50)) + assert(copied.uid !== solver.uid) + assert(copied.getInputCol === solver.getInputCol) + assert(copied.getMaxIter === 50) + } + + test("ParamValidate") { + val alwaysTrue = ParamValidators.alwaysTrue[Int] + assert(alwaysTrue(1)) + + val gt1Int = ParamValidators.gt[Int](1) + assert(!gt1Int(1) && gt1Int(2)) + val gt1Double = ParamValidators.gt[Double](1) + assert(!gt1Double(1.0) && gt1Double(1.1)) + + val gtEq1Int = ParamValidators.gtEq[Int](1) + assert(!gtEq1Int(0) && gtEq1Int(1)) + val gtEq1Double = ParamValidators.gtEq[Double](1) + assert(!gtEq1Double(0.9) && gtEq1Double(1.0)) + + val lt1Int = ParamValidators.lt[Int](1) + assert(lt1Int(0) && !lt1Int(1)) + val lt1Double = ParamValidators.lt[Double](1) + assert(lt1Double(0.9) && !lt1Double(1.0)) + + val ltEq1Int = ParamValidators.ltEq[Int](1) + assert(ltEq1Int(1) && !ltEq1Int(2)) + val ltEq1Double = ParamValidators.ltEq[Double](1) + assert(ltEq1Double(1.0) && !ltEq1Double(1.1)) + + val inRange02IntInclusive = ParamValidators.inRange[Int](0, 2) + assert(inRange02IntInclusive(0) && inRange02IntInclusive(1) && inRange02IntInclusive(2) && + !inRange02IntInclusive(-1) && !inRange02IntInclusive(3)) + val inRange02IntExclusive = + ParamValidators.inRange[Int](0, 2, lowerInclusive = false, upperInclusive = false) + assert(!inRange02IntExclusive(0) && inRange02IntExclusive(1) && !inRange02IntExclusive(2)) + + val inRange02DoubleInclusive = ParamValidators.inRange[Double](0, 2) + assert(inRange02DoubleInclusive(0) && inRange02DoubleInclusive(1) && + inRange02DoubleInclusive(2) && + !inRange02DoubleInclusive(-0.1) && !inRange02DoubleInclusive(2.1)) + val inRange02DoubleExclusive = + ParamValidators.inRange[Double](0, 2, lowerInclusive = false, upperInclusive = false) + assert(!inRange02DoubleExclusive(0) && inRange02DoubleExclusive(1) && + !inRange02DoubleExclusive(2)) + + val inArray = ParamValidators.inArray[Int](Array(1, 2)) + assert(inArray(1) && inArray(2) && !inArray(0)) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 1a65883d78a71..dc16073640407 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -17,20 +17,25 @@ package org.apache.spark.ml.param +import org.apache.spark.ml.param.shared.{HasInputCol, HasMaxIter} + /** A subclass of Params for testing. */ -class TestParams extends Params { +class TestParams extends Params with HasMaxIter with HasInputCol { - val maxIter = new IntParam(this, "maxIter", "max number of iterations", Some(100)) def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = get(maxIter) - val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = get(inputCol) - override def validate(paramMap: ParamMap) = { - val m = this.paramMap ++ paramMap - require(m(maxIter) >= 0) - require(m.contains(inputCol)) + setDefault(maxIter -> 10) + + def clearMaxIter(): this.type = clear(maxIter) + + override def validateParams(): Unit = { + super.validateParams() + require(isDefined(inputCol)) + } + + override def copy(extra: ParamMap): TestParams = { + super.copy(extra).asInstanceOf[TestParams] } } 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 0bb06e9e8ac9c..fc7349330cf86 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 @@ -22,6 +22,7 @@ import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.scalatest.FunSuite @@ -68,39 +69,42 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { } } - test("normal equation construction with explict feedback") { + test("normal equation construction") { val k = 2 val ne0 = new NormalEquation(k) - .add(Array(1.0f, 2.0f), 3.0f) - .add(Array(4.0f, 5.0f), 6.0f) + .add(Array(1.0f, 2.0f), 3.0) + .add(Array(4.0f, 5.0f), 6.0, 2.0) // weighted assert(ne0.k === k) assert(ne0.triK === k * (k + 1) / 2) - assert(ne0.n === 2) // NumPy code that computes the expected values: // A = np.matrix("1 2; 4 5") // b = np.matrix("3; 6") - // ata = A.transpose() * A - // atb = A.transpose() * b - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(17.0, 22.0, 29.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(27.0, 36.0) relTol 1e-8) + // C = np.matrix(np.diag([1, 2])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(33.0, 42.0, 54.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(51.0, 66.0) relTol 1e-8) val ne1 = new NormalEquation(2) - .add(Array(7.0f, 8.0f), 9.0f) + .add(Array(7.0f, 8.0f), 9.0) ne0.merge(ne1) - assert(ne0.n === 3) // NumPy code that computes the expected values: // A = np.matrix("1 2; 4 5; 7 8") // b = np.matrix("3; 6; 9") - // ata = A.transpose() * A - // atb = A.transpose() * b - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(66.0, 78.0, 93.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(90.0, 108.0) relTol 1e-8) + // C = np.matrix(np.diag([1, 2, 1])) + // ata = A.transpose() * C * A + // atb = A.transpose() * C * b + assert(Vectors.dense(ne0.ata) ~== Vectors.dense(82.0, 98.0, 118.0) relTol 1e-8) + assert(Vectors.dense(ne0.atb) ~== Vectors.dense(114.0, 138.0) relTol 1e-8) intercept[IllegalArgumentException] { - ne0.add(Array(1.0f), 2.0f) + ne0.add(Array(1.0f), 2.0) } intercept[IllegalArgumentException] { - ne0.add(Array(1.0f, 2.0f, 3.0f), 4.0f) + ne0.add(Array(1.0f, 2.0f, 3.0f), 4.0) + } + intercept[IllegalArgumentException] { + ne0.add(Array(1.0f, 2.0f), 0.0, -1.0) } intercept[IllegalArgumentException] { val ne2 = new NormalEquation(3) @@ -108,41 +112,16 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { } ne0.reset() - assert(ne0.n === 0) assert(ne0.ata.forall(_ == 0.0)) assert(ne0.atb.forall(_ == 0.0)) } - test("normal equation construction with implicit feedback") { - val k = 2 - val alpha = 0.5 - val ne0 = new NormalEquation(k) - .addImplicit(Array(-5.0f, -4.0f), -3.0f, alpha) - .addImplicit(Array(-2.0f, -1.0f), 0.0f, alpha) - .addImplicit(Array(1.0f, 2.0f), 3.0f, alpha) - assert(ne0.k === k) - assert(ne0.triK === k * (k + 1) / 2) - assert(ne0.n === 0) // addImplicit doesn't increase the count. - // NumPy code that computes the expected values: - // alpha = 0.5 - // A = np.matrix("-5 -4; -2 -1; 1 2") - // b = np.matrix("-3; 0; 3") - // b1 = b > 0 - // c = 1.0 + alpha * np.abs(b) - // C = np.diag(c.A1) - // I = np.eye(3) - // ata = A.transpose() * (C - I) * A - // atb = A.transpose() * C * b1 - assert(Vectors.dense(ne0.ata) ~== Vectors.dense(39.0, 33.0, 30.0) relTol 1e-8) - assert(Vectors.dense(ne0.atb) ~== Vectors.dense(2.5, 5.0) relTol 1e-8) - } - test("CholeskySolver") { val k = 2 val ne0 = new NormalEquation(k) - .add(Array(1.0f, 2.0f), 4.0f) - .add(Array(1.0f, 3.0f), 9.0f) - .add(Array(1.0f, 4.0f), 16.0f) + .add(Array(1.0f, 2.0f), 4.0) + .add(Array(1.0f, 3.0f), 9.0) + .add(Array(1.0f, 4.0f), 16.0) val ne1 = new NormalEquation(k) .merge(ne0) @@ -154,13 +133,12 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { // x0 = np.linalg.lstsq(A, b)[0] assert(Vectors.dense(x0) ~== Vectors.dense(-8.333333, 6.0) relTol 1e-6) - assert(ne0.n === 0) assert(ne0.ata.forall(_ == 0.0)) assert(ne0.atb.forall(_ == 0.0)) - val x1 = chol.solve(ne1, 0.5).map(_.toDouble) + val x1 = chol.solve(ne1, 1.5).map(_.toDouble) // NumPy code that computes the expected solution, where lambda is scaled by n: - // x0 = np.linalg.solve(A.transpose() * A + 0.5 * 3 * np.eye(2), A.transpose() * b) + // x0 = np.linalg.solve(A.transpose() * A + 1.5 * np.eye(2), A.transpose() * b) assert(Vectors.dense(x1) ~== Vectors.dense(-0.1155556, 3.28) relTol 1e-6) } 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 new file mode 100644 index 0000000000000..5aa81b44ddaf9 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -0,0 +1,91 @@ +/* + * 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.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, + DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +class DecisionTreeRegressorSuite extends FunSuite with MLlibTestSparkContext { + + import DecisionTreeRegressorSuite.compareAPIs + + private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + categoricalDataPointsRDD = + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + test("Regression stump with 3-ary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 3, 1-> 3) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + test("Regression stump with binary (ordered) categorical features") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + val categoricalFeatures = Map(0 -> 2, 1-> 2) + compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: test("model save/load") SPARK-6725 +} + +private[ml] object DecisionTreeRegressorSuite extends FunSuite { + + /** + * Train 2 decision trees on the given dataset, one using the old API and one using the new API. + * Convert the old tree to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + dt: DecisionTreeRegressor, + categoricalFeatures: Map[Int, Int]): Unit = { + val oldStrategy = dt.getOldStrategy(categoricalFeatures) + val oldTree = OldDecisionTree.train(data, oldStrategy) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) + val newTree = dt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldTreeAsNew = DecisionTreeRegressionModel.fromOld( + oldTree, newTree.parent, categoricalFeatures) + TreeTests.checkEqual(oldTreeAsNew, newTree) + } +} 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 new file mode 100644 index 0000000000000..25b36ab08b67c --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -0,0 +1,136 @@ +/* + * 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.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +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 +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * Test suite for [[GBTRegressor]]. + */ +class GBTRegressorSuite extends FunSuite with MLlibTestSparkContext { + + import GBTRegressorSuite.compareAPIs + + // Combinations for estimators, learning rates and subsamplingRate + private val testCombinations = + Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) + + private var data: RDD[LabeledPoint] = _ + private var trainData: RDD[LabeledPoint] = _ + private var validationData: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + trainData = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + validationData = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + } + + test("Regression with continuous features: SquaredError") { + val categoricalFeatures = Map.empty[Int, Int] + GBTRegressor.supportedLossTypes.foreach { loss => + testCombinations.foreach { + case (maxIter, learningRate, subsamplingRate) => + val gbt = new GBTRegressor() + .setMaxDepth(2) + .setSubsamplingRate(subsamplingRate) + .setLossType(loss) + .setMaxIter(maxIter) + .setStepSize(learningRate) + compareAPIs(data, None, gbt, categoricalFeatures) + } + } + } + + // TODO: Reinstate test once runWithValidation is implemented SPARK-7132 + /* + test("runWithValidation stops early and performs better on a validation dataset") { + val categoricalFeatures = Map.empty[Int, Int] + // Set maxIter large enough so that it stops early. + val maxIter = 20 + GBTRegressor.supportedLossTypes.foreach { loss => + val gbt = new GBTRegressor() + .setMaxIter(maxIter) + .setMaxDepth(2) + .setLossType(loss) + .setValidationTol(0.0) + compareAPIs(trainData, None, gbt, categoricalFeatures) + compareAPIs(trainData, Some(validationData), gbt, categoricalFeatures) + } + } + */ + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented SPARK-6725 + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray + val treeWeights = Array(0.1, 0.3, 1.1) + val oldModel = new OldGBTModel(OldAlgo.Regression, trees, treeWeights) + val newModel = GBTRegressionModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = GBTRegressionModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private object GBTRegressorSuite { + + /** + * Train 2 models on the given dataset, one using the old API and one using the new API. + * Convert the old model to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + validationData: Option[RDD[LabeledPoint]], + gbt: GBTRegressor, + categoricalFeatures: Map[Int, Int]): Unit = { + val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) + val oldGBT = new OldGBT(oldBoostingStrategy) + val oldModel = oldGBT.run(data) + val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) + val newModel = gbt.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldModelAsNew = GBTRegressionModel.fromOld(oldModel, newModel.parent, categoricalFeatures) + TreeTests.checkEqual(oldModelAsNew, newModel) + } +} 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 bbb44c3e2dfc2..80323ef5201a6 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 @@ -19,47 +19,149 @@ package org.apache.spark.ml.regression import org.scalatest.FunSuite -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.mllib.linalg.DenseVector +import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{Row, SQLContext, DataFrame} class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { @transient var sqlContext: SQLContext = _ @transient var dataset: DataFrame = _ + /** + * In `LinearRegressionSuite`, we will make sure that the model trained by SparkML + * is the same as the one trained by R's glmnet package. The following instruction + * describes how to reproduce the data in R. + * + * import org.apache.spark.mllib.util.LinearDataGenerator + * val data = + * sc.parallelize(LinearDataGenerator.generateLinearInput(6.3, Array(4.7, 7.2), 10000, 42), 2) + * data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1)).saveAsTextFile("path") + */ override def beforeAll(): Unit = { super.beforeAll() sqlContext = new SQLContext(sc) dataset = sqlContext.createDataFrame( - sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) + sc.parallelize(LinearDataGenerator.generateLinearInput( + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 10000, 42, 0.1), 2)) } - test("linear regression: default params") { - val lr = new LinearRegression - assert(lr.getLabelCol == "label") - val model = lr.fit(dataset) - model.transform(dataset) - .select("label", "prediction") - .collect() - // Check defaults - assert(model.getFeaturesCol == "features") - assert(model.getPredictionCol == "prediction") + test("linear regression with intercept without regularization") { + val trainer = new LinearRegression + val model = trainer.fit(dataset) + + /** + * Using the following R code to load the data and train the model using glmnet package. + * + * library("glmnet") + * data <- read.csv("path", header=FALSE, stringsAsFactors=FALSE) + * features <- as.matrix(data.frame(as.numeric(data$V2), as.numeric(data$V3))) + * label <- as.numeric(data$V1) + * weights <- coef(glmnet(features, label, family="gaussian", alpha = 0, lambda = 0)) + * > weights + * 3 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * (Intercept) 6.300528 + * as.numeric.data.V2. 4.701024 + * as.numeric.data.V3. 7.198257 + */ + val interceptR = 6.298698 + val weightsR = Array(4.700706, 7.199082) + + assert(model.intercept ~== interceptR relTol 1E-3) + assert(model.weights(0) ~== weightsR(0) relTol 1E-3) + assert(model.weights(1) ~== weightsR(1) relTol 1E-3) + + model.transform(dataset).select("features", "prediction").collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model.weights(0) + features(1) * model.weights(1) + model.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) + } + } + + test("linear regression with intercept with L1 regularization") { + val trainer = (new LinearRegression).setElasticNetParam(1.0).setRegParam(0.57) + val model = trainer.fit(dataset) + + /** + * weights <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, lambda = 0.57)) + * > weights + * 3 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * (Intercept) 6.311546 + * as.numeric.data.V2. 2.123522 + * as.numeric.data.V3. 4.605651 + */ + val interceptR = 6.243000 + val weightsR = Array(4.024821, 6.679841) + + assert(model.intercept ~== interceptR relTol 1E-3) + assert(model.weights(0) ~== weightsR(0) relTol 1E-3) + assert(model.weights(1) ~== weightsR(1) relTol 1E-3) + + model.transform(dataset).select("features", "prediction").collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model.weights(0) + features(1) * model.weights(1) + model.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) + } } - test("linear regression with setters") { - // Set params, train, and check as many as we can. - val lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(1.0) - val model = lr.fit(dataset) - assert(model.fittingParamMap.get(lr.maxIter).get === 10) - assert(model.fittingParamMap.get(lr.regParam).get === 1.0) - - // Call fit() with new params, and check as many as we can. - val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.predictionCol -> "thePred") - assert(model2.fittingParamMap.get(lr.maxIter).get === 5) - assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) - assert(model2.getPredictionCol == "thePred") + test("linear regression with intercept with L2 regularization") { + val trainer = (new LinearRegression).setElasticNetParam(0.0).setRegParam(2.3) + val model = trainer.fit(dataset) + + /** + * weights <- coef(glmnet(features, label, family="gaussian", alpha = 0.0, lambda = 2.3)) + * > weights + * 3 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * (Intercept) 6.328062 + * as.numeric.data.V2. 3.222034 + * as.numeric.data.V3. 4.926260 + */ + val interceptR = 5.269376 + val weightsR = Array(3.736216, 5.712356) + + assert(model.intercept ~== interceptR relTol 1E-3) + assert(model.weights(0) ~== weightsR(0) relTol 1E-3) + assert(model.weights(1) ~== weightsR(1) relTol 1E-3) + + model.transform(dataset).select("features", "prediction").collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model.weights(0) + features(1) * model.weights(1) + model.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) + } + } + + test("linear regression with intercept with ElasticNet regularization") { + val trainer = (new LinearRegression).setElasticNetParam(0.3).setRegParam(1.6) + val model = trainer.fit(dataset) + + /** + * weights <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, lambda = 1.6)) + * > weights + * 3 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * (Intercept) 6.324108 + * as.numeric.data.V2. 3.168435 + * as.numeric.data.V3. 5.200403 + */ + val interceptR = 5.696056 + val weightsR = Array(3.670489, 6.001122) + + assert(model.intercept ~== interceptR relTol 1E-3) + assert(model.weights(0) ~== weightsR(0) relTol 1E-3) + assert(model.weights(1) ~== weightsR(1) relTol 1E-3) + + model.transform(dataset).select("features", "prediction").collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model.weights(0) + features(1) * model.weights(1) + model.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) + } } } 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 new file mode 100644 index 0000000000000..45f09f4fdab81 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -0,0 +1,122 @@ +/* + * 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.regression + +import org.scalatest.FunSuite + +import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.mllib.regression.LabeledPoint +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 +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + + +/** + * Test suite for [[RandomForestRegressor]]. + */ +class RandomForestRegressorSuite extends FunSuite with MLlibTestSparkContext { + + import RandomForestRegressorSuite.compareAPIs + + private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ + + override def beforeAll() { + super.beforeAll() + orderedLabeledPoints50_1000 = + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// + + def regressionTestWithContinuousFeatures(rf: RandomForestRegressor) { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val newRF = rf + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(10) + .setNumTrees(1) + .setFeatureSubsetStrategy("auto") + .setSeed(123) + compareAPIs(orderedLabeledPoints50_1000, newRF, categoricalFeaturesInfo) + } + + test("Regression with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val rf = new RandomForestRegressor() + regressionTestWithContinuousFeatures(rf) + } + + test("Regression with continuous features and node Id cache :" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val rf = new RandomForestRegressor() + .setCacheNodeIds(true) + regressionTestWithContinuousFeatures(rf) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// + + // TODO: Reinstate test once save/load are implemented SPARK-6725 + /* + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray + val oldModel = new OldRandomForestModel(OldAlgo.Regression, trees) + val newModel = RandomForestRegressionModel.fromOld(oldModel) + + // Save model, load it back, and compare. + try { + newModel.save(sc, path) + val sameNewModel = RandomForestRegressionModel.load(sc, path) + TreeTests.checkEqual(newModel, sameNewModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + */ +} + +private object RandomForestRegressorSuite extends FunSuite { + + /** + * Train 2 models on the given dataset, one using the old API and one using the new API. + * Convert the old model to the new format, compare them, and fail if they are not exactly equal. + */ + def compareAPIs( + data: RDD[LabeledPoint], + rf: RandomForestRegressor, + categoricalFeatures: Map[Int, Int]): Unit = { + 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 newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) + val newModel = rf.fit(newData) + // Use parent, fittingParamMap from newTree since these are not checked anyways. + val oldModelAsNew = RandomForestRegressionModel.fromOld( + oldModel, newModel.parent, categoricalFeatures) + TreeTests.checkEqual(oldModelAsNew, newModel) + } +} 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 761ea821ef7c6..05313d440fbf6 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 @@ -49,8 +49,8 @@ class CrossValidatorSuite extends FunSuite with MLlibTestSparkContext { .setEvaluator(eval) .setNumFolds(3) val cvModel = cv.fit(dataset) - val bestParamMap = cvModel.bestModel.fittingParamMap - assert(bestParamMap(lr.regParam) === 0.001) - assert(bestParamMap(lr.maxIter) === 10) + val parent = cvModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(parent.getRegParam === 0.001) + assert(parent.getMaxIter === 10) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index db8ed62fa46ce..a629dba8a426f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors} +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors, SparseMatrix} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.recommendation.Rating @@ -77,6 +77,16 @@ class PythonMLLibAPISuite extends FunSuite { val emptyMatrix = Matrices.dense(0, 0, empty) val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) + + val sm = new SparseMatrix(3, 2, Array(0, 1, 3), Array(1, 0, 2), Array(0.9, 1.2, 3.4)) + val nsm = SerDe.loads(SerDe.dumps(sm)).asInstanceOf[SparseMatrix] + assert(sm.toArray === nsm.toArray) + + val smt = new SparseMatrix( + 3, 3, Array(0, 2, 3, 5), Array(0, 2, 1, 0, 2), Array(0.9, 1.2, 3.4, 5.7, 8.9), + isTransposed=true) + val nsmt = SerDe.loads(SerDe.dumps(smt)).asInstanceOf[SparseMatrix] + assert(smt.toArray === nsmt.toArray) } test("pickle rating") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index aaa81da9e273c..a26c52852c4d7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -425,6 +425,12 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M val model = lr.run(testRDD) + val numFeatures = testRDD.map(_.features.size).first() + val initialWeights = Vectors.dense(new Array[Double]((numFeatures + 1) * 2)) + val model2 = lr.run(testRDD, initialWeights) + + LogisticRegressionSuite.checkModelsEqual(model, model2) + /** * The following is the instruction to reproduce the model using R's glmnet package. * diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 5a27c7d2309c5..ea89b17b7c08f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.mllib.classification import scala.util.Random +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum, Axis} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} + import org.scalatest.FunSuite import org.apache.spark.SparkException @@ -41,37 +44,48 @@ object NaiveBayesSuite { // Generate input of the form Y = (theta * x).argmax() def generateNaiveBayesInput( - pi: Array[Double], // 1XC - theta: Array[Array[Double]], // CXD - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { + 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 = Array.tabulate[Double](D) { j => - if (rnd.nextDouble() < _theta(y)(j)) 1 else 0 + 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"NaiveBayesSuite found unknown ModelType: $modelType") } LabeledPoint(y, Vectors.dense(xi)) } } - private val smallPi = Array(0.5, 0.3, 0.2).map(math.log) + /** Bernoulli NaiveBayes with binary labels, 3 features */ + private val binaryBernoulliModel = new NaiveBayesModel(labels = Array(0.0, 1.0), + pi = Array(0.2, 0.8), theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4)), + "Bernoulli") - private val smallTheta = Array( - Array(0.91, 0.03, 0.03, 0.03), // label 0 - Array(0.03, 0.91, 0.03, 0.03), // label 1 - Array(0.03, 0.03, 0.91, 0.03) // label 2 - ).map(_.map(math.log)) - - /** Binary labels, 3 features */ - private val binaryModel = new NaiveBayesModel(labels = Array(0.0, 1.0), pi = Array(0.2, 0.8), - theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4))) + /** Multinomial NaiveBayes with binary labels, 3 features */ + private val binaryMultinomialModel = new NaiveBayesModel(labels = Array(0.0, 1.0), + pi = Array(0.2, 0.8), theta = Array(Array(0.1, 0.3, 0.6), Array(0.2, 0.4, 0.4)), + "Multinomial") } class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { @@ -85,6 +99,24 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(numOfPredictions < input.length / 5) } + def validateModelFit( + piData: Array[Double], + thetaData: Array[Array[Double]], + model: NaiveBayesModel): Unit = { + def closeFit(d1: Double, d2: Double, precision: Double): Boolean = { + (d1 - d2).abs <= precision + } + val modelIndex = (0 until piData.length).zip(model.labels.map(_.toInt)) + for (i <- modelIndex) { + assert(closeFit(math.exp(piData(i._2)), math.exp(model.pi(i._1)), 0.05)) + } + for (i <- modelIndex) { + for (j <- 0 until thetaData(i._2).length) { + assert(closeFit(math.exp(thetaData(i._2)(j)), math.exp(model.theta(i._1)(j)), 0.05)) + } + } + } + test("get, set params") { val nb = new NaiveBayes() nb.setLambda(2.0) @@ -93,19 +125,53 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(nb.getLambda === 3.0) } - test("Naive Bayes") { - val nPoints = 10000 + test("Naive Bayes Multinomial") { + val nPoints = 1000 + val pi = Array(0.5, 0.1, 0.4).map(math.log) + val theta = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0 + Array(0.10, 0.70, 0.10, 0.10), // label 1 + Array(0.10, 0.10, 0.70, 0.10) // label 2 + ).map(_.map(math.log)) + + val testData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 42, "Multinomial") + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val model = NaiveBayes.train(testRDD, 1.0, "Multinomial") + validateModelFit(pi, theta, model) + + val validationData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 17, "Multinomial") + val validationRDD = sc.parallelize(validationData, 2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) - val pi = NaiveBayesSuite.smallPi - val theta = NaiveBayesSuite.smallTheta + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } - val testData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 42) + test("Naive Bayes Bernoulli") { + val nPoints = 10000 + val pi = Array(0.5, 0.3, 0.2).map(math.log) + val theta = Array( + Array(0.50, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.40), // label 0 + Array(0.02, 0.70, 0.10, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02), // label 1 + Array(0.02, 0.02, 0.60, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.02, 0.30) // label 2 + ).map(_.map(math.log)) + + val testData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 45, "Bernoulli") val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val model = NaiveBayes.train(testRDD) + val model = NaiveBayes.train(testRDD, 1.0, "Bernoulli") + validateModelFit(pi, theta, model) - val validationData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 17) + val validationData = NaiveBayesSuite.generateNaiveBayesInput( + pi, theta, nPoints, 20, "Bernoulli") val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. @@ -142,19 +208,41 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { } } - test("model save/load") { - val model = NaiveBayesSuite.binaryModel + test("model save/load: 2.0 to 2.0") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + Seq(NaiveBayesSuite.binaryBernoulliModel, NaiveBayesSuite.binaryMultinomialModel).map { + model => + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = NaiveBayesModel.load(sc, path) + assert(model.labels === sameModel.labels) + assert(model.pi === sameModel.pi) + assert(model.theta === sameModel.theta) + assert(model.modelType === sameModel.modelType) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } + + test("model save/load: 1.0 to 2.0") { + val model = NaiveBayesSuite.binaryMultinomialModel val tempDir = Utils.createTempDir() val path = tempDir.toURI.toString - // Save model, load it back, and compare. + // Save model as version 1.0, load it back, and compare. try { - model.save(sc, path) + val data = NaiveBayesModel.SaveLoadV1_0.Data(model.labels, model.pi, model.theta) + NaiveBayesModel.SaveLoadV1_0.save(sc, path, data) val sameModel = NaiveBayesModel.load(sc, path) assert(model.labels === sameModel.labels) assert(model.pi === sameModel.pi) assert(model.theta === sameModel.theta) + assert(model.modelType === "Multinomial") } finally { Utils.deleteRecursively(tempDir) } @@ -172,8 +260,8 @@ class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) } } - // If we serialize data directly in the task closure, the size of the serialized task would be - // greater than 1MB and hence Spark would throw an error. + // If we serialize data directly in the task closure, the size of the serialized task + // would be greater than 1MB and hence Spark would throw an error. val model = NaiveBayes.train(examples) val predictions = model.predict(examples.map(_.features)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index 8b3e6e5ce9249..5683b55e8500a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 // Test if we can accurately learn B for Y = logistic(BX) on streaming data test("parameter accuracy") { @@ -132,4 +132,31 @@ class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { assert(errors.forall(x => x <= 0.4)) } + // Test training combined with prediction + test("training and prediction") { + // create model initialized with zero weights + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.dense(-0.1)) + .setStepSize(0.01) + .setNumIterations(10) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LogisticRegressionSuite.generateLogisticInput(0.0, 5.0, nPoints, 42 * (i + 1)) + } + + // train and predict + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // assert that prediction error improves, ensuring that the updated model is being used + val error = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints).toList + assert(error.head > 0.8 & error.last < 0.2) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index 1b46a4012d731..f356ffa3e3a26 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Matrices} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { test("single cluster") { @@ -48,13 +49,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { } test("two clusters") { - val data = sc.parallelize(Array( - Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), - Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), - Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), - Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), - Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) - )) + val data = sc.parallelize(GaussianTestData.data) // we set an initial gaussian to induce expected results val initialGmm = new GaussianMixtureModel( @@ -105,14 +100,7 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { } test("two clusters with sparse data") { - val data = sc.parallelize(Array( - Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), - Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), - Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), - Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), - Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) - )) - + val data = sc.parallelize(GaussianTestData.data) val sparseData = data.map(point => Vectors.sparse(1, Array(0), point.toArray)) // we set an initial gaussian to induce expected results val initialGmm = new GaussianMixtureModel( @@ -138,4 +126,38 @@ class GaussianMixtureSuite extends FunSuite with MLlibTestSparkContext { assert(sparseGMM.gaussians(0).sigma ~== Esigma(0) absTol 1E-3) assert(sparseGMM.gaussians(1).sigma ~== Esigma(1) absTol 1E-3) } + + test("model save / load") { + val data = sc.parallelize(GaussianTestData.data) + + val gmm = new GaussianMixture().setK(2).setSeed(0).run(data) + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + try { + gmm.save(sc, path) + + // TODO: GaussianMixtureModel should implement equals/hashcode directly. + val sameModel = GaussianMixtureModel.load(sc, path) + assert(sameModel.k === gmm.k) + (0 until sameModel.k).foreach { i => + assert(sameModel.gaussians(i).mu === gmm.gaussians(i).mu) + assert(sameModel.gaussians(i).sigma === gmm.gaussians(i).sigma) + } + } finally { + Utils.deleteRecursively(tempDir) + } + } + + object GaussianTestData { + + val data = Array( + Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), + Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), + Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), + Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), + Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) + ) + + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 7bf250eb5a383..0f2b26d462ad2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -199,9 +199,13 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext { test("k-means|| initialization") { case class VectorWithCompare(x: Vector) extends Ordered[VectorWithCompare] { - @Override def compare(that: VectorWithCompare): Int = { - if(this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > - that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) -1 else 1 + override def compare(that: VectorWithCompare): Int = { + if (this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > + that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) { + -1 + } else { + 1 + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 302d751eb8a94..2dcc881f5abd2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.mllib.clustering +import breeze.linalg.{DenseMatrix => BDM} + import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} +import org.apache.spark.mllib.linalg.{Vector, DenseMatrix, Matrix, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -37,7 +39,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Check: describeTopics() with all terms val fullTopicSummary = model.describeTopics() - assert(fullTopicSummary.size === tinyK) + assert(fullTopicSummary.length === tinyK) fullTopicSummary.zip(tinyTopicDescription).foreach { case ((algTerms, algTermWeights), (terms, termWeights)) => assert(algTerms === terms) @@ -54,7 +56,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { } } - test("running and DistributedLDAModel") { + test("running and DistributedLDAModel with default Optimizer (EM)") { val k = 3 val topicSmoothing = 1.2 val termSmoothing = 1.2 @@ -68,7 +70,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { .setSeed(12345) val corpus = sc.parallelize(tinyCorpus, 2) - val model: DistributedLDAModel = lda.run(corpus) + val model: DistributedLDAModel = lda.run(corpus).asInstanceOf[DistributedLDAModel] // Check: basic parameters val localModel = model.toLocal @@ -99,7 +101,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Check: per-doc topic distributions val topicDistributions = model.topicDistributions.collect() // Ensure all documents are covered. - assert(topicDistributions.size === tinyCorpus.size) + assert(topicDistributions.length === tinyCorpus.length) assert(tinyCorpus.map(_._1).toSet === topicDistributions.map(_._1).toSet) // Ensure we have proper distributions topicDistributions.foreach { case (docId, topicDistribution) => @@ -123,6 +125,95 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { assert(termVertexIds.map(i => LDA.index2term(i.toLong)) === termIds) assert(termVertexIds.forall(i => LDA.isTermVertex((i.toLong, 0)))) } + + test("setter alias") { + val lda = new LDA().setAlpha(2.0).setBeta(3.0) + assert(lda.getAlpha === 2.0) + assert(lda.getDocConcentration === 2.0) + assert(lda.getBeta === 3.0) + assert(lda.getTopicConcentration === 3.0) + } + + test("OnlineLDAOptimizer initialization") { + val lda = new LDA().setK(2) + val corpus = sc.parallelize(tinyCorpus, 2) + val op = new OnlineLDAOptimizer().initialize(corpus, lda) + op.setKappa(0.9876).setMiniBatchFraction(0.123).setTau_0(567) + assert(op.getAlpha == 0.5) // default 1.0 / k + assert(op.getEta == 0.5) // default 1.0 / k + assert(op.getKappa == 0.9876) + assert(op.getMiniBatchFraction == 0.123) + assert(op.getTau_0 == 567) + } + + test("OnlineLDAOptimizer one iteration") { + // run OnlineLDAOptimizer for 1 iteration to verify it's consistency with Blei-lab, + // [[https://github.com/Blei-Lab/onlineldavb]] + val k = 2 + val vocabSize = 6 + + def docs: Array[(Long, Vector)] = Array( + Vectors.sparse(vocabSize, Array(0, 1, 2), Array(1, 1, 1)), // apple, orange, banana + Vectors.sparse(vocabSize, Array(3, 4, 5), Array(1, 1, 1)) // tiger, cat, dog + ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) } + val corpus = sc.parallelize(docs, 2) + + // Set GammaShape large to avoid the stochastic impact. + val op = new OnlineLDAOptimizer().setTau_0(1024).setKappa(0.51).setGammaShape(1e40) + .setMiniBatchFraction(1) + val lda = new LDA().setK(k).setMaxIterations(1).setOptimizer(op).setSeed(12345) + + val state = op.initialize(corpus, lda) + // override lambda to simulate an intermediate state + // [[ 1.1 1.2 1.3 0.9 0.8 0.7] + // [ 0.9 0.8 0.7 1.1 1.2 1.3]] + op.setLambda(new BDM[Double](k, vocabSize, + Array(1.1, 0.9, 1.2, 0.8, 1.3, 0.7, 0.9, 1.1, 0.8, 1.2, 0.7, 1.3))) + + // run for one iteration + state.submitMiniBatch(corpus) + + // verify the result, Note this generate the identical result as + // [[https://github.com/Blei-Lab/onlineldavb]] + val topic1 = op.getLambda(0, ::).inner.toArray.map("%.4f".format(_)).mkString(", ") + val topic2 = op.getLambda(1, ::).inner.toArray.map("%.4f".format(_)).mkString(", ") + assert("1.1101, 1.2076, 1.3050, 0.8899, 0.7924, 0.6950" == topic1) + assert("0.8899, 0.7924, 0.6950, 1.1101, 1.2076, 1.3050" == topic2) + } + + test("OnlineLDAOptimizer with toy data") { + def toydata: Array[(Long, Vector)] = Array( + Vectors.sparse(6, Array(0, 1), Array(1, 1)), + Vectors.sparse(6, Array(1, 2), Array(1, 1)), + Vectors.sparse(6, Array(0, 2), Array(1, 1)), + Vectors.sparse(6, Array(3, 4), Array(1, 1)), + Vectors.sparse(6, Array(3, 5), Array(1, 1)), + Vectors.sparse(6, Array(4, 5), Array(1, 1)) + ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) } + + val docs = sc.parallelize(toydata) + val op = new OnlineLDAOptimizer().setMiniBatchFraction(1).setTau_0(1024).setKappa(0.51) + .setGammaShape(1e10) + val lda = new LDA().setK(2) + .setDocConcentration(0.01) + .setTopicConcentration(0.01) + .setMaxIterations(100) + .setOptimizer(op) + .setSeed(12345) + + val ldaModel = lda.run(docs) + val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) + val topics = topicIndices.map { case (terms, termWeights) => + terms.zip(termWeights) + } + + // check distribution for each topic, typical distribution is (0.3, 0.3, 0.3, 0.02, 0.02, 0.02) + topics.foreach { topic => + val smalls = topic.filter(t => t._2 < 0.1).map(_._2) + assert(smalls.length == 3 && smalls.sum < 0.2) + } + } + } private[clustering] object LDASuite { @@ -141,7 +232,7 @@ private[clustering] object LDASuite { (terms.toArray, termWeights.toArray) } - def tinyCorpus = Array( + def tinyCorpus: Array[(Long, Vector)] = Array( Vectors.dense(1, 3, 0, 2, 8), Vectors.dense(0, 2, 1, 0, 4), Vectors.dense(2, 3, 12, 3, 1), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 6315c03a700f1..6d6fe6fe46bab 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -18,12 +18,15 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable +import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkContext import org.apache.spark.graphx.{Edge, Graph} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext { @@ -110,4 +113,35 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext assert(x ~== u1(i.toInt) absTol 1e-14) } } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val model = PowerIterationClusteringSuite.createModel(sc, 3, 10) + try { + model.save(sc, path) + val sameModel = PowerIterationClusteringModel.load(sc, path) + PowerIterationClusteringSuite.checkEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} + +object PowerIterationClusteringSuite extends FunSuite { + def createModel(sc: SparkContext, k: Int, nPoints: Int): PowerIterationClusteringModel = { + val assignments = sc.parallelize( + (0 until nPoints).map(p => PowerIterationClustering.Assignment(p, Random.nextInt(k)))) + new PowerIterationClusteringModel(k, assignments) + } + + def checkEqual(a: PowerIterationClusteringModel, b: PowerIterationClusteringModel): Unit = { + assert(a.k === b.k) + + val aAssignments = a.assignments.map(x => (x.id, x.cluster)) + val bAssignments = b.assignments.map(x => (x.id, x.cluster)) + val unequalElements = aAssignments.join(bAssignments).filter { + case (id, (c1, c2)) => c1 != c2 }.count() + assert(unequalElements === 0L) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index 850c9fce507cd..f90025d535e45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.random.XORShiftRandom class StreamingKMeansSuite extends FunSuite with TestSuiteBase { - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 test("accuracy for single center and equivalence to grand average") { // set parameters @@ -59,7 +59,7 @@ class StreamingKMeansSuite extends FunSuite with TestSuiteBase { // estimated center from streaming should exactly match the arithmetic mean of all data points // because the decay factor is set to 1.0 val grandMean = - input.flatten.map(x => x.toBreeze).reduce(_+_) / (numBatches * numPoints).toDouble + input.flatten.map(x => x.toBreeze).reduce(_ + _) / (numBatches * numPoints).toDouble assert(model.latestModel().clusterCenters(0) ~== Vectors.dense(grandMean.toArray) absTol 1E-5) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index 52278690dbd89..98a98a7599bcb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -21,6 +21,9 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils + class Word2VecSuite extends FunSuite with MLlibTestSparkContext { // TODO: add more tests @@ -51,4 +54,27 @@ class Word2VecSuite extends FunSuite with MLlibTestSparkContext { assert(syms(0)._1 == "taiwan") assert(syms(1)._1 == "japan") } + + test("model load / save") { + + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + try { + model.save(sc, path) + val sameModel = Word2VecModel.load(sc, path) + assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + } finally { + Utils.deleteRecursively(tempDir) + } + + } } 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 96f677db3f377..86119ec38101e 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 @@ -436,5 +436,23 @@ class MatricesSuite extends FunSuite { Seq(dm1, dm2, dm3, sm1, sm2, sm3).foreach { mat => assert(mat.toArray === mUDT.deserialize(mUDT.serialize(mat)).toArray) } + assert(mUDT.typeName == "matrix") + assert(mUDT.simpleString == "matrix") + } + + test("toString") { + val empty = Matrices.ones(0, 0) + empty.toString(0, 0) + + val mat = Matrices.rand(5, 10, new Random()) + mat.toString(-1, -5) + mat.toString(0, 0) + mat.toString(Int.MinValue, Int.MinValue) + mat.toString(Int.MaxValue, Int.MaxValue) + var lines = mat.toString(6, 50).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 50)) + + lines = mat.toString(5, 100).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 100)) } } 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 5def899cea117..24755e9ff46fc 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 @@ -187,6 +187,8 @@ class VectorsSuite extends FunSuite { for (v <- Seq(dv0, dv1, sv0, sv1)) { assert(v === udt.deserialize(udt.serialize(v))) } + assert(udt.typeName == "vector") + assert(udt.simpleString == "vector") } test("fromBreeze") { @@ -268,4 +270,48 @@ class VectorsSuite extends FunSuite { assert(Vectors.norm(sv, 3.7) ~== math.pow(sv.toArray.foldLeft(0.0)((a, v) => a + math.pow(math.abs(v), 3.7)), 1.0 / 3.7) relTol 1E-8) } + + test("Vector numActive and numNonzeros") { + val dv = Vectors.dense(0.0, 2.0, 3.0, 0.0) + assert(dv.numActives === 4) + assert(dv.numNonzeros === 2) + + val sv = Vectors.sparse(4, Array(0, 1, 2), Array(0.0, 2.0, 3.0)) + assert(sv.numActives === 3) + assert(sv.numNonzeros === 2) + } + + test("Vector toSparse and toDense") { + val dv0 = Vectors.dense(0.0, 2.0, 3.0, 0.0) + assert(dv0.toDense === dv0) + val dv0s = dv0.toSparse + assert(dv0s.numActives === 2) + assert(dv0s === dv0) + + val sv0 = Vectors.sparse(4, Array(0, 1, 2), Array(0.0, 2.0, 3.0)) + assert(sv0.toDense === sv0) + val sv0s = sv0.toSparse + assert(sv0s.numActives === 2) + assert(sv0s === sv0) + } + + test("Vector.compressed") { + val dv0 = Vectors.dense(1.0, 2.0, 3.0, 0.0) + val dv0c = dv0.compressed.asInstanceOf[DenseVector] + assert(dv0c === dv0) + + val dv1 = Vectors.dense(0.0, 2.0, 0.0, 0.0) + val dv1c = dv1.compressed.asInstanceOf[SparseVector] + assert(dv1 === dv1c) + assert(dv1c.numActives === 1) + + val sv0 = Vectors.sparse(4, Array(1, 2), Array(2.0, 0.0)) + val sv0c = sv0.compressed.asInstanceOf[SparseVector] + assert(sv0 === sv0c) + assert(sv0c.numActives === 1) + + val sv1 = Vectors.sparse(4, Array(0, 1, 2), Array(1.0, 2.0, 3.0)) + val sv1c = sv1.compressed.asInstanceOf[DenseVector] + assert(sv1 === sv1c) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..0b646cf1ce6c4 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import org.dmg.pmml.RegressionModel +import org.dmg.pmml.RegressionNormalizationMethodType +import org.scalatest.FunSuite + +import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.classification.SVMModel +import org.apache.spark.mllib.util.LinearDataGenerator + +class BinaryClassificationPMMLModelExportSuite extends FunSuite { + + test("logistic regression PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val logisticRegressionModel = + new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) + + val logisticModelExport = PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) + + // assert that the PMML format is as expected + assert(logisticModelExport.isInstanceOf[PMMLModelExport]) + val pmml = logisticModelExport.asInstanceOf[PMMLModelExport].getPmml + assert(pmml.getHeader.getDescription === "logistic regression") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === logisticRegressionModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table (for target category 1) + // with the same number of predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getTargetCategory === "1") + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === logisticRegressionModel.weights.size) + // verify if there is a second table with target category 0 and no predictors + assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") + assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) + // ensure logistic regression has normalization method set to LOGIT + assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.LOGIT) + } + + test("linear SVM PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + + // assert that the PMML format is as expected + assert(svmModelExport.isInstanceOf[PMMLModelExport]) + val pmml = svmModelExport.getPmml + assert(pmml.getHeader.getDescription + === "linear SVM") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === svmModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table (for target category 1) + // with the same number of predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getTargetCategory === "1") + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === svmModel.weights.size) + // verify if there is a second table with target category 0 and no predictors + assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") + assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) + // ensure linear SVM has normalization method set to NONE + assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.NONE) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..f9afbd888dfc5 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import org.dmg.pmml.RegressionModel +import org.scalatest.FunSuite + +import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} +import org.apache.spark.mllib.util.LinearDataGenerator + +class GeneralizedLinearPMMLModelExportSuite extends FunSuite { + + test("linear regression PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val linearRegressionModel = + new LinearRegressionModel(linearInput(0).features, linearInput(0).label) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + // assert that the PMML format is as expected + assert(linearModelExport.isInstanceOf[PMMLModelExport]) + val pmml = linearModelExport.getPmml + assert(pmml.getHeader.getDescription === "linear regression") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === linearRegressionModel.weights.size + 1) + // This verifies that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === linearRegressionModel.weights.size) + } + + test("ridge regression PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val ridgeRegressionModel = + new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + // assert that the PMML format is as expected + assert(ridgeModelExport.isInstanceOf[PMMLModelExport]) + val pmml = ridgeModelExport.getPmml + assert(pmml.getHeader.getDescription === "ridge regression") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === ridgeRegressionModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === ridgeRegressionModel.weights.size) + } + + test("lasso PMML export") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) + // assert that the PMML format is as expected + assert(lassoModelExport.isInstanceOf[PMMLModelExport]) + val pmml = lassoModelExport.getPmml + assert(pmml.getHeader.getDescription === "lasso regression") + // check that the number of fields match the weights size + assert(pmml.getDataDictionary.getNumberOfFields === lassoModel.weights.size + 1) + // This verify that there is a model attached to the pmml object and the model is a regression + // one. It also verifies that the pmml model has a regression table with the same number of + // predictors of the model weights. + val pmmlRegressionModel = pmml.getModels.get(0).asInstanceOf[RegressionModel] + assert(pmmlRegressionModel.getRegressionTables.get(0).getNumericPredictors.size + === lassoModel.weights.size) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala new file mode 100644 index 0000000000000..b985d0446d7b0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import org.dmg.pmml.ClusteringModel +import org.scalatest.FunSuite + +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.linalg.Vectors + +class KMeansPMMLModelExportSuite extends FunSuite { + + test("KMeansPMMLModelExport generate PMML format") { + val clusterCenters = Array( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0)) + val kmeansModel = new KMeansModel(clusterCenters) + + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) + + // assert that the PMML format is as expected + assert(modelExport.isInstanceOf[PMMLModelExport]) + val pmml = modelExport.asInstanceOf[PMMLModelExport].getPmml + assert(pmml.getHeader.getDescription === "k-means clustering") + // check that the number of fields match the single vector size + assert(pmml.getDataDictionary.getNumberOfFields === clusterCenters(0).size) + // This verify that there is a model attached to the pmml object and the model is a clustering + // one. It also verifies that the pmml model has the same number of clusters of the spark model. + val pmmlClusteringModel = pmml.getModels.get(0).asInstanceOf[ClusteringModel] + assert(pmmlClusteringModel.getNumberOfClusters === clusterCenters.length) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala new file mode 100644 index 0000000000000..f28a4ac8ad01f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactorySuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.pmml.export + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.classification.{LogisticRegressionModel, SVMModel} +import org.apache.spark.mllib.clustering.KMeansModel +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} +import org.apache.spark.mllib.util.LinearDataGenerator + +class PMMLModelExportFactorySuite extends FunSuite { + + test("PMMLModelExportFactory create KMeansPMMLModelExport when passing a KMeansModel") { + val clusterCenters = Array( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0)) + val kmeansModel = new KMeansModel(clusterCenters) + + val modelExport = PMMLModelExportFactory.createPMMLModelExport(kmeansModel) + + assert(modelExport.isInstanceOf[KMeansPMMLModelExport]) + } + + test("PMMLModelExportFactory create GeneralizedLinearPMMLModelExport when passing a " + + "LinearRegressionModel, RidgeRegressionModel or LassoModel") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + + val linearRegressionModel = + new LinearRegressionModel(linearInput(0).features, linearInput(0).label) + val linearModelExport = PMMLModelExportFactory.createPMMLModelExport(linearRegressionModel) + assert(linearModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + val ridgeRegressionModel = + new RidgeRegressionModel(linearInput(0).features, linearInput(0).label) + val ridgeModelExport = PMMLModelExportFactory.createPMMLModelExport(ridgeRegressionModel) + assert(ridgeModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + + val lassoModel = new LassoModel(linearInput(0).features, linearInput(0).label) + val lassoModelExport = PMMLModelExportFactory.createPMMLModelExport(lassoModel) + assert(lassoModelExport.isInstanceOf[GeneralizedLinearPMMLModelExport]) + } + + test("PMMLModelExportFactory create BinaryClassificationPMMLModelExport " + + "when passing a LogisticRegressionModel or SVMModel") { + val linearInput = LinearDataGenerator.generateLinearInput(3.0, Array(10.0, 10.0), 1, 17) + + val logisticRegressionModel = + new LogisticRegressionModel(linearInput(0).features, linearInput(0).label) + val logisticRegressionModelExport = + PMMLModelExportFactory.createPMMLModelExport(logisticRegressionModel) + assert(logisticRegressionModelExport.isInstanceOf[BinaryClassificationPMMLModelExport]) + + val svmModel = new SVMModel(linearInput(0).features, linearInput(0).label) + val svmModelExport = PMMLModelExportFactory.createPMMLModelExport(svmModel) + assert(svmModelExport.isInstanceOf[BinaryClassificationPMMLModelExport]) + } + + test("PMMLModelExportFactory throw IllegalArgumentException " + + "when passing a Multinomial Logistic Regression") { + /** 3 classes, 2 features */ + val multiclassLogisticRegressionModel = new LogisticRegressionModel( + weights = Vectors.dense(0.1, 0.2, 0.3, 0.4), intercept = 1.0, + numFeatures = 2, numClasses = 3) + + intercept[IllegalArgumentException] { + PMMLModelExportFactory.createPMMLModelExport(multiclassLogisticRegressionModel) + } + } + + test("PMMLModelExportFactory throw IllegalArgumentException when passing an unsupported model") { + val invalidModel = new Object + + intercept[IllegalArgumentException] { + PMMLModelExportFactory.createPMMLModelExport(invalidModel) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 6395188a0842a..63f2ea916d457 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -181,7 +181,8 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) - val exponential = RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) + val exponential = + RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) testGeneratedVectorRDD(exponential, rows, cols, parts, exponentialMean, exponentialMean, 0.1) val gamma = RandomRDDs.gammaVectorRDD(sc, gammaShape, gammaScale, rows, cols, parts, seed) @@ -197,7 +198,7 @@ private[random] class MockDistro extends RandomDataGenerator[Double] { // This allows us to check that each partition has a different seed override def nextValue(): Double = seed.toDouble - override def setSeed(seed: Long) = this.seed = seed + override def setSeed(seed: Long): Unit = this.seed = seed override def copy(): MockDistro = new MockDistro } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala index 1ac7c12c4e8e6..cb8fe4dba96f5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ class MLPairRDDFunctionsSuite extends FunSuite with MLlibTestSparkContext { test("topByKey") { - val topMap = sc.parallelize(Array((1, 1), (1, 2), (3, 2), (3, 7), (3, 5), (5, 1), (5, 3)), 2) + val topMap = sc.parallelize(Array((1, 1), (1, 2), (3, 2), (3, 7), (5, 1), (3, 5)), 2) .topByKey(2) .collectAsMap() assert(topMap.size === 3) assert(topMap(1) === Array(2, 1)) assert(topMap(3) === Array(7, 5)) - assert(topMap(5) === Array(3, 1)) + assert(topMap(5) === Array(1)) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 8775c0ca9df84..b3798940ddc38 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -203,6 +203,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { * @param numProductBlocks number of product blocks to partition products into * @param negativeFactors whether the generated user/product factors can have negative entries */ + // scalastyle:off def testALS( users: Int, products: Int, @@ -216,6 +217,8 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { numUserBlocks: Int = -1, numProductBlocks: Int = -1, negativeFactors: Boolean = true) { + // scalastyle:on + val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights, negativeFactors) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala index 9801e87576744..2c92866f3893d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala @@ -72,4 +72,24 @@ class MatrixFactorizationModelSuite extends FunSuite with MLlibTestSparkContext Utils.deleteRecursively(tempDir) } } + + test("batch predict API recommendProductsForUsers") { + val model = new MatrixFactorizationModel(rank, userFeatures, prodFeatures) + val topK = 10 + val recommendations = model.recommendProductsForUsers(topK).collectAsMap() + + assert(recommendations(0)(0).rating ~== 17.0 relTol 1e-14) + assert(recommendations(1)(0).rating ~== 39.0 relTol 1e-14) + } + + test("batch predict API recommendUsersForProducts") { + val model = new MatrixFactorizationModel(rank, userFeatures, prodFeatures) + val topK = 10 + val recommendations = model.recommendUsersForProducts(topK).collectAsMap() + + assert(recommendations(2)(0).user == 1) + assert(recommendations(2)(0).rating ~== 39.0 relTol 1e-14) + assert(recommendations(2)(1).user == 0) + assert(recommendations(2)(1).rating ~== 17.0 relTol 1e-14) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index 7ef45248281e9..8e12340bbd9d6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.{Matchers, FunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class IsotonicRegressionSuite extends FunSuite with MLlibTestSparkContext with Matchers { @@ -73,6 +74,26 @@ class IsotonicRegressionSuite extends FunSuite with MLlibTestSparkContext with M assert(model.isotonic) } + test("model save/load") { + val boundaries = Array(0.0, 1.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0) + val predictions = Array(1, 2, 2, 6, 16.5, 16.5, 17.0, 18.0) + val model = new IsotonicRegressionModel(boundaries, predictions, true) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = IsotonicRegressionModel.load(sc, path) + assert(model.boundaries === sameModel.boundaries) + assert(model.predictions === sameModel.predictions) + assert(model.isotonic === model.isotonic) + } finally { + Utils.deleteRecursively(tempDir) + } + } + test("isotonic regression with size 0") { val model = runIsotonicRegression(Seq(), true) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 43d61151e2471..d6c93cc0e49cd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -35,7 +35,7 @@ private object RidgeRegressionSuite { class RidgeRegressionSuite extends FunSuite with MLlibTestSparkContext { - def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { + def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = { predictions.zip(input).map { case (prediction, expected) => (prediction - expected.label) * (prediction - expected.label) }.reduceLeft(_ + _) / predictions.size diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 70b43ddb7daf5..26604dbe6c1ef 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 20000 + override def maxWaitTimeMillis: Int = 20000 // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { @@ -139,4 +139,32 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) assert(errors.forall(x => x <= 0.1)) } + + // Test training combined with prediction + test("training and prediction") { + // create model initialized with zero weights + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0, 0.0)) + .setStepSize(0.2) + .setNumIterations(25) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), nPoints, 42 * (i + 1)) + } + + // train and predict + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // assert that prediction error improves, ensuring that the updated model is being used + val error = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints).toList + assert((error.head - error.last) > 2) + } } 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 4c162df810bb2..ce983eb27fa35 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 @@ -36,6 +36,10 @@ import org.apache.spark.util.Utils class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { + ///////////////////////////////////////////////////////////////////////////// + // Tests examining individual elements of training + ///////////////////////////////////////////////////////////////////////////// + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -254,6 +258,165 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(bins(0).length === 0) } + test("Avoid aggregation on the last level") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue leaf nodes into node queue + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Avoid aggregation if impurity is 0.0") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue a node into node queue if its impurity is 0.0 + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Second level node building with vs. without groups") { + val arr = DecisionTreeSuite.generateOrderedLabeledPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, + numClasses = 2, maxBins = 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val rootNode1 = modelOneNode.topNode.deepCopy() + val rootNode2 = modelOneNode.topNode.deepCopy() + assert(rootNode1.leftNode.nonEmpty) + assert(rootNode1.rightNode.nonEmpty) + + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) + + // Single group second level tree construction. + val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) + val treeToNodeToIndexInfo = Map((0, Map( + (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), + (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + val children1 = new Array[Node](2) + children1(0) = rootNode1.leftNode.get + children1(1) = rootNode1.rightNode.get + + // Train one second-level node at a time. + val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) + val treeToNodeToIndexInfoA = Map((0, Map( + (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) + val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) + val treeToNodeToIndexInfoB = Map((0, Map( + (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) + val children2 = new Array[Node](2) + children2(0) = rootNode2.leftNode.get + children2(1) = rootNode2.rightNode.get + + // Verify whether the splits obtained using single group and multiple group level + // construction strategies are the same. + for (i <- 0 until 2) { + assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) + assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) + val stats1 = children1(i).stats.get + val stats2 = children2(i).stats.get + assert(stats1.gain === stats2.gain) + assert(stats1.impurity === stats2.impurity) + assert(stats1.leftImpurity === stats2.leftImpurity) + assert(stats1.rightImpurity === stats2.rightImpurity) + assert(children1(i).predict.predict === children2(i).predict.predict) + } + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests calling train() + ///////////////////////////////////////////////////////////////////////////// test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() @@ -438,76 +601,6 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(rootNode.predict.predict === 1) } - test("Second level node building with vs. without groups") { - val arr = DecisionTreeSuite.generateOrderedLabeledPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - - // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClasses = 2, maxBins = 100) - val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNode1 = modelOneNode.topNode.deepCopy() - val rootNode2 = modelOneNode.topNode.deepCopy() - assert(rootNode1.leftNode.nonEmpty) - assert(rootNode1.rightNode.nonEmpty) - - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - // Single group second level tree construction. - val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) - val treeToNodeToIndexInfo = Map((0, Map( - (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), - (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - val children1 = new Array[Node](2) - children1(0) = rootNode1.leftNode.get - children1(1) = rootNode1.rightNode.get - - // Train one second-level node at a time. - val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) - val treeToNodeToIndexInfoA = Map((0, Map( - (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) - val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) - val treeToNodeToIndexInfoB = Map((0, Map( - (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) - val children2 = new Array[Node](2) - children2(0) = rootNode2.leftNode.get - children2(1) = rootNode2.rightNode.get - - // Verify whether the splits obtained using single group and multiple group level - // construction strategies are the same. - for (i <- 0 until 2) { - assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) - assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) - assert(children1(i).split === children2(i).split) - assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) - val stats1 = children1(i).stats.get - val stats2 = children2(i).stats.get - assert(stats1.gain === stats2.gain) - assert(stats1.impurity === stats2.impurity) - assert(stats1.leftImpurity === stats2.leftImpurity) - assert(stats1.rightImpurity === stats2.rightImpurity) - assert(children1(i).predict.predict === children2(i).predict.predict) - } - } - test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) @@ -528,11 +621,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) - arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(3.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 2) @@ -544,11 +637,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("Binary classification stump with 2 continuous features") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -668,11 +761,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min instances per node requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) - + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, numClasses = 2, minInstancesPerNode = 2) @@ -695,11 +787,11 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { test("do not choose split that does not satisfy min instance per node requirements") { // if a split does not satisfy min instances per node requirements, // this split is invalid, even though the information gain of split is large. - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) - arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0))) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, @@ -715,10 +807,10 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { } test("split must satisfy min info gain requirements") { - val arr = new Array[LabeledPoint](3) - arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) - arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) - arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + val arr = Array( + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), + LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, @@ -739,91 +831,9 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(gain == InformationGainStats.invalidInformationGainStats) } - test("Avoid aggregation on the last level") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue leaf nodes into node queue - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } - - test("Avoid aggregation if impurity is 0.0") { - val arr = new Array[LabeledPoint](4) - arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) - arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) - arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) - arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue a node into node queue if its impurity is 0.0 - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } + ///////////////////////////////////////////////////////////////////////////// + // Tests of model save/load + ///////////////////////////////////////////////////////////////////////////// test("Node.subtreeIterator") { val model = DecisionTreeSuite.createModel(Classification) @@ -988,7 +998,7 @@ object DecisionTreeSuite extends FunSuite { node.split = Some(new Split(feature = 1, threshold = 0.0, Categorical, categories = List(0.0, 1.0))) } - // TODO: The information gain stats should be consistent with the same info stored in children. + // TODO: The information gain stats should be consistent with info in children: SPARK-7131 node.stats = Some(new InformationGainStats(gain = 0.1, impurity = 0.2, leftImpurity = 0.3, rightImpurity = 0.4, new Predict(1.0, 0.4), new Predict(0.0, 0.6))) node @@ -996,8 +1006,9 @@ object DecisionTreeSuite extends FunSuite { /** * Create a tree model. This is deterministic and contains a variety of node and feature types. + * TODO: Update to be a correct tree (with matching probabilities, impurities, etc.): SPARK-7131 */ - private[tree] def createModel(algo: Algo): DecisionTreeModel = { + private[spark] def createModel(algo: Algo): DecisionTreeModel = { val topNode = createInternalNode(id = 1, Continuous) val (node2, node3) = (createLeafNode(id = 2), createInternalNode(id = 3, Categorical)) val (node6, node7) = (createLeafNode(id = 6), createLeafNode(id = 7)) @@ -1017,7 +1028,7 @@ object DecisionTreeSuite extends FunSuite { * make mistakes such as creating loops of Nodes. * If the trees are not equal, this prints the two trees and throws an exception. */ - private[tree] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + private[mllib] def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { try { assert(a.algo === b.algo) checkEqual(a.topNode, b.topNode) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index e957fa5d25f4c..352193a67860c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -95,16 +95,16 @@ object TestingUtils { /** * Comparison using absolute tolerance. */ - def absTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(AbsoluteErrorComparison, - x, eps, ABS_TOL_MSG) + def absTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(AbsoluteErrorComparison, x, eps, ABS_TOL_MSG) /** * Comparison using relative tolerance. */ - def relTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(RelativeErrorComparison, - x, eps, REL_TOL_MSG) + def relTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(RelativeErrorComparison, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareVectorRightSide( @@ -166,7 +166,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareMatrixRightSide( @@ -229,7 +229,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index b0ecb33c28483..59e6c778806f4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -88,16 +88,20 @@ class TestingUtilsSuite extends FunSuite { assert(!(17.8 ~= 17.59 absTol 0.2)) // Comparisons of numbers very close to zero, and both side of zeros - assert(Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - - assert(-Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert( + -Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) } test("Comparing vectors using relative error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) @@ -130,7 +134,7 @@ class TestingUtilsSuite extends FunSuite { test("Comparing vectors using absolute error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) diff --git a/network/common/pom.xml b/network/common/pom.xml index 7b51845206f4a..0c3147761cfc5 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -80,6 +80,11 @@ mockito-all test + + org.slf4j + slf4j-log4j12 + test + @@ -90,7 +95,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.2 test-jar-on-test-compile diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index f0a89c9d9116c..b8d073fa16b4b 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import io.netty.channel.Channel; import io.netty.channel.socket.SocketChannel; +import io.netty.handler.timeout.IdleStateHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,6 +36,7 @@ import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.server.TransportRequestHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -81,13 +83,21 @@ public TransportClientFactory createClientFactory() { } /** Create a server which will attempt to bind to a specific port. */ - public TransportServer createServer(int port) { - return new TransportServer(this, port); + public TransportServer createServer(int port, List bootstraps) { + return new TransportServer(this, port, rpcHandler, bootstraps); } /** Creates a new server, binding to any available ephemeral port. */ + public TransportServer createServer(List bootstraps) { + return createServer(0, bootstraps); + } + public TransportServer createServer() { - return new TransportServer(this, 0); + return createServer(0, Lists.newArrayList()); + } + + public TransportChannelHandler initializePipeline(SocketChannel channel) { + return initializePipeline(channel, rpcHandler); } /** @@ -95,17 +105,23 @@ public TransportServer createServer() { * has a {@link org.apache.spark.network.server.TransportChannelHandler} to handle request or * response messages. * + * @param channel The channel to initialize. + * @param channelRpcHandler The RPC handler to use for the channel. + * * @return Returns the created TransportChannelHandler, which includes a TransportClient that can * be used to communicate on this channel. The TransportClient is directly associated with a * ChannelHandler to ensure all users of the same channel get the same TransportClient object. */ - public TransportChannelHandler initializePipeline(SocketChannel channel) { + public TransportChannelHandler initializePipeline( + SocketChannel channel, + RpcHandler channelRpcHandler) { try { - TransportChannelHandler channelHandler = createChannelHandler(channel); + TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); channel.pipeline() .addLast("encoder", encoder) .addLast("frameDecoder", NettyUtils.createFrameDecoder()) .addLast("decoder", decoder) + .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this // would require more logic to guarantee if this were not part of the same event loop. .addLast("handler", channelHandler); @@ -121,12 +137,13 @@ public TransportChannelHandler initializePipeline(SocketChannel channel) { * ResponseMessages. The channel is expected to have been successfully created, though certain * properties (such as the remoteAddress()) may not be available yet. */ - private TransportChannelHandler createChannelHandler(Channel channel) { + private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler rpcHandler) { TransportResponseHandler responseHandler = new TransportResponseHandler(channel); TransportClient client = new TransportClient(channel, responseHandler); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, rpcHandler); - return new TransportChannelHandler(client, responseHandler, requestHandler); + return new TransportChannelHandler(client, responseHandler, requestHandler, + conf.connectionTimeoutMs()); } public TransportConf getConf() { return conf; } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java index 65e8020e34121..eaae2ee043c5a 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java @@ -17,6 +17,8 @@ package org.apache.spark.network.client; +import io.netty.channel.Channel; + /** * A bootstrap which is executed on a TransportClient before it is returned to the user. * This enables an initial exchange of information (e.g., SASL authentication tokens) on a once-per- @@ -28,5 +30,5 @@ */ public interface TransportClientBootstrap { /** Performs the bootstrapping operation, throwing an exception on failure. */ - public void doBootstrap(TransportClient client) throws RuntimeException; + void doBootstrap(TransportClient client, Channel channel) throws RuntimeException; } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index d26b9b4d6055f..4952ffb44bb8b 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -172,12 +172,14 @@ private TransportClient createClient(InetSocketAddress address) throws IOExcepti .option(ChannelOption.ALLOCATOR, pooledAllocator); final AtomicReference clientRef = new AtomicReference(); + final AtomicReference channelRef = new AtomicReference(); bootstrap.handler(new ChannelInitializer() { @Override public void initChannel(SocketChannel ch) { TransportChannelHandler clientHandler = context.initializePipeline(ch); clientRef.set(clientHandler.getClient()); + channelRef.set(ch); } }); @@ -192,6 +194,7 @@ public void initChannel(SocketChannel ch) { } TransportClient client = clientRef.get(); + Channel channel = channelRef.get(); assert client != null : "Channel future completed successfully with null client"; // Execute any client bootstraps synchronously before marking the Client as successful. @@ -199,7 +202,7 @@ public void initChannel(SocketChannel ch) { logger.debug("Connection to {} successful, running bootstraps...", address); try { for (TransportClientBootstrap clientBootstrap : clientBootstraps) { - clientBootstrap.doBootstrap(client); + clientBootstrap.doBootstrap(client, channel); } } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 2044afb0d85db..94fc21af5e606 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; -import com.google.common.annotations.VisibleForTesting; import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,13 +50,18 @@ public class TransportResponseHandler extends MessageHandler { private final Map outstandingRpcs; + /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ + private final AtomicLong timeOfLastRequestNs; + public TransportResponseHandler(Channel channel) { this.channel = channel; this.outstandingFetches = new ConcurrentHashMap(); this.outstandingRpcs = new ConcurrentHashMap(); + this.timeOfLastRequestNs = new AtomicLong(0); } public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + timeOfLastRequestNs.set(System.nanoTime()); outstandingFetches.put(streamChunkId, callback); } @@ -65,6 +70,7 @@ public void removeFetchRequest(StreamChunkId streamChunkId) { } public void addRpcRequest(long requestId, RpcResponseCallback callback) { + timeOfLastRequestNs.set(System.nanoTime()); outstandingRpcs.put(requestId, callback); } @@ -161,8 +167,12 @@ public void handle(ResponseMessage message) { } /** Returns total number of outstanding requests (fetch requests + rpcs) */ - @VisibleForTesting public int numOutstandingRequests() { return outstandingFetches.size() + outstandingRpcs.size(); } + + /** Returns the time in nanoseconds of when the last request was sent out. */ + public long getTimeOfLastRequestNs() { + return timeOfLastRequestNs.get(); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 91d1e8a538a77..0f999f5dfe8d8 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -72,9 +72,11 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) { in.encode(header); assert header.writableBytes() == 0; - out.add(header); if (body != null && bodyLength > 0) { - out.add(body); + out.add(new MessageWithHeader(header, body, bodyLength)); + } else { + out.add(header); } } + } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java new file mode 100644 index 0000000000000..d686a951467cf --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -0,0 +1,109 @@ +/* + * 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.protocol; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import io.netty.util.ReferenceCountUtil; + +/** + * A wrapper message that holds two separate pieces (a header and a body). + * + * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. + */ +class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { + + private final ByteBuf header; + private final int headerLength; + private final Object body; + private final long bodyLength; + private long totalBytesTransferred; + + MessageWithHeader(ByteBuf header, Object body, long bodyLength) { + Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, + "Body must be a ByteBuf or a FileRegion."); + this.header = header; + this.headerLength = header.readableBytes(); + this.body = body; + this.bodyLength = bodyLength; + } + + @Override + public long count() { + return headerLength + bodyLength; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return totalBytesTransferred; + } + + /** + * This code is more complicated than you would think because we might require multiple + * transferTo invocations in order to transfer a single MessageWithHeader to avoid busy waiting. + * + * The contract is that the caller will ensure position is properly set to the total number + * of bytes transferred so far (i.e. value returned by transfered()). + */ + @Override + public long transferTo(final WritableByteChannel target, final long position) throws IOException { + Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); + // Bytes written for header in this call. + long writtenHeader = 0; + if (header.readableBytes() > 0) { + writtenHeader = copyByteBuf(header, target); + totalBytesTransferred += writtenHeader; + if (header.readableBytes() > 0) { + return writtenHeader; + } + } + + // Bytes written for body in this call. + long writtenBody = 0; + if (body instanceof FileRegion) { + writtenBody = ((FileRegion) body).transferTo(target, totalBytesTransferred - headerLength); + } else if (body instanceof ByteBuf) { + writtenBody = copyByteBuf((ByteBuf) body, target); + } + totalBytesTransferred += writtenBody; + + return writtenHeader + writtenBody; + } + + @Override + protected void deallocate() { + header.release(); + ReferenceCountUtil.release(body); + } + + private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { + int written = target.write(buf.nioBuffer()); + buf.skipBytes(written); + return written; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java index 33aa1344345ff..185ba2ef3bb1f 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -17,8 +17,12 @@ package org.apache.spark.network.sasl; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; + import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,14 +37,24 @@ public class SaslClientBootstrap implements TransportClientBootstrap { private final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); + private final boolean encrypt; private final TransportConf conf; private final String appId; private final SecretKeyHolder secretKeyHolder; public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder secretKeyHolder) { + this(conf, appId, secretKeyHolder, false); + } + + public SaslClientBootstrap( + TransportConf conf, + String appId, + SecretKeyHolder secretKeyHolder, + boolean encrypt) { this.conf = conf; this.appId = appId; this.secretKeyHolder = secretKeyHolder; + this.encrypt = encrypt; } /** @@ -49,8 +63,8 @@ public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder sec * due to mismatch. */ @Override - public void doBootstrap(TransportClient client) { - SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder); + public void doBootstrap(TransportClient client, Channel channel) { + SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, encrypt); try { byte[] payload = saslClient.firstToken(); @@ -62,13 +76,26 @@ public void doBootstrap(TransportClient client) { byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeoutMs()); payload = saslClient.response(response); } + + if (encrypt) { + if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslClient.getNegotiatedProperty(Sasl.QOP))) { + throw new RuntimeException( + new SaslException("Encryption requests by negotiated non-encrypted connection.")); + } + SaslEncryption.addToChannel(channel, saslClient, conf.maxSaslEncryptedBlockSize()); + saslClient = null; + logger.debug("Channel {} configured for SASL encryption.", client); + } } finally { - try { - // Once authentication is complete, the server will trust all remaining communication. - saslClient.dispose(); - } catch (RuntimeException e) { - logger.error("Error while disposing SASL client", e); + if (saslClient != null) { + try { + // Once authentication is complete, the server will trust all remaining communication. + saslClient.dispose(); + } catch (RuntimeException e) { + logger.error("Error while disposing SASL client", e); + } } } } + } diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java new file mode 100644 index 0000000000000..127335e4d35fb --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java @@ -0,0 +1,291 @@ +/* + * 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.sasl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.FileRegion; +import io.netty.handler.codec.MessageToMessageDecoder; +import io.netty.util.AbstractReferenceCounted; +import io.netty.util.ReferenceCountUtil; + +import org.apache.spark.network.util.ByteArrayWritableChannel; +import org.apache.spark.network.util.NettyUtils; + +/** + * Provides SASL-based encription for transport channels. The single method exposed by this + * class installs the needed channel handlers on a connected channel. + */ +class SaslEncryption { + + @VisibleForTesting + static final String ENCRYPTION_HANDLER_NAME = "saslEncryption"; + + /** + * Adds channel handlers that perform encryption / decryption of data using SASL. + * + * @param channel The channel. + * @param backend The SASL backend. + * @param maxOutboundBlockSize Max size in bytes of outgoing encrypted blocks, to control + * memory usage. + */ + static void addToChannel( + Channel channel, + SaslEncryptionBackend backend, + int maxOutboundBlockSize) { + channel.pipeline() + .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(backend, maxOutboundBlockSize)) + .addFirst("saslDecryption", new DecryptionHandler(backend)) + .addFirst("saslFrameDecoder", NettyUtils.createFrameDecoder()); + } + + private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + + private final int maxOutboundBlockSize; + private final SaslEncryptionBackend backend; + + EncryptionHandler(SaslEncryptionBackend backend, int maxOutboundBlockSize) { + this.backend = backend; + this.maxOutboundBlockSize = maxOutboundBlockSize; + } + + /** + * Wrap the incoming message in an implementation that will perform encryption lazily. This is + * needed to guarantee ordering of the outgoing encrypted packets - they need to be decrypted in + * the same order, and netty doesn't have an atomic ChannelHandlerContext.write() API, so it + * does not guarantee any ordering. + */ + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + + ctx.write(new EncryptedMessage(backend, msg, maxOutboundBlockSize), promise); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + try { + backend.dispose(); + } finally { + super.handlerRemoved(ctx); + } + } + + } + + private static class DecryptionHandler extends MessageToMessageDecoder { + + private final SaslEncryptionBackend backend; + + DecryptionHandler(SaslEncryptionBackend backend) { + this.backend = backend; + } + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf msg, List out) + throws Exception { + + byte[] data; + int offset; + int length = msg.readableBytes(); + if (msg.hasArray()) { + data = msg.array(); + offset = msg.arrayOffset(); + msg.skipBytes(length); + } else { + data = new byte[length]; + msg.readBytes(data); + offset = 0; + } + + out.add(Unpooled.wrappedBuffer(backend.unwrap(data, offset, length))); + } + + } + + @VisibleForTesting + static class EncryptedMessage extends AbstractReferenceCounted implements FileRegion { + + private final SaslEncryptionBackend backend; + private final boolean isByteBuf; + private final ByteBuf buf; + private final FileRegion region; + + /** + * A channel used to buffer input data for encryption. The channel has an upper size bound + * so that if the input is larger than the allowed buffer, it will be broken into multiple + * chunks. + */ + private final ByteArrayWritableChannel byteChannel; + + private ByteBuf currentHeader; + private ByteBuffer currentChunk; + private long currentChunkSize; + private long currentReportedBytes; + private long unencryptedChunkSize; + private long transferred; + + EncryptedMessage(SaslEncryptionBackend backend, Object msg, int maxOutboundBlockSize) { + Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, + "Unrecognized message type: %s", msg.getClass().getName()); + this.backend = backend; + this.isByteBuf = msg instanceof ByteBuf; + this.buf = isByteBuf ? (ByteBuf) msg : null; + this.region = isByteBuf ? null : (FileRegion) msg; + this.byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize); + } + + /** + * Returns the size of the original (unencrypted) message. + * + * This makes assumptions about how netty treats FileRegion instances, because there's no way + * to know beforehand what will be the size of the encrypted message. Namely, it assumes + * that netty will try to transfer data from this message while + * transfered() < count(). So these two methods return, technically, wrong data, + * but netty doesn't know better. + */ + @Override + public long count() { + return isByteBuf ? buf.readableBytes() : region.count(); + } + + @Override + public long position() { + return 0; + } + + /** + * Returns an approximation of the amount of data transferred. See {@link #count()}. + */ + @Override + public long transfered() { + return transferred; + } + + /** + * Transfers data from the original message to the channel, encrypting it in the process. + * + * This method also breaks down the original message into smaller chunks when needed. This + * is done to keep memory usage under control. This avoids having to copy the whole message + * data into memory at once, and can avoid ballooning memory usage when transferring large + * messages such as shuffle blocks. + * + * The {@link #transfered()} counter also behaves a little funny, in that it won't go forward + * until a whole chunk has been written. This is done because the code can't use the actual + * number of bytes written to the channel as the transferred count (see {@link #count()}). + * Instead, once an encrypted chunk is written to the output (including its header), the + * size of the original block will be added to the {@link #transfered()} amount. + */ + @Override + public long transferTo(final WritableByteChannel target, final long position) + throws IOException { + + Preconditions.checkArgument(position == transfered(), "Invalid position."); + + long reportedWritten = 0L; + long actuallyWritten = 0L; + do { + if (currentChunk == null) { + nextChunk(); + } + + if (currentHeader.readableBytes() > 0) { + int bytesWritten = target.write(currentHeader.nioBuffer()); + currentHeader.skipBytes(bytesWritten); + actuallyWritten += bytesWritten; + if (currentHeader.readableBytes() > 0) { + // Break out of loop if there are still header bytes left to write. + break; + } + } + + actuallyWritten += target.write(currentChunk); + if (!currentChunk.hasRemaining()) { + // Only update the count of written bytes once a full chunk has been written. + // See method javadoc. + long chunkBytesRemaining = unencryptedChunkSize - currentReportedBytes; + reportedWritten += chunkBytesRemaining; + transferred += chunkBytesRemaining; + currentHeader.release(); + currentHeader = null; + currentChunk = null; + currentChunkSize = 0; + currentReportedBytes = 0; + } + } while (currentChunk == null && transfered() + reportedWritten < count()); + + // Returning 0 triggers a backoff mechanism in netty which may harm performance. Instead, + // we return 1 until we can (i.e. until the reported count would actually match the size + // of the current chunk), at which point we resort to returning 0 so that the counts still + // match, at the cost of some performance. That situation should be rare, though. + if (reportedWritten != 0L) { + return reportedWritten; + } + + if (actuallyWritten > 0 && currentReportedBytes < currentChunkSize - 1) { + transferred += 1L; + currentReportedBytes += 1L; + return 1L; + } + + return 0L; + } + + private void nextChunk() throws IOException { + byteChannel.reset(); + if (isByteBuf) { + int copied = byteChannel.write(buf.nioBuffer()); + buf.skipBytes(copied); + } else { + region.transferTo(byteChannel, region.transfered()); + } + + byte[] encrypted = backend.wrap(byteChannel.getData(), 0, byteChannel.length()); + this.currentChunk = ByteBuffer.wrap(encrypted); + this.currentChunkSize = encrypted.length; + this.currentHeader = Unpooled.copyLong(8 + currentChunkSize); + this.unencryptedChunkSize = byteChannel.length(); + } + + @Override + protected void deallocate() { + if (currentHeader != null) { + currentHeader.release(); + } + if (buf != null) { + buf.release(); + } + if (region != null) { + region.release(); + } + } + + } + +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java similarity index 67% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java index b2c6d3029031d..89b78bc7e1df1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions +package org.apache.spark.network.sasl; -import java.util.Random -import org.apache.spark.sql.types.DoubleType +import javax.security.sasl.SaslException; +interface SaslEncryptionBackend { -case object Rand extends LeafExpression { - override def dataType = DoubleType - override def nullable = false + /** Disposes of resources used by the backend. */ + void dispose(); - private[this] lazy val rand = new Random + /** Encrypt data. */ + byte[] wrap(byte[] data, int offset, int len) throws SaslException; - override def eval(input: Row = null) = rand.nextDouble().asInstanceOf[EvaluatedType] + /** Decrypt data. */ + byte[] unwrap(byte[] data, int offset, int len) throws SaslException; - override def toString = "RAND()" } diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java index 026cbd260d16c..be6165caf3c74 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java @@ -17,10 +17,10 @@ package org.apache.spark.network.sasl; -import java.util.concurrent.ConcurrentMap; +import javax.security.sasl.Sasl; -import com.google.common.collect.Maps; import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,6 +28,7 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.TransportConf; /** * RPC Handler which performs SASL authentication before delegating to a child RPC handler. @@ -37,8 +38,14 @@ * Note that the authentication process consists of multiple challenge-response pairs, each of * which are individual RPCs. */ -public class SaslRpcHandler extends RpcHandler { - private final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); +class SaslRpcHandler extends RpcHandler { + private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; /** RpcHandler we will delegate to for authenticated connections. */ private final RpcHandler delegate; @@ -46,19 +53,25 @@ public class SaslRpcHandler extends RpcHandler { /** Class which provides secret keys which are shared by server and client on a per-app basis. */ private final SecretKeyHolder secretKeyHolder; - /** Maps each channel to its SASL authentication state. */ - private final ConcurrentMap channelAuthenticationMap; + private SparkSaslServer saslServer; + private boolean isComplete; - public SaslRpcHandler(RpcHandler delegate, SecretKeyHolder secretKeyHolder) { + SaslRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.channel = channel; this.delegate = delegate; this.secretKeyHolder = secretKeyHolder; - this.channelAuthenticationMap = Maps.newConcurrentMap(); + this.saslServer = null; + this.isComplete = false; } @Override public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - SparkSaslServer saslServer = channelAuthenticationMap.get(client); - if (saslServer != null && saslServer.isComplete()) { + if (isComplete) { // Authentication complete, delegate to base handler. delegate.receive(client, message, callback); return; @@ -68,15 +81,30 @@ public void receive(TransportClient client, byte[] message, RpcResponseCallback if (saslServer == null) { // First message in the handshake, setup the necessary state. - saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder); - channelAuthenticationMap.put(client, saslServer); + saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder, + conf.saslServerAlwaysEncrypt()); } byte[] response = saslServer.response(saslMessage.payload); + callback.onSuccess(response); + + // Setup encryption after the SASL response is sent, otherwise the client can't parse the + // response. It's ok to change the channel pipeline here since we are processing an incoming + // message, so the pipeline is busy and no new incoming messages will be fed to it before this + // method returns. This assumes that the code ensures, through other means, that no outbound + // messages are being written to the channel while negotiation is still going on. if (saslServer.isComplete()) { logger.debug("SASL authentication successful for channel {}", client); + isComplete = true; + if (SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) { + logger.debug("Enabling encryption for channel {}", client); + SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); + saslServer = null; + } else { + saslServer.dispose(); + saslServer = null; + } } - callback.onSuccess(response); } @Override @@ -86,9 +114,9 @@ public StreamManager getStreamManager() { @Override public void connectionTerminated(TransportClient client) { - SparkSaslServer saslServer = channelAuthenticationMap.remove(client); if (saslServer != null) { saslServer.dispose(); } } + } diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java new file mode 100644 index 0000000000000..f2f983856f444 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java @@ -0,0 +1,49 @@ +/* + * 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.sasl; + +import io.netty.channel.Channel; + +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.TransportConf; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server. This allows customizing the client channel to allow for things such as SASL + * authentication. + */ +public class SaslServerBootstrap implements TransportServerBootstrap { + + private final TransportConf conf; + private final SecretKeyHolder secretKeyHolder; + + public SaslServerBootstrap(TransportConf conf, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + } + + /** + * Wrap the given application handler in a SaslRpcHandler that will handle the initial SASL + * negotiation. + */ + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + return new SaslRpcHandler(conf, channel, rpcHandler, secretKeyHolder); + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java index 9abad1f30a259..94685e91b862e 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java @@ -17,6 +17,8 @@ package org.apache.spark.network.sasl; +import java.io.IOException; +import java.util.Map; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.NameCallback; @@ -27,9 +29,9 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; import javax.security.sasl.SaslException; -import java.io.IOException; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,19 +42,25 @@ * initial state to the "authenticated" state. This client initializes the protocol via a * firstToken, which is then followed by a set of challenges and responses. */ -public class SparkSaslClient { +public class SparkSaslClient implements SaslEncryptionBackend { private final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class); private final String secretKeyId; private final SecretKeyHolder secretKeyHolder; + private final String expectedQop; private SaslClient saslClient; - public SparkSaslClient(String secretKeyId, SecretKeyHolder secretKeyHolder) { + public SparkSaslClient(String secretKeyId, SecretKeyHolder secretKeyHolder, boolean encrypt) { this.secretKeyId = secretKeyId; this.secretKeyHolder = secretKeyHolder; + this.expectedQop = encrypt ? QOP_AUTH_CONF : QOP_AUTH; + + Map saslProps = ImmutableMap.builder() + .put(Sasl.QOP, expectedQop) + .build(); try { this.saslClient = Sasl.createSaslClient(new String[] { DIGEST }, null, null, DEFAULT_REALM, - SASL_PROPS, new ClientCallbackHandler()); + saslProps, new ClientCallbackHandler()); } catch (SaslException e) { throw Throwables.propagate(e); } @@ -76,6 +84,11 @@ public synchronized boolean isComplete() { return saslClient != null && saslClient.isComplete(); } + /** Returns the value of a negotiated property. */ + public Object getNegotiatedProperty(String name) { + return saslClient.getNegotiatedProperty(name); + } + /** * Respond to server's SASL token. * @param token contains server's SASL token @@ -93,6 +106,7 @@ public synchronized byte[] response(byte[] token) { * Disposes of any system resources or security-sensitive information the * SaslClient might be using. */ + @Override public synchronized void dispose() { if (saslClient != null) { try { @@ -134,4 +148,15 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback } } } + + @Override + public byte[] wrap(byte[] data, int offset, int len) throws SaslException { + return saslClient.wrap(data, offset, len); + } + + @Override + public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { + return saslClient.unwrap(data, offset, len); + } + } diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java index e87b17ead1e1a..431cb67a2ae0b 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java +++ b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java @@ -44,7 +44,7 @@ * initial state to the "authenticated" state. (It is not a server in the sense of accepting * connections on some socket.) */ -public class SparkSaslServer { +public class SparkSaslServer implements SaslEncryptionBackend { private final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class); /** @@ -60,26 +60,37 @@ public class SparkSaslServer { static final String DIGEST = "DIGEST-MD5"; /** - * The quality of protection is just "auth". This means that we are doing - * authentication only, we are not supporting integrity or privacy protection of the - * communication channel after authentication. This could be changed to be configurable - * in the future. + * Quality of protection value that includes encryption. */ - static final Map SASL_PROPS = ImmutableMap.builder() - .put(Sasl.QOP, "auth") - .put(Sasl.SERVER_AUTH, "true") - .build(); + static final String QOP_AUTH_CONF = "auth-conf"; + + /** + * Quality of protection value that does not include encryption. + */ + static final String QOP_AUTH = "auth"; /** Identifier for a certain secret key within the secretKeyHolder. */ private final String secretKeyId; private final SecretKeyHolder secretKeyHolder; private SaslServer saslServer; - public SparkSaslServer(String secretKeyId, SecretKeyHolder secretKeyHolder) { + public SparkSaslServer( + String secretKeyId, + SecretKeyHolder secretKeyHolder, + boolean alwaysEncrypt) { this.secretKeyId = secretKeyId; this.secretKeyHolder = secretKeyHolder; + + // Sasl.QOP is a comma-separated list of supported values. The value that allows encryption + // is listed first since it's preferred over the non-encrypted one (if the client also + // lists both in the request). + String qop = alwaysEncrypt ? QOP_AUTH_CONF : String.format("%s,%s", QOP_AUTH_CONF, QOP_AUTH); + Map saslProps = ImmutableMap.builder() + .put(Sasl.SERVER_AUTH, "true") + .put(Sasl.QOP, qop) + .build(); try { - this.saslServer = Sasl.createSaslServer(DIGEST, null, DEFAULT_REALM, SASL_PROPS, + this.saslServer = Sasl.createSaslServer(DIGEST, null, DEFAULT_REALM, saslProps, new DigestCallbackHandler()); } catch (SaslException e) { throw Throwables.propagate(e); @@ -93,6 +104,11 @@ public synchronized boolean isComplete() { return saslServer != null && saslServer.isComplete(); } + /** Returns the value of a negotiated property. */ + public Object getNegotiatedProperty(String name) { + return saslServer.getNegotiatedProperty(name); + } + /** * Used to respond to server SASL tokens. * @param token Server's SASL token @@ -110,6 +126,7 @@ public synchronized byte[] response(byte[] token) { * Disposes of any system resources or security-sensitive information the * SaslServer might be using. */ + @Override public synchronized void dispose() { if (saslServer != null) { try { @@ -122,6 +139,16 @@ public synchronized void dispose() { } } + @Override + public byte[] wrap(byte[] data, int offset, int len) throws SaslException { + return saslServer.wrap(data, offset, len); + } + + @Override + public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { + return saslServer.unwrap(data, offset, len); + } + /** * Implementation of javax.security.auth.callback.CallbackHandler for SASL DIGEST-MD5 mechanism. */ diff --git a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index a6d390e13f396..c95e64e8e2cda 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -20,14 +20,18 @@ import java.util.Iterator; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.buffer.ManagedBuffer; +import com.google.common.base.Preconditions; + /** * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually * fetched as chunks by the client. Each registered buffer is one chunk. @@ -36,18 +40,21 @@ public class OneForOneStreamManager extends StreamManager { private final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); private final AtomicLong nextStreamId; - private final Map streams; + private final ConcurrentHashMap streams; /** State of a single stream. */ private static class StreamState { final Iterator buffers; + // The channel associated to the stream + Channel associatedChannel = null; + // Used to keep track of the index of the buffer that the user has retrieved, just to ensure // that the caller only requests each chunk one at a time, in order. int curChunk = 0; StreamState(Iterator buffers) { - this.buffers = buffers; + this.buffers = Preconditions.checkNotNull(buffers); } } @@ -58,6 +65,13 @@ public OneForOneStreamManager() { streams = new ConcurrentHashMap(); } + @Override + public void registerChannel(Channel channel, long streamId) { + if (streams.containsKey(streamId)) { + streams.get(streamId).associatedChannel = channel; + } + } + @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { StreamState state = streams.get(streamId); @@ -80,12 +94,17 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { } @Override - public void connectionTerminated(long streamId) { - // Release all remaining buffers. - StreamState state = streams.remove(streamId); - if (state != null && state.buffers != null) { - while (state.buffers.hasNext()) { - state.buffers.next().release(); + public void connectionTerminated(Channel channel) { + // Close all streams which have been associated with the channel. + for (Map.Entry entry: streams.entrySet()) { + StreamState state = entry.getValue(); + if (state.associatedChannel == channel) { + streams.remove(entry.getKey()); + + // Release all remaining buffers. + while (state.buffers.hasNext()) { + state.buffers.next().release(); + } } } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java index 5a9a14a180c10..929f789bf9d24 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java +++ b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -17,6 +17,8 @@ package org.apache.spark.network.server; +import io.netty.channel.Channel; + import org.apache.spark.network.buffer.ManagedBuffer; /** @@ -44,9 +46,18 @@ public abstract class StreamManager { public abstract ManagedBuffer getChunk(long streamId, int chunkIndex); /** - * Indicates that the TCP connection that was tied to the given stream has been terminated. After - * this occurs, we are guaranteed not to read from the stream again, so any state can be cleaned - * up. + * Associates a stream with a single client connection, which is guaranteed to be the only reader + * of the stream. The getChunk() method will be called serially on this connection and once the + * connection is closed, the stream will never be used again, enabling cleanup. + * + * This must be called before the first getChunk() on the stream, but it may be invoked multiple + * times with the same channel and stream id. + */ + public void registerChannel(Channel channel, long streamId) { } + + /** + * Indicates that the given channel has been terminated. After this occurs, we are guaranteed not + * to read from the associated streams again, so any state can be cleaned up. */ - public void connectionTerminated(long streamId) { } + public void connectionTerminated(Channel channel) { } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index e491367fa4528..8e0ee709e38e3 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -19,6 +19,8 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +42,11 @@ * Client. * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, * for the Client's responses to the Server's requests. + * + * This class also handles timeouts from a {@link io.netty.handler.timeout.IdleStateHandler}. + * We consider a connection timed out if there are outstanding fetch or RPC requests but no traffic + * on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not + * timeout if the client is continuously sending but getting no responses, for simplicity. */ public class TransportChannelHandler extends SimpleChannelInboundHandler { private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); @@ -47,14 +54,17 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler 0; + boolean isActuallyOverdue = + System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; + if (e.state() == IdleState.ALL_IDLE && hasInFlightRequests && isActuallyOverdue) { + String address = NettyUtils.getRemoteAddress(ctx.channel()); + logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + + "requests. Assuming connection is dead; please adjust spark.network.timeout if this " + + "is wrong.", address, requestTimeoutNs / 1000 / 1000); + ctx.close(); + } + } + } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 1580180cc17e9..e5159ab56d0d4 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -17,10 +17,7 @@ package org.apache.spark.network.server; -import java.util.Set; - import com.google.common.base.Throwables; -import com.google.common.collect.Sets; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; @@ -62,9 +59,6 @@ public class TransportRequestHandler extends MessageHandler { /** Returns each chunk part of a stream. */ private final StreamManager streamManager; - /** List of all stream ids that have been read on this handler, used for cleanup. */ - private final Set streamIds; - public TransportRequestHandler( Channel channel, TransportClient reverseClient, @@ -73,7 +67,6 @@ public TransportRequestHandler( this.reverseClient = reverseClient; this.rpcHandler = rpcHandler; this.streamManager = rpcHandler.getStreamManager(); - this.streamIds = Sets.newHashSet(); } @Override @@ -82,10 +75,7 @@ public void exceptionCaught(Throwable cause) { @Override public void channelUnregistered() { - // Inform the StreamManager that these streams will no longer be read from. - for (long streamId : streamIds) { - streamManager.connectionTerminated(streamId); - } + streamManager.connectionTerminated(channel); rpcHandler.connectionTerminated(reverseClient); } @@ -102,12 +92,12 @@ public void handle(RequestMessage request) { private void processFetchRequest(final ChunkFetchRequest req) { final String client = NettyUtils.getRemoteAddress(channel); - streamIds.add(req.streamChunkId.streamId); logger.trace("Received req from {} to fetch block {}", client, req.streamChunkId); ManagedBuffer buf; try { + streamManager.registerChannel(channel, req.streamChunkId.streamId); buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); } catch (Exception e) { logger.error(String.format( diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index b7ce8541e565e..941ef95772e16 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -19,8 +19,11 @@ import java.io.Closeable; import java.net.InetSocketAddress; +import java.util.List; import java.util.concurrent.TimeUnit; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; import io.netty.bootstrap.ServerBootstrap; import io.netty.buffer.PooledByteBufAllocator; import io.netty.channel.ChannelFuture; @@ -44,15 +47,23 @@ public class TransportServer implements Closeable { private final TransportContext context; private final TransportConf conf; + private final RpcHandler appRpcHandler; + private final List bootstraps; private ServerBootstrap bootstrap; private ChannelFuture channelFuture; private int port = -1; /** Creates a TransportServer that binds to the given port, or to any available if 0. */ - public TransportServer(TransportContext context, int portToBind) { + public TransportServer( + TransportContext context, + int portToBind, + RpcHandler appRpcHandler, + List bootstraps) { this.context = context; this.conf = context.getConf(); + this.appRpcHandler = appRpcHandler; + this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); init(portToBind); } @@ -95,7 +106,11 @@ private void init(int portToBind) { bootstrap.childHandler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) throws Exception { - context.initializePipeline(ch); + RpcHandler rpcHandler = appRpcHandler; + for (TransportServerBootstrap bootstrap : bootstraps) { + rpcHandler = bootstrap.doBootstrap(ch, rpcHandler); + } + context.initializePipeline(ch, rpcHandler); } }); diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java new file mode 100644 index 0000000000000..05803ab1bb059 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java @@ -0,0 +1,36 @@ +/* + * 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.server; + +import io.netty.channel.Channel; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server. This allows customizing the client channel to allow for things such as SASL + * authentication. + */ +public interface TransportServerBootstrap { + /** + * Customizes the channel to include new features, if needed. + * + * @param channel The connected channel opened by the client. + * @param rpcHandler The RPC handler for the server. + * @return The RPC handler to use for the channel. + */ + RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler); +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java b/network/common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java new file mode 100644 index 0000000000000..b1415720045e2 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java @@ -0,0 +1,69 @@ +/* + * 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.util; + +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + * A writable channel that stores the written data in a byte array in memory. + */ +public class ByteArrayWritableChannel implements WritableByteChannel { + + private final byte[] data; + private int offset; + + public ByteArrayWritableChannel(int size) { + this.data = new byte[size]; + } + + public byte[] getData() { + return data; + } + + public int length() { + return offset; + } + + /** Resets the channel so that writing to it will overwrite the existing buffer. */ + public void reset() { + offset = 0; + } + + /** + * Reads from the given buffer into the internal byte array. + */ + @Override + public int write(ByteBuffer src) { + int toTransfer = Math.min(src.remaining(), data.length - offset); + src.get(data, offset, toTransfer); + offset += toTransfer; + return toTransfer; + } + + @Override + public void close() { + + } + + @Override + public boolean isOpen() { + return true; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java new file mode 100644 index 0000000000000..36d655017fb0d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -0,0 +1,67 @@ +/* + * 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.util; + +public enum ByteUnit { + BYTE (1), + KiB (1024L), + MiB ((long) Math.pow(1024L, 2L)), + GiB ((long) Math.pow(1024L, 3L)), + TiB ((long) Math.pow(1024L, 4L)), + PiB ((long) Math.pow(1024L, 5L)); + + private ByteUnit(long multiplier) { + this.multiplier = multiplier; + } + + // Interpret the provided number (d) with suffix (u) as this unit type. + // E.g. KiB.interpret(1, MiB) interprets 1MiB as its KiB representation = 1024k + public long convertFrom(long d, ByteUnit u) { + return u.convertTo(d, this); + } + + // Convert the provided number (d) interpreted as this unit type to unit type (u). + public long convertTo(long d, ByteUnit u) { + if (multiplier > u.multiplier) { + long ratio = multiplier / u.multiplier; + if (Long.MAX_VALUE / ratio < d) { + throw new IllegalArgumentException("Conversion of " + d + " exceeds Long.MAX_VALUE in " + + name() + ". Try a larger unit (e.g. MiB instead of KiB)"); + } + return d * ratio; + } else { + // Perform operations in this order to avoid potential overflow + // when computing d * multiplier + return d / (u.multiplier / multiplier); + } + } + + public double toBytes(long d) { + if (d < 0) { + throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); + } + return d * multiplier; + } + + public long toKiB(long d) { return convertTo(d, KiB); } + public long toMiB(long d) { return convertTo(d, MiB); } + public long toGiB(long d) { return convertTo(d, GiB); } + public long toTiB(long d) { return convertTo(d, TiB); } + public long toPiB(long d) { return convertTo(d, PiB); } + + private final long multiplier; +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 73da9b7346f4d..6b514aaa1290d 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -21,9 +21,13 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,4 +125,157 @@ private static boolean isSymlink(File file) throws IOException { } return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + + private static final ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + private static final ImmutableMap byteSuffixes = + ImmutableMap.builder() + .put("b", ByteUnit.BYTE) + .put("k", ByteUnit.KiB) + .put("kb", ByteUnit.KiB) + .put("m", ByteUnit.MiB) + .put("mb", ByteUnit.MiB) + .put("g", ByteUnit.GiB) + .put("gb", ByteUnit.GiB) + .put("t", ByteUnit.TiB) + .put("tb", ByteUnit.TiB) + .put("p", ByteUnit.PiB) + .put("pb", ByteUnit.PiB) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for + * internal use. If no suffix is provided a direct conversion is attempted. + */ + private static long parseTimeString(String str, TimeUnit unit) { + String lower = str.toLowerCase().trim(); + + try { + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (!m.matches()) { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return parseTimeString(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return parseTimeString(str, TimeUnit.SECONDS); + } + + /** + * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to a ByteUnit for + * internal use. If no suffix is provided a direct conversion of the provided default is + * attempted. + */ + private static long parseByteString(String str, ByteUnit unit) { + String lower = str.toLowerCase().trim(); + + try { + Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); + Matcher fractionMatcher = Pattern.compile("([0-9]+\\.[0-9]+)([a-z]+)?").matcher(lower); + + if (m.matches()) { + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !byteSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convertFrom(val, suffix != null ? byteSuffixes.get(suffix) : unit); + } else if (fractionMatcher.matches()) { + throw new NumberFormatException("Fractional values are not supported. Input was: " + + fractionMatcher.group(1)); + } else { + throw new NumberFormatException("Failed to parse byte string: " + str); + } + + } catch (NumberFormatException e) { + String timeError = "Size must be specified as bytes (b), " + + "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " + + "E.g. 50b, 100k, or 250m."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in bytes. + */ + public static long byteStringAsBytes(String str) { + return parseByteString(str, ByteUnit.BYTE); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in kibibytes. + */ + public static long byteStringAsKb(String str) { + return parseByteString(str, ByteUnit.KiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in mebibytes. + */ + public static long byteStringAsMb(String str) { + return parseByteString(str, ByteUnit.MiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to gibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in gibibytes. + */ + public static long byteStringAsGb(String str) { + return parseByteString(str, ByteUnit.GiB); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java new file mode 100644 index 0000000000000..668d2356b955d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java @@ -0,0 +1,41 @@ +/* + * 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.util; + +import com.google.common.collect.Maps; + +import java.util.Map; +import java.util.NoSuchElementException; + +/** ConfigProvider based on a Map (copied in the constructor). */ +public class MapConfigProvider extends ConfigProvider { + private final Map config; + + public MapConfigProvider(Map config) { + this.config = Maps.newHashMap(config); + } + + @Override + public String get(String name) { + String value = config.get(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index dabd6261d2aa0..26c6399ce7dbc 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -98,7 +98,7 @@ public static ByteToMessageDecoder createFrameDecoder() { return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 8, -8, 8); } - /** Returns the remote address on the channel or "<remote address>" if none exists. */ + /** Returns the remote address on the channel or "<unknown remote>" if none exists. */ public static String getRemoteAddress(Channel channel) { if (channel != null && channel.remoteAddress() != null) { return channel.remoteAddress().toString(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 2eaf3b71d9a49..3b2eff377955a 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -17,6 +17,8 @@ package org.apache.spark.network.util; +import com.google.common.primitives.Ints; + /** * A central location that tracks all the settings we expose to users. */ @@ -37,8 +39,11 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int defaultTimeout = conf.getInt("spark.network.timeout", 120); - return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; } /** Number of concurrent connections between two nodes for fetching data. */ @@ -68,7 +73,9 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } + public int saslRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -80,7 +87,9 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + } /** * Minimum size of a block that we should start using memory map rather than reading in through @@ -105,4 +114,20 @@ public boolean lazyFileDescriptor() { public int portMaxRetries() { return conf.getInt("spark.port.maxRetries", 16); } + + /** + * Maximum number of bytes to be encrypted at a time when SASL encryption is enabled. + */ + public int maxSaslEncryptedBlockSize() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k"))); + } + + /** + * Whether the server should enforce encryption on SASL-authenticated connections. + */ + public boolean saslServerAlwaysEncrypt() { + return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); + } + } diff --git a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java index 43dc0cf8c7194..d500bc3c98a78 100644 --- a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java @@ -17,26 +17,35 @@ package org.apache.spark.network; +import java.util.List; + +import com.google.common.primitives.Ints; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.FileRegion; import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.MessageToMessageEncoder; import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.network.protocol.Message; -import org.apache.spark.network.protocol.StreamChunkId; -import org.apache.spark.network.protocol.ChunkFetchRequest; import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchRequest; import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.RpcRequest; -import org.apache.spark.network.protocol.RpcFailure; -import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.Message; import org.apache.spark.network.protocol.MessageDecoder; import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.util.ByteArrayWritableChannel; import org.apache.spark.network.util.NettyUtils; public class ProtocolSuite { private void testServerToClient(Message msg) { - EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder()); + EmbeddedChannel serverChannel = new EmbeddedChannel(new FileRegionEncoder(), + new MessageEncoder()); serverChannel.writeOutbound(msg); EmbeddedChannel clientChannel = new EmbeddedChannel( @@ -51,7 +60,8 @@ private void testServerToClient(Message msg) { } private void testClientToServer(Message msg) { - EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder()); + EmbeddedChannel clientChannel = new EmbeddedChannel(new FileRegionEncoder(), + new MessageEncoder()); clientChannel.writeOutbound(msg); EmbeddedChannel serverChannel = new EmbeddedChannel( @@ -83,4 +93,25 @@ public void responses() { testServerToClient(new RpcFailure(0, "this is an error")); testServerToClient(new RpcFailure(0, "")); } + + /** + * Handler to transform a FileRegion into a byte buffer. EmbeddedChannel doesn't actually transfer + * bytes, but messages, so this is needed so that the frame decoder on the receiving side can + * understand what MessageWithHeader actually contains. + */ + private static class FileRegionEncoder extends MessageToMessageEncoder { + + @Override + public void encode(ChannelHandlerContext ctx, FileRegion in, List out) + throws Exception { + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(Ints.checkedCast(in.count())); + while (in.transfered() < in.count()) { + in.transferTo(channel, in.transfered()); + } + out.add(Unpooled.wrappedBuffer(channel.getData())); + } + + } + } diff --git a/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java new file mode 100644 index 0000000000000..84ebb337e6d54 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java @@ -0,0 +1,277 @@ +/* + * 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; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; +import org.junit.*; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +/** + * Suite which ensures that requests that go without a response for the network timeout period are + * failed, and the connection closed. + * + * In this suite, we use 2 seconds as the connection timeout, with some slack given in the tests, + * to ensure stability in different test environments. + */ +public class RequestTimeoutIntegrationSuite { + + private TransportServer server; + private TransportClientFactory clientFactory; + + private StreamManager defaultManager; + private TransportConf conf; + + // A large timeout that "shouldn't happen", for the sake of faulty tests not hanging forever. + private final int FOREVER = 60 * 1000; + + @Before + public void setUp() throws Exception { + Map configMap = Maps.newHashMap(); + configMap.put("spark.shuffle.io.connectionTimeout", "2s"); + conf = new TransportConf(new MapConfigProvider(configMap)); + + defaultManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + throw new UnsupportedOperationException(); + } + }; + } + + @After + public void tearDown() { + if (server != null) { + server.close(); + } + if (clientFactory != null) { + clientFactory.close(); + } + } + + // Basic suite: First request completes quickly, and second waits for longer than network timeout. + @Test + public void timeoutInactiveRequests() throws Exception { + final Semaphore semaphore = new Semaphore(1); + final byte[] response = new byte[16]; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + try { + semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); + callback.onSuccess(response); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // First completes quickly (semaphore starts at 1). + TestCallback callback0 = new TestCallback(); + synchronized (callback0) { + client.sendRpc(new byte[0], callback0); + callback0.wait(FOREVER); + assert (callback0.success.length == response.length); + } + + // Second times out after 2 seconds, with slack. Must be IOException. + TestCallback callback1 = new TestCallback(); + synchronized (callback1) { + client.sendRpc(new byte[0], callback1); + callback1.wait(4 * 1000); + assert (callback1.failure != null); + assert (callback1.failure instanceof IOException); + } + semaphore.release(); + } + + // A timeout will cause the connection to be closed, invalidating the current TransportClient. + // It should be the case that requesting a client from the factory produces a new, valid one. + @Test + public void timeoutCleanlyClosesClient() throws Exception { + final Semaphore semaphore = new Semaphore(0); + final byte[] response = new byte[16]; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + try { + semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); + callback.onSuccess(response); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + + // First request should eventually fail. + TransportClient client0 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback0 = new TestCallback(); + synchronized (callback0) { + client0.sendRpc(new byte[0], callback0); + callback0.wait(FOREVER); + assert (callback0.failure instanceof IOException); + assert (!client0.isActive()); + } + + // Increment the semaphore and the second request should succeed quickly. + semaphore.release(2); + TransportClient client1 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback1 = new TestCallback(); + synchronized (callback1) { + client1.sendRpc(new byte[0], callback1); + callback1.wait(FOREVER); + assert (callback1.success.length == response.length); + assert (callback1.failure == null); + } + } + + // The timeout is relative to the LAST request sent, which is kinda weird, but still. + // This test also makes sure the timeout works for Fetch requests as well as RPCs. + @Test + public void furtherRequestsDelay() throws Exception { + final byte[] response = new byte[16]; + final StreamManager manager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS); + return new NioManagedBuffer(ByteBuffer.wrap(response)); + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return manager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // Send one request, which will eventually fail. + TestCallback callback0 = new TestCallback(); + client.fetchChunk(0, 0, callback0); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // Send a second request before the first has failed. + TestCallback callback1 = new TestCallback(); + client.fetchChunk(0, 1, callback1); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + synchronized (callback0) { + // not complete yet, but should complete soon + assert (callback0.success == null && callback0.failure == null); + callback0.wait(2 * 1000); + assert (callback0.failure instanceof IOException); + } + + synchronized (callback1) { + // failed at same time as previous + assert (callback0.failure instanceof IOException); + } + } + + /** + * Callback which sets 'success' or 'failure' on completion. + * Additionally notifies all waiters on this callback when invoked. + */ + class TestCallback implements RpcResponseCallback, ChunkReceivedCallback { + + byte[] success; + Throwable failure; + + @Override + public void onSuccess(byte[] response) { + synchronized(this) { + success = response; + this.notifyAll(); + } + } + + @Override + public void onFailure(Throwable e) { + synchronized(this) { + failure = e; + this.notifyAll(); + } + } + + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + synchronized(this) { + try { + success = buffer.nioByteBuffer().array(); + this.notifyAll(); + } catch (IOException e) { + // weird + } + } + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + synchronized(this) { + failure = e; + this.notifyAll(); + } + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 416dc1b969fa4..35de5e57ccb98 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -20,10 +20,11 @@ import java.io.IOException; import java.util.Collections; import java.util.HashSet; -import java.util.NoSuchElementException; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import com.google.common.collect.Maps; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -36,9 +37,9 @@ import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.ConfigProvider; -import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class TransportClientFactorySuite { @@ -70,16 +71,10 @@ public void tearDown() { */ private void testClientReuse(final int maxConnections, boolean concurrent) throws IOException, InterruptedException { - TransportConf conf = new TransportConf(new ConfigProvider() { - @Override - public String get(String name) { - if (name.equals("spark.shuffle.io.numConnectionsPerPeer")) { - return Integer.toString(maxConnections); - } else { - throw new NoSuchElementException(); - } - } - }); + + Map configMap = Maps.newHashMap(); + configMap.put("spark.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); + TransportConf conf = new TransportConf(new MapConfigProvider(configMap)); RpcHandler rpcHandler = new NoOpRpcHandler(); TransportContext context = new TransportContext(conf, rpcHandler); diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java new file mode 100644 index 0000000000000..6c98e733b462f --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -0,0 +1,129 @@ +/* + * 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.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.util.ByteArrayWritableChannel; + +public class MessageWithHeaderSuite { + + @Test + public void testSingleWrite() throws Exception { + testFileRegionBody(8, 8); + } + + @Test + public void testShortWrite() throws Exception { + testFileRegionBody(8, 1); + } + + @Test + public void testByteBufBody() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ByteBuf body = Unpooled.copyLong(84); + MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); + + ByteBuf result = doWrite(msg, 1); + assertEquals(msg.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + assertEquals(84, result.readLong()); + } + + private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { + ByteBuf header = Unpooled.copyLong(42); + int headerLength = header.readableBytes(); + TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); + MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); + + ByteBuf result = doWrite(msg, totalWrites / writesPerCall); + assertEquals(headerLength + region.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + for (long i = 0; i < 8; i++) { + assertEquals(i, result.readLong()); + } + } + + private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { + int writes = 0; + ByteArrayWritableChannel channel = new ByteArrayWritableChannel((int) msg.count()); + while (msg.transfered() < msg.count()) { + msg.transferTo(channel, msg.transfered()); + writes++; + } + assertTrue("Not enough writes!", minExpectedWrites <= writes); + return Unpooled.wrappedBuffer(channel.getData()); + } + + private static class TestFileRegion extends AbstractReferenceCounted implements FileRegion { + + private final int writeCount; + private final int writesPerCall; + private int written; + + TestFileRegion(int totalWrites, int writesPerCall) { + this.writeCount = totalWrites; + this.writesPerCall = writesPerCall; + } + + @Override + public long count() { + return 8 * writeCount; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return 8 * written; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + for (int i = 0; i < writesPerCall; i++) { + ByteBuf buf = Unpooled.copyLong((position / 8) + i); + ByteBuffer nio = buf.nioBuffer(); + while (nio.remaining() > 0) { + target.write(nio); + } + buf.release(); + written++; + } + return 8 * writesPerCall; + } + + @Override + protected void deallocate() { + } + + } + +} diff --git a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 23b4e06f064e1..be6632bb8cf49 100644 --- a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -17,12 +17,47 @@ package org.apache.spark.network.sasl; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static com.google.common.base.Charsets.UTF_8; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import javax.security.sasl.SaslException; + +import com.google.common.collect.Lists; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.ByteArrayWritableChannel; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; /** * Jointly tests SparkSaslClient and SparkSaslServer, as both are black boxes. @@ -44,8 +79,8 @@ public String getSecretKey(String appId) { @Test public void testMatching() { - SparkSaslClient client = new SparkSaslClient("shared-secret", secretKeyHolder); - SparkSaslServer server = new SparkSaslServer("shared-secret", secretKeyHolder); + SparkSaslClient client = new SparkSaslClient("shared-secret", secretKeyHolder, false); + SparkSaslServer server = new SparkSaslServer("shared-secret", secretKeyHolder, false); assertFalse(client.isComplete()); assertFalse(server.isComplete()); @@ -64,11 +99,10 @@ public void testMatching() { assertFalse(client.isComplete()); } - @Test public void testNonMatching() { - SparkSaslClient client = new SparkSaslClient("my-secret", secretKeyHolder); - SparkSaslServer server = new SparkSaslServer("your-secret", secretKeyHolder); + SparkSaslClient client = new SparkSaslClient("my-secret", secretKeyHolder, false); + SparkSaslServer server = new SparkSaslServer("your-secret", secretKeyHolder, false); assertFalse(client.isComplete()); assertFalse(server.isComplete()); @@ -86,4 +120,312 @@ public void testNonMatching() { assertFalse(server.isComplete()); } } + + @Test + public void testSaslAuthentication() throws Exception { + testBasicSasl(false); + } + + @Test + public void testSaslEncryption() throws Exception { + testBasicSasl(true); + } + + private void testBasicSasl(boolean encrypt) throws Exception { + RpcHandler rpcHandler = mock(RpcHandler.class); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) { + byte[] message = (byte[]) invocation.getArguments()[1]; + RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; + assertEquals("Ping", new String(message, UTF_8)); + cb.onSuccess("Pong".getBytes(UTF_8)); + return null; + } + }) + .when(rpcHandler) + .receive(any(TransportClient.class), any(byte[].class), any(RpcResponseCallback.class)); + + SaslTestCtx ctx = new SaslTestCtx(rpcHandler, encrypt, false); + try { + byte[] response = ctx.client.sendRpcSync("Ping".getBytes(UTF_8), TimeUnit.SECONDS.toMillis(10)); + assertEquals("Pong", new String(response, UTF_8)); + } finally { + ctx.close(); + } + } + + @Test + public void testEncryptedMessage() throws Exception { + SaslEncryptionBackend backend = mock(SaslEncryptionBackend.class); + byte[] data = new byte[1024]; + new Random().nextBytes(data); + when(backend.wrap(any(byte[].class), anyInt(), anyInt())).thenReturn(data); + + ByteBuf msg = Unpooled.buffer(); + try { + msg.writeBytes(data); + + // Create a channel with a really small buffer compared to the data. This means that on each + // call, the outbound data will not be fully written, so the write() method should return a + // dummy count to keep the channel alive when possible. + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(32); + + SaslEncryption.EncryptedMessage emsg = + new SaslEncryption.EncryptedMessage(backend, msg, 1024); + long count = emsg.transferTo(channel, emsg.transfered()); + assertTrue(count < data.length); + assertTrue(count > 0); + + // Here, the output buffer is full so nothing should be transferred. + assertEquals(0, emsg.transferTo(channel, emsg.transfered())); + + // Now there's room in the buffer, but not enough to transfer all the remaining data, + // so the dummy count should be returned. + channel.reset(); + assertEquals(1, emsg.transferTo(channel, emsg.transfered())); + + // Eventually, the whole message should be transferred. + for (int i = 0; i < data.length / 32 - 2; i++) { + channel.reset(); + assertEquals(1, emsg.transferTo(channel, emsg.transfered())); + } + + channel.reset(); + count = emsg.transferTo(channel, emsg.transfered()); + assertTrue("Unexpected count: " + count, count > 1 && count < data.length); + assertEquals(data.length, emsg.transfered()); + } finally { + msg.release(); + } + } + + @Test + public void testEncryptedMessageChunking() throws Exception { + File file = File.createTempFile("sasltest", ".txt"); + try { + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + + byte[] data = new byte[8 * 1024]; + new Random().nextBytes(data); + Files.write(data, file); + + SaslEncryptionBackend backend = mock(SaslEncryptionBackend.class); + // It doesn't really matter what we return here, as long as it's not null. + when(backend.wrap(any(byte[].class), anyInt(), anyInt())).thenReturn(data); + + FileSegmentManagedBuffer msg = new FileSegmentManagedBuffer(conf, file, 0, file.length()); + SaslEncryption.EncryptedMessage emsg = + new SaslEncryption.EncryptedMessage(backend, msg.convertToNetty(), data.length / 8); + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(data.length); + while (emsg.transfered() < emsg.count()) { + channel.reset(); + emsg.transferTo(channel, emsg.transfered()); + } + + verify(backend, times(8)).wrap(any(byte[].class), anyInt(), anyInt()); + } finally { + file.delete(); + } + } + + @Test + public void testFileRegionEncryption() throws Exception { + final String blockSizeConf = "spark.network.sasl.maxEncryptedBlockSize"; + System.setProperty(blockSizeConf, "1k"); + + final AtomicReference response = new AtomicReference(); + final File file = File.createTempFile("sasltest", ".txt"); + SaslTestCtx ctx = null; + try { + final TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + StreamManager sm = mock(StreamManager.class); + when(sm.getChunk(anyLong(), anyInt())).thenAnswer(new Answer() { + @Override + public ManagedBuffer answer(InvocationOnMock invocation) { + return new FileSegmentManagedBuffer(conf, file, 0, file.length()); + } + }); + + RpcHandler rpcHandler = mock(RpcHandler.class); + when(rpcHandler.getStreamManager()).thenReturn(sm); + + byte[] data = new byte[8 * 1024]; + new Random().nextBytes(data); + Files.write(data, file); + + ctx = new SaslTestCtx(rpcHandler, true, false); + + final Object lock = new Object(); + + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) { + response.set((ManagedBuffer) invocation.getArguments()[1]); + response.get().retain(); + synchronized (lock) { + lock.notifyAll(); + } + return null; + } + }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); + + synchronized (lock) { + ctx.client.fetchChunk(0, 0, callback); + lock.wait(10 * 1000); + } + + verify(callback, times(1)).onSuccess(anyInt(), any(ManagedBuffer.class)); + verify(callback, never()).onFailure(anyInt(), any(Throwable.class)); + + byte[] received = ByteStreams.toByteArray(response.get().createInputStream()); + assertTrue(Arrays.equals(data, received)); + } finally { + file.delete(); + if (ctx != null) { + ctx.close(); + } + if (response.get() != null) { + response.get().release(); + } + System.clearProperty(blockSizeConf); + } + } + + @Test + public void testServerAlwaysEncrypt() throws Exception { + final String alwaysEncryptConfName = "spark.network.sasl.serverAlwaysEncrypt"; + System.setProperty(alwaysEncryptConfName, "true"); + + SaslTestCtx ctx = null; + try { + ctx = new SaslTestCtx(mock(RpcHandler.class), false, false); + fail("Should have failed to connect without encryption."); + } catch (Exception e) { + assertTrue(e.getCause() instanceof SaslException); + } finally { + if (ctx != null) { + ctx.close(); + } + System.clearProperty(alwaysEncryptConfName); + } + } + + @Test + public void testDataEncryptionIsActuallyEnabled() throws Exception { + // This test sets up an encrypted connection but then, using a client bootstrap, removes + // the encryption handler from the client side. This should cause the server to not be + // able to understand RPCs sent to it and thus close the connection. + SaslTestCtx ctx = null; + try { + ctx = new SaslTestCtx(mock(RpcHandler.class), true, true); + ctx.client.sendRpcSync("Ping".getBytes(UTF_8), TimeUnit.SECONDS.toMillis(10)); + fail("Should have failed to send RPC to server."); + } catch (Exception e) { + assertFalse(e.getCause() instanceof TimeoutException); + } finally { + if (ctx != null) { + ctx.close(); + } + } + } + + private static class SaslTestCtx { + + final TransportClient client; + final TransportServer server; + + private final boolean encrypt; + private final boolean disableClientEncryption; + private final EncryptionCheckerBootstrap checker; + + SaslTestCtx( + RpcHandler rpcHandler, + boolean encrypt, + boolean disableClientEncryption) + throws Exception { + + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + + SecretKeyHolder keyHolder = mock(SecretKeyHolder.class); + when(keyHolder.getSaslUser(anyString())).thenReturn("user"); + when(keyHolder.getSecretKey(anyString())).thenReturn("secret"); + + TransportContext ctx = new TransportContext(conf, rpcHandler); + + this.checker = new EncryptionCheckerBootstrap(); + this.server = ctx.createServer(Arrays.asList(new SaslServerBootstrap(conf, keyHolder), + checker)); + + try { + List clientBootstraps = Lists.newArrayList(); + clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder, encrypt)); + if (disableClientEncryption) { + clientBootstraps.add(new EncryptionDisablerBootstrap()); + } + + this.client = ctx.createClientFactory(clientBootstraps) + .createClient(TestUtils.getLocalHost(), server.getPort()); + } catch (Exception e) { + close(); + throw e; + } + + this.encrypt = encrypt; + this.disableClientEncryption = disableClientEncryption; + } + + void close() { + if (!disableClientEncryption) { + assertEquals(encrypt, checker.foundEncryptionHandler); + } + if (client != null) { + client.close(); + } + if (server != null) { + server.close(); + } + } + + } + + private static class EncryptionCheckerBootstrap extends ChannelOutboundHandlerAdapter + implements TransportServerBootstrap { + + boolean foundEncryptionHandler; + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + if (!foundEncryptionHandler) { + foundEncryptionHandler = + ctx.channel().pipeline().get(SaslEncryption.ENCRYPTION_HANDLER_NAME) != null; + } + ctx.write(msg, promise); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + super.handlerRemoved(ctx); + } + + @Override + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + channel.pipeline().addFirst("encryptionChecker", this); + return rpcHandler; + } + + } + + private static class EncryptionDisablerBootstrap implements TransportClientBootstrap { + + @Override + public void doBootstrap(TransportClient client, Channel channel) { + channel.pipeline().remove(SaslEncryption.ENCRYPTION_HANDLER_NAME); + } + + } + } diff --git a/network/common/src/test/resources/log4j.properties b/network/common/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e8da774f7ca9e --- /dev/null +++ b/network/common/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 6e8018b723dc6..612bce571a493 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.List; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,6 +47,7 @@ public class ExternalShuffleClient extends ShuffleClient { private final TransportConf conf; private final boolean saslEnabled; + private final boolean saslEncryptionEnabled; private final SecretKeyHolder secretKeyHolder; private TransportClientFactory clientFactory; @@ -58,10 +60,15 @@ public class ExternalShuffleClient extends ShuffleClient { public ExternalShuffleClient( TransportConf conf, SecretKeyHolder secretKeyHolder, - boolean saslEnabled) { + boolean saslEnabled, + boolean saslEncryptionEnabled) { + Preconditions.checkArgument( + !saslEncryptionEnabled || saslEnabled, + "SASL encryption can only be enabled if SASL is also enabled."); this.conf = conf; this.secretKeyHolder = secretKeyHolder; this.saslEnabled = saslEnabled; + this.saslEncryptionEnabled = saslEncryptionEnabled; } @Override @@ -70,7 +77,7 @@ public void init(String appId) { TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); List bootstraps = Lists.newArrayList(); if (saslEnabled) { - bootstraps.add(new SaslClientBootstrap(conf, appId, secretKeyHolder)); + bootstraps.add(new SaslClientBootstrap(conf, appId, secretKeyHolder, saslEncryptionEnabled)); } clientFactory = context.createClientFactory(bootstraps); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index d25283e46ef96..382f613ecbb1b 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.network.sasl; import java.io.IOException; +import java.util.Arrays; import com.google.common.collect.Lists; import org.junit.After; @@ -37,6 +38,7 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -72,10 +74,11 @@ public String getSecretKey(String appId) { @BeforeClass public static void beforeAll() throws IOException { SecretKeyHolder secretKeyHolder = new TestSecretKeyHolder("good-key"); - SaslRpcHandler handler = new SaslRpcHandler(new TestRpcHandler(), secretKeyHolder); conf = new TransportConf(new SystemPropertyConfigProvider()); - context = new TransportContext(conf, handler); - server = context.createServer(); + context = new TransportContext(conf, new TestRpcHandler()); + + TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder); + server = context.createServer(Arrays.asList(bootstrap)); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 02c10bcb7b261..39aa49911d9cb 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -136,7 +136,7 @@ private FetchResult fetchBlocks(String execId, String[] blockIds, int port) thro final Semaphore requestsRemaining = new Semaphore(0); - ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, false); client.init(APP_ID); client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, new BlockFetchingListener() { @@ -274,7 +274,7 @@ public void testFetchNoServer() throws Exception { private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) throws IOException { - ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, false); client.init(APP_ID); client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), executorId, executorInfo); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 759a12910c94d..d4ec1956c1e29 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -18,6 +18,7 @@ package org.apache.spark.network.shuffle; import java.io.IOException; +import java.util.Arrays; import org.junit.After; import org.junit.Before; @@ -27,10 +28,11 @@ import org.apache.spark.network.TestUtils; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.SaslServerBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -42,10 +44,10 @@ public class ExternalShuffleSecuritySuite { @Before public void beforeEach() { - RpcHandler handler = new SaslRpcHandler(new ExternalShuffleBlockHandler(conf), - new TestSecretKeyHolder("my-app-id", "secret")); - TransportContext context = new TransportContext(conf, handler); - this.server = context.createServer(); + TransportContext context = new TransportContext(conf, new ExternalShuffleBlockHandler(conf)); + TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, + new TestSecretKeyHolder("my-app-id", "secret")); + this.server = context.createServer(Arrays.asList(bootstrap)); } @After @@ -58,13 +60,13 @@ public void afterEach() { @Test public void testValid() throws IOException { - validate("my-app-id", "secret"); + validate("my-app-id", "secret", false); } @Test public void testBadAppId() { try { - validate("wrong-app-id", "secret"); + validate("wrong-app-id", "secret", false); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("Wrong appId!")); } @@ -73,16 +75,21 @@ public void testBadAppId() { @Test public void testBadSecret() { try { - validate("my-app-id", "bad-secret"); + validate("my-app-id", "bad-secret", false); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("Mismatched response")); } } + @Test + public void testEncryption() throws IOException { + validate("my-app-id", "secret", true); + } + /** Creates an ExternalShuffleClient and attempts to register with the server. */ - private void validate(String appId, String secretKey) throws IOException { + private void validate(String appId, String secretKey, boolean encrypt) throws IOException { ExternalShuffleClient client = - new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true); + new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true, encrypt); client.init(appId); // Registration either succeeds or throws an exception. client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 63b21222e7b77..463f99ef3352d 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -17,9 +17,10 @@ package org.apache.spark.network.yarn; -import java.lang.Override; import java.nio.ByteBuffer; +import java.util.List; +import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -32,10 +33,11 @@ import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.SaslServerBootstrap; import org.apache.spark.network.sasl.ShuffleSecretManager; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; import org.apache.spark.network.util.TransportConf; import org.apache.spark.network.yarn.util.HadoopConfigProvider; @@ -103,16 +105,17 @@ protected void serviceInit(Configuration conf) { // special RPC handler that filters out unauthenticated fetch requests boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); blockHandler = new ExternalShuffleBlockHandler(transportConf); - RpcHandler rpcHandler = blockHandler; + + List bootstraps = Lists.newArrayList(); if (authEnabled) { secretManager = new ShuffleSecretManager(); - rpcHandler = new SaslRpcHandler(rpcHandler, secretManager); + bootstraps.add(new SaslServerBootstrap(transportConf, secretManager)); } int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportContext transportContext = new TransportContext(transportConf, rpcHandler); - shuffleServer = transportContext.createServer(port); + TransportContext transportContext = new TransportContext(transportConf, blockHandler); + shuffleServer = transportContext.createServer(port, bootstraps); String authEnabledString = authEnabled ? "enabled" : "not enabled"; logger.info("Started YARN shuffle service for Spark on port {}. " + "Authentication is {}.", port, authEnabledString); diff --git a/pom.xml b/pom.xml index 23bb16130b504..94cf28332cf47 100644 --- a/pom.xml +++ b/pom.xml @@ -97,6 +97,7 @@ sql/catalyst sql/core sql/hive + unsafe assembly external/twitter external/flume @@ -116,7 +117,7 @@ 1.6 spark 2.0.1 - 0.21.0 + 0.21.1 shaded-protobuf 1.7.10 1.2.17 @@ -141,13 +142,13 @@ 2.4.0 2.0.8 3.1.0 - 1.7.6 + 1.7.7 0.7.1 1.8.3 1.1.0 - 4.2.6 - 3.1.1 + 4.3.2 + 3.4.1 ${project.build.directory}/spark-test-classpath.txt 2.10.4 2.10 @@ -156,9 +157,11 @@ 3.6.3 1.8.8 2.4.4 - 1.1.1.6 + 1.1.1.7 1.1.2 + ${java.home} + @@ -1194,6 +1224,7 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + ${test.java.home} true @@ -1202,6 +1233,7 @@ false false true + true false @@ -1224,6 +1256,7 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + ${test.java.home} true @@ -1246,17 +1279,17 @@ org.apache.maven.plugins maven-jar-plugin - 2.4 + 2.6 org.apache.maven.plugins maven-antrun-plugin - 1.7 + 1.8 org.apache.maven.plugins maven-source-plugin - 2.2.1 + 2.4 true @@ -1265,6 +1298,7 @@ create-source-jar jar-no-fork + test-jar-no-fork @@ -1272,7 +1306,7 @@ org.apache.maven.plugins maven-clean-plugin - 2.5 + 2.6.1 @@ -1290,7 +1324,27 @@ org.apache.maven.plugins maven-javadoc-plugin - 2.10.1 + 2.10.3 + + + org.codehaus.mojo + exec-maven-plugin + 1.4.0 + + + org.apache.maven.plugins + maven-assembly-plugin + 2.5.3 + + + org.apache.maven.plugins + maven-install-plugin + 2.5.2 + + + org.apache.maven.plugins + maven-deploy-plugin + 2.8.2 @@ -1300,7 +1354,7 @@ org.apache.maven.plugins maven-dependency-plugin - 2.9 + 2.10 test-compile @@ -1319,7 +1373,7 @@ org.codehaus.gmavenplus gmavenplus-plugin - 1.2 + 1.5 process-test-classes @@ -1344,7 +1398,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.2 + 2.3 false @@ -1442,7 +1496,7 @@ org.scalastyle scalastyle-maven-plugin - 0.4.0 + 0.7.0 false true @@ -1451,12 +1505,12 @@ ${basedir}/src/main/scala ${basedir}/src/test/scala scalastyle-config.xml - scalastyle-output.xml - UTF-8 + ${basedir}/target/scalastyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} - package check @@ -1472,6 +1526,25 @@ org.scalatest scalatest-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + prepare-package + + test-jar + + + + log4j.properties + + + + + @@ -1558,20 +1631,6 @@ http://hadoop.apache.org/docs/ra.b.c/hadoop-project-dist/hadoop-common/dependency-analysis.html --> - - hadoop-0.23 - - - - org.apache.avro - avro - - - - 0.23.10 - - - hadoop-2.2 @@ -1695,15 +1754,25 @@ + + test-java-home + + env.JAVA_HOME + + + ${env.JAVA_HOME} + + + scala-2.11 scala-2.11 - 2.11.2 + 2.11.6 2.11 - 2.12 + 2.12.1 jline @@ -1728,5 +1797,8 @@ parquet-provided + + sparkr + diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 328d59485a731..bf343d4b7e40b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -44,7 +44,66 @@ object MimaExcludes { // the maven-generated artifacts in 1.3. excludePackage("org.spark-project.jetty"), MimaBuild.excludeSparkPackage("unused"), - ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional") + ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.rdd.JdbcRDD.compute"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.scheduler.OutputCommitCoordinator$OutputCommitCoordinatorActor") + ) ++ Seq( + // SPARK-4655 - Making Stage an Abstract class broke binary compatility even though + // the stage class is defined as private[spark] + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.scheduler.Stage") + ) ++ Seq( + // SPARK-6510 Add a Graph#minus method acting as Set#difference + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.minus") + ) ++ Seq( + // SPARK-6492 Fix deadlock in SparkContext.stop() + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.org$" + + "apache$spark$SparkContext$$SPARK_CONTEXT_CONSTRUCTOR_LOCK") + )++ Seq( + // SPARK-6693 add tostring with max lines and width for matrix + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrix.toString") + )++ Seq( + // SPARK-6703 Add getOrCreate method to SparkContext + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkContext.org$apache$spark$SparkContext$$activeContext") + )++ Seq( + // SPARK-7090 Introduce LDAOptimizer to LDA to further improve extensibility + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.mllib.clustering.LDA$EMOptimizer") + ) ++ Seq( + // SPARK-6756 add toSparse, toDense, numActives, numNonzeros, and compressed to Vector + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Vector.compressed"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Vector.toDense"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Vector.numNonzeros"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Vector.toSparse"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Vector.numActives") + ) ++ Seq( + // This `protected[sql]` method was removed in 1.3.1 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.sql.SQLContext.checkAnalysis"), + // This `private[sql]` class was removed in 1.4.0: + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.execution.AddExchange"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.execution.AddExchange$"), + // These test support classes were moved out of src/main and into src/test: + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.ParquetTestData"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.ParquetTestData$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.TestGroupWriteSupport") ) case v if v.startsWith("1.3") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ac37c605de4b6..b4431c7ee05b6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -import java.io.File +import java.io._ import scala.util.Properties import scala.collection.JavaConversions._ @@ -34,11 +34,11 @@ object BuildCommons { val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, - streamingMqtt, streamingTwitter, streamingZeromq, launcher) = + streamingMqtt, streamingTwitter, streamingZeromq, launcher, unsafe) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq", "launcher").map(ProjectRef(buildLocation, _)) + "streaming-zeromq", "launcher", "unsafe").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", @@ -119,7 +119,9 @@ object SparkBuild extends PomBuild { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") lazy val sharedSettings = graphSettings ++ genjavadocSettings ++ Seq ( - javaHome := Properties.envOrNone("JAVA_HOME").map(file), + javaHome := sys.env.get("JAVA_HOME") + .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) + .map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -154,16 +156,21 @@ object SparkBuild extends PomBuild { /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - // TODO: Add Sql to mima checks - // TODO: remove launcher from this list after 1.3. - allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - networkCommon, networkShuffle, networkYarn, launcher).contains(x)).foreach { + // TODO: remove launcher from this list after 1.4.0 + allProjects.filterNot(x => Seq(spark, hive, hiveThriftServer, catalyst, repl, + networkCommon, networkShuffle, networkYarn, launcher, unsafe).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } + /* Unsafe settings */ + enable(Unsafe.settings)(unsafe) + /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) + /* Package pyspark artifacts in the main assembly. */ + enable(PySparkAssembly.settings)(assembly) + /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) @@ -211,6 +218,13 @@ object SparkBuild extends PomBuild { } +object Unsafe { + lazy val settings = Seq( + // This option is needed to suppress warnings from sun.misc.Unsafe usage + javacOptions in Compile += "-XDignore.symbol.file" + ) +} + object Flume { lazy val settings = sbtavro.SbtAvro.avroSettings } @@ -314,6 +328,7 @@ object Hive { } object Assembly { + import sbtassembly.AssemblyUtils._ import sbtassembly.Plugin._ import AssemblyKeys._ @@ -345,6 +360,60 @@ object Assembly { ) } +object PySparkAssembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ + + lazy val settings = Seq( + unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, + // Use a resource generator to copy all .py files from python/pyspark into a managed directory + // to be included in the assembly. We can't just add "python/" to the assembly's resource dir + // list since that will copy unneeded / unwanted files. + resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val dst = new File(outDir, "pyspark") + if (!dst.isDirectory()) { + require(dst.mkdirs()) + } + + val src = new File(BuildCommons.sparkHome, "python/pyspark") + copy(src, dst) + } + ) + + private def copy(src: File, dst: File): Seq[File] = { + src.listFiles().flatMap { f => + val child = new File(dst, f.getName()) + if (f.isDirectory()) { + child.mkdir() + copy(f, child) + } else if (f.getName().endsWith(".py")) { + var in: Option[FileInputStream] = None + var out: Option[FileOutputStream] = None + try { + in = Some(new FileInputStream(f)) + out = Some(new FileOutputStream(child)) + + val bytes = new Array[Byte](1024) + var read = 0 + while (read >= 0) { + read = in.get.read(bytes) + if (read > 0) { + out.get.write(bytes, 0, read) + } + } + + Some(child) + } finally { + in.foreach(_.close()) + out.foreach(_.close()) + } + } else { + None + } + } + } +} + object Unidoc { import BuildCommons._ @@ -360,15 +429,16 @@ object Unidoc { packages .map(_.filterNot(_.getName.contains("$"))) .map(_.filterNot(_.getCanonicalPath.contains("akka"))) - .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) - .map(_.filterNot(_.getCanonicalPath.contains("network"))) - .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) - .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/shuffle"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/unsafe"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) - .map(_.filterNot(_.getCanonicalPath.contains("collection"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/catalyst"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/execution"))) - .map(_.filterNot(_.getCanonicalPath.contains("sql/hive/test"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/execution"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test"))) } lazy val settings = scalaJavaUnidocSettings ++ Seq ( @@ -407,7 +477,7 @@ object Unidoc { "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation", "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss", "ml", "ml.attribute", "ml.classification", "ml.evaluation", "ml.feature", "ml.param", - "ml.tuning" + "ml.recommendation", "ml.regression", "ml.tuning" ), "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" @@ -426,14 +496,17 @@ object TestSettings { fork := true, // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. - envVars in Test += ("SPARK_DIST_CLASSPATH" -> - (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":")), + envVars in Test ++= Map( + "SPARK_DIST_CLASSPATH" -> + (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), + "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true", + javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/project/plugins.sbt b/project/plugins.sbt index ee45b6a51905e..7096b0d3ee7de 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,7 +19,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") -addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.7.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 8863f272da415..471d00bd8223f 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -24,20 +24,6 @@ import sbt.Keys._ * becomes available for scalastyle sbt plugin. */ object SparkPluginDef extends Build { - lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle, sbtPomReader) - lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) + lazy val root = Project("plugins", file(".")) dependsOn(sbtPomReader) lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git#ignore_artifact_id") - - // There is actually no need to publish this artifact. - def styleSettings = Defaults.defaultSettings ++ Seq ( - name := "spark-style", - organization := "org.apache.spark", - scalaVersion := "2.10.4", - scalacOptions := Seq("-unchecked", "-deprecation"), - libraryDependencies ++= Dependencies.scalaStyle - ) - - object Dependencies { - val scalaStyle = Seq("org.scalastyle" %% "scalastyle" % "0.4.0") - } } diff --git a/python/docs/index.rst b/python/docs/index.rst index d150de9d5c502..f7eede9c3c82a 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -29,6 +29,14 @@ Core classes: A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. + :class:`pyspark.sql.SQLContext` + + Main entry point for DataFrame and SQL functionality. + + :class:`pyspark.sql.DataFrame` + + A distributed collection of data grouped into named columns. + Indices and tables ================== diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index 15101470afc07..26ece4c2c389a 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -31,6 +31,13 @@ pyspark.mllib.feature module :undoc-members: :show-inheritance: +pyspark.mllib.fpm module +------------------------ + +.. automodule:: pyspark.mllib.fpm + :members: + :undoc-members: + pyspark.mllib.linalg module --------------------------- diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index 7890d9dcaac21..50822c93faba1 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -10,7 +10,7 @@ Module contents :show-inheritance: pyspark.streaming.kafka module ----------------------------- +------------------------------ .. automodule:: pyspark.streaming.kafka :members: :undoc-members: diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..0d21a132048a5 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -54,7 +54,7 @@ ... def zero(self, value): ... return [0.0] * len(value) ... def addInPlace(self, val1, val2): -... for i in xrange(len(val1)): +... for i in range(len(val1)): ... val1[i] += val2[i] ... return val1 >>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam()) @@ -83,12 +83,16 @@ >>> sc.accumulator([1.0, 2.0, 3.0]) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... -Exception:... +TypeError:... """ +import sys import select import struct -import SocketServer +if sys.version < '3': + import SocketServer +else: + import socketserver as SocketServer import threading from pyspark.cloudpickle import CloudPickler from pyspark.serializers import read_int, PickleSerializer @@ -247,6 +251,7 @@ class AccumulatorServer(SocketServer.TCPServer): def shutdown(self): self.server_shutdown = True SocketServer.TCPServer.shutdown(self) + self.server_close() def _start_update_server(): diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 6b8a8b256a891..3de4615428bb6 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -16,10 +16,15 @@ # import os -import cPickle +import sys import gc from tempfile import NamedTemporaryFile +if sys.version < '3': + import cPickle as pickle +else: + import pickle + unicode = str __all__ = ['Broadcast'] @@ -70,33 +75,19 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None): self._path = path def dump(self, value, f): - if isinstance(value, basestring): - if isinstance(value, unicode): - f.write('U') - value = value.encode('utf8') - else: - f.write('S') - f.write(value) - else: - f.write('P') - cPickle.dump(value, f, 2) + pickle.dump(value, f, 2) f.close() return f.name def load(self, path): with open(path, 'rb', 1 << 20) as f: - flag = f.read(1) - data = f.read() - if flag == 'P': - # cPickle.loads() may create lots of objects, disable GC - # temporary for better performance - gc.disable() - try: - return cPickle.loads(data) - finally: - gc.enable() - else: - return data.decode('utf8') if flag == 'U' else data + # pickle.load() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return pickle.load(f) + finally: + gc.enable() @property def value(self): diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index bb0783555aa77..9ef93071d2e77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -40,164 +40,126 @@ NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ - +from __future__ import print_function import operator import os +import io import pickle import struct import sys import types from functools import partial import itertools -from copy_reg import _extension_registry, _inverted_registry, _extension_cache -import new import dis import traceback -import platform - -PyImp = platform.python_implementation() - -import logging -cloudLog = logging.getLogger("Cloud.Transport") +if sys.version < '3': + from pickle import Pickler + try: + from cStringIO import StringIO + except ImportError: + from StringIO import StringIO + PY3 = False +else: + types.ClassType = type + from pickle import _Pickler as Pickler + from io import BytesIO as StringIO + PY3 = True #relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +STORE_GLOBAL = dis.opname.index('STORE_GLOBAL') +DELETE_GLOBAL = dis.opname.index('DELETE_GLOBAL') +LOAD_GLOBAL = dis.opname.index('LOAD_GLOBAL') GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) - -if PyImp == "PyPy": - # register builtin type in `new` - new.method = types.MethodType - -try: - from cStringIO import StringIO -except ImportError: - from StringIO import StringIO -# These helper functions were copied from PiCloud's util module. def islambda(func): - return getattr(func,'func_name') == '' + return getattr(func,'__name__') == '' -def xrange_params(xrangeobj): - """Returns a 3 element tuple describing the xrange start, step, and len - respectively - Note: Only guarentees that elements of xrange are the same. parameters may - be different. - e.g. xrange(1,1) is interpretted as xrange(0,0); both behave the same - though w/ iteration - """ - - xrange_len = len(xrangeobj) - if not xrange_len: #empty - return (0,1,0) - start = xrangeobj[0] - if xrange_len == 1: #one element - return start, 1, 1 - return (start, xrangeobj[1] - xrangeobj[0], xrange_len) - -#debug variables intended for developer use: -printSerialization = False -printMemoization = False +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k -useForcedImports = True #Should I use forced imports for tracking? +def _builtin_type(name): + return getattr(types, name) -class CloudPickler(pickle.Pickler): +class CloudPickler(Pickler): - dispatch = pickle.Pickler.dispatch.copy() - savedForceImports = False - savedDjangoEnv = False #hack tro transport django environment + dispatch = Pickler.dispatch.copy() - def __init__(self, file, protocol=None, min_size_to_save= 0): - pickle.Pickler.__init__(self,file,protocol) - self.modules = set() #set of modules needed to depickle - self.globals_ref = {} # map ids to dictionary. used to ensure that functions can share global env + def __init__(self, file, protocol=None): + Pickler.__init__(self, file, protocol) + # set of modules to unpickle + self.modules = set() + # map ids to dictionary. used to ensure that functions can share global env + self.globals_ref = {} def dump(self, obj): - # note: not thread safe - # minimal side-effects, so not fixing - recurse_limit = 3000 - base_recurse = sys.getrecursionlimit() - if base_recurse < recurse_limit: - sys.setrecursionlimit(recurse_limit) self.inject_addons() try: - return pickle.Pickler.dump(self, obj) - except RuntimeError, e: + return Pickler.dump(self, obj) + except RuntimeError as e: if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required. - Try _fast_serialization=2 or contact PiCloud support""" + msg = """Could not pickle object as excessively deep recursion required.""" raise pickle.PicklingError(msg) - finally: - new_recurse = sys.getrecursionlimit() - if new_recurse == recurse_limit: - sys.setrecursionlimit(base_recurse) + + def save_memoryview(self, obj): + """Fallback to save_string""" + Pickler.save_string(self, str(obj)) def save_buffer(self, obj): """Fallback to save_string""" - pickle.Pickler.save_string(self,str(obj)) - dispatch[buffer] = save_buffer + Pickler.save_string(self,str(obj)) + if PY3: + dispatch[memoryview] = save_memoryview + else: + dispatch[buffer] = save_buffer - #block broken objects - def save_unsupported(self, obj, pack=None): + def save_unsupported(self, obj): raise pickle.PicklingError("Cannot pickle objects of type %s" % type(obj)) dispatch[types.GeneratorType] = save_unsupported - #python2.6+ supports slice pickling. some py2.5 extensions might as well. We just test it - try: - slice(0,1).__reduce__() - except TypeError: #can't pickle - - dispatch[slice] = save_unsupported - - #itertools objects do not pickle! + # itertools objects do not pickle! for v in itertools.__dict__.values(): if type(v) is type: dispatch[v] = save_unsupported - - def save_dict(self, obj): - """hack fix - If the dict is a global, deal with it in a special way - """ - #print 'saving', obj - if obj is __builtins__: - self.save_reduce(_get_module_builtins, (), obj=obj) - else: - pickle.Pickler.save_dict(self, obj) - dispatch[pickle.DictionaryType] = save_dict - - - def save_module(self, obj, pack=struct.pack): + def save_module(self, obj): """ Save a module as an import """ - #print 'try save import', obj.__name__ self.modules.add(obj) - self.save_reduce(subimport,(obj.__name__,), obj=obj) - dispatch[types.ModuleType] = save_module #new type + self.save_reduce(subimport, (obj.__name__,), obj=obj) + dispatch[types.ModuleType] = save_module - def save_codeobject(self, obj, pack=struct.pack): + def save_codeobject(self, obj): """ Save a code object """ - #print 'try to save codeobj: ', obj - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) + if PY3: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, obj.co_varnames, + obj.co_filename, obj.co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, + obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars + ) self.save_reduce(types.CodeType, args, obj=obj) - dispatch[types.CodeType] = save_codeobject #new type + dispatch[types.CodeType] = save_codeobject - def save_function(self, obj, name=None, pack=struct.pack): + def save_function(self, obj, name=None): """ Registered with the dispatch to handle all function types. Determines what kind of function obj is (e.g. lambda, defined at @@ -205,12 +167,14 @@ def save_function(self, obj, name=None, pack=struct.pack): """ write = self.write - name = obj.__name__ + if name is None: + name = obj.__name__ modname = pickle.whichmodule(obj, name) - #print 'which gives %s %s %s' % (modname, obj, name) + # print('which gives %s %s %s' % (modname, obj, name)) try: themodule = sys.modules[modname] - except KeyError: # eval'd items such as namedtuple give invalid items for their function __module__ + except KeyError: + # eval'd items such as namedtuple give invalid items for their function __module__ modname = '__main__' if modname == '__main__': @@ -221,37 +185,18 @@ def save_function(self, obj, name=None, pack=struct.pack): if getattr(themodule, name, None) is obj: return self.save_global(obj, name) - if not self.savedDjangoEnv: - #hack for django - if we detect the settings module, we transport it - django_settings = os.environ.get('DJANGO_SETTINGS_MODULE', '') - if django_settings: - django_mod = sys.modules.get(django_settings) - if django_mod: - cloudLog.debug('Transporting django settings %s during save of %s', django_mod, name) - self.savedDjangoEnv = True - self.modules.add(django_mod) - write(pickle.MARK) - self.save_reduce(django_settings_load, (django_mod.__name__,), obj=django_mod) - write(pickle.POP_MARK) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: - #Force server to import modules that have been imported in main - modList = None - if themodule is None and not self.savedForceImports: - mainmod = sys.modules['__main__'] - if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): - modList = list(mainmod.___pyc_forcedImports__) - self.savedForceImports = True - self.save_function_tuple(obj, modList) + if islambda(obj) or obj.__code__.co_filename == '' or themodule is None: + #print("save global", islambda(obj), obj.__code__.co_filename, modname, themodule) + self.save_function_tuple(obj) return - else: # func is nested + else: + # func is nested klass = getattr(themodule, name, None) if klass is None or klass is not obj: - self.save_function_tuple(obj, [themodule]) + self.save_function_tuple(obj) return if obj.__dict__: @@ -266,7 +211,7 @@ def save_function(self, obj, name=None, pack=struct.pack): self.memoize(obj) dispatch[types.FunctionType] = save_function - def save_function_tuple(self, func, forced_imports): + def save_function_tuple(self, func): """ Pickles an actual func object. A func comprises: code, globals, defaults, closure, and dict. We @@ -281,19 +226,6 @@ def save_function_tuple(self, func, forced_imports): save = self.save write = self.write - # save the modules (if any) - if forced_imports: - write(pickle.MARK) - save(_modules_to_main) - #print 'forced imports are', forced_imports - - forced_names = map(lambda m: m.__name__, forced_imports) - save((forced_names,)) - - #save((forced_imports,)) - write(pickle.REDUCE) - write(pickle.POP_MARK) - code, f_globals, defaults, closure, dct, base_globals = self.extract_func_data(func) save(_fill_function) # skeleton function updater @@ -318,6 +250,8 @@ def extract_code_globals(co): Find all globals names read or written to by codeblock co """ code = co.co_code + if not PY3: + code = [ord(c) for c in code] names = co.co_names out_names = set() @@ -327,18 +261,18 @@ def extract_code_globals(co): while i < n: op = code[i] - i = i+1 + i += 1 if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + oparg = code[i] + code[i+1] * 256 + extended_arg extended_arg = 0 - i = i+2 + i += 2 if op == EXTENDED_ARG: - extended_arg = oparg*65536L + extended_arg = oparg*65536 if op in GLOBAL_OPS: out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - if co.co_consts: # see if nested function have any global refs + # see if nested function have any global refs + if co.co_consts: for const in co.co_consts: if type(const) is types.CodeType: out_names |= CloudPickler.extract_code_globals(const) @@ -350,46 +284,28 @@ def extract_func_data(self, func): Turn the function into a tuple of data necessary to recreate it: code, globals, defaults, closure, dict """ - code = func.func_code + code = func.__code__ # extract all global ref's - func_global_refs = CloudPickler.extract_code_globals(code) + func_global_refs = self.extract_code_globals(code) # process all variables referenced by global environment f_globals = {} for var in func_global_refs: - #Some names, such as class functions are not global - we don't need them - if func.func_globals.has_key(var): - f_globals[var] = func.func_globals[var] + if var in func.__globals__: + f_globals[var] = func.__globals__[var] # defaults requires no processing - defaults = func.func_defaults - - def get_contents(cell): - try: - return cell.cell_contents - except ValueError, e: #cell is empty error on not yet assigned - raise pickle.PicklingError('Function to be pickled has free variables that are referenced before assignment in enclosing scope') - + defaults = func.__defaults__ # process closure - if func.func_closure: - closure = map(get_contents, func.func_closure) - else: - closure = [] + closure = [c.cell_contents for c in func.__closure__] if func.__closure__ else [] # save the dict - dct = func.func_dict - - if printSerialization: - outvars = ['code: ' + str(code) ] - outvars.append('globals: ' + str(f_globals)) - outvars.append('defaults: ' + str(defaults)) - outvars.append('closure: ' + str(closure)) - print 'function ', func, 'is extracted to: ', ', '.join(outvars) + dct = func.__dict__ - base_globals = self.globals_ref.get(id(func.func_globals), {}) - self.globals_ref[id(func.func_globals)] = base_globals + base_globals = self.globals_ref.get(id(func.__globals__), {}) + self.globals_ref[id(func.__globals__)] = base_globals return (code, f_globals, defaults, closure, dct, base_globals) @@ -400,8 +316,9 @@ def save_builtin_function(self, obj): dispatch[types.BuiltinFunctionType] = save_builtin_function def save_global(self, obj, name=None, pack=struct.pack): - write = self.write - memo = self.memo + if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": + if obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce(_builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) if name is None: name = obj.__name__ @@ -410,98 +327,57 @@ def save_global(self, obj, name=None, pack=struct.pack): if modname is None: modname = pickle.whichmodule(obj, name) - try: - __import__(modname) - themodule = sys.modules[modname] - except (ImportError, KeyError, AttributeError): #should never occur - raise pickle.PicklingError( - "Can't pickle %r: Module %s cannot be found" % - (obj, modname)) - if modname == '__main__': themodule = None - - if themodule: + else: + __import__(modname) + themodule = sys.modules[modname] self.modules.add(themodule) - sendRef = True - typ = type(obj) - #print 'saving', obj, typ - try: - try: #Deal with case when getattribute fails with exceptions - klass = getattr(themodule, name) - except (AttributeError): - if modname == '__builtin__': #new.* are misrepeported - modname = 'new' - __import__(modname) - themodule = sys.modules[modname] - try: - klass = getattr(themodule, name) - except AttributeError, a: - # print themodule, name, obj, type(obj) - raise pickle.PicklingError("Can't pickle builtin %s" % obj) - else: - raise + if hasattr(themodule, name) and getattr(themodule, name) is obj: + return Pickler.save_global(self, obj, name) - except (ImportError, KeyError, AttributeError): - if typ == types.TypeType or typ == types.ClassType: - sendRef = False - else: #we can't deal with this - raise - else: - if klass is not obj and (typ == types.TypeType or typ == types.ClassType): - sendRef = False - if not sendRef: - #note: Third party types might crash this - add better checks! - d = dict(obj.__dict__) #copy dict proxy to a dict - if not isinstance(d.get('__dict__', None), property): # don't extract dict that are properties - d.pop('__dict__',None) - d.pop('__weakref__',None) + typ = type(obj) + if typ is not obj and isinstance(obj, (type, types.ClassType)): + d = dict(obj.__dict__) # copy dict proxy to a dict + if not isinstance(d.get('__dict__', None), property): + # don't extract dict that are properties + d.pop('__dict__', None) + d.pop('__weakref__', None) # hack as __new__ is stored differently in the __dict__ new_override = d.get('__new__', None) if new_override: d['__new__'] = obj.__new__ - self.save_reduce(type(obj),(obj.__name__,obj.__bases__, - d),obj=obj) - #print 'internal reduce dask %s %s' % (obj, d) - return - - if self.proto >= 2: - code = _extension_registry.get((modname, name)) - if code: - assert code > 0 - if code <= 0xff: - write(pickle.EXT1 + chr(code)) - elif code <= 0xffff: - write("%c%c%c" % (pickle.EXT2, code&0xff, code>>8)) - else: - write(pickle.EXT4 + pack("> sys.stderr, 'Cloud not import django settings %s:' % (name) - print_exec(sys.stderr) - if modified_env: - del os.environ['DJANGO_SETTINGS_MODULE'] - else: - #add project directory to sys,path: - if hasattr(module,'__file__'): - dirname = os.path.split(module.__file__)[0] + '/' - sys.path.append(dirname) # restores function attributes def _restore_attr(obj, attr): @@ -851,13 +636,16 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj + def _get_module_builtins(): return pickle.__builtins__ + def print_exec(stream): ei = sys.exc_info() traceback.print_exception(ei[0], ei[1], ei[2], None, stream) + def _modules_to_main(modList): """Force every module in modList to be placed into main""" if not modList: @@ -868,22 +656,16 @@ def _modules_to_main(modList): if type(modname) is str: try: mod = __import__(modname) - except Exception, i: #catch all... - sys.stderr.write('warning: could not import %s\n. Your function may unexpectedly error due to this import failing; \ -A version mismatch is likely. Specific error was:\n' % modname) + except Exception as e: + sys.stderr.write('warning: could not import %s\n. ' + 'Your function may unexpectedly error due to this import failing;' + 'A version mismatch is likely. Specific error was:\n' % modname) print_exec(sys.stderr) else: - setattr(main,mod.__name__, mod) - else: - #REVERSE COMPATIBILITY FOR CLOUD CLIENT 1.5 (WITH EPD) - #In old version actual module was sent - setattr(main,modname.__name__, modname) + setattr(main, mod.__name__, mod) -#object generators: -def _build_xrange(start, step, len): - """Built xrange explicitly""" - return xrange(start, start + step*len, step) +#object generators: def _genpartial(func, args, kwds): if not args: args = () @@ -891,22 +673,26 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) + def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ - func.func_globals.update(globals) - func.func_defaults = defaults - func.func_dict = dict + func.__globals__.update(globals) + func.__defaults__ = defaults + func.__dict__ = dict return func + def _make_cell(value): - return (lambda: value).func_closure[0] + return (lambda: value).__closure__[0] + def _reconstruct_closure(values): return tuple([_make_cell(v) for v in values]) + def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other @@ -928,40 +714,3 @@ def _make_skel_func(code, closures, base_globals = None): def _getobject(modname, attribute): mod = __import__(modname, fromlist=[attribute]) return mod.__dict__[attribute] - -def _generateImage(size, mode, str_rep): - """Generate image from string representation""" - import Image - i = Image.new(mode, size) - i.fromstring(str_rep) - return i - -def _lazyloadImage(fp): - import Image - fp.seek(0) #works in almost any case - return Image.open(fp) - -"""Timeseries""" -def _genTimeSeries(reduce_args, state): - import scikits.timeseries.tseries as ts - from numpy import ndarray - from numpy.ma import MaskedArray - - - time_series = ts._tsreconstruct(*reduce_args) - - #from setstate modified - (ver, shp, typ, isf, raw, msk, flv, dsh, dtm, dtyp, frq, infodict) = state - #print 'regenerating %s' % dtyp - - MaskedArray.__setstate__(time_series, (ver, shp, typ, isf, raw, msk, flv)) - _dates = time_series._dates - #_dates.__setstate__((ver, dsh, typ, isf, dtm, frq)) #use remote typ - ndarray.__setstate__(_dates,(dsh,dtyp, isf, dtm)) - _dates.freq = frq - _dates._cachedinfo.update(dict(full=None, hasdups=None, steps=None, - toobj=None, toord=None, tostr=None)) - # Update the _optinfo dictionary - time_series._optinfo.update(infodict) - return time_series - diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index dc7cd0bce56f3..924da3eecf214 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -44,7 +44,7 @@ >>> conf.get("spark.executorEnv.VAR1") u'value1' ->>> print conf.toDebugString() +>>> print(conf.toDebugString()) spark.executorEnv.VAR1=value1 spark.executorEnv.VAR3=value3 spark.executorEnv.VAR4=value4 @@ -56,6 +56,13 @@ __all__ = ['SparkConf'] +import sys +import re + +if sys.version > '3': + unicode = str + __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__) + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 78dccc40470e3..31992795a9e45 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -15,14 +15,14 @@ # limitations under the License. # +from __future__ import print_function + import os import shutil import sys from threading import Lock from tempfile import NamedTemporaryFile -from py4j.java_collections import ListConverter - from pyspark import accumulators from pyspark.accumulators import Accumulator from pyspark.broadcast import Broadcast @@ -32,11 +32,14 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +if sys.version > '3': + xrange = range + __all__ = ['SparkContext'] @@ -133,7 +136,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if sparkHome: self._conf.setSparkHome(sparkHome) if environment: - for key, value in environment.iteritems(): + for key, value in environment.items(): self._conf.setExecutorEnv(key, value) for key, value in DEFAULT_CONFIGS.items(): self._conf.setIfMissing(key, value) @@ -153,6 +156,10 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] self.environment[varName] = v + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + # disable randomness of hash of string in worker, if this is not + # launched by spark-submit + self.environment["PYTHONHASHSEED"] = "0" # Create the Java SparkContext through Py4J self._jsc = jsc or self._initialize_context(self._conf._jconf) @@ -260,6 +267,13 @@ def __exit__(self, type, value, trace): """ self.stop() + def setLogLevel(self, logLevel): + """ + Control our logLevel. This overrides any user-defined log settings. + Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN + """ + self._jsc.setLogLevel(logLevel) + @classmethod def setSystemProperty(cls, key, value): """ @@ -323,7 +337,7 @@ def parallelize(self, c, numSlices=None): start0 = c[0] def getStart(split): - return start0 + (split * size / numSlices) * step + return start0 + int((split * size / numSlices)) * step def f(split, iterator): return xrange(getStart(split), getStart(split + 1), step) @@ -357,6 +371,7 @@ def pickleFile(self, name, minPartitions=None): minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.objectFile(name, minPartitions), self) + @ignore_unicode_prefix def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all @@ -369,7 +384,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello world!") + ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello world!'] @@ -378,6 +393,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer(use_unicode)) + @ignore_unicode_prefix def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system @@ -411,9 +427,9 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): >>> dirPath = os.path.join(tempdir, "files") >>> os.mkdir(dirPath) >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: - ... file1.write("1") + ... _ = file1.write("1") >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: - ... file2.write("2") + ... _ = file2.write("2") >>> textFiles = sc.wholeTextFiles(dirPath) >>> sorted(textFiles.collect()) [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] @@ -456,7 +472,7 @@ def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() if not d: d = {} - for k, v in d.iteritems(): + for k, v in d.items(): jm[k] = v return jm @@ -608,6 +624,7 @@ def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) + @ignore_unicode_prefix def union(self, rdds): """ Build the union of a list of RDDs. @@ -618,7 +635,7 @@ def union(self, rdds): >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello") + ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] @@ -631,7 +648,6 @@ def union(self, rdds): rdds = [x._reserialize() for x in rdds] first = rdds[0]._jrdd rest = [x._jrdd for x in rdds[1:]] - rest = ListConverter().convert(rest, self._gateway._gateway_client) return RDD(self._jsc.union(first, rest), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): @@ -659,7 +675,7 @@ def accumulator(self, value, accum_param=None): elif isinstance(value, complex): accum_param = accumulators.COMPLEX_ACCUMULATOR_PARAM else: - raise Exception("No default accumulator param for type %s" % type(value)) + raise TypeError("No default accumulator param for type %s" % type(value)) SparkContext._next_accum_id += 1 return Accumulator(SparkContext._next_accum_id - 1, value, accum_param) @@ -677,7 +693,7 @@ def addFile(self, path): >>> from pyspark import SparkFiles >>> path = os.path.join(tempdir, "test.txt") >>> with open(path, "w") as testFile: - ... testFile.write("100") + ... _ = testFile.write("100") >>> sc.addFile(path) >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: @@ -705,11 +721,13 @@ def addPyFile(self, path): """ self.addFile(path) (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix - if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() def setCheckpointDir(self, dirName): """ @@ -744,7 +762,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): The application can use L{SparkContext.cancelJobGroup} to cancel all running jobs in this group. - >>> import thread, threading + >>> import threading >>> from time import sleep >>> result = "Not Set" >>> lock = threading.Lock() @@ -763,10 +781,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ... sleep(5) ... sc.cancelJobGroup("job_to_cancel") >>> supress = lock.acquire() - >>> supress = thread.start_new_thread(start_job, (10,)) - >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = threading.Thread(target=start_job, args=(10,)).start() + >>> supress = threading.Thread(target=stop_job).start() >>> supress = lock.acquire() - >>> print result + >>> print(result) Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result @@ -832,13 +850,12 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): """ if partitions is None: partitions = range(rdd._jrdd.partitions().size()) - javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) # Implementation note: This is implemented as a mapPartitions followed # by runJob() in order to avoid having to pass a Python lambda into # SparkContext#runJob. mappedRDD = rdd.mapPartitions(partitionFunc) - port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, + port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, partitions, allowLocal) return list(_load_from_socket(port, mappedRDD._jrdd_deserializer)) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 93885985fe377..7f06d4288c872 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -24,9 +24,10 @@ import traceback import time import gc -from errno import EINTR, ECHILD, EAGAIN +from errno import EINTR, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT + from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -53,8 +54,8 @@ def worker(sock): # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536) exit_code = 0 try: worker_main(infile, outfile) @@ -68,17 +69,6 @@ def worker(sock): return exit_code -# Cleanup zombie children -def cleanup_dead_children(): - try: - while True: - pid, _ = os.waitpid(0, os.WNOHANG) - if not pid: - break - except: - pass - - def manager(): # Create a new process group to corral our children os.setpgid(0, 0) @@ -88,8 +78,12 @@ def manager(): listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() - write_int(listen_port, sys.stdout) - sys.stdout.flush() + + # re-open stdin/stdout in 'wb' mode + stdin_bin = os.fdopen(sys.stdin.fileno(), 'rb', 4) + stdout_bin = os.fdopen(sys.stdout.fileno(), 'wb', 4) + write_int(listen_port, stdout_bin) + stdout_bin.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -101,6 +95,7 @@ def handle_sigterm(*args): shutdown(1) signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + signal.signal(SIGCHLD, SIG_IGN) reuse = os.environ.get("SPARK_REUSE_WORKER") @@ -115,12 +110,9 @@ def handle_sigterm(*args): else: raise - # cleanup in signal handler will cause deadlock - cleanup_dead_children() - if 0 in ready_fds: try: - worker_pid = read_int(sys.stdin) + worker_pid = read_int(stdin_bin) except EOFError: # Spark told us to exit by closing stdin shutdown(0) @@ -145,7 +137,7 @@ def handle_sigterm(*args): time.sleep(1) pid = os.fork() # error here will shutdown daemon else: - outfile = sock.makefile('w') + outfile = sock.makefile(mode='wb') write_int(e.errno, outfile) # Signal that the fork failed outfile.flush() outfile.close() @@ -157,7 +149,7 @@ def handle_sigterm(*args): listen_sock.close() try: # Acknowledge that the fork was successful - outfile = sock.makefile("w") + outfile = sock.makefile(mode="wb") write_int(os.getpid(), outfile) outfile.flush() outfile.close() diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py index bc441f138f7fc..4ef2afe03544f 100644 --- a/python/pyspark/heapq3.py +++ b/python/pyspark/heapq3.py @@ -627,51 +627,49 @@ def merge(iterables, key=None, reverse=False): if key is None: for order, it in enumerate(map(iter, iterables)): try: - next = it.next - h_append([next(), order * direction, next]) + h_append([next(it), order * direction, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - value, order, next = s = h[0] + value, order, it = s = h[0] yield value - s[0] = next() # raises StopIteration when exhausted + s[0] = next(it) # raises StopIteration when exhausted _heapreplace(h, s) # restore heap condition except StopIteration: _heappop(h) # remove empty iterator if h: # fast case when only a single iterator remains - value, order, next = h[0] + value, order, it = h[0] yield value - for value in next.__self__: + for value in it: yield value return for order, it in enumerate(map(iter, iterables)): try: - next = it.next - value = next() - h_append([key(value), order * direction, value, next]) + value = next(it) + h_append([key(value), order * direction, value, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - key_value, order, value, next = s = h[0] + key_value, order, value, it = s = h[0] yield value - value = next() + value = next(it) s[0] = key(value) s[2] = value _heapreplace(h, s) except StopIteration: _heappop(h) if h: - key_value, order, value, next = h[0] + key_value, order, value, it = h[0] yield value - for value in next.__self__: + for value in it: yield value diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0a16cbd8bff62..3cee4ea6e3a35 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -17,23 +17,35 @@ import atexit import os +import sys import select import signal import shlex import socket import platform from subprocess import Popen, PIPE + +if sys.version >= '3': + xrange = range + from py4j.java_gateway import java_import, JavaGateway, GatewayClient +from py4j.java_collections import ListConverter from pyspark.serializers import read_int -def launch_gateway(): - SPARK_HOME = os.environ["SPARK_HOME"] +# patching ListConverter, or it will convert bytearray into Java ArrayList +def can_convert_list(self, obj): + return isinstance(obj, (list, tuple, xrange)) +ListConverter.can_convert = can_convert_list + + +def launch_gateway(): if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: + SPARK_HOME = os.environ["SPARK_HOME"] # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" @@ -70,7 +82,7 @@ def preexec_func(): if callback_socket in readable: gateway_connection = callback_socket.accept()[0] # Determine which ephemeral port the server started on: - gateway_port = read_int(gateway_connection.makefile()) + gateway_port = read_int(gateway_connection.makefile(mode="rb")) gateway_connection.close() callback_socket.close() if gateway_port is None: @@ -93,7 +105,7 @@ def killChild(): atexit.register(killChild) # Connect to the gateway - gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False) + gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=True) # Import the classes used by PySpark java_import(gateway.jvm, "org.apache.spark.SparkConf") diff --git a/python/pyspark/join.py b/python/pyspark/join.py index efc1ef9396412..94df3990164d6 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -32,6 +32,7 @@ """ from pyspark.resultiterable import ResultIterable +from functools import reduce def _do_python_join(rdd, other, numPartitions, dispatch): @@ -48,7 +49,7 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -62,7 +63,7 @@ def dispatch(seq): wbuf.append(v) if not vbuf: vbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -76,7 +77,7 @@ def dispatch(seq): wbuf.append(v) if not wbuf: wbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -104,8 +105,9 @@ def make_mapper(i): rdd_len = len(vrdds) def dispatch(seq): - bufs = [[] for i in range(rdd_len)] - for (n, v) in seq: + bufs = [[] for _ in range(rdd_len)] + for n, v in seq: bufs[n].append(v) - return tuple(map(ResultIterable, bufs)) + return tuple(ResultIterable(vs) for vs in bufs) + return union_vrdds.groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4ff7463498cce..45754bc9d4b10 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -39,10 +39,10 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> lr = LogisticRegression(maxIter=5, regParam=0.01) >>> model = lr.fit(df) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0))]).toDF() - >>> print model.transform(test0).head().prediction + >>> model.transform(test0).head().prediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))]).toDF() - >>> print model.transform(test1).head().prediction + >>> model.transform(test1).head().prediction 1.0 >>> lr.setParams("vector") Traceback (most recent call last): @@ -59,6 +59,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxIter=100, regParam=0.1) """ super(LogisticRegression, self).__init__() + self._setDefault(maxIter=100, regParam=0.1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -71,7 +72,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre Sets params for logistic regression. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def _create_model(self, java_model): return LogisticRegressionModel(java_model) @@ -91,9 +92,9 @@ class LogisticRegressionModel(JavaModel): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 433b4fb5d22bf..4e4614b859ac6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,6 +15,7 @@ # limitations under the License. # +from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaTransformer @@ -24,6 +25,7 @@ @inherit_doc +@ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ A tokenizer that converts the input string to lowercase and then @@ -32,15 +34,15 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(text="a b c")]).toDF() >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) >>> # Change a parameter. - >>> print tokenizer.setParams(outputCol="tokens").transform(df).head() + >>> tokenizer.setParams(outputCol="tokens").transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Temporarily modify a parameter. - >>> print tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Must use keyword arguments to specify params. >>> tokenizer.setParams("text") @@ -52,22 +54,22 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): _java_class = "org.apache.spark.ml.feature.Tokenizer" @keyword_only - def __init__(self, inputCol="input", outputCol="output"): + def __init__(self, inputCol=None, outputCol=None): """ - __init__(self, inputCol="input", outputCol="output") + __init__(self, inputCol=None, outputCol=None) """ super(Tokenizer, self).__init__() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, inputCol="input", outputCol="output"): + def setParams(self, inputCol=None, outputCol=None): """ setParams(self, inputCol="input", outputCol="output") Sets params for this Tokenizer. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) @inherit_doc @@ -79,34 +81,35 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") - >>> print hashingTF.transform(df).head().features - (10,[7,8,9],[1.0,1.0,1.0]) - >>> print hashingTF.setParams(outputCol="freqs").transform(df).head().freqs - (10,[7,8,9],[1.0,1.0,1.0]) + >>> hashingTF.transform(df).head().features + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(df, params).head().vector - (5,[2,3,4],[1.0,1.0,1.0]) + >>> hashingTF.transform(df, params).head().vector + SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) """ _java_class = "org.apache.spark.ml.feature.HashingTF" @keyword_only - def __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) """ super(HashingTF, self).__init__() + self._setDefault(numFeatures=1 << 18) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) Sets params for this HashingTF. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) if __name__ == "__main__": @@ -117,9 +120,9 @@ def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index e3a53dd780c4c..49c20b4cf70cf 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -25,23 +25,21 @@ class Param(object): """ - A param with self-contained documentation and optionally default value. + A param with self-contained documentation. """ - def __init__(self, parent, name, doc, defaultValue=None): - if not isinstance(parent, Identifiable): - raise ValueError("Parent must be identifiable but got type %s." % type(parent).__name__) + def __init__(self, parent, name, doc): + if not isinstance(parent, Params): + raise TypeError("Parent must be a Params but got type %s." % type(parent)) self.parent = parent self.name = str(name) self.doc = str(doc) - self.defaultValue = defaultValue def __str__(self): - return str(self.parent) + "-" + self.name + return str(self.parent) + "__" + self.name def __repr__(self): - return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ - (self.parent, self.name, self.doc, self.defaultValue) + return "Param(parent=%r, name=%r, doc=%r)" % (self.parent, self.name, self.doc) class Params(Identifiable): @@ -52,26 +50,128 @@ class Params(Identifiable): __metaclass__ = ABCMeta - def __init__(self): - super(Params, self).__init__() - #: embedded param map - self.paramMap = {} + #: internal param map for user-supplied values param map + paramMap = {} + + #: internal param map for default values + defaultParamMap = {} @property def params(self): """ - Returns all params. The default implementation uses - :py:func:`dir` to get all attributes of type + Returns all params ordered by name. The default implementation + uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return filter(lambda attr: isinstance(attr, Param), - [getattr(self, x) for x in dir(self) if x != "params"]) + return list(filter(lambda attr: isinstance(attr, Param), + [getattr(self, x) for x in dir(self) if x != "params"])) + + def _explain(self, param): + """ + Explains a single param and returns its name, doc, and optional + default value and user-supplied value in a string. + """ + param = self._resolveParam(param) + values = [] + if self.isDefined(param): + if param in self.defaultParamMap: + values.append("default: %s" % self.defaultParamMap[param]) + if param in self.paramMap: + values.append("current: %s" % self.paramMap[param]) + else: + values.append("undefined") + valueStr = "(" + ", ".join(values) + ")" + return "%s: %s %s" % (param.name, param.doc, valueStr) + + def explainParams(self): + """ + Returns the documentation of all params with their optionally + default values and user-supplied values. + """ + return "\n".join([self._explain(param) for param in self.params]) + + def getParam(self, paramName): + """ + Gets a param by its name. + """ + param = getattr(self, paramName) + if isinstance(param, Param): + return param + else: + raise ValueError("Cannot find param with name %s." % paramName) + + def isSet(self, param): + """ + Checks whether a param is explicitly set by user. + """ + param = self._resolveParam(param) + return param in self.paramMap + + def hasDefault(self, param): + """ + Checks whether a param has a default value. + """ + param = self._resolveParam(param) + return param in self.defaultParamMap + + def isDefined(self, param): + """ + Checks whether a param is explicitly set by user or has a default value. + """ + return self.isSet(param) or self.hasDefault(param) - def _merge_params(self, params): - paramMap = self.paramMap.copy() - paramMap.update(params) + def getOrDefault(self, param): + """ + Gets the value of a param in the user-supplied param map or its + default value. Raises an error if either is set. + """ + if isinstance(param, Param): + if param in self.paramMap: + return self.paramMap[param] + else: + return self.defaultParamMap[param] + elif isinstance(param, str): + return self.getOrDefault(self.getParam(param)) + else: + raise KeyError("Cannot recognize %r as a param." % param) + + def extractParamMap(self, extraParamMap={}): + """ + Extracts the embedded default param values and user-supplied + values, and then merges them with extra values from input into + a flat param map, where the latter value is used if there exist + conflicts, i.e., with ordering: default param values < + user-supplied values < extraParamMap. + :param extraParamMap: extra param values + :return: merged param map + """ + paramMap = self.defaultParamMap.copy() + paramMap.update(self.paramMap) + paramMap.update(extraParamMap) return paramMap + def _shouldOwn(self, param): + """ + Validates that the input param belongs to this Params instance. + """ + if param.parent is not self: + raise ValueError("Param %r does not belong to %r." % (param, self)) + + def _resolveParam(self, param): + """ + Resolves a param and validates the ownership. + :param param: param name or the param instance, which must + belong to this Params instance + :return: resolved param instance + """ + if isinstance(param, Param): + self._shouldOwn(param) + return param + elif isinstance(param, str): + return self.getParam(param) + else: + raise ValueError("Cannot resolve %r as a param." % param) + @staticmethod def _dummy(): """ @@ -81,10 +181,18 @@ def _dummy(): dummy.uid = "undefined" return dummy - def _set_params(self, **kwargs): + def _set(self, **kwargs): """ - Sets params. + Sets user-supplied params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.paramMap[getattr(self, param)] = value return self + + def _setDefault(self, **kwargs): + """ + Sets default params. + """ + for param, value in kwargs.items(): + self.defaultParamMap[getattr(self, param)] = value + return self diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_shared_params_code_gen.py similarity index 67% rename from python/pyspark/ml/param/_gen_shared_params.py rename to python/pyspark/ml/param/_shared_params_code_gen.py index 5eb81106f116c..6a3192465d66d 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + header = """# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -32,29 +34,34 @@ # limitations under the License. #""" +# Code generator for shared params (shared.py). Run under this folder with: +# python _shared_params_code_gen.py > shared.py + -def _gen_param_code(name, doc, defaultValue): +def _gen_param_code(name, doc, defaultValueStr): """ Generates Python code for a shared param class. :param name: param name :param doc: param doc - :param defaultValue: string representation of the param + :param defaultValueStr: string representation of the default value :return: code string """ # TODO: How to correctly inherit instance attributes? template = '''class Has$Name(Params): """ - Params with $name. + Mixin for param $name: $doc. """ # a placeholder to make it appear in the generated doc - $name = Param(Params._dummy(), "$name", "$doc", $defaultValue) + $name = Param(Params._dummy(), "$name", "$doc") def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc", $defaultValue) + self.$name = Param(self, "$name", "$doc") + if $defaultValueStr is not None: + self._setDefault($name=$defaultValueStr) def set$Name(self, value): """ @@ -67,32 +74,29 @@ def get$Name(self): """ Gets the value of $name or its default value. """ - if self.$name in self.paramMap: - return self.paramMap[self.$name] - else: - return self.$name.defaultValue''' + return self.getOrDefault(self.$name)''' - upperCamelName = name[0].upper() + name[1:] + Name = name[0].upper() + name[1:] return template \ .replace("$name", name) \ - .replace("$Name", upperCamelName) \ + .replace("$Name", Name) \ .replace("$doc", doc) \ - .replace("$defaultValue", defaultValue) + .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": - print header - print "\n# DO NOT MODIFY. The code is generated by _gen_shared_params.py.\n" - print "from pyspark.ml.param import Param, Params\n\n" + print(header) + print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") + print("from pyspark.ml.param import Param, Params\n\n") shared = [ - ("maxIter", "max number of iterations", "100"), - ("regParam", "regularization constant", "0.1"), + ("maxIter", "max number of iterations", None), + ("regParam", "regularization constant", None), ("featuresCol", "features column name", "'features'"), ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), - ("inputCol", "input column name", "'input'"), - ("outputCol", "output column name", "'output'"), - ("numFeatures", "number of features", "1 << 18")] + ("inputCol", "input column name", None), + ("outputCol", "output column name", None), + ("numFeatures", "number of features", None)] code = [] - for name, doc, defaultValue in shared: - code.append(_gen_param_code(name, doc, defaultValue)) - print "\n\n\n".join(code) + for name, doc, defaultValueStr in shared: + code.append(_gen_param_code(name, doc, defaultValueStr)) + print("\n\n\n".join(code)) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 586822f2de423..13b6749998ad0 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -15,23 +15,25 @@ # limitations under the License. # -# DO NOT MODIFY. The code is generated by _gen_shared_params.py. +# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py. from pyspark.ml.param import Param, Params class HasMaxIter(Params): """ - Params with maxIter. + Mixin for param maxIter: max number of iterations. """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations", 100) + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations") def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations - self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + self.maxIter = Param(self, "maxIter", "max number of iterations") + if None is not None: + self._setDefault(maxIter=None) def setMaxIter(self, value): """ @@ -44,24 +46,23 @@ def getMaxIter(self): """ Gets the value of maxIter or its default value. """ - if self.maxIter in self.paramMap: - return self.paramMap[self.maxIter] - else: - return self.maxIter.defaultValue + return self.getOrDefault(self.maxIter) class HasRegParam(Params): """ - Params with regParam. + Mixin for param regParam: regularization constant. """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization constant", 0.1) + regParam = Param(Params._dummy(), "regParam", "regularization constant") def __init__(self): super(HasRegParam, self).__init__() #: param for regularization constant - self.regParam = Param(self, "regParam", "regularization constant", 0.1) + self.regParam = Param(self, "regParam", "regularization constant") + if None is not None: + self._setDefault(regParam=None) def setRegParam(self, value): """ @@ -74,24 +75,23 @@ def getRegParam(self): """ Gets the value of regParam or its default value. """ - if self.regParam in self.paramMap: - return self.paramMap[self.regParam] - else: - return self.regParam.defaultValue + return self.getOrDefault(self.regParam) class HasFeaturesCol(Params): """ - Params with featuresCol. + Mixin for param featuresCol: features column name. """ # a placeholder to make it appear in the generated doc - featuresCol = Param(Params._dummy(), "featuresCol", "features column name", 'features') + featuresCol = Param(Params._dummy(), "featuresCol", "features column name") def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name - self.featuresCol = Param(self, "featuresCol", "features column name", 'features') + self.featuresCol = Param(self, "featuresCol", "features column name") + if 'features' is not None: + self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ @@ -104,24 +104,23 @@ def getFeaturesCol(self): """ Gets the value of featuresCol or its default value. """ - if self.featuresCol in self.paramMap: - return self.paramMap[self.featuresCol] - else: - return self.featuresCol.defaultValue + return self.getOrDefault(self.featuresCol) class HasLabelCol(Params): """ - Params with labelCol. + Mixin for param labelCol: label column name. """ # a placeholder to make it appear in the generated doc - labelCol = Param(Params._dummy(), "labelCol", "label column name", 'label') + labelCol = Param(Params._dummy(), "labelCol", "label column name") def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name - self.labelCol = Param(self, "labelCol", "label column name", 'label') + self.labelCol = Param(self, "labelCol", "label column name") + if 'label' is not None: + self._setDefault(labelCol='label') def setLabelCol(self, value): """ @@ -134,24 +133,23 @@ def getLabelCol(self): """ Gets the value of labelCol or its default value. """ - if self.labelCol in self.paramMap: - return self.paramMap[self.labelCol] - else: - return self.labelCol.defaultValue + return self.getOrDefault(self.labelCol) class HasPredictionCol(Params): """ - Params with predictionCol. + Mixin for param predictionCol: prediction column name. """ # a placeholder to make it appear in the generated doc - predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name", 'prediction') + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name") def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name - self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') + self.predictionCol = Param(self, "predictionCol", "prediction column name") + if 'prediction' is not None: + self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ @@ -164,24 +162,23 @@ def getPredictionCol(self): """ Gets the value of predictionCol or its default value. """ - if self.predictionCol in self.paramMap: - return self.paramMap[self.predictionCol] - else: - return self.predictionCol.defaultValue + return self.getOrDefault(self.predictionCol) class HasInputCol(Params): """ - Params with inputCol. + Mixin for param inputCol: input column name. """ # a placeholder to make it appear in the generated doc - inputCol = Param(Params._dummy(), "inputCol", "input column name", 'input') + inputCol = Param(Params._dummy(), "inputCol", "input column name") def __init__(self): super(HasInputCol, self).__init__() #: param for input column name - self.inputCol = Param(self, "inputCol", "input column name", 'input') + self.inputCol = Param(self, "inputCol", "input column name") + if None is not None: + self._setDefault(inputCol=None) def setInputCol(self, value): """ @@ -194,24 +191,23 @@ def getInputCol(self): """ Gets the value of inputCol or its default value. """ - if self.inputCol in self.paramMap: - return self.paramMap[self.inputCol] - else: - return self.inputCol.defaultValue + return self.getOrDefault(self.inputCol) class HasOutputCol(Params): """ - Params with outputCol. + Mixin for param outputCol: output column name. """ # a placeholder to make it appear in the generated doc - outputCol = Param(Params._dummy(), "outputCol", "output column name", 'output') + outputCol = Param(Params._dummy(), "outputCol", "output column name") def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name - self.outputCol = Param(self, "outputCol", "output column name", 'output') + self.outputCol = Param(self, "outputCol", "output column name") + if None is not None: + self._setDefault(outputCol=None) def setOutputCol(self, value): """ @@ -224,24 +220,23 @@ def getOutputCol(self): """ Gets the value of outputCol or its default value. """ - if self.outputCol in self.paramMap: - return self.paramMap[self.outputCol] - else: - return self.outputCol.defaultValue + return self.getOrDefault(self.outputCol) class HasNumFeatures(Params): """ - Params with numFeatures. + Mixin for param numFeatures: number of features. """ # a placeholder to make it appear in the generated doc - numFeatures = Param(Params._dummy(), "numFeatures", "number of features", 1 << 18) + numFeatures = Param(Params._dummy(), "numFeatures", "number of features") def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features - self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + self.numFeatures = Param(self, "numFeatures", "number of features") + if None is not None: + self._setDefault(numFeatures=None) def setNumFeatures(self, value): """ @@ -254,7 +249,4 @@ def getNumFeatures(self): """ Gets the value of numFeatures or its default value. """ - if self.numFeatures in self.paramMap: - return self.paramMap[self.numFeatures] - else: - return self.numFeatures.defaultValue + return self.getOrDefault(self.numFeatures) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 83880a5afcd1d..7c1ec3026da6f 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -124,15 +124,15 @@ def setParams(self, stages=[]): Sets params for Pipeline. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def fit(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): - raise ValueError( - "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) + raise TypeError( + "Cannot recognize a pipeline stage of type %s." % type(stage)) indexOfLastEstimator = -1 for i, stage in enumerate(stages): if isinstance(stage, Estimator): @@ -164,7 +164,7 @@ def __init__(self, transformers): self.transformers = transformers def transform(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for t in self.transformers: dataset = t.transform(dataset, paramMap) return dataset diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index b627c2b4e930b..3a42bcf723894 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -33,6 +33,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasMaxIter, HasInputCol from pyspark.ml.pipeline import Transformer, Estimator, Pipeline @@ -46,7 +47,7 @@ class MockTransformer(Transformer): def __init__(self): super(MockTransformer, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None @@ -62,7 +63,7 @@ class MockEstimator(Estimator): def __init__(self): super(MockEstimator, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None self.model = None @@ -111,5 +112,52 @@ def test_pipeline(self): self.assertEqual(6, dataset.index) +class TestParams(HasMaxIter, HasInputCol): + """ + A subclass of Params mixed with HasMaxIter and HasInputCol. + """ + + def __init__(self): + super(TestParams, self).__init__() + self._setDefault(maxIter=10) + + +class ParamTests(PySparkTestCase): + + def test_param(self): + testParams = TestParams() + maxIter = testParams.maxIter + self.assertEqual(maxIter.name, "maxIter") + self.assertEqual(maxIter.doc, "max number of iterations") + self.assertTrue(maxIter.parent is testParams) + + def test_params(self): + testParams = TestParams() + maxIter = testParams.maxIter + inputCol = testParams.inputCol + + params = testParams.params + self.assertEqual(params, [inputCol, maxIter]) + + self.assertTrue(testParams.hasDefault(maxIter)) + self.assertFalse(testParams.isSet(maxIter)) + self.assertTrue(testParams.isDefined(maxIter)) + self.assertEqual(testParams.getMaxIter(), 10) + testParams.setMaxIter(100) + self.assertTrue(testParams.isSet(maxIter)) + self.assertEquals(testParams.getMaxIter(), 100) + + self.assertFalse(testParams.hasDefault(inputCol)) + self.assertFalse(testParams.isSet(inputCol)) + self.assertFalse(testParams.isDefined(inputCol)) + with self.assertRaises(KeyError): + testParams.getInputCol() + + self.assertEquals( + testParams.explainParams(), + "\n".join(["inputCol: input column name (undefined)", + "maxIter: max number of iterations (default: 10, current: 100)"])) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py new file mode 100644 index 0000000000000..1773ab5bdcdb1 --- /dev/null +++ b/python/pyspark/ml/tuning.py @@ -0,0 +1,84 @@ +# +# 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. +# + +import itertools + +__all__ = ['ParamGridBuilder'] + + +class ParamGridBuilder(object): + """ + Builder for a param grid used in grid search-based model selection. + + >>> from classification import LogisticRegression + >>> lr = LogisticRegression() + >>> output = ParamGridBuilder().baseOn({lr.labelCol: 'l'}) \ + .baseOn([lr.predictionCol, 'p']) \ + .addGrid(lr.regParam, [1.0, 2.0, 3.0]) \ + .addGrid(lr.maxIter, [1, 5]) \ + .addGrid(lr.featuresCol, ['f']) \ + .build() + >>> expected = [ \ +{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ +{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ +{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 1, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ +{lr.regParam: 1.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ +{lr.regParam: 2.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}, \ +{lr.regParam: 3.0, lr.featuresCol: 'f', lr.maxIter: 5, lr.labelCol: 'l', lr.predictionCol: 'p'}] + >>> len(output) == len(expected) + True + >>> all([m in expected for m in output]) + True + """ + + def __init__(self): + self._param_grid = {} + + def addGrid(self, param, values): + """ + Sets the given parameters in this grid to fixed values. + """ + self._param_grid[param] = values + + return self + + def baseOn(self, *args): + """ + Sets the given parameters in this grid to fixed values. + Accepts either a parameter dictionary or a list of (parameter, value) pairs. + """ + if isinstance(args[0], dict): + self.baseOn(*args[0].items()) + else: + for (param, value) in args: + self.addGrid(param, [value]) + + return self + + def build(self): + """ + Builds and returns all combinations of parameters specified + by the param grid. + """ + keys = self._param_grid.keys() + grid_values = self._param_grid.values() + return [dict(zip(keys, prod)) for prod in itertools.product(*grid_values)] + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 6f7f39c40eb5a..d3cb100a9efa5 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -40,8 +40,8 @@ class Identifiable(object): def __init__(self): #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + #: concatenates the class name, "_", and 8 random hex chars. + self.uid = type(self).__name__ + "_" + uuid.uuid4().hex[:8] def __repr__(self): return self.uid diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 31a66b3d2f730..394f23c5e9b12 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -64,7 +64,7 @@ def _transfer_params_to_java(self, params, java_obj): :param params: additional params (overwriting embedded values) :param java_obj: Java object to receive the params """ - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for param in self.params: if param in paramMap: java_obj.set(param.name, paramMap[param]) diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index 6449800d9c120..07507b2ad0d05 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -18,6 +18,7 @@ """ Python bindings for MLlib. """ +from __future__ import absolute_import # MLlib currently needs NumPy 1.4+, so complain if lower @@ -25,11 +26,13 @@ if numpy.version.version < '1.4': raise Exception("MLlib requires NumPy 1.4+") -__all__ = ['classification', 'clustering', 'feature', 'linalg', 'random', +__all__ = ['classification', 'clustering', 'feature', 'fpm', 'linalg', 'random', 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys -import rand as random -random.__name__ = 'random' -random.RandomRDDs.__module__ = __name__ + '.random' -sys.modules[__name__ + '.random'] = random +from . import rand as random +modname = __name__ + '.random' +random.__name__ = modname +random.RandomRDDs.__module__ = modname +sys.modules[modname] = random +del modname, sys diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 6766f3ebb8894..a70c664a71fdb 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -22,7 +22,7 @@ from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, _py2java, _java2py -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper from pyspark.mllib.util import Saveable, Loader, inherit_doc @@ -31,13 +31,13 @@ 'SVMModel', 'SVMWithSGD', 'NaiveBayesModel', 'NaiveBayes'] -class LinearBinaryClassificationModel(LinearModel): +class LinearClassificationModel(LinearModel): """ - Represents a linear binary classification model that predicts to whether an - example is positive (1.0) or negative (0.0). + A private abstract class representing a multiclass classification model. + The categories are represented by int values: 0, 1, 2, etc. """ def __init__(self, weights, intercept): - super(LinearBinaryClassificationModel, self).__init__(weights, intercept) + super(LinearClassificationModel, self).__init__(weights, intercept) self._threshold = None def setThreshold(self, value): @@ -47,14 +47,26 @@ def setThreshold(self, value): Sets the threshold that separates positive predictions from negative predictions. An example with prediction score greater than or equal to this threshold is identified as an positive, and negative otherwise. + It is used for binary classification only. """ self._threshold = value + @property + def threshold(self): + """ + .. note:: Experimental + + Returns the threshold (if any) used for converting raw prediction scores + into 0/1 predictions. It is used for binary classification only. + """ + return self._threshold + def clearThreshold(self): """ .. note:: Experimental Clears the threshold so that `predict` will output raw prediction scores. + It is used for binary classification only. """ self._threshold = None @@ -66,7 +78,7 @@ def predict(self, test): raise NotImplementedError -class LogisticRegressionModel(LinearBinaryClassificationModel): +class LogisticRegressionModel(LinearClassificationModel): """A linear binary classification model derived from logistic regression. @@ -74,7 +86,7 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): ... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(1.0, [1.0, 0.0]), ... ] - >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(data)) + >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(data), iterations=10) >>> lrm.predict([1.0, 0.0]) 1 >>> lrm.predict([0.0, 1.0]) @@ -83,7 +95,7 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): [1, 0] >>> lrm.clearThreshold() >>> lrm.predict([0.0, 1.0]) - 0.123... + 0.279... >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), @@ -91,7 +103,7 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): ... LabeledPoint(0.0, SparseVector(2, {0: 1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] - >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data)) + >>> lrm = LogisticRegressionWithSGD.train(sc.parallelize(sparse_data), iterations=10) >>> lrm.predict(array([0.0, 1.0])) 1 >>> lrm.predict(array([1.0, 0.0])) @@ -112,10 +124,40 @@ class LogisticRegressionModel(LinearBinaryClassificationModel): ... os.removedirs(path) ... except: ... pass + >>> multi_class_data = [ + ... LabeledPoint(0.0, [0.0, 1.0, 0.0]), + ... LabeledPoint(1.0, [1.0, 0.0, 0.0]), + ... LabeledPoint(2.0, [0.0, 0.0, 1.0]) + ... ] + >>> data = sc.parallelize(multi_class_data) + >>> mcm = LogisticRegressionWithLBFGS.train(data, iterations=10, numClasses=3) + >>> mcm.predict([0.0, 0.5, 0.0]) + 0 + >>> mcm.predict([0.8, 0.0, 0.0]) + 1 + >>> mcm.predict([0.0, 0.0, 0.3]) + 2 """ - def __init__(self, weights, intercept): + def __init__(self, weights, intercept, numFeatures, numClasses): super(LogisticRegressionModel, self).__init__(weights, intercept) + self._numFeatures = int(numFeatures) + self._numClasses = int(numClasses) self._threshold = 0.5 + if self._numClasses == 2: + self._dataWithBiasSize = None + self._weightsMatrix = None + else: + self._dataWithBiasSize = self._coeff.size / (self._numClasses - 1) + self._weightsMatrix = self._coeff.toArray().reshape(self._numClasses - 1, + self._dataWithBiasSize) + + @property + def numFeatures(self): + return self._numFeatures + + @property + def numClasses(self): + return self._numClasses def predict(self, x): """ @@ -126,20 +168,38 @@ def predict(self, x): return x.map(lambda v: self.predict(v)) x = _convert_to_vector(x) - margin = self.weights.dot(x) + self._intercept - if margin > 0: - prob = 1 / (1 + exp(-margin)) + if self.numClasses == 2: + margin = self.weights.dot(x) + self._intercept + if margin > 0: + prob = 1 / (1 + exp(-margin)) + else: + exp_margin = exp(margin) + prob = exp_margin / (1 + exp_margin) + if self._threshold is None: + return prob + else: + return 1 if prob > self._threshold else 0 else: - exp_margin = exp(margin) - prob = exp_margin / (1 + exp_margin) - if self._threshold is None: - return prob - else: - return 1 if prob > self._threshold else 0 + best_class = 0 + max_margin = 0.0 + if x.size + 1 == self._dataWithBiasSize: + for i in range(0, self._numClasses - 1): + margin = x.dot(self._weightsMatrix[i][0:x.size]) + \ + self._weightsMatrix[i][x.size] + if margin > max_margin: + max_margin = margin + best_class = i + 1 + else: + for i in range(0, self._numClasses - 1): + margin = x.dot(self._weightsMatrix[i]) + if margin > max_margin: + max_margin = margin + best_class = i + 1 + return best_class def save(self, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.LogisticRegressionModel( - _py2java(sc, self._coeff), self.intercept) + _py2java(sc, self._coeff), self.intercept, self.numFeatures, self.numClasses) java_model.save(sc._jsc.sc(), path) @classmethod @@ -148,8 +208,10 @@ def load(cls, sc, path): sc._jsc.sc(), path) weights = _java2py(sc, java_model.weights()) intercept = java_model.intercept() + numFeatures = java_model.numFeatures() + numClasses = java_model.numClasses() threshold = java_model.getThreshold().get() - model = LogisticRegressionModel(weights, intercept) + model = LogisticRegressionModel(weights, intercept, numFeatures, numClasses) model.setThreshold(threshold) return model @@ -158,7 +220,8 @@ class LogisticRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=0.01, regType="l2", intercept=False): + initialWeights=None, regParam=0.01, regType="l2", intercept=False, + validateData=True): """ Train a logistic regression model on the given data. @@ -184,11 +247,14 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, or not of the augmented representation for training data (i.e. whether bias features are activated or not). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainLogisticRegressionModelWithSGD", rdd, int(iterations), float(step), float(miniBatchFraction), i, float(regParam), regType, - bool(intercept)) + bool(intercept), bool(validateData)) return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) @@ -197,7 +263,7 @@ class LogisticRegressionWithLBFGS(object): @classmethod def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType="l2", - intercept=False, corrections=10, tolerance=1e-4): + intercept=False, corrections=10, tolerance=1e-4, validateData=True, numClasses=2): """ Train a logistic regression model on the given data. @@ -223,12 +289,17 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType update (default: 10). :param tolerance: The convergence tolerance of iterations for L-BFGS (default: 1e-4). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) + :param numClasses: The number of classes (i.e., outcomes) a label can take + in Multinomial Logistic Regression (default: 2). >>> data = [ ... LabeledPoint(0.0, [0.0, 1.0]), ... LabeledPoint(1.0, [1.0, 0.0]), ... ] - >>> lrm = LogisticRegressionWithLBFGS.train(sc.parallelize(data)) + >>> lrm = LogisticRegressionWithLBFGS.train(sc.parallelize(data), iterations=10) >>> lrm.predict([1.0, 0.0]) 1 >>> lrm.predict([0.0, 1.0]) @@ -237,12 +308,20 @@ def train(cls, data, iterations=100, initialWeights=None, regParam=0.01, regType def train(rdd, i): return callMLlibFunc("trainLogisticRegressionModelWithLBFGS", rdd, int(iterations), i, float(regParam), regType, bool(intercept), int(corrections), - float(tolerance)) - + float(tolerance), bool(validateData), int(numClasses)) + + if initialWeights is None: + if numClasses == 2: + initialWeights = [0.0] * len(data.first().features) + else: + if intercept: + initialWeights = [0.0] * (len(data.first().features) + 1) * (numClasses - 1) + else: + initialWeights = [0.0] * len(data.first().features) * (numClasses - 1) return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) -class SVMModel(LinearBinaryClassificationModel): +class SVMModel(LinearClassificationModel): """A support vector machine. @@ -252,14 +331,14 @@ class SVMModel(LinearBinaryClassificationModel): ... LabeledPoint(1.0, [2.0]), ... LabeledPoint(1.0, [3.0]) ... ] - >>> svm = SVMWithSGD.train(sc.parallelize(data)) + >>> svm = SVMWithSGD.train(sc.parallelize(data), iterations=10) >>> svm.predict([1.0]) 1 >>> svm.predict(sc.parallelize([[1.0]])).collect() [1] >>> svm.clearThreshold() >>> svm.predict(array([1.0])) - 1.25... + 1.44... >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})), @@ -267,7 +346,7 @@ class SVMModel(LinearBinaryClassificationModel): ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] - >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data)) + >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data), iterations=10) >>> svm.predict(SparseVector(2, {1: 1.0})) 1 >>> svm.predict(SparseVector(2, {0: -1.0})) @@ -325,7 +404,8 @@ class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None, regType="l2", intercept=False): + miniBatchFraction=1.0, initialWeights=None, regType="l2", + intercept=False, validateData=True): """ Train a support vector machine on the given data. @@ -351,11 +431,14 @@ def train(cls, data, iterations=100, step=1.0, regParam=0.01, or not of the augmented representation for training data (i.e. whether bias features are activated or not). + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainSVMModelWithSGD", rdd, int(iterations), float(step), float(regParam), float(miniBatchFraction), i, regType, - bool(intercept)) + bool(intercept), bool(validateData)) return _regression_train_wrapper(train, SVMModel, data, initialWeights) @@ -428,9 +511,10 @@ def save(self, sc, path): def load(cls, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel.load( sc._jsc.sc(), path) - py_labels = _java2py(sc, java_model.labels()) - py_pi = _java2py(sc, java_model.pi()) - py_theta = _java2py(sc, java_model.theta()) + # Can not unpickle array.array from Pyrolite in Python3 with "bytes" + py_labels = _java2py(sc, java_model.labels(), "latin1") + py_pi = _java2py(sc, java_model.pi(), "latin1") + py_theta = _java2py(sc, java_model.theta(), "latin1") return NaiveBayesModel(py_labels, py_pi, numpy.array(py_theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 464f49aeee3cd..04e67158514f5 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,6 +15,12 @@ # limitations under the License. # +import sys +import array as pyarray + +if sys.version > '3': + xrange = range + from numpy import array from pyspark import RDD @@ -34,11 +40,16 @@ class KMeansModel(Saveable, Loader): >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4, 2) >>> model = KMeans.train( - ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") + ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random", + ... seed=50, initializationSteps=5, epsilon=1e-4) >>> model.predict(array([0.0, 0.0])) == model.predict(array([1.0, 1.0])) True >>> model.predict(array([8.0, 9.0])) == model.predict(array([9.0, 8.0])) True + >>> model.k + 2 + >>> model.computeCost(sc.parallelize(data)) + 2.0000000000000004 >>> model = KMeans.train(sc.parallelize(data), 2) >>> sparse_data = [ ... SparseVector(3, {1: 1.0}), @@ -46,7 +57,8 @@ class KMeansModel(Saveable, Loader): ... SparseVector(3, {2: 1.0}), ... SparseVector(3, {2: 1.1}) ... ] - >>> model = KMeans.train(sc.parallelize(sparse_data), 2, initializationMode="k-means||") + >>> model = KMeans.train(sc.parallelize(sparse_data), 2, initializationMode="k-means||", + ... seed=50, initializationSteps=5, epsilon=1e-4) >>> model.predict(array([0., 1., 0.])) == model.predict(array([0, 1.1, 0.])) True >>> model.predict(array([0., 0., 1.])) == model.predict(array([0, 0, 1.1])) @@ -55,8 +67,8 @@ class KMeansModel(Saveable, Loader): True >>> model.predict(sparse_data[2]) == model.predict(sparse_data[3]) True - >>> type(model.clusterCenters) - + >>> isinstance(model.clusterCenters, list) + True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -77,6 +89,11 @@ def clusterCenters(self): """Get the cluster centers, represented as a list of NumPy arrays.""" return self.centers + @property + def k(self): + """Total number of clusters.""" + return len(self.centers) + def predict(self, x): """Find the cluster to which x belongs in this model.""" best = 0 @@ -89,8 +106,17 @@ def predict(self, x): best_distance = distance return best + def computeCost(self, rdd): + """ + Return the K-means cost (sum of squared distances of points to + their nearest center) for this model on the given data. + """ + cost = callMLlibFunc("computeCostKmeansModel", rdd.map(_convert_to_vector), + [_convert_to_vector(c) for c in self.centers]) + return cost + def save(self, sc, path): - java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_centers = _py2java(sc, [_convert_to_vector(c) for c in self.centers]) java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) java_model.save(sc._jsc.sc(), path) @@ -103,10 +129,11 @@ def load(cls, sc, path): class KMeans(object): @classmethod - def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||", seed=None): + def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||", + seed=None, initializationSteps=5, epsilon=1e-4): """Train a k-means clustering model.""" model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations, - runs, initializationMode, seed) + runs, initializationMode, seed, initializationSteps, epsilon) centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) @@ -133,7 +160,7 @@ class GaussianMixtureModel(object): ... 5.7048, 4.6567, 5.5026, ... 4.5605, 5.2043, 6.2734]).reshape(5, 3)) >>> model = GaussianMixture.train(clusterdata_2, 2, convergenceTol=0.0001, - ... maxIterations=150, seed=10) + ... maxIterations=150, seed=10) >>> labels = model.predict(clusterdata_2).collect() >>> labels[0]==labels[1]==labels[2] True @@ -168,8 +195,8 @@ def predictSoft(self, x): if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) membership_matrix = callMLlibFunc("predictSoftGMM", x.map(_convert_to_vector), - self.weights, means, sigmas) - return membership_matrix + _convert_to_vector(self.weights), means, sigmas) + return membership_matrix.map(lambda x: pyarray.array('d', x)) class GaussianMixture(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index a539d2f2846f9..ba6058978880a 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -15,6 +15,11 @@ # limitations under the License. # +import sys +if sys.version >= '3': + long = int + unicode = str + import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -36,7 +41,7 @@ def _new_smart_decode(obj): if isinstance(obj, float): - s = unicode(obj) + s = str(obj) return _float_str_mapping.get(s, s) return _old_smart_decode(obj) @@ -74,15 +79,15 @@ def _py2java(sc, obj): obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, basestring)): + elif isinstance(obj, (int, long, float, bool, bytes, unicode)): pass else: - bytes = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.SerDe.loads(bytes) + data = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(data) return obj -def _java2py(sc, r): +def _java2py(sc, r, encoding="bytes"): if isinstance(r, JavaObject): clsName = r.getClass().getSimpleName() # convert RDD into JavaRDD @@ -102,8 +107,8 @@ def _java2py(sc, r): except Py4JJavaError: pass # not pickable - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) + if isinstance(r, (bytearray, bytes)): + r = PickleSerializer().loads(bytes(r), encoding=encoding) return r diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 0ffe092a07365..1140539a24e95 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -23,12 +23,17 @@ import sys import warnings import random +import binascii +if sys.version >= '3': + basestring = str + unicode = str from py4j.protocol import Py4JJavaError -from pyspark import RDD, SparkContext +from pyspark import SparkContext +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector +from pyspark.mllib.linalg import Vectors, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -132,6 +137,22 @@ def transform(self, vector): """ return JavaVectorTransformer.transform(self, vector) + def setWithMean(self, withMean): + """ + Setter of the boolean which decides + whether it uses mean or not + """ + self.call("setWithMean", withMean) + return self + + def setWithStd(self, withStd): + """ + Setter of the boolean which decides + whether it uses std or not + """ + self.call("setWithStd", withStd) + return self + class StandardScaler(object): """ @@ -190,7 +211,7 @@ class HashingTF(object): >>> htf = HashingTF(100) >>> doc = "a a b b c d".split(" ") >>> htf.transform(doc) - SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + SparseVector(100, {...}) """ def __init__(self, numFeatures=1 << 20): """ @@ -244,6 +265,12 @@ def transform(self, x): x = _convert_to_vector(x) return JavaVectorTransformer.transform(self, x) + def idf(self): + """ + Returns the current IDF vector. + """ + return self.call('idf') + class IDF(object): """ @@ -331,7 +358,14 @@ def findSynonyms(self, word, num): words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) + def getVectors(self): + """ + Returns a map of words to their vector representations. + """ + return self.call("getVectors") + +@ignore_unicode_prefix class Word2Vec(object): """ Word2Vec creates vector representation of words in a text corpus. @@ -354,7 +388,7 @@ class Word2Vec(object): >>> sentence = "a b " * 100 + "a c " * 10 >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) - >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] @@ -372,7 +406,8 @@ def __init__(self): self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = random.randint(0, sys.maxint) + self.seed = random.randint(0, sys.maxsize) + self.minCount = 5 def setVectorSize(self, vectorSize): """ @@ -411,6 +446,14 @@ def setSeed(self, seed): self.seed = seed return self + def setMinCount(self, minCount): + """ + Sets minCount, the minimum number of times a token must appear + to be included in the word2vec model's vocabulary (default: 5). + """ + self.minCount = minCount + return self + def fit(self, data): """ Computes the vector representation of each word in vocabulary. @@ -422,7 +465,8 @@ def fit(self, data): raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed)) + int(self.numIterations), int(self.seed), + int(self.minCount)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py new file mode 100644 index 0000000000000..d8df02bdbaba9 --- /dev/null +++ b/python/pyspark/mllib/fpm.py @@ -0,0 +1,92 @@ +# +# 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. +# + +import numpy +from numpy import array +from collections import namedtuple + +from pyspark import SparkContext +from pyspark.rdd import ignore_unicode_prefix +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc + +__all__ = ['FPGrowth', 'FPGrowthModel'] + + +@inherit_doc +@ignore_unicode_prefix +class FPGrowthModel(JavaModelWrapper): + + """ + .. note:: Experimental + + A FP-Growth model for mining frequent itemsets + using the Parallel FP-Growth algorithm. + + >>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]] + >>> rdd = sc.parallelize(data, 2) + >>> model = FPGrowth.train(rdd, 0.6, 2) + >>> sorted(model.freqItemsets().collect()) + [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'c'], freq=3), ... + """ + + def freqItemsets(self): + """ + Returns the frequent itemsets of this model. + """ + return self.call("getFreqItemsets").map(lambda x: (FPGrowth.FreqItemset(x[0], x[1]))) + + +class FPGrowth(object): + """ + .. note:: Experimental + + A Parallel FP-growth algorithm to mine frequent itemsets. + """ + + @classmethod + def train(cls, data, minSupport=0.3, numPartitions=-1): + """ + Computes an FP-Growth model that contains frequent itemsets. + :param data: The input data set, each element + contains a transaction. + :param minSupport: The minimal support level + (default: `0.3`). + :param numPartitions: The number of partitions used by parallel + FP-growth (default: same as input data). + """ + model = callMLlibFunc("trainFPGrowthModel", data, float(minSupport), int(numPartitions)) + return FPGrowthModel(model) + + class FreqItemset(namedtuple("FreqItemset", ["items", "freq"])): + """ + Represents an (items, freq) tuple. + """ + + +def _test(): + import doctest + import pyspark.mllib.fpm + globs = pyspark.mllib.fpm.__dict__.copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest') + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index f5aad28afda0f..9f3b0baf9f19f 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -25,7 +25,13 @@ import sys import array -import copy_reg + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg +else: + import copy_reg import numpy as np @@ -33,7 +39,8 @@ IntegerType, ByteType -__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', 'DenseMatrix', 'Matrices'] +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', + 'Matrix', 'DenseMatrix', 'SparseMatrix', 'Matrices'] if sys.version_info[:2] == (2, 7): @@ -57,7 +64,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -88,7 +95,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple): + elif type(v) in (array.array, list, tuple, xrange): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -139,7 +146,7 @@ def serialize(self, obj): values = [float(v) for v in obj] return (1, None, None, values) else: - raise ValueError("cannot serialize %r of type %r" % (obj, type(obj))) + raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) def deserialize(self, datum): assert len(datum) == 4, \ @@ -173,10 +180,27 @@ def toArray(self): class DenseVector(Vector): """ - A dense vector represented by a value array. + A dense vector represented by a value array. We use numpy array for + storage and arithmetics will be delegated to the underlying numpy + array. + + >>> v = Vectors.dense([1.0, 2.0]) + >>> u = Vectors.dense([3.0, 4.0]) + >>> v + u + DenseVector([4.0, 6.0]) + >>> 2 - v + DenseVector([1.0, 0.0]) + >>> v / 2 + DenseVector([0.5, 1.0]) + >>> v * u + DenseVector([3.0, 8.0]) + >>> u / v + DenseVector([3.0, 2.0]) + >>> u % 2 + DenseVector([1.0, 0.0]) """ def __init__(self, ar): - if isinstance(ar, basestring): + if isinstance(ar, bytes): ar = np.frombuffer(ar, dtype=np.float64) elif not isinstance(ar, np.ndarray): ar = np.array(ar, dtype=np.float64) @@ -292,6 +316,27 @@ def __ne__(self, other): def __getattr__(self, item): return getattr(self.array, item) + def _delegate(op): + def func(self, other): + if isinstance(other, DenseVector): + other = other.array + return DenseVector(getattr(self.array, op)(other)) + return func + + __neg__ = _delegate("__neg__") + __add__ = _delegate("__add__") + __sub__ = _delegate("__sub__") + __mul__ = _delegate("__mul__") + __div__ = _delegate("__div__") + __truediv__ = _delegate("__truediv__") + __mod__ = _delegate("__mod__") + __radd__ = _delegate("__radd__") + __rsub__ = _delegate("__rsub__") + __rmul__ = _delegate("__rmul__") + __rdiv__ = _delegate("__rdiv__") + __rtruediv__ = _delegate("__rtruediv__") + __rmod__ = _delegate("__rmod__") + class SparseVector(Vector): """ @@ -308,12 +353,12 @@ def __init__(self, size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print SparseVector(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> SparseVector(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" @@ -325,8 +370,8 @@ def __init__(self, size, *args): self.indices = np.array([p[0] for p in pairs], dtype=np.int32) self.values = np.array([p[1] for p in pairs], dtype=np.float64) else: - if isinstance(args[0], basestring): - assert isinstance(args[1], str), "values should be string too" + if isinstance(args[0], bytes): + assert isinstance(args[1], bytes), "values should be string too" if args[0]: self.indices = np.frombuffer(args[0], np.int32) self.values = np.frombuffer(args[1], np.float64) @@ -517,7 +562,7 @@ def __getitem__(self, index): inds = self.indices vals = self.values if not isinstance(index, int): - raise ValueError( + raise TypeError( "Indices must be of type integer, got type %s" % type(index)) if index < 0: index += self.size @@ -555,12 +600,12 @@ def sparse(size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> Vectors.sparse(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ return SparseVector(size, *args) @@ -594,9 +639,10 @@ class Matrix(object): Represents a local matrix. """ - def __init__(self, numRows, numCols): + def __init__(self, numRows, numCols, isTransposed=False): self.numRows = numRows self.numCols = numCols + self.isTransposed = isTransposed def toArray(self): """ @@ -604,24 +650,30 @@ def toArray(self): """ raise NotImplementedError + @staticmethod + def _convert_to_array(array_like, dtype): + """ + Convert Matrix attributes which are array-like or buffer to array. + """ + if isinstance(array_like, bytes): + return np.frombuffer(array_like, dtype=dtype) + return np.asarray(array_like, dtype=dtype) + class DenseMatrix(Matrix): """ Column-major dense matrix. """ - def __init__(self, numRows, numCols, values): - Matrix.__init__(self, numRows, numCols) - if isinstance(values, basestring): - values = np.frombuffer(values, dtype=np.float64) - elif not isinstance(values, np.ndarray): - values = np.array(values, dtype=np.float64) + def __init__(self, numRows, numCols, values, isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) + values = self._convert_to_array(values, np.float64) assert len(values) == numRows * numCols - if values.dtype != np.float64: - values.astype(np.float64) self.values = values def __reduce__(self): - return DenseMatrix, (self.numRows, self.numCols, self.values.tostring()) + return DenseMatrix, ( + self.numRows, self.numCols, self.values.tostring(), + int(self.isTransposed)) def toArray(self): """ @@ -632,13 +684,124 @@ def toArray(self): array([[ 0., 2.], [ 1., 3.]]) """ - return self.values.reshape((self.numRows, self.numCols), order='F') + if self.isTransposed: + return np.asfortranarray( + self.values.reshape((self.numRows, self.numCols))) + else: + return self.values.reshape((self.numRows, self.numCols), order='F') + + def toSparse(self): + """Convert to SparseMatrix""" + if self.isTransposed: + values = np.ravel(self.toArray(), order='F') + else: + values = self.values + indices = np.nonzero(values)[0] + colCounts = np.bincount(indices // self.numRows) + colPtrs = np.cumsum(np.hstack( + (0, colCounts, np.zeros(self.numCols - colCounts.size)))) + values = values[indices] + rowIndices = indices % self.numRows + + return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j >= self.numCols or j < 0: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + if self.isTransposed: + return self.values[i * self.numCols + j] + else: + return self.values[i + j * self.numRows] def __eq__(self, other): - return (isinstance(other, DenseMatrix) and - self.numRows == other.numRows and - self.numCols == other.numCols and - all(self.values == other.values)) + if (not isinstance(other, DenseMatrix) or + self.numRows != other.numRows or + self.numCols != other.numCols): + return False + + self_values = np.ravel(self.toArray(), order='F') + other_values = np.ravel(other.toArray(), order='F') + return all(self_values == other_values) + + +class SparseMatrix(Matrix): + """Sparse Matrix stored in CSC format.""" + def __init__(self, numRows, numCols, colPtrs, rowIndices, values, + isTransposed=False): + Matrix.__init__(self, numRows, numCols, isTransposed) + self.colPtrs = self._convert_to_array(colPtrs, np.int32) + self.rowIndices = self._convert_to_array(rowIndices, np.int32) + self.values = self._convert_to_array(values, np.float64) + + if self.isTransposed: + if self.colPtrs.size != numRows + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numRows + 1, self.colPtrs.size)) + else: + if self.colPtrs.size != numCols + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numCols + 1, self.colPtrs.size)) + if self.rowIndices.size != self.values.size: + raise ValueError("Expected rowIndices of length %d, got %d." + % (self.rowIndices.size, self.values.size)) + + def __reduce__(self): + return SparseMatrix, ( + self.numRows, self.numCols, self.colPtrs.tostring(), + self.rowIndices.tostring(), self.values.tostring(), + int(self.isTransposed)) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j < 0 or j >= self.numCols: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + # If a CSR matrix is given, then the row index should be searched + # for in ColPtrs, and the column index should be searched for in the + # corresponding slice obtained from rowIndices. + if self.isTransposed: + j, i = i, j + + colStart = self.colPtrs[j] + colEnd = self.colPtrs[j + 1] + nz = self.rowIndices[colStart: colEnd] + ind = np.searchsorted(nz, i) + colStart + if ind < colEnd and self.rowIndices[ind] == i: + return self.values[ind] + else: + return 0.0 + + def toArray(self): + """ + Return an numpy.ndarray + """ + A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') + for k in xrange(self.colPtrs.size - 1): + startptr = self.colPtrs[k] + endptr = self.colPtrs[k + 1] + if self.isTransposed: + A[k, self.rowIndices[startptr:endptr]] = self.values[startptr:endptr] + else: + A[self.rowIndices[startptr:endptr], k] = self.values[startptr:endptr] + return A + + def toDense(self): + densevals = np.ravel(self.toArray(), order='F') + return DenseMatrix(self.numRows, self.numCols, densevals) + + # TODO: More efficient implementation: + def __eq__(self, other): + return np.all(self.toArray() == other.toArray()) class Matrices(object): @@ -649,6 +812,13 @@ def dense(numRows, numCols, values): """ return DenseMatrix(numRows, numCols, values) + @staticmethod + def sparse(numRows, numCols, colPtrs, rowIndices, values): + """ + Create a SparseMatrix + """ + return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + def _test(): import doctest diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py index 20ee9d78bf5b0..06fbc0eb6aef0 100644 --- a/python/pyspark/mllib/rand.py +++ b/python/pyspark/mllib/rand.py @@ -88,10 +88,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): :param seed: Random seed (default: a random long integer). :return: RDD of float comprised of i.i.d. samples ~ N(0.0, 1.0). - >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - 0.0) < 0.1 True >>> abs(stats.stdev() - 1.0) < 0.1 @@ -118,10 +118,10 @@ def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None): >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L) + >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> from math import sqrt @@ -145,10 +145,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Pois(mean). >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -171,10 +171,10 @@ def exponentialRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Exp(mean). >>> mean = 2.0 - >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -202,10 +202,10 @@ def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None): >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L) + >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> abs(stats.stdev() - expStd) < 0.5 @@ -254,7 +254,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): :return: RDD of Vector with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. >>> import numpy as np - >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -286,8 +286,8 @@ def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \ - 100, 100, seed=1L).collect()) + >>> m = RandomRDDs.logNormalVectorRDD(sc, mean, std, 100, 100, seed=1).collect() + >>> mat = np.matrix(m) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 @@ -315,7 +315,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -345,7 +345,7 @@ def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=No >>> import numpy as np >>> mean = 0.5 - >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -380,8 +380,7 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \ - 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 1a4527b12cef2..9c4647ddfdcfd 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -15,12 +15,14 @@ # limitations under the License. # +import array from collections import namedtuple from pyspark import SparkContext from pyspark.rdd import RDD from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc from pyspark.mllib.util import JavaLoader, JavaSaveable +from pyspark.sql import DataFrame __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -52,7 +54,7 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> ratings = sc.parallelize([r1, r2, r3]) >>> model = ALS.trainImplicit(ratings, 1, seed=10) >>> model.predict(2, 2) - 0.43... + 0.4... >>> testset = sc.parallelize([(1, 2), (1, 1)]) >>> model = ALS.train(ratings, 2, seed=0) @@ -63,6 +65,13 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> model.userFeatures().collect() [(1, array('d', [...])), (2, array('d', [...]))] + >>> model.recommendUsers(1, 2) + [Rating(user=2, product=1, rating=1.9...), Rating(user=1, product=1, rating=1.0...)] + >>> model.recommendProducts(1, 2) + [Rating(user=1, product=2, rating=1.9...), Rating(user=1, product=1, rating=1.0...)] + >>> model.rank + 4 + >>> first_user = model.userFeatures().take(1)[0] >>> latents = first_user[1] >>> len(latents) == 4 @@ -77,52 +86,104 @@ class MatrixFactorizationModel(JavaModelWrapper, JavaSaveable, JavaLoader): True >>> model = ALS.train(ratings, 1, nonnegative=True, seed=10) - >>> model.predict(2,2) + >>> model.predict(2, 2) + 3.8... + + >>> df = sqlContext.createDataFrame([Rating(1, 1, 1.0), Rating(1, 2, 2.0), Rating(2, 1, 2.0)]) + >>> model = ALS.train(df, 1, nonnegative=True, seed=10) + >>> model.predict(2, 2) 3.8... >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) - >>> model.predict(2,2) - 0.43... + >>> model.predict(2, 2) + 0.4... >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) >>> sameModel = MatrixFactorizationModel.load(sc, path) - >>> sameModel.predict(2,2) - 0.43... + >>> sameModel.predict(2, 2) + 0.4... + >>> sameModel.predictAll(testset).collect() + [Rating(... >>> try: ... os.removedirs(path) ... except OSError: ... pass """ def predict(self, user, product): + """ + Predicts rating for the given user and product. + """ return self._java_model.predict(int(user), int(product)) def predictAll(self, user_product): + """ + Returns a list of predicted ratings for input user and product pairs. + """ assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() assert len(first) == 2, "user_product should be RDD of (user, product)" - user_product = user_product.map(lambda (u, p): (int(u), int(p))) + user_product = user_product.map(lambda u_p: (int(u_p[0]), int(u_p[1]))) return self.call("predict", user_product) def userFeatures(self): - return self.call("getUserFeatures") + """ + Returns a paired RDD, where the first element is the user and the + second is an array of features corresponding to that user. + """ + return self.call("getUserFeatures").mapValues(lambda v: array.array('d', v)) def productFeatures(self): - return self.call("getProductFeatures") + """ + Returns a paired RDD, where the first element is the product and the + second is an array of features corresponding to that product. + """ + return self.call("getProductFeatures").mapValues(lambda v: array.array('d', v)) + + def recommendUsers(self, product, num): + """ + Recommends the top "num" number of users for a given product and returns a list + of Rating objects sorted by the predicted rating in descending order. + """ + return list(self.call("recommendUsers", product, num)) + + def recommendProducts(self, user, num): + """ + Recommends the top "num" number of products for a given user and returns a list + of Rating objects sorted by the predicted rating in descending order. + """ + return list(self.call("recommendProducts", user, num)) + + @property + def rank(self): + return self.call("rank") + + @classmethod + def load(cls, sc, path): + model = cls._load_java(sc, path) + wrapper = sc._jvm.MatrixFactorizationModelWrapper(model) + return MatrixFactorizationModel(wrapper) class ALS(object): @classmethod def _prepare(cls, ratings): - assert isinstance(ratings, RDD), "ratings should be RDD" + if isinstance(ratings, RDD): + pass + elif isinstance(ratings, DataFrame): + ratings = ratings.rdd + else: + raise TypeError("Ratings should be represented by either an RDD or a DataFrame, " + "but got %s." % type(ratings)) first = ratings.first() - if not isinstance(first, Rating): - if isinstance(first, (tuple, list)): - ratings = ratings.map(lambda x: Rating(*x)) - else: - raise ValueError("rating should be RDD of Rating or tuple/list") + if isinstance(first, Rating): + pass + elif isinstance(first, (tuple, list)): + ratings = ratings.map(lambda x: Rating(*x)) + else: + raise TypeError("Expect a Rating or a tuple/list, but got %s." % type(first)) return ratings @classmethod @@ -143,8 +204,11 @@ def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alp def _test(): import doctest import pyspark.mllib.recommendation + from pyspark.sql import SQLContext globs = pyspark.mllib.recommendation.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + sc = SparkContext('local[4]', 'PythonTest') + globs['sc'] = sc + globs['sqlContext'] = SQLContext(sc) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 414a0ada80787..4bc6351bdf02f 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -108,7 +108,8 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=np.array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=np.array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -135,7 +136,15 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) + >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, + ... miniBatchFraction=1.0, initialWeights=array([1.0]), regParam=0.1, regType="l2", + ... intercept=True, validateData=True) >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 @@ -160,20 +169,27 @@ def load(cls, sc, path): # return the result of a call to the appropriate JVM stub. # _regression_train_wrapper is responsible for setup and error checking. def _regression_train_wrapper(train_func, modelClass, data, initial_weights): + from pyspark.mllib.classification import LogisticRegressionModel first = data.first() if not isinstance(first, LabeledPoint): - raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) + raise TypeError("data should be an RDD of LabeledPoint, but got %s" % type(first)) if initial_weights is None: initial_weights = [0.0] * len(data.first().features) - weights, intercept = train_func(data, _convert_to_vector(initial_weights)) - return modelClass(weights, intercept) + if (modelClass == LogisticRegressionModel): + weights, intercept, numFeatures, numClasses = train_func( + data, _convert_to_vector(initial_weights)) + return modelClass(weights, intercept, numFeatures, numClasses) + else: + weights, intercept = train_func(data, _convert_to_vector(initial_weights)) + return modelClass(weights, intercept) class LinearRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=0.0, regType=None, intercept=False): + initialWeights=None, regParam=0.0, regType=None, intercept=False, + validateData=True): """ Train a linear regression model on the given data. @@ -195,15 +211,18 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, (default: None) - @param intercept: Boolean parameter which indicates the use + :param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features are activated or not). (default: False) + :param validateData: Boolean parameter which indicates if the + algorithm should validate data before training. + (default: True) """ def train(rdd, i): return callMLlibFunc("trainLinearRegressionModelWithSGD", rdd, int(iterations), float(step), float(miniBatchFraction), i, float(regParam), - regType, bool(intercept)) + regType, bool(intercept), bool(validateData)) return _regression_train_wrapper(train, LinearRegressionModel, data, initialWeights) @@ -221,7 +240,7 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LassoWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -248,7 +267,15 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 @@ -273,11 +300,13 @@ class LassoWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None): + miniBatchFraction=1.0, initialWeights=None, intercept=False, + validateData=True): """Train a Lasso regression model on the given data.""" def train(rdd, i): return callMLlibFunc("trainLassoModelWithSGD", rdd, int(iterations), float(step), - float(regParam), float(miniBatchFraction), i) + float(regParam), float(miniBatchFraction), i, bool(intercept), + bool(validateData)) return _regression_train_wrapper(train, LassoModel, data, initialWeights) @@ -295,7 +324,8 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 @@ -322,7 +352,15 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, SparseVector(1, {0: 2.0})), ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, + ... initialWeights=array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + True + >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + True + >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, + ... regParam=0.01, miniBatchFraction=1.0, initialWeights=array([1.0]), intercept=True, + ... validateData=True) >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 @@ -347,11 +385,13 @@ class RidgeRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=0.01, - miniBatchFraction=1.0, initialWeights=None): + miniBatchFraction=1.0, initialWeights=None, intercept=False, + validateData=True): """Train a ridge regression model on the given data.""" def train(rdd, i): return callMLlibFunc("trainRidgeModelWithSGD", rdd, int(iterations), float(step), - float(regParam), float(miniBatchFraction), i) + float(regParam), float(miniBatchFraction), i, bool(intercept), + bool(validateData)) return _regression_train_wrapper(train, RidgeRegressionModel, data, initialWeights) @@ -361,7 +401,7 @@ def _test(): from pyspark import SparkContext import pyspark.mllib.regression globs = pyspark.mllib.regression.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 218ac148ca992..b475be4b4d953 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import RDD +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -38,7 +38,7 @@ def variance(self): return self.call("variance").toArray() def count(self): - return self.call("count") + return int(self.call("count")) def numNonzeros(self): return self.call("numNonzeros").toArray() @@ -49,6 +49,12 @@ def max(self): def min(self): return self.call("min").toArray() + def normL1(self): + return self.call("normL1").toArray() + + def normL2(self): + return self.call("normL2").toArray() + class Statistics(object): @@ -72,7 +78,7 @@ def colStats(rdd): >>> cStats.variance() array([ 4., 13., 0., 25.]) >>> cStats.count() - 3L + 3 >>> cStats.numNonzeros() array([ 3., 2., 0., 3.]) >>> cStats.max() @@ -118,20 +124,20 @@ def corr(x, y=None, method=None): >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) - >>> print str(pearsonCorr).replace('nan', 'NaN') + >>> print(str(pearsonCorr).replace('nan', 'NaN')) [[ 1. 0.05564149 NaN 0.40047142] [ 0.05564149 1. NaN 0.91359586] [ NaN NaN 1. NaN] [ 0.40047142 0.91359586 NaN 1. ]] >>> spearmanCorr = Statistics.corr(rdd, method="spearman") - >>> print str(spearmanCorr).replace('nan', 'NaN') + >>> print(str(spearmanCorr).replace('nan', 'NaN')) [[ 1. 0.10540926 NaN 0.4 ] [ 0.10540926 1. NaN 0.9486833 ] [ NaN NaN 1. NaN] [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") - ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... print("Method name as second argument without 'method=' shouldn't be allowed.") ... except TypeError: ... pass """ @@ -147,6 +153,7 @@ def corr(x, y=None, method=None): return callMLlibFunc("corr", x.map(float), y.map(float), method) @staticmethod + @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ .. note:: Experimental @@ -182,11 +189,11 @@ def chiSqTest(observed, expected=None): >>> from pyspark.mllib.linalg import Vectors, Matrices >>> observed = Vectors.dense([4, 6, 5]) >>> pearson = Statistics.chiSqTest(observed) - >>> print pearson.statistic + >>> print(pearson.statistic) 0.4 >>> pearson.degreesOfFreedom 2 - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.8187 >>> pearson.method u'pearson' @@ -196,12 +203,12 @@ def chiSqTest(observed, expected=None): >>> observed = Vectors.dense([21, 38, 43, 80]) >>> expected = Vectors.dense([3, 5, 7, 20]) >>> pearson = Statistics.chiSqTest(observed, expected) - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.0027 >>> data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0] >>> chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) - >>> print round(chi.statistic, 4) + >>> print(round(chi.statistic, 4)) 21.9958 >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), @@ -212,9 +219,9 @@ def chiSqTest(observed, expected=None): ... LabeledPoint(1.0, Vectors.dense([3.5, 40.0])),] >>> rdd = sc.parallelize(data, 4) >>> chi = Statistics.chiSqTest(rdd) - >>> print chi[0].statistic + >>> print(chi[0].statistic) 0.75 - >>> print chi[1].statistic + >>> print(chi[1].statistic) 1.5 """ if isinstance(observed, RDD): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 155019638f806..d05cfe2af04b2 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -24,7 +24,7 @@ import tempfile import array as pyarray -from numpy import array, array_equal +from numpy import array, array_equal, zeros from py4j.protocol import Py4JJavaError if sys.version_info[:2] <= (2, 6): @@ -36,14 +36,18 @@ else: import unittest +from pyspark import SparkContext +from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ - DenseMatrix, Vectors, Matrices + DenseMatrix, SparseMatrix, Vectors, Matrices from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics +from pyspark.mllib.feature import Word2Vec +from pyspark.mllib.feature import IDF +from pyspark.mllib.feature import StandardScaler from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext -from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase _have_scipy = False try: @@ -54,6 +58,12 @@ pass ser = PickleSerializer() +sc = SparkContext('local[4]', "MLlib tests") + + +class MLlibTestCase(unittest.TestCase): + def setUp(self): + self.sc = sc def _squared_distance(a, b): @@ -63,16 +73,16 @@ def _squared_distance(a, b): return b.squared_distance(a) -class VectorTests(PySparkTestCase): +class VectorTests(MLlibTestCase): def _test_serialize(self, v): self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) - nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + nv = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): @@ -82,6 +92,9 @@ def test_serialize(self): self._test_serialize(SparseVector(4, {1: 1, 3: 2})) self._test_serialize(SparseVector(3, {})) self._test_serialize(DenseMatrix(2, 3, range(6))) + sm1 = SparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self._test_serialize(sm1) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) @@ -131,11 +144,84 @@ def test_sparse_vector_indexing(self): self.assertEquals(sv[-1], 2) self.assertEquals(sv[-2], 0) self.assertEquals(sv[-4], 0) - for ind in [4, -5, 7.8]: + for ind in [4, -5]: self.assertRaises(ValueError, sv.__getitem__, ind) - - -class ListTests(PySparkTestCase): + for ind in [7.8, '1']: + self.assertRaises(TypeError, sv.__getitem__, ind) + + def test_matrix_indexing(self): + mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10]) + expected = [[0, 6], [1, 8], [4, 10]] + for i in range(3): + for j in range(2): + self.assertEquals(mat[i, j], expected[i][j]) + + def test_sparse_matrix(self): + # Test sparse matrix creation. + sm1 = SparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self.assertEquals(sm1.numRows, 3) + self.assertEquals(sm1.numCols, 4) + self.assertEquals(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4]) + self.assertEquals(sm1.rowIndices.tolist(), [1, 2, 1, 2]) + self.assertEquals(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0]) + + # Test indexing + expected = [ + [0, 0, 0, 0], + [1, 0, 4, 0], + [2, 0, 5, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1[i, j]) + self.assertTrue(array_equal(sm1.toArray(), expected)) + + # Test conversion to dense and sparse. + smnew = sm1.toDense().toSparse() + self.assertEquals(sm1.numRows, smnew.numRows) + self.assertEquals(sm1.numCols, smnew.numCols) + self.assertTrue(array_equal(sm1.colPtrs, smnew.colPtrs)) + self.assertTrue(array_equal(sm1.rowIndices, smnew.rowIndices)) + self.assertTrue(array_equal(sm1.values, smnew.values)) + + sm1t = SparseMatrix( + 3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], + isTransposed=True) + self.assertEquals(sm1t.numRows, 3) + self.assertEquals(sm1t.numCols, 4) + self.assertEquals(sm1t.colPtrs.tolist(), [0, 2, 3, 5]) + self.assertEquals(sm1t.rowIndices.tolist(), [0, 1, 2, 0, 2]) + self.assertEquals(sm1t.values.tolist(), [3.0, 2.0, 4.0, 9.0, 8.0]) + + expected = [ + [3, 2, 0, 0], + [0, 0, 4, 0], + [9, 0, 8, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1t[i, j]) + self.assertTrue(array_equal(sm1t.toArray(), expected)) + + def test_dense_matrix_is_transposed(self): + mat1 = DenseMatrix(3, 2, [0, 4, 1, 6, 3, 9], isTransposed=True) + mat = DenseMatrix(3, 2, [0, 1, 3, 4, 6, 9]) + self.assertEquals(mat1, mat) + + expected = [[0, 4], [1, 6], [3, 9]] + for i in range(3): + for j in range(2): + self.assertEquals(mat1[i, j], expected[i][j]) + self.assertTrue(array_equal(mat1.toArray(), expected)) + + sm = mat1.toSparse() + self.assertTrue(array_equal(sm.rowIndices, [1, 2, 0, 1, 2])) + self.assertTrue(array_equal(sm.colPtrs, [0, 2, 5])) + self.assertTrue(array_equal(sm.values, [1, 3, 4, 6, 9])) + + +class ListTests(MLlibTestCase): """ Test MLlib algorithms on plain lists, to make sure they're passed through @@ -150,7 +236,8 @@ def test_kmeans(self): [1.1, 0], [1.2, 0], ] - clusters = KMeans.train(self.sc.parallelize(data), 2, initializationMode="k-means||") + clusters = KMeans.train(self.sc.parallelize(data), 2, initializationMode="k-means||", + initializationSteps=7, epsilon=1e-4) self.assertEquals(clusters.predict(data[0]), clusters.predict(data[1])) self.assertEquals(clusters.predict(data[2]), clusters.predict(data[3])) @@ -160,9 +247,11 @@ def test_kmeans_deterministic(self): Y = range(0, 100, 10) data = [[x, y] for x, y in zip(X, Y)] clusters1 = KMeans.train(self.sc.parallelize(data), - 3, initializationMode="k-means||", seed=42) + 3, initializationMode="k-means||", + seed=42, initializationSteps=7, epsilon=1e-4) clusters2 = KMeans.train(self.sc.parallelize(data), - 3, initializationMode="k-means||", seed=42) + 3, initializationMode="k-means||", + seed=42, initializationSteps=7, epsilon=1e-4) centers1 = clusters1.centers centers2 = clusters2.centers for c1, c2 in zip(centers1, centers2): @@ -178,7 +267,7 @@ def test_gmm(self): [-6, -7], ]) clusters = GaussianMixture.train(data, 2, convergenceTol=0.001, - maxIterations=100, seed=56) + maxIterations=10, seed=56) labels = clusters.predict(data).collect() self.assertEquals(labels[0], labels[1]) self.assertEquals(labels[2], labels[3]) @@ -189,9 +278,9 @@ def test_gmm_deterministic(self): y = range(0, 100, 10) data = self.sc.parallelize([[a, b] for a, b in zip(x, y)]) clusters1 = GaussianMixture.train(data, 5, convergenceTol=0.001, - maxIterations=100, seed=63) + maxIterations=10, seed=63) clusters2 = GaussianMixture.train(data, 5, convergenceTol=0.001, - maxIterations=100, seed=63) + maxIterations=10, seed=63) for c1, c2 in zip(clusters1.weights, clusters2.weights): self.assertEquals(round(c1, 7), round(c2, 7)) @@ -210,13 +299,13 @@ def test_classification(self): temp_dir = tempfile.mkdtemp() - lr_model = LogisticRegressionWithSGD.train(rdd) + lr_model = LogisticRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(lr_model.predict(features[0]) <= 0) self.assertTrue(lr_model.predict(features[1]) > 0) self.assertTrue(lr_model.predict(features[2]) <= 0) self.assertTrue(lr_model.predict(features[3]) > 0) - svm_model = SVMWithSGD.train(rdd) + svm_model = SVMWithSGD.train(rdd, iterations=10) self.assertTrue(svm_model.predict(features[0]) <= 0) self.assertTrue(svm_model.predict(features[1]) > 0) self.assertTrue(svm_model.predict(features[2]) <= 0) @@ -230,7 +319,7 @@ def test_classification(self): categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories dt_model = DecisionTree.trainClassifier( - rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, maxBins=4) self.assertTrue(dt_model.predict(features[0]) <= 0) self.assertTrue(dt_model.predict(features[1]) > 0) self.assertTrue(dt_model.predict(features[2]) <= 0) @@ -242,7 +331,8 @@ def test_classification(self): self.assertEqual(same_dt_model.toDebugString(), dt_model.toDebugString()) rf_model = RandomForest.trainClassifier( - rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100) + rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=10, + maxBins=4, seed=1) self.assertTrue(rf_model.predict(features[0]) <= 0) self.assertTrue(rf_model.predict(features[1]) > 0) self.assertTrue(rf_model.predict(features[2]) <= 0) @@ -254,7 +344,7 @@ def test_classification(self): self.assertEqual(same_rf_model.toDebugString(), rf_model.toDebugString()) gbt_model = GradientBoostedTrees.trainClassifier( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numIterations=4) self.assertTrue(gbt_model.predict(features[0]) <= 0) self.assertTrue(gbt_model.predict(features[1]) > 0) self.assertTrue(gbt_model.predict(features[2]) <= 0) @@ -283,19 +373,19 @@ def test_regression(self): rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] - lr_model = LinearRegressionWithSGD.train(rdd) + lr_model = LinearRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(lr_model.predict(features[0]) <= 0) self.assertTrue(lr_model.predict(features[1]) > 0) self.assertTrue(lr_model.predict(features[2]) <= 0) self.assertTrue(lr_model.predict(features[3]) > 0) - lasso_model = LassoWithSGD.train(rdd) + lasso_model = LassoWithSGD.train(rdd, iterations=10) self.assertTrue(lasso_model.predict(features[0]) <= 0) self.assertTrue(lasso_model.predict(features[1]) > 0) self.assertTrue(lasso_model.predict(features[2]) <= 0) self.assertTrue(lasso_model.predict(features[3]) > 0) - rr_model = RidgeRegressionWithSGD.train(rdd) + rr_model = RidgeRegressionWithSGD.train(rdd, iterations=10) self.assertTrue(rr_model.predict(features[0]) <= 0) self.assertTrue(rr_model.predict(features[1]) > 0) self.assertTrue(rr_model.predict(features[2]) <= 0) @@ -303,35 +393,35 @@ def test_regression(self): categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories dt_model = DecisionTree.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, maxBins=4) self.assertTrue(dt_model.predict(features[0]) <= 0) self.assertTrue(dt_model.predict(features[1]) > 0) self.assertTrue(dt_model.predict(features[2]) <= 0) self.assertTrue(dt_model.predict(features[3]) > 0) rf_model = RandomForest.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100, seed=1) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=10, maxBins=4, seed=1) self.assertTrue(rf_model.predict(features[0]) <= 0) self.assertTrue(rf_model.predict(features[1]) > 0) self.assertTrue(rf_model.predict(features[2]) <= 0) self.assertTrue(rf_model.predict(features[3]) > 0) gbt_model = GradientBoostedTrees.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numIterations=4) self.assertTrue(gbt_model.predict(features[0]) <= 0) self.assertTrue(gbt_model.predict(features[1]) > 0) self.assertTrue(gbt_model.predict(features[2]) <= 0) self.assertTrue(gbt_model.predict(features[3]) > 0) try: - LinearRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) - LassoWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) - RidgeRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0])) + LinearRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) + LassoWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) + RidgeRegressionWithSGD.train(rdd, initialWeights=array([1.0, 1.0]), iterations=10) except ValueError: self.fail() -class StatTests(PySparkTestCase): +class StatTests(MLlibTestCase): # SPARK-4023 def test_col_with_different_rdds(self): # numpy @@ -347,8 +437,21 @@ def test_col_with_different_rdds(self): summary = Statistics.colStats(data) self.assertEqual(10, summary.count()) + def test_col_norms(self): + data = RandomRDDs.normalVectorRDD(self.sc, 1000, 10, 10) + summary = Statistics.colStats(data) + self.assertEqual(10, len(summary.normL1())) + self.assertEqual(10, len(summary.normL2())) -class VectorUDTTests(PySparkTestCase): + data2 = self.sc.parallelize(range(10)).map(lambda x: Vectors.dense(x)) + summary2 = Statistics.colStats(data2) + self.assertEqual(array([45.0]), summary2.normL1()) + import math + expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, range(10)))) + self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) + + +class VectorUDTTests(MLlibTestCase): dv0 = DenseVector([]) dv1 = DenseVector([1.0, 2.0]) @@ -366,11 +469,11 @@ def test_serialization(self): def test_infer_schema(self): sqlCtx = SQLContext(self.sc) rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) - srdd = sqlCtx.inferSchema(rdd) - schema = srdd.schema + df = rdd.toDF() + schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] self.assertEqual(field.dataType, self.udt) - vectors = srdd.map(lambda p: p.features).collect() + vectors = df.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: if isinstance(v, SparseVector): @@ -378,11 +481,11 @@ def test_infer_schema(self): elif isinstance(v, DenseVector): self.assertEqual(v, self.dv1) else: - raise ValueError("expecting a vector but got %r of type %r" % (v, type(v))) + raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) @unittest.skipIf(not _have_scipy, "SciPy not installed") -class SciPyTests(PySparkTestCase): +class SciPyTests(MLlibTestCase): """ Test both vector operations and MLlib algorithms with SciPy sparse matrices, @@ -523,7 +626,7 @@ def test_regression(self): self.assertTrue(dt_model.predict(features[3]) > 0) -class ChiSqTestTests(PySparkTestCase): +class ChiSqTestTests(MLlibTestCase): def test_goodness_of_fit(self): from numpy import inf @@ -620,9 +723,82 @@ def test_right_number_of_results(self): self.assertEqual(len(chi), num_cols) self.assertIsNotNone(chi[1000]) + +class SerDeTest(MLlibTestCase): + def test_to_java_object_rdd(self): # SPARK-6660 + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0) + self.assertEqual(_to_java_object_rdd(data).count(), 10) + + +class FeatureTest(MLlibTestCase): + def test_idf_model(self): + data = [ + Vectors.dense([1, 2, 6, 0, 2, 3, 1, 1, 0, 0, 3]), + Vectors.dense([1, 3, 0, 1, 3, 0, 0, 2, 0, 0, 1]), + Vectors.dense([1, 4, 1, 0, 0, 4, 9, 0, 1, 2, 0]), + Vectors.dense([2, 1, 0, 3, 0, 0, 5, 0, 2, 3, 9]) + ] + model = IDF().fit(self.sc.parallelize(data, 2)) + idf = model.idf() + self.assertEqual(len(idf), 11) + + +class Word2VecTests(MLlibTestCase): + def test_word2vec_setters(self): + model = Word2Vec() \ + .setVectorSize(2) \ + .setLearningRate(0.01) \ + .setNumPartitions(2) \ + .setNumIterations(10) \ + .setSeed(1024) \ + .setMinCount(3) + self.assertEquals(model.vectorSize, 2) + self.assertTrue(model.learningRate < 0.02) + self.assertEquals(model.numPartitions, 2) + self.assertEquals(model.numIterations, 10) + self.assertEquals(model.seed, 1024) + self.assertEquals(model.minCount, 3) + + def test_word2vec_get_vectors(self): + data = [ + ["a", "b", "c", "d", "e", "f", "g"], + ["a", "b", "c", "d", "e", "f"], + ["a", "b", "c", "d", "e"], + ["a", "b", "c", "d"], + ["a", "b", "c"], + ["a", "b"], + ["a"] + ] + model = Word2Vec().fit(self.sc.parallelize(data)) + self.assertEquals(len(model.getVectors()), 3) + + +class StandardScalerTests(MLlibTestCase): + def test_model_setters(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertIsNotNone(model.setWithMean(True)) + self.assertIsNotNone(model.setWithStd(True)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([-1.0, -1.0, -1.0])) + + def test_model_transform(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([1.0, 2.0, 3.0])) + + if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") + sc.stop() diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index bf288d76447bd..cfcbea573fd22 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -163,14 +163,16 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) - >>> print model, # it already has newline + >>> print(model) DecisionTreeModel classifier of depth 1 with 3 nodes - >>> print model.toDebugString(), # it already has newline + + >>> print(model.toDebugString()) DecisionTreeModel classifier of depth 1 with 3 nodes If (feature 0 <= 0.0) Predict: 0.0 Else (feature 0 > 0.0) Predict: 1.0 + >>> model.predict(array([1.0])) 1.0 >>> model.predict(array([0.0])) @@ -286,21 +288,18 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, :param numTrees: Number of trees in the random forest. :param featureSubsetStrategy: Number of features to consider for splits at each node. - Supported: "auto" (default), "all", "sqrt", "log2", - "onethird". - If "auto" is set, this parameter is set based on - numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "sqrt". - :param impurity: Criterion used for information gain - calculation. + Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + If "auto" is set, this parameter is set based on numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "sqrt". + :param impurity: Criterion used for information gain calculation. Supported values: "gini" (recommended) or "entropy". :param maxDepth: Maximum depth of the tree. E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. (default: 4) :param maxBins: maximum number of bins used for splitting features - (default: 100) + (default: 100) :param seed: Random seed for bootstrapping and choosing feature subsets. :return: RandomForestModel that can be used for prediction @@ -321,9 +320,10 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, 3 >>> model.totalNumNodes() 7 - >>> print model, + >>> print(model) TreeEnsembleModel classifier with 3 trees - >>> print model.toDebugString(), + + >>> print(model.toDebugString()) TreeEnsembleModel classifier with 3 trees Tree 0: @@ -338,6 +338,7 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, Predict: 0.0 Else (feature 0 > 1.0) Predict: 1.0 + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) @@ -365,13 +366,10 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, numTrees, featureSubsetSt :param numTrees: Number of trees in the random forest. :param featureSubsetStrategy: Number of features to consider for splits at each node. - Supported: "auto" (default), "all", "sqrt", "log2", - "onethird". - If "auto" is set, this parameter is set based on - numTrees: - if numTrees == 1, set to "all"; - if numTrees > 1 (forest) set to "onethird" for - regression. + Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + If "auto" is set, this parameter is set based on numTrees: + if numTrees == 1, set to "all"; + if numTrees > 1 (forest) set to "onethird" for regression. :param impurity: Criterion used for information gain calculation. Supported values: "variance". @@ -484,13 +482,14 @@ def trainClassifier(cls, data, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> - >>> model = GradientBoostedTrees.trainClassifier(sc.parallelize(data), {}) + >>> model = GradientBoostedTrees.trainClassifier(sc.parallelize(data), {}, numIterations=10) >>> model.numTrees() - 100 + 10 >>> model.totalNumNodes() - 300 - >>> print model, # it already has newline - TreeEnsembleModel classifier with 100 trees + 30 + >>> print(model) # it already has newline + TreeEnsembleModel classifier with 10 trees + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) @@ -542,11 +541,12 @@ def trainRegressor(cls, data, categoricalFeaturesInfo, ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> - >>> model = GradientBoostedTrees.trainRegressor(sc.parallelize(sparse_data), {}) + >>> data = sc.parallelize(sparse_data) + >>> model = GradientBoostedTrees.trainRegressor(data, {}, numIterations=10) >>> model.numTrees() - 100 + 10 >>> model.totalNumNodes() - 102 + 12 >>> model.predict(SparseVector(2, {1: 1.0})) 1.0 >>> model.predict(SparseVector(2, {0: 1.0})) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index c5c3468eb95e9..16a90db146ef0 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -15,10 +15,14 @@ # limitations under the License. # +import sys import numpy as np import warnings -from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc +if sys.version > '3': + xrange = range + +from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector @@ -94,22 +98,16 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> from pyspark.mllib.util import MLUtils >>> from pyspark.mllib.regression import LabeledPoint >>> tempFile = NamedTemporaryFile(delete=True) - >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") + >>> _ = tempFile.write(b"+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() - >>> type(examples[0]) == LabeledPoint - True - >>> print examples[0] - (1.0,(6,[0,2,4],[1.0,2.0,3.0])) - >>> type(examples[1]) == LabeledPoint - True - >>> print examples[1] - (-1.0,(6,[],[])) - >>> type(examples[2]) == LabeledPoint - True - >>> print examples[2] - (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) + >>> examples[0] + LabeledPoint(1.0, (6,[0,2,4],[1.0,2.0,3.0])) + >>> examples[1] + LabeledPoint(-1.0, (6,[],[])) + >>> examples[2] + LabeledPoint(-1.0, (6,[1,3,5],[4.0,5.0,6.0])) """ from pyspark.mllib.regression import LabeledPoint if multiclass is not None: diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 4408996db0790..d18daaabfcb3c 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -84,11 +84,11 @@ class Profiler(object): >>> from pyspark import BasicProfiler >>> class MyCustomProfiler(BasicProfiler): ... def show(self, id): - ... print "My custom profiles for RDD:%s" % id + ... print("My custom profiles for RDD:%s" % id) ... >>> conf = SparkConf().set("spark.python.profile", "true") >>> sc = SparkContext('local', 'test', conf=conf, profiler_cls=MyCustomProfiler) - >>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10) + >>> sc.parallelize(range(1000)).map(lambda x: 2 * x).take(10) [0, 2, 4, 6, 8, 10, 12, 14, 16, 18] >>> sc.show_profiles() My custom profiles for RDD:1 @@ -111,9 +111,9 @@ def show(self, id): """ Print the profile stats to stdout, id is the RDD id """ stats = self.stats() if stats: - print "=" * 60 - print "Profile of RDD" % id - print "=" * 60 + print("=" * 60) + print("Profile of RDD" % id) + print("=" * 60) stats.sort_stats("time", "cumulative").print_stats() def dump(self, id, path): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c337a43c8a7fc..d254deb527d10 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -16,21 +16,29 @@ # import copy -from collections import defaultdict -from itertools import chain, ifilter, imap -import operator import sys +import os +import re +import operator import shlex -from subprocess import Popen, PIPE -from tempfile import NamedTemporaryFile -from threading import Thread import warnings import heapq import bisect import random import socket +from subprocess import Popen, PIPE +from tempfile import NamedTemporaryFile +from threading import Thread +from collections import defaultdict +from itertools import chain +from functools import reduce from math import sqrt, log, isinf, isnan, pow, ceil +if sys.version > '3': + basestring = unicode = str +else: + from itertools import imap as map, ifilter as filter + from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -41,7 +49,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ - get_used_memory, ExternalSorter + get_used_memory, ExternalSorter, ExternalGroupBy from pyspark.traceback_utils import SCCallSiteSync from py4j.java_collections import ListConverter, MapConverter @@ -50,20 +58,21 @@ __all__ = ["RDD"] -# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized -# hash for string def portable_hash(x): """ - This function returns consistant hash code for builtin types, especially + This function returns consistent hash code for builtin types, especially for None and tuple with None. - The algrithm is similar to that one used by CPython 2.7 + The algorithm is similar to that one used by CPython 2.7 >>> portable_hash(None) 0 >>> portable_hash((None, 1)) & 0xffffffff 219750521 """ + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED") + if x is None: return 0 if isinstance(x, tuple): @@ -71,7 +80,7 @@ def portable_hash(x): for i in x: h ^= portable_hash(i) h *= 1000003 - h &= sys.maxint + h &= sys.maxsize h ^= len(x) if h == -1: h = -2 @@ -113,6 +122,7 @@ def _parse_memory(s): def _load_from_socket(port, serializer): sock = socket.socket() + sock.settimeout(3) try: sock.connect(("localhost", port)) rf = sock.makefile("rb", 65536) @@ -122,6 +132,19 @@ def _load_from_socket(port, serializer): sock.close() +def ignore_unicode_prefix(f): + """ + Ignore the 'u' prefix of string in doc tests, to make it works + in both python 2 and 3 + """ + if sys.version >= '3': + # the representation of unicode string in Python 3 does not have prefix 'u', + # so remove the prefix 'u' for doc tests + literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) + f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) + return f + + class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -250,7 +273,7 @@ def map(self, f, preservesPartitioning=False): [('a', 1), ('b', 1), ('c', 1)] """ def func(_, iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -265,7 +288,7 @@ def flatMap(self, f, preservesPartitioning=False): [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -328,7 +351,7 @@ def filter(self, f): [2, 4] """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def distinct(self, numPartitions=None): @@ -340,7 +363,7 @@ def distinct(self, numPartitions=None): """ return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x, numPartitions) \ - .map(lambda (x, _): x) + .map(lambda x: x[0]) def sample(self, withReplacement, fraction, seed=None): """ @@ -353,8 +376,8 @@ def sample(self, withReplacement, fraction, seed=None): :param seed: seed for the random number generator >>> rdd = sc.parallelize(range(100), 4) - >>> rdd.sample(False, 0.1, 81).count() - 10 + >>> 6 <= rdd.sample(False, 0.1, 81).count() <= 14 + True """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) @@ -367,12 +390,14 @@ def randomSplit(self, weights, seed=None): :param seed: random seed :return: split RDDs in a list - >>> rdd = sc.parallelize(range(5), 1) + >>> rdd = sc.parallelize(range(500), 1) >>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17) - >>> rdd1.collect() - [1, 3] - >>> rdd2.collect() - [0, 2, 4] + >>> len(rdd1.collect() + rdd2.collect()) + 500 + >>> 150 < rdd1.count() < 250 + True + >>> 250 < rdd2.count() < 350 + True """ s = float(sum(weights)) cweights = [0.0] @@ -415,7 +440,7 @@ def takeSample(self, withReplacement, num, seed=None): rand.shuffle(samples) return samples - maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) + maxSampleSize = sys.maxsize - int(numStDev * sqrt(sys.maxsize)) if num > maxSampleSize: raise ValueError( "Sample size cannot be greater than %d." % maxSampleSize) @@ -429,7 +454,7 @@ def takeSample(self, withReplacement, num, seed=None): # See: scala/spark/RDD.scala while len(samples) < num: # TODO: add log warning for when more than one iteration was run - seed = rand.randint(0, sys.maxint) + seed = rand.randint(0, sys.maxsize) samples = self.sample(withReplacement, fraction, seed).collect() rand.shuffle(samples) @@ -506,7 +531,7 @@ def intersection(self, other): """ return self.map(lambda v: (v, None)) \ .cogroup(other.map(lambda v: (v, None))) \ - .filter(lambda (k, vs): all(vs)) \ + .filter(lambda k_vs: all(k_vs[1])) \ .keys() def _reserialize(self, serializer=None): @@ -548,7 +573,7 @@ def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=p def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda k_v: keyfunc(k_v[0]), reverse=(not ascending))) return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) @@ -572,13 +597,13 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') - memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() serializer = self._jrdd_deserializer def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda kv: keyfunc(kv[0]), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: @@ -593,12 +618,12 @@ def sortPartition(iterator): return self # empty RDD maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() - samples = sorted(samples, reverse=(not ascending), key=keyfunc) + samples = self.sample(False, fraction, 1).map(lambda kv: kv[0]).collect() + samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary - bounds = [samples[len(samples) * (i + 1) / numPartitions] + bounds = [samples[int(len(samples) * (i + 1) / numPartitions)] for i in range(0, numPartitions - 1)] def rangePartitioner(k): @@ -661,12 +686,13 @@ def groupBy(self, f, numPartitions=None): """ return self.map(lambda x: (f(x), x)).groupByKey(numPartitions) + @ignore_unicode_prefix def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() - ['1', '2', '', '3'] + [u'1', u'2', u'', u'3'] """ def func(iterator): pipe = Popen( @@ -674,17 +700,18 @@ def func(iterator): def pipe_objs(out): for obj in iterator: - out.write(str(obj).rstrip('\n') + '\n') + s = str(obj).rstrip('\n') + '\n' + out.write(s.encode('utf-8')) out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) + return (x.rstrip(b'\n').decode('utf-8') for x in iter(pipe.stdout.readline, b'')) return self.mapPartitions(func) def foreach(self, f): """ Applies a function to all elements of this RDD. - >>> def f(x): print x + >>> def f(x): print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ def processPartition(iterator): @@ -699,7 +726,7 @@ def foreachPartition(self, f): >>> def f(iterator): ... for x in iterator: - ... print x + ... print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f) """ def func(it): @@ -873,7 +900,7 @@ def aggregatePartition(iterator): # aggregation. while numPartitions > scale + numPartitions / scale: numPartitions /= scale - curNumPartitions = numPartitions + curNumPartitions = int(numPartitions) def mapPartition(i, iterator): for obj in iterator: @@ -983,7 +1010,7 @@ def histogram(self, buckets): (('a', 'b', 'c'), [2, 2]) """ - if isinstance(buckets, (int, long)): + if isinstance(buckets, int): if buckets < 1: raise ValueError("number of buckets must be >= 1") @@ -1019,6 +1046,7 @@ def minmax(a, b): raise ValueError("Can not generate buckets with infinite value") # keep them as integer if possible + inc = int(inc) if inc * buckets != maxv - minv: inc = (maxv - minv) * 1.0 / buckets @@ -1136,7 +1164,7 @@ def countPartition(iterator): yield counts def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) @@ -1196,7 +1224,7 @@ def take(self, num): [91, 92, 93] """ items = [] - totalParts = self._jrdd.partitions().size() + totalParts = self.getNumPartitions() partsScanned = 0 while len(items) < num and partsScanned < totalParts: @@ -1259,7 +1287,7 @@ def isEmpty(self): >>> sc.parallelize([1]).isEmpty() False """ - return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0 + return self.getNumPartitions() == 0 or len(self.take(1)) == 0 def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): """ @@ -1377,8 +1405,8 @@ def saveAsPickleFile(self, path, batchSize=10): >>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile.close() >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) - >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) - [1, 2, 'rdd', 'spark'] + >>> sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect()) + ['1', '2', 'rdd', 'spark'] """ if batchSize == 0: ser = AutoBatchedSerializer(PickleSerializer()) @@ -1386,6 +1414,7 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) + @ignore_unicode_prefix def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. @@ -1417,12 +1446,13 @@ def saveAsTextFile(self, path, compressionCodecClass=None): >>> codec = "org.apache.hadoop.io.compress.GzipCodec" >>> sc.parallelize(['foo', 'bar']).saveAsTextFile(tempFile3.name, codec) >>> from fileinput import input, hook_compressed - >>> ''.join(sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed))) - 'bar\\nfoo\\n' + >>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed)) + >>> b''.join(result).decode('utf-8') + u'bar\\nfoo\\n' """ def func(split, iterator): for x in iterator: - if not isinstance(x, basestring): + if not isinstance(x, (unicode, bytes)): x = unicode(x) if isinstance(x, unicode): x = x.encode("utf-8") @@ -1457,7 +1487,7 @@ def keys(self): >>> m.collect() [1, 3] """ - return self.map(lambda (k, v): k) + return self.map(lambda x: x[0]) def values(self): """ @@ -1467,7 +1497,7 @@ def values(self): >>> m.collect() [2, 4] """ - return self.map(lambda (k, v): v) + return self.map(lambda x: x[1]) def reduceByKey(self, func, numPartitions=None): """ @@ -1506,7 +1536,7 @@ def reducePartition(iterator): yield m def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] = func(m1[k], v) if k in m1 else v return m1 return self.mapPartitions(reducePartition).reduce(mergeMaps) @@ -1603,8 +1633,8 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x)) >>> sets = pairs.partitionBy(2).glom().collect() - >>> set(sets[0]).intersection(set(sets[1])) - set([]) + >>> len(set(sets[0]).intersection(set(sets[1]))) + 0 """ if numPartitions is None: numPartitions = self._defaultReducePartitions() @@ -1636,22 +1666,22 @@ def add_shuffle_key(split, iterator): if (c % 1000 == 0 and get_used_memory() > limit or c > batch): n, size = len(buckets), 0 - for split in buckets.keys(): + for split in list(buckets.keys()): yield pack_long(split) d = outputSerializer.dumps(buckets[split]) del buckets[split] yield d size += len(d) - avg = (size / n) >> 20 + avg = int(size / n) >> 20 # let 1M < avg < 10M if avg < 1: batch *= 1.5 elif avg > 10: - batch = max(batch / 1.5, 1) + batch = max(int(batch / 1.5), 1) c = 0 - for split, items in buckets.iteritems(): + for split, items in buckets.items(): yield pack_long(split) yield outputSerializer.dumps(items) @@ -1698,17 +1728,15 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numPartitions = self._defaultReducePartitions() serializer = self.ctx.serializer - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() - == 'true') - memory = _parse_memory(self.ctx._conf.get( - "spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() agg = Aggregator(createCombiner, mergeValue, mergeCombiners) def combineLocally(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1717,7 +1745,7 @@ def _mergeCombiners(iterator): merger = ExternalMerger(agg, memory, serializer) \ if spill else InMemoryMerger(agg) merger.mergeCombiners(iterator) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True) @@ -1746,7 +1774,7 @@ def foldByKey(self, zeroValue, func, numPartitions=None): >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> from operator import add - >>> rdd.foldByKey(0, add).collect() + >>> sorted(rdd.foldByKey(0, add).collect()) [('a', 2), ('b', 1)] """ def createZero(): @@ -1754,21 +1782,28 @@ def createZero(): return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions) + def _can_spill(self): + return self.ctx._conf.get("spark.shuffle.spill", "True").lower() == "true" + + def _memory_limit(self): + return _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ Group the values for each key in the RDD into a single sequence. - Hash-partitions the resulting RDD with into numPartitions partitions. + Hash-partitions the resulting RDD with numPartitions partitions. Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. - >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.groupByKey().mapValues(len).collect()) + [('a', 2), ('b', 1)] + >>> sorted(rdd.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ - def createCombiner(x): return [x] @@ -1780,8 +1815,27 @@ def mergeCombiners(a, b): a.extend(b) return a - return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultIterable(x)) + spill = self._can_spill() + memory = self._memory_limit() + serializer = self._jrdd_deserializer + agg = Aggregator(createCombiner, mergeValue, mergeCombiners) + + def combine(iterator): + merger = ExternalMerger(agg, memory * 0.9, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeValues(iterator) + return merger.items() + + locally_combined = self.mapPartitions(combine, preservesPartitioning=True) + shuffled = locally_combined.partitionBy(numPartitions) + + def groupByKey(it): + merger = ExternalGroupBy(agg, memory, serializer)\ + if spill else InMemoryMerger(agg) + merger.mergeCombiners(it) + return merger.items() + + return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) def flatMapValues(self, f): """ @@ -1794,7 +1848,7 @@ def flatMapValues(self, f): >>> x.flatMapValues(f).collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')] """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def mapValues(self, f): @@ -1808,7 +1862,7 @@ def mapValues(self, f): >>> x.mapValues(f).collect() [('a', 3), ('b', 1)] """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def groupWith(self, other, *others): @@ -1819,8 +1873,7 @@ def groupWith(self, other, *others): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) >>> z = sc.parallelize([("b", 42)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1]), list(y[2]), list(y[3])))), \ - sorted(list(w.groupWith(x, y, z).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))] [('a', ([5], [1], [2], [])), ('b', ([6], [4], [], [42]))] """ @@ -1835,7 +1888,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))] [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup((self, other), numPartitions) @@ -1871,8 +1924,9 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - def filter_func((key, vals)): - return vals[0] and not vals[1] + def filter_func(pair): + key, (val1, val2) = pair + return val1 and not val2 return self.cogroup(other, numPartitions).filter(filter_func).flatMapValues(lambda x: x[0]) def subtract(self, other, numPartitions=None): @@ -1894,8 +1948,8 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) - [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] + >>> [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())] + [(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])] """ return self.map(lambda x: (f(x), x)) @@ -2024,17 +2078,18 @@ def name(self): """ Return the name of this RDD. """ - name_ = self._jrdd.name() - if name_: - return name_.encode('utf-8') + n = self._jrdd.name() + if n: + return n + @ignore_unicode_prefix def setName(self, name): """ Assign a name to this RDD. - >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() - 'RDD1' + u'RDD1' """ self._jrdd.setName(name) return self @@ -2096,7 +2151,7 @@ def lookup(self, key): >>> sorted.lookup(1024) [] """ - values = self.filter(lambda (k, v): k == key).values() + values = self.filter(lambda kv: kv[0] == key).values() if self.partitioner is not None: return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)], False) @@ -2134,7 +2189,7 @@ def sumApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) + >>> r = sum(range(1000)) >>> (rdd.sumApprox(1000) - r) / r < 0.05 True """ @@ -2151,7 +2206,7 @@ def meanApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) / 1000.0 + >>> r = sum(range(1000)) / 1000.0 >>> (rdd.meanApprox(1000) - r) / r < 0.05 True """ @@ -2176,10 +2231,10 @@ def countApproxDistinct(self, relativeSD=0.05): It must be greater than 0.000017. >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() - >>> 950 < n < 1050 + >>> 900 < n < 1100 True >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() - >>> 18 < n < 22 + >>> 16 < n < 24 True """ if relativeSD < 0.000017: @@ -2198,8 +2253,7 @@ def toLocalIterator(self): >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ - partitions = xrange(self.getNumPartitions()) - for partition in partitions: + for partition in range(self.getNumPartitions()): rows = self.context.runJob(self, lambda x: x, [partition]) for row in rows: yield row @@ -2208,13 +2262,14 @@ def toLocalIterator(self): def _prepare_for_python_RDD(sc, command, obj=None): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() - pickled_command = ser.dumps(command) + pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M + # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) - # tracking the life cycle by obj - if obj is not None: - obj._broadcast = broadcast + # There is a bug in py4j.java_gateway.JavaClass with auto_convert + # https://github.com/bartdag/py4j/issues/161 + # TODO: use auto_convert once py4j fix the bug broadcast_vars = ListConverter().convert( [x._jbroadcast for x in sc._pickled_broadcast_vars], sc._gateway._gateway_client) @@ -2269,12 +2324,9 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self.partitioner = prev.partitioner if self.preservesPartitioning else None - self._broadcast = None - def __del__(self): - if self._broadcast: - self._broadcast.unpersist() - self._broadcast = None + def getNumPartitions(self): + return self._prev_jrdd.partitions().size() @property def _jrdd(self): diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 459e1427803cb..fe8f87324804b 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -23,7 +23,7 @@ class RDDSamplerBase(object): def __init__(self, withReplacement, seed=None): - self._seed = seed if seed is not None else random.randint(0, sys.maxint) + self._seed = seed if seed is not None else random.randint(0, sys.maxsize) self._withReplacement = withReplacement self._random = None @@ -31,7 +31,7 @@ def initRandomGenerator(self, split): self._random = random.Random(self._seed ^ split) # mixing because the initial seeds are close to each other - for _ in xrange(10): + for _ in range(10): self._random.randint(0, 1) def getUniformSample(self): diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index ef04c82866e6c..1ab5ce14c3531 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -15,15 +15,16 @@ # limitations under the License. # -__all__ = ["ResultIterable"] - import collections +__all__ = ["ResultIterable"] + class ResultIterable(collections.Iterable): """ - A special result iterable. This is used because the standard iterator can not be pickled + A special result iterable. This is used because the standard + iterator can not be pickled """ def __init__(self, data): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 0ffb41d02f6f6..d8cdcda3a3783 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -49,16 +49,24 @@ >>> sc.stop() """ -import cPickle -from itertools import chain, izip, product +import sys +from itertools import chain, product import marshal import struct -import sys import types import collections import zlib import itertools +if sys.version < '3': + import cPickle as pickle + protocol = 2 + from itertools import izip as zip +else: + import pickle + protocol = 3 + xrange = range + from pyspark import cloudpickle @@ -97,7 +105,7 @@ def _load_stream_without_unbatching(self, stream): # subclasses should override __eq__ as appropriate. def __eq__(self, other): - return isinstance(other, self.__class__) + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not self.__eq__(other) @@ -212,14 +220,33 @@ def load_stream(self, stream): def _load_stream_without_unbatching(self, stream): return self.serializer.load_stream(stream) - def __eq__(self, other): - return (isinstance(other, BatchedSerializer) and - other.serializer == self.serializer and other.batchSize == self.batchSize) - def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) +class FlattenedValuesSerializer(BatchedSerializer): + + """ + Serializes a stream of list of pairs, split the list of values + which contain more than a certain number of objects to make them + have similar sizes. + """ + def __init__(self, serializer, batchSize=10): + BatchedSerializer.__init__(self, serializer, batchSize) + + def _batched(self, iterator): + n = self.batchSize + for key, values in iterator: + for i in range(0, len(values), n): + yield key, values[i:i + n] + + def load_stream(self, stream): + return self.serializer.load_stream(stream) + + def __repr__(self): + return "FlattenedValuesSerializer(%s, %d)" % (self.serializer, self.batchSize) + + class AutoBatchedSerializer(BatchedSerializer): """ Choose the size of batch automatically based on the size of object @@ -247,12 +274,8 @@ def dump_stream(self, iterator, stream): elif size > best * 10 and batch > 1: batch /= 2 - def __eq__(self, other): - return (isinstance(other, AutoBatchedSerializer) and - other.serializer == self.serializer and other.bestSize == self.bestSize) - - def __str__(self): - return "AutoBatchedSerializer(%s)" % str(self.serializer) + def __repr__(self): + return "AutoBatchedSerializer(%s)" % self.serializer class CartesianDeserializer(FramedSerializer): @@ -262,6 +285,7 @@ class CartesianDeserializer(FramedSerializer): """ def __init__(self, key_ser, val_ser): + FramedSerializer.__init__(self) self.key_ser = key_ser self.val_ser = val_ser @@ -270,7 +294,7 @@ def prepare_keys_values(self, stream): val_stream = self.val_ser._load_stream_without_unbatching(stream) key_is_batched = isinstance(self.key_ser, BatchedSerializer) val_is_batched = isinstance(self.val_ser, BatchedSerializer) - for (keys, vals) in izip(key_stream, val_stream): + for (keys, vals) in zip(key_stream, val_stream): keys = keys if key_is_batched else [keys] vals = vals if val_is_batched else [vals] yield (keys, vals) @@ -280,10 +304,6 @@ def load_stream(self, stream): for pair in product(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, CartesianDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "CartesianDeserializer(%s, %s)" % \ (str(self.key_ser), str(self.val_ser)) @@ -295,22 +315,14 @@ class PairDeserializer(CartesianDeserializer): Deserializes the JavaRDD zip() of two PythonRDDs. """ - def __init__(self, key_ser, val_ser): - self.key_ser = key_ser - self.val_ser = val_ser - def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): if len(keys) != len(vals): raise ValueError("Can not deserialize RDD with different number of items" " in pair: (%d, %d)" % (len(keys), len(vals))) - for pair in izip(keys, vals): + for pair in zip(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, PairDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "PairDeserializer(%s, %s)" % (str(self.key_ser), str(self.val_ser)) @@ -359,8 +371,8 @@ def _hijack_namedtuple(): global _old_namedtuple # or it will put in closure def _copy_func(f): - return types.FunctionType(f.func_code, f.func_globals, f.func_name, - f.func_defaults, f.func_closure) + return types.FunctionType(f.__code__, f.__globals__, f.__name__, + f.__defaults__, f.__closure__) _old_namedtuple = _copy_func(collections.namedtuple) @@ -369,15 +381,15 @@ def namedtuple(*args, **kwargs): return _hack_namedtuple(cls) # replace namedtuple with new one - collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple - collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple - collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__globals__["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.__globals__["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.__code__ = namedtuple.__code__ collections.namedtuple.__hijack = 1 # hack the cls already generated by namedtuple # those created in other module can be pickled as normal, # so only hack those in __main__ module - for n, o in sys.modules["__main__"].__dict__.iteritems(): + for n, o in sys.modules["__main__"].__dict__.items(): if (type(o) is type and o.__base__ is tuple and hasattr(o, "_fields") and "__reduce__" not in o.__dict__): @@ -390,7 +402,7 @@ def namedtuple(*args, **kwargs): class PickleSerializer(FramedSerializer): """ - Serializes objects using Python's cPickle serializer: + Serializes objects using Python's pickle serializer: http://docs.python.org/2/library/pickle.html @@ -399,10 +411,14 @@ class PickleSerializer(FramedSerializer): """ def dumps(self, obj): - return cPickle.dumps(obj, 2) + return pickle.dumps(obj, protocol) - def loads(self, obj): - return cPickle.loads(obj) + if sys.version >= '3': + def loads(self, obj, encoding="bytes"): + return pickle.loads(obj, encoding=encoding) + else: + def loads(self, obj, encoding=None): + return pickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -431,7 +447,7 @@ def loads(self, obj): class AutoSerializer(FramedSerializer): """ - Choose marshal or cPickle as serialization protocol automatically + Choose marshal or pickle as serialization protocol automatically """ def __init__(self): @@ -440,19 +456,19 @@ def __init__(self): def dumps(self, obj): if self._type is not None: - return 'P' + cPickle.dumps(obj, -1) + return b'P' + pickle.dumps(obj, -1) try: - return 'M' + marshal.dumps(obj) + return b'M' + marshal.dumps(obj) except Exception: - self._type = 'P' - return 'P' + cPickle.dumps(obj, -1) + self._type = b'P' + return b'P' + pickle.dumps(obj, -1) def loads(self, obj): _type = obj[0] - if _type == 'M': + if _type == b'M': return marshal.loads(obj[1:]) - elif _type == 'P': - return cPickle.loads(obj[1:]) + elif _type == b'P': + return pickle.loads(obj[1:]) else: raise ValueError("invalid sevialization type: %s" % _type) @@ -472,8 +488,8 @@ def dumps(self, obj): def loads(self, obj): return self.serializer.loads(zlib.decompress(obj)) - def __eq__(self, other): - return isinstance(other, CompressedSerializer) and self.serializer == other.serializer + def __repr__(self): + return "CompressedSerializer(%s)" % self.serializer class UTF8Deserializer(Serializer): @@ -482,7 +498,7 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ - def __init__(self, use_unicode=False): + def __init__(self, use_unicode=True): self.use_unicode = use_unicode def loads(self, stream): @@ -503,13 +519,13 @@ def load_stream(self, stream): except EOFError: return - def __eq__(self, other): - return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode + def __repr__(self): + return "UTF8Deserializer(%s)" % self.use_unicode def read_long(stream): length = stream.read(8) - if length == "": + if not length: raise EOFError return struct.unpack("!q", length)[0] @@ -524,7 +540,7 @@ def pack_long(value): def read_int(stream): length = stream.read(4) - if length == "": + if not length: raise EOFError return struct.unpack("!i", length)[0] diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 1a02fece9c5a5..144cdf0b0cdd5 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -21,13 +21,6 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ -import sys -if sys.version_info[0] != 2: - print("Error: Default Python used is Python%s" % sys.version_info.major) - print("\tSet env variable PYSPARK_PYTHON to Python2 binary and re-run it.") - sys.exit(1) - - import atexit import os import platform @@ -53,9 +46,14 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = HiveContext(sc) + sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) +except TypeError: + sqlContext = SQLContext(sc) + +# for compatibility +sqlCtx = sqlContext print("""Welcome to ____ __ @@ -68,7 +66,7 @@ platform.python_version(), platform.python_build()[0], platform.python_build()[1])) -print("SparkContext available as sc, %s available as sqlCtx." % sqlCtx.__class__.__name__) +print("SparkContext available as sc, %s available as sqlContext." % sqlContext.__class__.__name__) if add_files is not None: print("Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead") diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 10a7ccd502000..1d0b16cade8bb 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -16,28 +16,35 @@ # import os -import sys import platform import shutil import warnings import gc import itertools +import operator import random import pyspark.heapq3 as heapq -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ + CompressedSerializer, AutoBatchedSerializer + try: import psutil + process = None + def get_used_memory(): """ Return the used memory in MB """ - process = psutil.Process(os.getpid()) + global process + if process is None or process._pid != os.getpid(): + process = psutil.Process(os.getpid()) if hasattr(process, "memory_info"): info = process.memory_info() else: info = process.get_memory_info() return info.rss >> 20 + except ImportError: def get_used_memory(): @@ -46,6 +53,7 @@ def get_used_memory(): for line in open('/proc/self/status'): if line.startswith('VmRSS:'): return int(line.split()[1]) >> 10 + else: warnings.warn("Please install psutil to have better " "support with spilling") @@ -54,6 +62,7 @@ def get_used_memory(): rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss return rss >> 20 # TODO: support windows + return 0 @@ -69,8 +78,8 @@ def _get_local_dirs(sub): # global stats -MemoryBytesSpilled = 0L -DiskBytesSpilled = 0L +MemoryBytesSpilled = 0 +DiskBytesSpilled = 0 class Aggregator(object): @@ -117,7 +126,7 @@ def mergeCombiners(self, iterator): """ Merge the combined items by mergeCombiner """ raise NotImplementedError - def iteritems(self): + def items(self): """ Return the merged items ad iterator """ raise NotImplementedError @@ -147,9 +156,15 @@ def mergeCombiners(self, iterator): for k, v in iterator: d[k] = comb(d[k], v) if k in d else v - def iteritems(self): + def items(self): """ Return the merged items ad iterator """ - return self.data.iteritems() + return iter(self.data.items()) + + +def _compressed_serializer(self, serializer=None): + # always use PickleSerializer to simplify implementation + ser = PickleSerializer() + return AutoBatchedSerializer(CompressedSerializer(ser)) class ExternalMerger(Merger): @@ -173,7 +188,7 @@ class ExternalMerger(Merger): dict. Repeat this again until combine all the items. - Before return any items, it will load each partition and - combine them seperately. Yield them before loading next + combine them separately. Yield them before loading next partition. - During loading a partition, if the memory goes over limit, @@ -182,7 +197,7 @@ class ExternalMerger(Merger): `data` and `pdata` are used to hold the merged items in memory. At first, all the data are merged into `data`. Once the used - memory goes over limit, the items in `data` are dumped indo + memory goes over limit, the items in `data` are dumped into disks, `data` will be cleared, all rest of items will be merged into `pdata` and then dumped into disks. Before returning, all the items in `pdata` will be dumped into disks. @@ -193,16 +208,16 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeValues(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) - 499950000 + >>> sum(v for k,v in merger.items()) + 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeCombiners(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) - 499950000 + >>> sum(v for k,v in merger.items()) + 49995000 """ # the max total partitions created recursively @@ -212,8 +227,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, localdirs=None, scale=1, partitions=59, batch=1000): Merger.__init__(self, aggregator) self.memory_limit = memory_limit - # default serializer is only used for tests - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions @@ -221,7 +235,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, self.batch = batch # scale is used to scale down the hash of key for recursive hash map self.scale = scale - # unpartitioned merged data + # un-partitioned merged data self.data = {} # partitioned merged data, list of dicts self.pdata = [] @@ -244,72 +258,63 @@ def _next_limit(self): def mergeValues(self, iterator): """ Combine the items by creator and combiner """ - iterator = iter(iterator) # speedup attribute lookup creator, comb = self.agg.createCombiner, self.agg.mergeValue - d, c, batch = self.data, 0, self.batch + c, data, pdata, hfun, batch = 0, self.data, self.pdata, self._partition, self.batch + limit = self.memory_limit for k, v in iterator: + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else creator(v) c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeValues(iterator, self._next_limit()) - break + if c >= batch: + if get_used_memory() >= limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if get_used_memory() >= limit: + self._spill() def _partition(self, key): """ Return the partition for key """ return hash((key, self._seed)) % self.partitions - def _partitioned_mergeValues(self, iterator, limit=0): - """ Partition the items by key, then combine them """ - # speedup attribute lookup - creator, comb = self.agg.createCombiner, self.agg.mergeValue - c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch - - for k, v in iterator: - d = pdata[hfun(k)] - d[k] = comb(d[k], v) if k in d else creator(v) - if not limit: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + def _object_size(self, obj): + """ How much of memory for this obj, assume that all the objects + consume similar bytes of memory + """ + return 1 - def mergeCombiners(self, iterator, check=True): + def mergeCombiners(self, iterator, limit=None): """ Merge (K,V) pair by mergeCombiner """ - iterator = iter(iterator) + if limit is None: + limit = self.memory_limit # speedup attribute lookup - d, comb, batch = self.data, self.agg.mergeCombiners, self.batch - c = 0 - for k, v in iterator: - d[k] = comb(d[k], v) if k in d else v - if not check: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeCombiners(iterator, self._next_limit()) - break - - def _partitioned_mergeCombiners(self, iterator, limit=0): - """ Partition the items by key, then merge them """ - comb, pdata = self.agg.mergeCombiners, self.pdata - c, hfun = 0, self._partition + comb, hfun, objsize = self.agg.mergeCombiners, self._partition, self._object_size + c, data, pdata, batch = 0, self.data, self.pdata, self.batch for k, v in iterator: - d = pdata[hfun(k)] + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else v if not limit: continue - c += 1 - if c % self.batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + c += objsize(v) + if c > batch: + if get_used_memory() > limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if limit and get_used_memory() >= limit: + self._spill() def _spill(self): """ @@ -330,12 +335,12 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) - # put one item in batch, make it compatitable with load_stream + # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch self.serializer.dump_stream([(k, v)], streams[h]) @@ -344,14 +349,14 @@ def _spill(self): s.close() self.data.clear() - self.pdata = [{} for i in range(self.partitions)] + self.pdata.extend([{} for i in range(self.partitions)]) else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(iter(self.pdata[i].items()), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -359,10 +364,10 @@ def _spill(self): gc.collect() # release the memory as much as possible MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 - def iteritems(self): + def items(self): """ Return all merged items as iterator """ if not self.pdata and not self.spills: - return self.data.iteritems() + return iter(self.data.items()) return self._external_items() def _external_items(self): @@ -370,29 +375,12 @@ def _external_items(self): assert not self.data if any(self.pdata): self._spill() - hard_limit = self._next_limit() + # disable partitioning and spilling when merge combiners from disk + self.pdata = [] try: for i in range(self.partitions): - self.data = {} - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), - False) - - # limit the total partitions - if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS - and j < self.spills - 1 - and get_used_memory() > hard_limit): - self.data.clear() # will read from disk again - gc.collect() # release the memory as much as possible - for v in self._recursive_merged_items(i): - yield v - return - - for v in self.data.iteritems(): + for v in self._merged_items(i): yield v self.data.clear() @@ -400,53 +388,58 @@ def _external_items(self): for j in range(self.spills): path = self._get_spill_dir(j) os.remove(os.path.join(path, str(i))) - finally: self._cleanup() - def _cleanup(self): - """ Clean up all the files in disks """ - for d in self.localdirs: - shutil.rmtree(d, True) - - def _recursive_merged_items(self, start): + def _merged_items(self, index): + self.data = {} + limit = self._next_limit() + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) + + # limit the total partitions + if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS + and j < self.spills - 1 + and get_used_memory() > limit): + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible + return self._recursive_merged_items(index) + + return self.data.items() + + def _recursive_merged_items(self, index): """ merge the partitioned items and return the as iterator If one partition can not be fit in memory, then them will be partitioned and merged recursively. """ - # make sure all the data are dumps into disks. - assert not self.data - if any(self.pdata): - self._spill() - assert self.spills > 0 - - for i in range(start, self.partitions): - subdirs = [os.path.join(d, "parts", str(i)) - for d in self.localdirs] - m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions, self.partitions) - m.pdata = [{} for _ in range(self.partitions)] - limit = self._next_limit() - - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - m._partitioned_mergeCombiners( - self.serializer.load_stream(open(p))) - - if get_used_memory() > limit: - m._spill() - limit = self._next_limit() + subdirs = [os.path.join(d, "parts", str(index)) for d in self.localdirs] + m = ExternalMerger(self.agg, self.memory_limit, self.serializer, subdirs, + self.scale * self.partitions, self.partitions, self.batch) + m.pdata = [{} for _ in range(self.partitions)] + limit = self._next_limit() + + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + with open(p, 'rb') as f: + m.mergeCombiners(self.serializer.load_stream(f), 0) + + if get_used_memory() > limit: + m._spill() + limit = self._next_limit() - for v in m._external_items(): - yield v + return m._external_items() - # remove the merged partition - for j in range(self.spills): - path = self._get_spill_dir(j) - os.remove(os.path.join(path, str(i))) + def _cleanup(self): + """ Clean up all the files in disks """ + for d in self.localdirs: + shutil.rmtree(d, True) class ExternalSorter(object): @@ -457,9 +450,10 @@ class ExternalSorter(object): The spilling will only happen when the used memory goes above the limit. + >>> sorter = ExternalSorter(1) # 1M >>> import random - >>> l = range(1024) + >>> l = list(range(1024)) >>> random.shuffle(l) >>> sorted(l) == list(sorter.sorted(l)) True @@ -469,7 +463,7 @@ class ExternalSorter(object): def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) def _get_path(self, n): """ Choose one directory for spill by number n """ @@ -492,7 +486,7 @@ def sorted(self, iterator, key=None, reverse=False): goes above the limit. """ global MemoryBytesSpilled, DiskBytesSpilled - batch, limit = 100, self._next_limit() + batch, limit = 100, self.memory_limit chunks, current_chunk = [], [] iterator = iter(iterator) while True: @@ -503,21 +497,30 @@ def sorted(self, iterator, key=None, reverse=False): break used_memory = get_used_memory() - if used_memory > self.memory_limit: + if used_memory > limit: # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) - with open(path, 'w') as f: + with open(path, 'wb') as f: self.serializer.dump_stream(current_chunk, f) - chunks.append(self.serializer.load_stream(open(path))) + + def load(f): + for v in self.serializer.load_stream(f): + yield v + # close the file explicit once we consume all the items + # to avoid ResourceWarning in Python3 + f.close() + chunks.append(load(open(path, 'rb'))) current_chunk = [] gc.collect() + batch //= 2 limit = self._next_limit() MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 DiskBytesSpilled += os.path.getsize(path) + os.unlink(path) # data will be deleted after close elif not chunks: - batch = min(batch * 2, 10000) + batch = min(int(batch * 1.5), 10000) current_chunk.sort(key=key, reverse=reverse) if not chunks: @@ -529,6 +532,313 @@ def sorted(self, iterator, key=None, reverse=False): return heapq.merge(chunks, key=key, reverse=reverse) +class ExternalList(object): + """ + ExternalList can have many items which cannot be hold in memory in + the same time. + + >>> l = ExternalList(list(range(100))) + >>> len(l) + 100 + >>> l.append(10) + >>> len(l) + 101 + >>> for i in range(20240): + ... l.append(i) + >>> len(l) + 20341 + >>> import pickle + >>> l2 = pickle.loads(pickle.dumps(l)) + >>> len(l2) + 20341 + >>> list(l2)[100] + 10 + """ + LIMIT = 10240 + + def __init__(self, values): + self.values = values + self.count = len(values) + self._file = None + self._ser = None + + def __getstate__(self): + if self._file is not None: + self._file.flush() + with os.fdopen(os.dup(self._file.fileno()), "rb") as f: + f.seek(0) + serialized = f.read() + else: + serialized = b'' + return self.values, self.count, serialized + + def __setstate__(self, item): + self.values, self.count, serialized = item + if serialized: + self._open_file() + self._file.write(serialized) + else: + self._file = None + self._ser = None + + def __iter__(self): + if self._file is not None: + self._file.flush() + # read all items from disks first + with os.fdopen(os.dup(self._file.fileno()), 'rb') as f: + f.seek(0) + for v in self._ser.load_stream(f): + yield v + + for v in self.values: + yield v + + def __len__(self): + return self.count + + def append(self, value): + self.values.append(value) + self.count += 1 + # dump them into disk if the key is huge + if len(self.values) >= self.LIMIT: + self._spill() + + def _open_file(self): + dirs = _get_local_dirs("objects") + d = dirs[id(self) % len(dirs)] + if not os.path.exists(d): + os.makedirs(d) + p = os.path.join(d, str(id(self))) + self._file = open(p, "wb+", 65536) + self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) + os.unlink(p) + + def __del__(self): + if self._file: + self._file.close() + self._file = None + + def _spill(self): + """ dump the values into disk """ + global MemoryBytesSpilled, DiskBytesSpilled + if self._file is None: + self._open_file() + + used_memory = get_used_memory() + pos = self._file.tell() + self._ser.dump_stream(self.values, self._file) + self.values = [] + gc.collect() + DiskBytesSpilled += self._file.tell() - pos + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + +class ExternalListOfList(ExternalList): + """ + An external list for list. + + >>> l = ExternalListOfList([[i, i] for i in range(100)]) + >>> len(l) + 200 + >>> l.append(range(10)) + >>> len(l) + 210 + >>> len(list(l)) + 210 + """ + + def __init__(self, values): + ExternalList.__init__(self, values) + self.count = sum(len(i) for i in values) + + def append(self, value): + ExternalList.append(self, value) + # already counted 1 in ExternalList.append + self.count += len(value) - 1 + + def __iter__(self): + for values in ExternalList.__iter__(self): + for v in values: + yield v + + +class GroupByKey(object): + """ + Group a sorted iterator as [(k1, it1), (k2, it2), ...] + + >>> k = [i // 3 for i in range(6)] + >>> v = [[i] for i in range(6)] + >>> g = GroupByKey(zip(k, v)) + >>> [(k, list(it)) for k, it in g] + [(0, [0, 1, 2]), (1, [3, 4, 5])] + """ + + def __init__(self, iterator): + self.iterator = iterator + + def __iter__(self): + key, values = None, None + for k, v in self.iterator: + if values is not None and k == key: + values.append(v) + else: + if values is not None: + yield (key, values) + key = k + values = ExternalListOfList([v]) + if values is not None: + yield (key, values) + + +class ExternalGroupBy(ExternalMerger): + + """ + Group by the items by key. If any partition of them can not been + hold in memory, it will do sort based group by. + + This class works as follows: + + - It repeatedly group the items by key and save them in one dict in + memory. + + - When the used memory goes above memory limit, it will split + the combined data into partitions by hash code, dump them + into disk, one file per partition. If the number of keys + in one partitions is smaller than 1000, it will sort them + by key before dumping into disk. + + - Then it goes through the rest of the iterator, group items + by key into different dict by hash. Until the used memory goes over + memory limit, it dump all the dicts into disks, one file per + dict. Repeat this again until combine all the items. It + also will try to sort the items by key in each partition + before dumping into disks. + + - It will yield the grouped items partitions by partitions. + If the data in one partitions can be hold in memory, then it + will load and combine them in memory and yield. + + - If the dataset in one partition cannot be hold in memory, + it will sort them first. If all the files are already sorted, + it merge them by heap.merge(), so it will do external sort + for all the files. + + - After sorting, `GroupByKey` class will put all the continuous + items with the same key as a group, yield the values as + an iterator. + """ + SORT_KEY_LIMIT = 1000 + + def flattened_serializer(self): + assert isinstance(self.serializer, BatchedSerializer) + ser = self.serializer + return FlattenedValuesSerializer(ser, 20) + + def _object_size(self, obj): + return len(obj) + + def _spill(self): + """ + dump already partitioned data into disks. + """ + global MemoryBytesSpilled, DiskBytesSpilled + path = self._get_spill_dir(self.spills) + if not os.path.exists(path): + os.makedirs(path) + + used_memory = get_used_memory() + if not self.pdata: + # The data has not been partitioned, it will iterator the + # data once, write them into different files, has no + # additional memory. It only called when the memory goes + # above limit at the first time. + + # open all the files for writing + streams = [open(os.path.join(path, str(i)), 'wb') + for i in range(self.partitions)] + + # If the number of keys is small, then the overhead of sort is small + # sort them before dumping into disks + self._sorted = len(self.data) < self.SORT_KEY_LIMIT + if self._sorted: + self.serializer = self.flattened_serializer() + for k in sorted(self.data.keys()): + h = self._partition(k) + self.serializer.dump_stream([(k, self.data[k])], streams[h]) + else: + for k, v in self.data.items(): + h = self._partition(k) + self.serializer.dump_stream([(k, v)], streams[h]) + + for s in streams: + DiskBytesSpilled += s.tell() + s.close() + + self.data.clear() + # self.pdata is cached in `mergeValues` and `mergeCombiners` + self.pdata.extend([{} for i in range(self.partitions)]) + + else: + for i in range(self.partitions): + p = os.path.join(path, str(i)) + with open(p, "wb") as f: + # dump items in batch + if self._sorted: + # sort by key only (stable) + sorted_items = sorted(self.pdata[i].items(), key=operator.itemgetter(0)) + self.serializer.dump_stream(sorted_items, f) + else: + self.serializer.dump_stream(self.pdata[i].items(), f) + self.pdata[i].clear() + DiskBytesSpilled += os.path.getsize(p) + + self.spills += 1 + gc.collect() # release the memory as much as possible + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + def _merged_items(self, index): + size = sum(os.path.getsize(os.path.join(self._get_spill_dir(j), str(index))) + for j in range(self.spills)) + # if the memory can not hold all the partition, + # then use sort based merge. Because of compression, + # the data on disks will be much smaller than needed memory + if size >= self.memory_limit << 17: # * 1M / 8 + return self._merge_sorted_items(index) + + self.data = {} + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) + return self.data.items() + + def _merge_sorted_items(self, index): + """ load a partition from disk, then sort and group by key """ + def load_partition(j): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + with open(p, 'rb', 65536) as f: + for v in self.serializer.load_stream(f): + yield v + + disk_items = [load_partition(j) for j in range(self.spills)] + + if self._sorted: + # all the partitions are already sorted + sorted_items = heapq.merge(disk_items, key=operator.itemgetter(0)) + + else: + # Flatten the combined values, so it will not consume huge + # memory during merging sort. + ser = self.flattened_serializer() + sorter = ExternalSorter(self.memory_limit, ser) + sorted_items = sorter.sorted(itertools.chain(*disk_items), + key=operator.itemgetter(0)) + return ((k, vs) for k, vs in GroupByKey(sorted_items)) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index b9ffd6945ea7e..b60b991dd4d8b 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -16,26 +16,47 @@ # """ -public classes of Spark SQL: +Important classes of Spark SQL and DataFrames: - L{SQLContext} - Main entry point for SQL functionality. + Main entry point for :class:`DataFrame` and SQL functionality. - L{DataFrame} - A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In - addition to normal RDD operations, DataFrames also support SQL. - - L{GroupedData} + A distributed collection of data grouped into named columns. - L{Column} - Column is a DataFrame with a single column. + A column expression in a :class:`DataFrame`. - L{Row} - A Row of data returned by a Spark SQL query. + A row of data in a :class:`DataFrame`. - L{HiveContext} - Main entry point for accessing data stored in Apache Hive.. + Main entry point for accessing data stored in Apache Hive. + - L{GroupedData} + Aggregation methods, returned by :func:`DataFrame.groupBy`. + - L{DataFrameNaFunctions} + Methods for handling missing data (null values). + - L{functions} + List of built-in functions available for :class:`DataFrame`. + - L{types} + List of data types available. """ +from __future__ import absolute_import + +# fix the module name conflict for Python 3+ +import sys +from . import _types as types +modname = __name__ + '.types' +types.__name__ = modname +# update the __module__ for all objects, make them picklable +for v in types.__dict__.values(): + if hasattr(v, "__module__") and v.__module__.endswith('._types'): + v.__module__ = modname +sys.modules[modname] = types +del modname, sys -from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row -from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD +from pyspark.sql.context import SQLContext, HiveContext +from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD, DataFrameNaFunctions +from pyspark.sql.dataframe import DataFrameStatFunctions __all__ = [ 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', + 'DataFrameNaFunctions', 'DataFrameStatFunctions' ] diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/_types.py similarity index 86% rename from python/pyspark/sql/types.py rename to python/pyspark/sql/_types.py index 0169028ccc4eb..95fb91ad43457 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/_types.py @@ -15,7 +15,9 @@ # limitations under the License. # +import sys import decimal +import time import datetime import keyword import warnings @@ -25,6 +27,12 @@ from array import array from operator import itemgetter +if sys.version >= "3": + long = int + unicode = str + +from py4j.protocol import register_input_converter +from py4j.java_gateway import JavaClass __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", @@ -33,8 +41,7 @@ class DataType(object): - - """Spark SQL DataType""" + """Base class for data types.""" def __repr__(self): return self.__class__.__name__ @@ -67,7 +74,6 @@ def json(self): # This singleton pattern does not work with pickle, you will get # another object after pickle and unpickle class PrimitiveTypeSingleton(type): - """Metaclass for PrimitiveType""" _instances = {} @@ -79,66 +85,45 @@ def __call__(cls): class PrimitiveType(DataType): - """Spark SQL PrimitiveType""" __metaclass__ = PrimitiveTypeSingleton class NullType(PrimitiveType): + """Null type. - """Spark SQL NullType - - The data type representing None, used for the types which has not - been inferred. + The data type representing None, used for the types that cannot be inferred. """ class StringType(PrimitiveType): - - """Spark SQL StringType - - The data type representing string values. + """String data type. """ class BinaryType(PrimitiveType): - - """Spark SQL BinaryType - - The data type representing bytearray values. + """Binary (byte array) data type. """ class BooleanType(PrimitiveType): - - """Spark SQL BooleanType - - The data type representing bool values. + """Boolean data type. """ class DateType(PrimitiveType): - - """Spark SQL DateType - - The data type representing datetime.date values. + """Date (datetime.date) data type. """ class TimestampType(PrimitiveType): - - """Spark SQL TimestampType - - The data type representing datetime.datetime values. + """Timestamp (datetime.datetime) data type. """ class DecimalType(DataType): - - """Spark SQL DecimalType - - The data type representing decimal.Decimal values. + """Decimal (decimal.Decimal) data type. """ def __init__(self, precision=None, scale=None): @@ -166,80 +151,55 @@ def __repr__(self): class DoubleType(PrimitiveType): - - """Spark SQL DoubleType - - The data type representing float values. + """Double data type, representing double precision floats. """ class FloatType(PrimitiveType): - - """Spark SQL FloatType - - The data type representing single precision floating-point values. + """Float data type, representing single precision floats. """ class ByteType(PrimitiveType): - - """Spark SQL ByteType - - The data type representing int values with 1 singed byte. + """Byte data type, i.e. a signed integer in a single byte. """ def simpleString(self): return 'tinyint' class IntegerType(PrimitiveType): - - """Spark SQL IntegerType - - The data type representing int values. + """Int data type, i.e. a signed 32-bit integer. """ def simpleString(self): return 'int' class LongType(PrimitiveType): + """Long data type, i.e. a signed 64-bit integer. - """Spark SQL LongType - - The data type representing long values. If the any value is - beyond the range of [-9223372036854775808, 9223372036854775807], - please use DecimalType. + If the values are beyond the range of [-9223372036854775808, 9223372036854775807], + please use :class:`DecimalType`. """ def simpleString(self): return 'bigint' class ShortType(PrimitiveType): - - """Spark SQL ShortType - - The data type representing int values with 2 signed bytes. + """Short data type, i.e. a signed 16-bit integer. """ def simpleString(self): return 'smallint' class ArrayType(DataType): + """Array data type. - """Spark SQL ArrayType - - The data type representing list values. An ArrayType object - comprises two fields, elementType (a DataType) and containsNull (a bool). - The field of elementType is used to specify the type of array elements. - The field of containsNull is used to specify if the array has None values. - + :param elementType: :class:`DataType` of each element in the array. + :param containsNull: boolean, whether the array can contain null (None) values. """ def __init__(self, elementType, containsNull=True): - """Creates an ArrayType - - :param elementType: the data type of elements. - :param containsNull: indicates whether the list contains None values. - + """ >>> ArrayType(StringType()) == ArrayType(StringType(), True) True >>> ArrayType(StringType(), False) == ArrayType(StringType()) @@ -268,29 +228,17 @@ def fromJson(cls, json): class MapType(DataType): + """Map data type. - """Spark SQL MapType - - The data type representing dict values. A MapType object comprises - three fields, keyType (a DataType), valueType (a DataType) and - valueContainsNull (a bool). - - The field of keyType is used to specify the type of keys in the map. - The field of valueType is used to specify the type of values in the map. - The field of valueContainsNull is used to specify if values of this - map has None values. - - For values of a MapType column, keys are not allowed to have None values. + :param keyType: :class:`DataType` of the keys in the map. + :param valueType: :class:`DataType` of the values in the map. + :param valueContainsNull: indicates whether values can contain null (None) values. + Keys in a map data type are not allowed to be null (None). """ def __init__(self, keyType, valueType, valueContainsNull=True): - """Creates a MapType - :param keyType: the data type of keys. - :param valueType: the data type of values. - :param valueContainsNull: indicates whether values contains - null values. - + """ >>> (MapType(StringType(), IntegerType()) ... == MapType(StringType(), IntegerType(), True)) True @@ -325,30 +273,16 @@ def fromJson(cls, json): class StructField(DataType): + """A field in :class:`StructType`. - """Spark SQL StructField - - Represents a field in a StructType. - A StructField object comprises three fields, name (a string), - dataType (a DataType) and nullable (a bool). The field of name - is the name of a StructField. The field of dataType specifies - the data type of a StructField. - - The field of nullable specifies if values of a StructField can - contain None values. - + :param name: string, name of the field. + :param dataType: :class:`DataType` of the field. + :param nullable: boolean, whether the field can be null (None) or not. + :param metadata: a dict from string to simple type that can be serialized to JSON automatically """ def __init__(self, name, dataType, nullable=True, metadata=None): - """Creates a StructField - :param name: the name of this field. - :param dataType: the data type of this field. - :param nullable: indicates whether values of this field - can be null. - :param metadata: metadata of this field, which is a map from string - to simple type that can be serialized to JSON - automatically - + """ >>> (StructField("f1", StringType(), True) ... == StructField("f1", StringType(), True)) True @@ -384,17 +318,13 @@ def fromJson(cls, json): class StructType(DataType): + """Struct type, consisting of a list of :class:`StructField`. - """Spark SQL StructType - - The data type representing rows. - A StructType object comprises a list of L{StructField}. - + This is the data type representing a :class:`Row`. """ def __init__(self, fields): - """Creates a StructType - + """ >>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct2 = StructType([StructField("f1", StringType(), True)]) >>> struct1 == struct2 @@ -425,9 +355,9 @@ def fromJson(cls, json): class UserDefinedType(DataType): - """ + """User-defined type (UDT). + .. note:: WARN: Spark Internal Use Only - SQL User-Defined Type (UDT). """ @classmethod @@ -488,7 +418,7 @@ def fromJson(cls, json): split = pyUDT.rfind(".") pyModule = pyUDT[:split] pyClass = pyUDT[split+1:] - m = __import__(pyModule, globals(), locals(), [pyClass], -1) + m = __import__(pyModule, globals(), locals(), [pyClass]) UDT = getattr(m, pyClass) return UDT() @@ -497,10 +427,9 @@ def __eq__(self, other): _all_primitive_types = dict((v.typeName(), v) - for v in globals().itervalues() - if type(v) is PrimitiveTypeSingleton and - v.__base__ == PrimitiveType) - + for v in list(globals().values()) + if (type(v) is type or type(v) is PrimitiveTypeSingleton) + and v.__base__ == PrimitiveType) _all_complex_types = dict((v.typeName(), v) for v in [ArrayType, MapType, StructType]) @@ -512,7 +441,7 @@ def _parse_datatype_json_string(json_string): >>> def check_datatype(datatype): ... pickled = pickle.loads(pickle.dumps(datatype)) ... assert datatype == pickled - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) + ... scala_datatype = sqlContext._ssql_ctx.parseDataType(datatype.json()) ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) ... assert datatype == python_datatype >>> for cls in _all_primitive_types.values(): @@ -564,10 +493,10 @@ def _parse_datatype_json_string(json_string): def _parse_datatype_json_value(json_value): - if type(json_value) is unicode: + if not isinstance(json_value, dict): if json_value in _all_primitive_types.keys(): return _all_primitive_types[json_value]() - elif json_value == u'decimal': + elif json_value == 'decimal': return DecimalType() elif _FIXED_DECIMAL.match(json_value): m = _FIXED_DECIMAL.match(json_value) @@ -589,10 +518,8 @@ def _parse_datatype_json_value(json_value): type(None): NullType, bool: BooleanType, int: LongType, - long: LongType, float: DoubleType, str: StringType, - unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.date: DateType, @@ -600,6 +527,12 @@ def _parse_datatype_json_value(json_value): datetime.time: TimestampType, } +if sys.version < "3": + _type_mappings.update({ + unicode: StringType, + long: LongType, + }) + def _infer_type(obj): """Infer the DataType from obj @@ -619,7 +552,7 @@ def _infer_type(obj): return dataType() if isinstance(obj, dict): - for key, value in obj.iteritems(): + for key, value in obj.items(): if key is not None and value is not None: return MapType(_infer_type(key), _infer_type(value), True) else: @@ -633,8 +566,8 @@ def _infer_type(obj): else: try: return _infer_schema(obj) - except ValueError: - raise ValueError("not supported type: %s" % type(obj)) + except TypeError: + raise TypeError("not supported type: %s" % type(obj)) def _infer_schema(row): @@ -643,10 +576,10 @@ def _infer_schema(row): items = sorted(row.items()) elif isinstance(row, (tuple, list)): - if hasattr(row, "_fields"): # namedtuple + if hasattr(row, "__fields__"): # Row + items = zip(row.__fields__, tuple(row)) + elif hasattr(row, "_fields"): # namedtuple items = zip(row._fields, tuple(row)) - elif hasattr(row, "__FIELDS__"): # Row - items = zip(row.__FIELDS__, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -655,7 +588,7 @@ def _infer_schema(row): items = sorted(row.__dict__.items()) else: - raise ValueError("Can not infer schema for type: %s" % type(row)) + raise TypeError("Can not infer schema for type: %s" % type(row)) fields = [StructField(k, _infer_type(v), True) for k, v in items] return StructType(fields) @@ -725,7 +658,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): + if hasattr(obj, "__fields__") or hasattr(obj, "_fields"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -767,7 +700,7 @@ def _merge_type(a, b): return a elif type(a) is not type(b): # TODO: type cast (such as int -> long) - raise TypeError("Can not merge type %s and %s" % (a, b)) + raise TypeError("Can not merge type %s and %s" % (type(a), type(b))) # same type if isinstance(a, StructType): @@ -811,12 +744,12 @@ def _create_converter(dataType): if isinstance(dataType, ArrayType): conv = _create_converter(dataType.elementType) - return lambda row: map(conv, row) + return lambda row: [conv(v) for v in row] elif isinstance(dataType, MapType): kconv = _create_converter(dataType.keyType) vconv = _create_converter(dataType.valueType) - return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.items()) elif isinstance(dataType, NullType): return lambda x: None @@ -844,7 +777,7 @@ def convert_struct(obj): elif hasattr(obj, "__dict__"): # object d = obj.__dict__ else: - raise ValueError("Unexpected obj: %s" % obj) + raise TypeError("Unexpected obj type: %s" % type(obj)) if convert_fields: return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) @@ -959,7 +892,7 @@ def _infer_schema_type(obj, dataType): >>> _infer_schema_type(row, schema) StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ - if dataType is NullType(): + if isinstance(dataType, NullType): return _infer_type(obj) if not obj: @@ -970,7 +903,7 @@ def _infer_schema_type(obj, dataType): return ArrayType(eType, True) elif isinstance(dataType, MapType): - k, v = obj.iteritems().next() + k, v = next(iter(obj.items())) return MapType(_infer_schema_type(k, dataType.keyType), _infer_schema_type(v, dataType.valueType)) @@ -983,7 +916,7 @@ def _infer_schema_type(obj, dataType): return StructType(fields) else: - raise ValueError("Unexpected dataType: %s" % dataType) + raise TypeError("Unexpected dataType: %s" % type(dataType)) _acceptable_types = { @@ -1013,7 +946,7 @@ def _verify_type(obj, dataType): >>> _verify_type(None, StructType([])) >>> _verify_type("", StringType()) >>> _verify_type(0, LongType()) - >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(list(range(3)), ArrayType(ShortType())) >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... @@ -1054,7 +987,7 @@ def _verify_type(obj, dataType): _verify_type(i, dataType.elementType) elif isinstance(dataType, MapType): - for k, v in obj.iteritems(): + for k, v in obj.items(): _verify_type(k, dataType.keyType) _verify_type(v, dataType.valueType) @@ -1075,12 +1008,13 @@ def _restore_object(dataType, obj): # same object in most cases. k = id(dataType) cls = _cached_cls.get(k) - if cls is None: + if cls is None or cls.__datatype is not dataType: # use dataType as key to avoid create multiple class cls = _cached_cls.get(dataType) if cls is None: cls = _create_cls(dataType) _cached_cls[dataType] = cls + cls.__datatype = dataType _cached_cls[k] = cls return cls(obj) @@ -1197,8 +1131,8 @@ def Dict(d): class Row(tuple): """ Row in DataFrame """ - __DATATYPE__ = dataType - __FIELDS__ = tuple(f.name for f in dataType.fields) + __datatype = dataType + __fields__ = tuple(f.name for f in dataType.fields) __slots__ = () # create property for fast access @@ -1206,22 +1140,22 @@ class Row(tuple): def asDict(self): """ Return as a dict """ - return dict((n, getattr(self, n)) for n in self.__FIELDS__) + return dict((n, getattr(self, n)) for n in self.__fields__) def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) - for n in self.__FIELDS__)) + for n in self.__fields__)) def __reduce__(self): - return (_restore_object, (self.__DATATYPE__, tuple(self))) + return (_restore_object, (self.__datatype, tuple(self))) return Row def _create_row(fields, values): row = Row(*values) - row.__FIELDS__ = fields + row.__fields__ = fields return row @@ -1261,7 +1195,7 @@ def __new__(self, *args, **kwargs): # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) - row.__FIELDS__ = names + row.__fields__ = names return row else: @@ -1271,11 +1205,11 @@ def asDict(self): """ Return as an dict """ - if not hasattr(self, "__FIELDS__"): + if not hasattr(self, "__fields__"): raise TypeError("Cannot convert a Row class into dict") - return dict(zip(self.__FIELDS__, self)) + return dict(zip(self.__fields__, self)) - # let obect acs like class + # let object acts like class def __call__(self, *args): """create new Row object""" return _create_row(self, args) @@ -1286,25 +1220,50 @@ def __getattr__(self, item): try: # it will be slow when it has many fields, # but this will not be used in normal cases - idx = self.__FIELDS__.index(item) + idx = self.__fields__.index(item) return self[idx] except IndexError: raise AttributeError(item) + except ValueError: + raise AttributeError(item) def __reduce__(self): - if hasattr(self, "__FIELDS__"): - return (_create_row, (self.__FIELDS__, tuple(self))) + if hasattr(self, "__fields__"): + return (_create_row, (self.__fields__, tuple(self))) else: return tuple.__reduce__(self) def __repr__(self): - if hasattr(self, "__FIELDS__"): + if hasattr(self, "__fields__"): return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) + for k, v in zip(self.__fields__, tuple(self))) else: return "" % ", ".join(self) +class DateConverter(object): + def can_convert(self, obj): + return isinstance(obj, datetime.date) + + def convert(self, obj, gateway_client): + Date = JavaClass("java.sql.Date", gateway_client) + return Date.valueOf(obj.strftime("%Y-%m-%d")) + + +class DatetimeConverter(object): + def can_convert(self, obj): + return isinstance(obj, datetime.datetime) + + def convert(self, obj, gateway_client): + Timestamp = JavaClass("java.sql.Timestamp", gateway_client) + return Timestamp(int(time.mktime(obj.timetuple())) * 1000 + obj.microsecond // 1000) + + +# datetime is a subclass of date, we should register DatetimeConverter first +register_input_converter(DatetimeConverter()) +register_input_converter(DateConverter()) + + def _test(): import doctest from pyspark.context import SparkContext @@ -1315,7 +1274,7 @@ def _test(): globs = pyspark.sql.types.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['ExamplePoint'] = ExamplePoint globs['ExamplePointUDT'] = ExamplePointUDT (failure_count, test_count) = doctest.testmod( diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 795ef0dbc4c47..f6f107ca32d2f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -15,14 +15,18 @@ # limitations under the License. # +import sys import warnings import json -from itertools import imap + +if sys.version >= '3': + basestring = unicode = str +else: + from itertools import imap as map from py4j.protocol import Py4JError -from py4j.java_collections import MapConverter -from pyspark.rdd import RDD, _prepare_for_python_RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.sql.types import Row, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter @@ -34,15 +38,15 @@ except ImportError: has_pandas = False -__all__ = ["SQLContext", "HiveContext"] +__all__ = ["SQLContext", "HiveContext", "UDFRegistration"] -def _monkey_patch_RDD(sqlCtx): +def _monkey_patch_RDD(sqlContext): def toDF(self, schema=None, sampleRatio=None): """ - Convert current :class:`RDD` into a :class:`DataFrame` + Converts current :class:`RDD` into a :class:`DataFrame` - This is a shorthand for `sqlCtx.createDataFrame(rdd, schema, sampleRatio)` + This is a shorthand for ``sqlContext.createDataFrame(rdd, schema, sampleRatio)`` :param schema: a StructType or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring @@ -51,43 +55,38 @@ def toDF(self, schema=None, sampleRatio=None): >>> rdd.toDF().collect() [Row(name=u'Alice', age=1)] """ - return sqlCtx.createDataFrame(self, schema, sampleRatio) + return sqlContext.createDataFrame(self, schema, sampleRatio) RDD.toDF = toDF class SQLContext(object): - """Main entry point for Spark SQL functionality. - A SQLContext can be used create L{DataFrame}, register L{DataFrame} as + A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. + + :param sparkContext: The :class:`SparkContext` backing this SQLContext. + :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instantiate a new + SQLContext in the JVM, instead we make all calls to this object. """ + @ignore_unicode_prefix def __init__(self, sparkContext, sqlContext=None): - """Create a new SQLContext. - - It will add a method called `toDF` to :class:`RDD`, which could be - used to convert an RDD into a DataFrame, it's a shorthand for - :func:`SQLContext.createDataFrame`. - - :param sparkContext: The SparkContext to wrap. - :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new - SQLContext in the JVM, instead we make all calls to this object. + """Creates a new SQLContext. >>> from datetime import datetime - >>> sqlCtx = SQLContext(sc) - >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + >>> sqlContext = SQLContext(sc) + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") - >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() - [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] - >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, - ... x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] + >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc @@ -118,6 +117,12 @@ def getConf(self, key, defaultValue): """ return self._ssql_ctx.getConf(key, defaultValue) + @property + def udf(self): + """Returns a :class:`UDFRegistration` for UDF registration.""" + return UDFRegistration(self) + + @ignore_unicode_prefix def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -125,16 +130,25 @@ def registerFunction(self, name, f, returnType=StringType()): When the return type is not given it default to a string and conversion will automatically be done. For any other return type, the produced object must match the specified type. - >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) - >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + :param name: name of the UDF + :param samplingRatio: lambda function + :param returnType: a :class:`DataType` object + + >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlContext.sql("SELECT stringLengthString('test')").collect() [Row(c0=u'4')] >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + >>> sqlContext.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + + >>> from pyspark.sql.types import IntegerType + >>> sqlContext.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) @@ -172,132 +186,73 @@ def _inferSchema(self, rdd, samplingRatio=None): schema = rdd.map(_infer_schema).reduce(_merge_type) return schema + @ignore_unicode_prefix def inferSchema(self, rdd, samplingRatio=None): - """Infer and apply a schema to an RDD of L{Row}. - - ::note: - Deprecated in 1.3, use :func:`createDataFrame` instead - - When samplingRatio is specified, the schema is inferred by looking - at the types of each row in the sampled dataset. Otherwise, the - first 100 rows of the RDD are inspected. Nested collections are - supported, which can include array, dict, list, Row, tuple, - namedtuple, or object. - - Each row could be L{pyspark.sql.Row} object or namedtuple or objects. - Using top level dicts is deprecated, as dict is used to represent Maps. - - If a single column has multiple distinct inferred types, it may cause - runtime exceptions. - - >>> rdd = sc.parallelize( - ... [Row(field1=1, field2="row1"), - ... Row(field1=2, field2="row2"), - ... Row(field1=3, field2="row3")]) - >>> df = sqlCtx.inferSchema(rdd) - >>> df.collect()[0] - Row(field1=1, field2=u'row1') + """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ + warnings.warn("inferSchema is deprecated, please use createDataFrame instead") if isinstance(rdd, DataFrame): raise TypeError("Cannot apply schema to DataFrame") - schema = self._inferSchema(rdd, samplingRatio) - converter = _create_converter(schema) - rdd = rdd.map(converter) - return self.applySchema(rdd, schema) + return self.createDataFrame(rdd, None, samplingRatio) + @ignore_unicode_prefix def applySchema(self, rdd, schema): + """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ - Applies the given schema to the given RDD of L{tuple} or L{list}. - - ::note: - Deprecated in 1.3, use :func:`createDataFrame` instead - - These tuples or lists can contain complex nested structures like - lists, maps or nested rows. - - The schema should be a StructType. - - It is important that the schema matches the types of the objects - in each row or exceptions could be thrown at runtime. - - >>> from pyspark.sql.types import * - >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) - >>> schema = StructType([StructField("field1", IntegerType(), False), - ... StructField("field2", StringType(), False)]) - >>> df = sqlCtx.applySchema(rdd2, schema) - >>> df.collect() - [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] - """ + warnings.warn("applySchema is deprecated, please use createDataFrame instead") if isinstance(rdd, DataFrame): raise TypeError("Cannot apply schema to DataFrame") if not isinstance(schema, StructType): - raise TypeError("schema should be StructType, but got %s" % schema) - - # take the first few rows to verify schema - rows = rdd.take(10) - # Row() cannot been deserialized by Pyrolite - if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': - rdd = rdd.map(tuple) - rows = rdd.take(10) + raise TypeError("schema should be StructType, but got %s" % type(schema)) - for row in rows: - _verify_type(row, schema) - - # convert python objects to sql data - converter = _python_to_sql_converter(schema) - rdd = rdd.map(converter) - - jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) - df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) - return DataFrame(df, self) + return self.createDataFrame(rdd, schema) + @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None): """ - Create a DataFrame from an RDD of tuple/list, list or pandas.DataFrame. - - `schema` could be :class:`StructType` or a list of column names. + Creates a :class:`DataFrame` from an :class:`RDD` of :class:`tuple`/:class:`list`, + list or :class:`pandas.DataFrame`. - When `schema` is a list of column names, the type of each column - will be inferred from `rdd`. + When ``schema`` is a list of column names, the type of each column + will be inferred from ``data``. - When `schema` is None, it will try to infer the column name and type - from `rdd`, which should be an RDD of :class:`Row`, or namedtuple, - or dict. + When ``schema`` is ``None``, it will try to infer the schema (column names and types) + from ``data``, which should be an RDD of :class:`Row`, + or :class:`namedtuple`, or :class:`dict`. - If referring needed, `samplingRatio` is used to determined how many - rows will be used to do referring. The first row will be used if - `samplingRatio` is None. + If schema inference is needed, ``samplingRatio`` is used to determined the ratio of + rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. - :param data: an RDD of Row/tuple/list/dict, list, or pandas.DataFrame - :param schema: a StructType or list of names of columns + :param data: an RDD of :class:`Row`/:class:`tuple`/:class:`list`/:class:`dict`, + :class:`list`, or :class:`pandas.DataFrame`. + :param schema: a :class:`StructType` or list of column names. default None. :param samplingRatio: the sample ratio of rows used for inferring - :return: a DataFrame >>> l = [('Alice', 1)] - >>> sqlCtx.createDataFrame(l).collect() + >>> sqlContext.createDataFrame(l).collect() [Row(_1=u'Alice', _2=1)] - >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect() + >>> sqlContext.createDataFrame(l, ['name', 'age']).collect() [Row(name=u'Alice', age=1)] >>> d = [{'name': 'Alice', 'age': 1}] - >>> sqlCtx.createDataFrame(d).collect() + >>> sqlContext.createDataFrame(d).collect() [Row(age=1, name=u'Alice')] >>> rdd = sc.parallelize(l) - >>> sqlCtx.createDataFrame(rdd).collect() + >>> sqlContext.createDataFrame(rdd).collect() [Row(_1=u'Alice', _2=1)] - >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) + >>> df = sqlContext.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name=u'Alice', age=1)] >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) - >>> df2 = sqlCtx.createDataFrame(person) + >>> df2 = sqlContext.createDataFrame(person) >>> df2.collect() [Row(name=u'Alice', age=1)] @@ -305,11 +260,11 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): >>> schema = StructType([ ... StructField("name", StringType(), True), ... StructField("age", IntegerType(), True)]) - >>> df3 = sqlCtx.createDataFrame(rdd, schema) + >>> df3 = sqlContext.createDataFrame(rdd, schema) >>> df3.collect() [Row(name=u'Alice', age=1)] - >>> sqlCtx.createDataFrame(df.toPandas()).collect() # doctest: +SKIP + >>> sqlContext.createDataFrame(df.toPandas()).collect() # doctest: +SKIP [Row(name=u'Alice', age=1)] """ if isinstance(data, DataFrame): @@ -323,45 +278,63 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): if not isinstance(data, RDD): try: # data could be list, tuple, generator ... - data = self._sc.parallelize(data) + rdd = self._sc.parallelize(data) except Exception: - raise ValueError("cannot create an RDD from type: %s" % type(data)) + raise TypeError("cannot create an RDD from type: %s" % type(data)) + else: + rdd = data if schema is None: - return self.inferSchema(data, samplingRatio) + schema = self._inferSchema(rdd, samplingRatio) + converter = _create_converter(schema) + rdd = rdd.map(converter) if isinstance(schema, (list, tuple)): - first = data.first() + first = rdd.first() if not isinstance(first, (list, tuple)): - raise ValueError("each row in `rdd` should be list or tuple, " - "but got %r" % type(first)) + raise TypeError("each row in `rdd` should be list or tuple, " + "but got %r" % type(first)) row_cls = Row(*schema) - schema = self._inferSchema(data.map(lambda r: row_cls(*r)), samplingRatio) + schema = self._inferSchema(rdd.map(lambda r: row_cls(*r)), samplingRatio) + + # take the first few rows to verify schema + rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + + for row in rows: + _verify_type(row, schema) + + # convert python objects to sql data + converter = _python_to_sql_converter(schema) + rdd = rdd.map(converter) - return self.applySchema(data, schema) + jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) + df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) + return DataFrame(df, self) - def registerDataFrameAsTable(self, rdd, tableName): - """Registers the given RDD as a temporary table in the catalog. + def registerDataFrameAsTable(self, df, tableName): + """Registers the given :class:`DataFrame` as a temporary table in the catalog. - Temporary tables exist only during the lifetime of this instance of - SQLContext. + Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") """ - if (rdd.__class__ is DataFrame): - df = rdd._jdf - self._ssql_ctx.registerDataFrameAsTable(df, tableName) + if (df.__class__ is DataFrame): + self._ssql_ctx.registerDataFrameAsTable(df._jdf, tableName) else: raise ValueError("Can only register DataFrame as table") def parquetFile(self, *paths): - """Loads a Parquet file, returning the result as a L{DataFrame}. + """Loads a Parquet file, returning the result as a :class:`DataFrame`. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -373,22 +346,17 @@ def parquetFile(self, *paths): return DataFrame(jdf, self) def jsonFile(self, path, schema=None, samplingRatio=1.0): - """ - Loads a text file storing one JSON object per line as a - L{DataFrame}. - - If the schema is provided, applies the given schema to this - JSON dataset. + """Loads a text file storing one JSON object per line as a :class:`DataFrame`. - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() >>> shutil.rmtree(jsonFile) >>> with open(jsonFile, 'w') as f: ... f.writelines(jsonStrings) - >>> df1 = sqlCtx.jsonFile(jsonFile) + >>> df1 = sqlContext.jsonFile(jsonFile) >>> df1.printSchema() root |-- field1: long (nullable = true) @@ -401,7 +369,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): ... StructField("field2", StringType()), ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))]))]) - >>> df2 = sqlCtx.jsonFile(jsonFile, schema) + >>> df2 = sqlContext.jsonFile(jsonFile, schema) >>> df2.printSchema() root |-- field2: string (nullable = true) @@ -416,20 +384,18 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): df = self._ssql_ctx.jsonFile(path, scala_datatype) return DataFrame(df, self) + @ignore_unicode_prefix def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): - """Loads an RDD storing one JSON object per string as a L{DataFrame}. + """Loads an RDD storing one JSON object per string as a :class:`DataFrame`. - If the schema is provided, applies the given schema to this - JSON dataset. + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. - Otherwise, it samples the dataset with ratio `samplingRatio` to - determine the schema. - - >>> df1 = sqlCtx.jsonRDD(json) + >>> df1 = sqlContext.jsonRDD(json) >>> df1.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) - >>> df2 = sqlCtx.jsonRDD(json, df1.schema) + >>> df2 = sqlContext.jsonRDD(json, df1.schema) >>> df2.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) @@ -439,10 +405,9 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))])) ... ]) - >>> df3 = sqlCtx.jsonRDD(json, schema) + >>> df3 = sqlContext.jsonRDD(json, schema) >>> df3.first() Row(field2=u'row1', field3=Row(field5=None)) - """ def func(iterator): @@ -463,11 +428,11 @@ def func(iterator): return DataFrame(df, self) def load(self, path=None, source=None, schema=None, **options): - """Returns the dataset in a data source as a DataFrame. + """Returns the dataset in a data source as a :class:`DataFrame`. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Optionally, a schema can be provided as the schema of the returned DataFrame. """ @@ -476,15 +441,13 @@ def load(self, path=None, source=None, schema=None, **options): if source is None: source = self.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) if schema is None: - df = self._ssql_ctx.load(source, joptions) + df = self._ssql_ctx.load(source, options) else: if not isinstance(schema, StructType): raise TypeError("schema should be StructType") scala_datatype = self._ssql_ctx.parseDataType(schema.json()) - df = self._ssql_ctx.load(source, scala_datatype, joptions) + df = self._ssql_ctx.load(source, scala_datatype, options) return DataFrame(df, self) def createExternalTable(self, tableName, path=None, source=None, @@ -493,11 +456,11 @@ def createExternalTable(self, tableName, path=None, source=None, It returns the DataFrame associated with the external table. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. - Optionally, a schema can be provided as the schema of the returned DataFrame and + Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and created external table. """ if path is not None: @@ -505,48 +468,48 @@ def createExternalTable(self, tableName, path=None, source=None, if source is None: source = self.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) if schema is None: - df = self._ssql_ctx.createExternalTable(tableName, source, joptions) + df = self._ssql_ctx.createExternalTable(tableName, source, options) else: if not isinstance(schema, StructType): raise TypeError("schema should be StructType") scala_datatype = self._ssql_ctx.parseDataType(schema.json()) df = self._ssql_ctx.createExternalTable(tableName, source, scala_datatype, - joptions) + options) return DataFrame(df, self) + @ignore_unicode_prefix def sql(self, sqlQuery): - """Return a L{DataFrame} representing the result of the given query. + """Returns a :class:`DataFrame` representing the result of the given query. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ return DataFrame(self._ssql_ctx.sql(sqlQuery), self) def table(self, tableName): - """Returns the specified table as a L{DataFrame}. + """Returns the specified table as a :class:`DataFrame`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.table("table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) True """ return DataFrame(self._ssql_ctx.table(tableName), self) + @ignore_unicode_prefix def tables(self, dbName=None): - """Returns a DataFrame containing names of tables in the given database. + """Returns a :class:`DataFrame` containing names of tables in the given database. - If `dbName` is not specified, the current database will be used. + If ``dbName`` is not specified, the current database will be used. - The returned DataFrame has two columns, tableName and isTemporary - (a column with BooleanType indicating if a table is a temporary one or not). + The returned DataFrame has two columns: ``tableName`` and ``isTemporary`` + (a column with :class:`BooleanType` indicating if a table is a temporary one or not). - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.tables() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.tables() >>> df2.filter("tableName = 'table1'").first() Row(tableName=u'table1', isTemporary=True) """ @@ -556,14 +519,14 @@ def tables(self, dbName=None): return DataFrame(self._ssql_ctx.tables(dbName), self) def tableNames(self, dbName=None): - """Returns a list of names of tables in the database `dbName`. + """Returns a list of names of tables in the database ``dbName``. - If `dbName` is not specified, the current database will be used. + If ``dbName`` is not specified, the current database will be used. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> "table1" in sqlCtx.tableNames() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> "table1" in sqlContext.tableNames() True - >>> "table1" in sqlCtx.tableNames("db") + >>> "table1" in sqlContext.tableNames("db") True """ if dbName is None: @@ -585,22 +548,18 @@ def clearCache(self): class HiveContext(SQLContext): - """A variant of Spark SQL that integrates with data stored in Hive. - Configuration for Hive is read from hive-site.xml on the classpath. + Configuration for Hive is read from ``hive-site.xml`` on the classpath. It supports running both SQL and HiveQL commands. + + :param sparkContext: The SparkContext to wrap. + :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new + :class:`HiveContext` in the JVM, instead we make all calls to this object. """ def __init__(self, sparkContext, hiveContext=None): - """Create a new HiveContext. - - :param sparkContext: The SparkContext to wrap. - :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new - HiveContext in the JVM, instead we make all calls to this object. - """ SQLContext.__init__(self, sparkContext) - if hiveContext: self._scala_HiveContext = hiveContext @@ -618,6 +577,27 @@ def _ssql_ctx(self): def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) + def refreshTable(self, tableName): + """Invalidate and refresh all the cached the metadata of the given + table. For performance reasons, Spark SQL or the external data source + library it uses might cache certain metadata about a table, such as the + location of blocks. When those change outside of Spark SQL, users should + call this function to invalidate the cache. + """ + self._ssql_ctx.refreshTable(tableName) + + +class UDFRegistration(object): + """Wrapper for user-defined function registration.""" + + def __init__(self, sqlContext): + self.sqlContext = sqlContext + + def register(self, name, f, returnType=StringType()): + return self.sqlContext.registerFunction(name, f, returnType) + + register.__doc__ = SQLContext.registerFunction.__doc__ + def _test(): import doctest @@ -627,13 +607,12 @@ def _test(): globs = pyspark.sql.context.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['rdd'] = rdd = sc.parallelize( [Row(field1=1, field2="row1"), Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) - _monkey_patch_RDD(sqlCtx) globs['df'] = rdd.toDF() jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 5cb89da7a8ed5..f30a92dfc8534 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -16,14 +16,17 @@ # import sys -import itertools import warnings import random -from py4j.java_collections import ListConverter, MapConverter +if sys.version >= '3': + basestring = unicode = str + long = int +else: + from itertools import imap as map from pyspark.context import SparkContext -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -31,12 +34,12 @@ from pyspark.sql.types import _create_cls, _parse_datatype_json_string -__all__ = ["DataFrame", "GroupedData", "Column", "SchemaRDD"] +__all__ = ["DataFrame", "GroupedData", "Column", "SchemaRDD", "DataFrameNaFunctions", + "DataFrameStatFunctions"] class DataFrame(object): - - """A collection of rows that have the same columns. + """A distributed collection of data grouped into named columns. A :class:`DataFrame` is equivalent to a relational table in Spark SQL, and can be created using various functions in :class:`SQLContext`:: @@ -50,13 +53,6 @@ class DataFrame(object): ageCol = people.age - Note that the :class:`Column` type can also be manipulated - through its various functions:: - - # The following creates a new column that increases everybody's age by 10. - people.age + 10 - - A more concrete example:: # To create DataFrame using SQLContext @@ -73,46 +69,60 @@ def __init__(self, jdf, sql_ctx): self._sc = sql_ctx and sql_ctx._sc self.is_cached = False self._schema = None # initialized lazily + self._lazy_rdd = None @property def rdd(self): + """Returns the content as an :class:`pyspark.RDD` of :class:`Row`. """ - Return the content of the :class:`DataFrame` as an :class:`RDD` - of :class:`Row` s. - """ - if not hasattr(self, '_lazy_rdd'): + if self._lazy_rdd is None: jrdd = self._jdf.javaToPython() rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) schema = self.schema def applySchema(it): cls = _create_cls(schema) - return itertools.imap(cls, it) + return map(cls, it) self._lazy_rdd = rdd.mapPartitions(applySchema) return self._lazy_rdd - def toJSON(self, use_unicode=False): - """Convert a :class:`DataFrame` into a MappedRDD of JSON documents; one document per row. + @property + def na(self): + """Returns a :class:`DataFrameNaFunctions` for handling missing values. + """ + return DataFrameNaFunctions(self) + + @property + def stat(self): + """Returns a :class:`DataFrameStatFunctions` for statistic functions. + """ + return DataFrameStatFunctions(self) + + @ignore_unicode_prefix + def toJSON(self, use_unicode=True): + """Converts a :class:`DataFrame` into a :class:`RDD` of string. + + Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() - '{"age":2,"name":"Alice"}' + u'{"age":2,"name":"Alice"}' """ rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) def saveAsParquetFile(self, path): - """Save the contents as a Parquet file, preserving the schema. + """Saves the contents as a Parquet file, preserving the schema. Files that are written out using this method can be read back in as - a :class:`DataFrame` using the L{SQLContext.parquetFile} method. + a :class:`DataFrame` using :func:`SQLContext.parquetFile`. >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df2.collect()) == sorted(df.collect()) True """ @@ -121,18 +131,18 @@ def saveAsParquetFile(self, path): def registerTempTable(self, name): """Registers this RDD as a temporary table using the given name. - The lifetime of this temporary table is tied to the L{SQLContext} - that was used to create this DataFrame. + The lifetime of this temporary table is tied to the :class:`SQLContext` + that was used to create this :class:`DataFrame`. >>> df.registerTempTable("people") - >>> df2 = sqlCtx.sql("select * from people") + >>> df2 = sqlContext.sql("select * from people") >>> sorted(df.collect()) == sorted(df2.collect()) True """ self._jdf.registerTempTable(name) def registerAsTable(self, name): - """DEPRECATED: use registerTempTable() instead""" + """DEPRECATED: use :func:`registerTempTable` instead""" warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) self.registerTempTable(name) @@ -163,46 +173,40 @@ def _java_save_mode(self, mode): return jmode def saveAsTable(self, tableName, source=None, mode="error", **options): - """Saves the contents of the :class:`DataFrame` to a data source as a table. + """Saves the contents of this :class:`DataFrame` to a data source as a table. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Additionally, mode is used to specify the behavior of the saveAsTable operation when table already exists in the data source. There are four modes: - * append: Contents of this :class:`DataFrame` are expected to be appended \ - to existing table. - * overwrite: Data in the existing table is expected to be overwritten by \ - the contents of this DataFrame. - * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of the \ - :class:`DataFrame` and to not change the existing table. + * `append`: Append contents of this :class:`DataFrame` to existing data. + * `overwrite`: Overwrite existing data. + * `error`: Throw an exception if data already exists. + * `ignore`: Silently ignore this operation if data already exists. """ if source is None: source = self.sql_ctx.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") jmode = self._java_save_mode(mode) - joptions = MapConverter().convert(options, - self.sql_ctx._sc._gateway._gateway_client) - self._jdf.saveAsTable(tableName, source, jmode, joptions) + self._jdf.saveAsTable(tableName, source, jmode, options) def save(self, path=None, source=None, mode="error", **options): """Saves the contents of the :class:`DataFrame` to a data source. - The data source is specified by the `source` and a set of `options`. - If `source` is not specified, the default data source configured by - spark.sql.sources.default will be used. + The data source is specified by the ``source`` and a set of ``options``. + If ``source`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. Additionally, mode is used to specify the behavior of the save operation when data already exists in the data source. There are four modes: - * append: Contents of this :class:`DataFrame` are expected to be appended to existing data. - * overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. - * error: An exception is expected to be thrown. - * ignore: The save operation is expected to not save the contents of \ - the :class:`DataFrame` and to not change the existing data. + * `append`: Append contents of this :class:`DataFrame` to existing data. + * `overwrite`: Overwrite existing data. + * `error`: Throw an exception if data already exists. + * `ignore`: Silently ignore this operation if data already exists. """ if path is not None: options["path"] = path @@ -210,14 +214,11 @@ def save(self, path=None, source=None, mode="error", **options): source = self.sql_ctx.getConf("spark.sql.sources.default", "org.apache.spark.sql.parquet") jmode = self._java_save_mode(mode) - joptions = MapConverter().convert(options, - self._sc._gateway._gateway_client) - self._jdf.save(source, jmode, joptions) + self._jdf.save(source, jmode, options) @property def schema(self): - """Returns the schema of this :class:`DataFrame` (represented by - a L{StructType}). + """Returns the schema of this :class:`DataFrame` as a :class:`types.StructType`. >>> df.schema StructType(List(StructField(age,IntegerType,true),StructField(name,StringType,true))) @@ -235,17 +236,16 @@ def printSchema(self): |-- name: string (nullable = true) """ - print (self._jdf.schema().treeString()) + print(self._jdf.schema().treeString()) def explain(self, extended=False): - """ - Prints the plans (logical and physical) to the console for - debugging purpose. + """Prints the (logical and physical) plans to the console for debugging purpose. - If extended is False, only prints the physical plan. + :param extended: boolean, default ``False``. If ``False``, prints only the physical plan. >>> df.explain() - PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at mapPartitions at SQLContext.scala:... + PhysicalRDD [age#0,name#1], MapPartitionsRDD[...] at applySchemaToPythonRDD at\ + NativeMethodAccessorImpl.java:... >>> df.explain(True) == Parsed Logical Plan == @@ -259,50 +259,45 @@ def explain(self, extended=False): == RDD == """ if extended: - print self._jdf.queryExecution().toString() + print(self._jdf.queryExecution().toString()) else: - print self._jdf.queryExecution().executedPlan().toString() + print(self._jdf.queryExecution().executedPlan().toString()) def isLocal(self): - """ - Returns True if the `collect` and `take` methods can be run locally + """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally (without any Spark executors). """ return self._jdf.isLocal() def show(self, n=20): - """ - Print the first n rows. + """Prints the first ``n`` rows to the console. >>> df DataFrame[age: int, name: string] >>> df.show() - age name - 2 Alice - 5 Bob + +---+-----+ + |age| name| + +---+-----+ + | 2|Alice| + | 5| Bob| + +---+-----+ """ - print self._jdf.showString(n).encode('utf8', 'ignore') + print(self._jdf.showString(n)) def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) def count(self): - """Return the number of elements in this RDD. - - Unlike the base RDD implementation of count, this implementation - leverages the query optimizer to compute the count on the DataFrame, - which supports features such as filter pushdown. + """Returns the number of rows in this :class:`DataFrame`. >>> df.count() - 2L + 2 """ - return self._jdf.count() + return int(self._jdf.count()) + @ignore_unicode_prefix def collect(self): - """Return a list that contains all of the rows. - - Each object in the list is a Row, the fields can be accessed as - attributes. + """Returns all the records as a list of :class:`Row`. >>> df.collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -313,8 +308,9 @@ def collect(self): cls = _create_cls(self.schema) return [cls(r) for r in rs] + @ignore_unicode_prefix def limit(self, num): - """Limit the result count to the number specified. + """Limits the result count to the number specified. >>> df.limit(1).collect() [Row(age=2, name=u'Alice')] @@ -324,32 +320,32 @@ def limit(self, num): jdf = self._jdf.limit(num) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def take(self, num): - """Take the first num rows of the RDD. - - Each object in the list is a Row, the fields can be accessed as - attributes. + """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. >>> df.take(2) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ return self.limit(num).collect() + @ignore_unicode_prefix def map(self, f): - """ Return a new RDD by applying a function to each Row + """ Returns a new :class:`RDD` by applying a the ``f`` function to each :class:`Row`. - It's a shorthand for df.rdd.map() + This is a shorthand for ``df.rdd.map()``. >>> df.map(lambda p: p.name).collect() [u'Alice', u'Bob'] """ return self.rdd.map(f) + @ignore_unicode_prefix def flatMap(self, f): - """ Return a new RDD by first applying a function to all elements of this, + """ Returns a new :class:`RDD` by first applying the ``f`` function to each :class:`Row`, and then flattening the results. - It's a shorthand for df.rdd.flatMap() + This is a shorthand for ``df.rdd.flatMap()``. >>> df.flatMap(lambda p: p.name).collect() [u'A', u'l', u'i', u'c', u'e', u'B', u'o', u'b'] @@ -357,10 +353,9 @@ def flatMap(self, f): return self.rdd.flatMap(f) def mapPartitions(self, f, preservesPartitioning=False): - """ - Return a new RDD by applying a function to each partition. + """Returns a new :class:`RDD` by applying the ``f`` function to each partition. - It's a shorthand for df.rdd.mapPartitions() + This is a shorthand for ``df.rdd.mapPartitions()``. >>> rdd = sc.parallelize([1, 2, 3, 4], 4) >>> def f(iterator): yield 1 @@ -370,39 +365,37 @@ def mapPartitions(self, f, preservesPartitioning=False): return self.rdd.mapPartitions(f, preservesPartitioning) def foreach(self, f): - """ - Applies a function to all rows of this DataFrame. + """Applies the ``f`` function to all :class:`Row` of this :class:`DataFrame`. - It's a shorthand for df.rdd.foreach() + This is a shorthand for ``df.rdd.foreach()``. >>> def f(person): - ... print person.name + ... print(person.name) >>> df.foreach(f) """ return self.rdd.foreach(f) def foreachPartition(self, f): - """ - Applies a function to each partition of this DataFrame. + """Applies the ``f`` function to each partition of this :class:`DataFrame`. - It's a shorthand for df.rdd.foreachPartition() + This a shorthand for ``df.rdd.foreachPartition()``. >>> def f(people): ... for person in people: - ... print person.name + ... print(person.name) >>> df.foreachPartition(f) """ return self.rdd.foreachPartition(f) def cache(self): - """ Persist with the default storage level (C{MEMORY_ONLY_SER}). + """ Persists with the default storage level (C{MEMORY_ONLY_SER}). """ self.is_cached = True self._jdf.cache() return self def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): - """ Set the storage level to persist its values across operations + """Sets the storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). @@ -413,7 +406,7 @@ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): return self def unpersist(self, blocking=True): - """ Mark it as non-persistent, and remove all blocks for it from + """Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from memory and disk. """ self.is_cached = False @@ -425,8 +418,7 @@ def unpersist(self, blocking=True): # return DataFrame(rdd, self.sql_ctx) def repartition(self, numPartitions): - """ Return a new :class:`DataFrame` that has exactly `numPartitions` - partitions. + """Returns a new :class:`DataFrame` that has exactly ``numPartitions`` partitions. >>> df.repartition(10).rdd.getNumPartitions() 10 @@ -434,29 +426,48 @@ def repartition(self, numPartitions): return DataFrame(self._jdf.repartition(numPartitions), self.sql_ctx) def distinct(self): - """ - Return a new :class:`DataFrame` containing the distinct rows in this DataFrame. + """Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`. >>> df.distinct().count() - 2L + 2 """ return DataFrame(self._jdf.distinct(), self.sql_ctx) def sample(self, withReplacement, fraction, seed=None): - """ - Return a sampled subset of this DataFrame. + """Returns a sampled subset of this :class:`DataFrame`. - >>> df.sample(False, 0.5, 97).count() - 1L + >>> df.sample(False, 0.5, 42).count() + 1 """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction - seed = seed if seed is not None else random.randint(0, sys.maxint) + seed = seed if seed is not None else random.randint(0, sys.maxsize) rdd = self._jdf.sample(withReplacement, fraction, long(seed)) return DataFrame(rdd, self.sql_ctx) + def randomSplit(self, weights, seed=None): + """Randomly splits this :class:`DataFrame` with the provided weights. + + :param weights: list of doubles as weights with which to split the DataFrame. Weights will + be normalized if they don't sum up to 1.0. + :param seed: The seed for sampling. + + >>> splits = df4.randomSplit([1.0, 2.0], 24) + >>> splits[0].count() + 1 + + >>> splits[1].count() + 3 + """ + for w in weights: + if w < 0.0: + raise ValueError("Weights must be positive. Found weight value: %s" % w) + seed = seed if seed is not None else random.randint(0, sys.maxsize) + rdd_array = self._jdf.randomSplit(_to_seq(self.sql_ctx._sc, weights), long(seed)) + return [DataFrame(rdd, self.sql_ctx) for rdd in rdd_array] + @property def dtypes(self): - """Return all column names and their data types as a list. + """Returns all column names and their data types as a list. >>> df.dtypes [('age', 'int'), ('name', 'string')] @@ -464,29 +475,53 @@ def dtypes(self): return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] @property + @ignore_unicode_prefix def columns(self): - """ Return all column names as a list. + """Returns all column names as a list. >>> df.columns [u'age', u'name'] """ return [f.name for f in self.schema.fields] - def join(self, other, joinExprs=None, joinType=None): + @ignore_unicode_prefix + def alias(self, alias): + """Returns a new :class:`DataFrame` with an alias set. + + >>> from pyspark.sql.functions import * + >>> df_as1 = df.alias("df_as1") + >>> df_as2 = df.alias("df_as2") + >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') + >>> joined_df.select(col("df_as1.name"), col("df_as2.name"), col("df_as2.age")).collect() + [Row(name=u'Alice', name=u'Alice', age=2), Row(name=u'Bob', name=u'Bob', age=5)] """ - Join with another :class:`DataFrame`, using the given join expression. - The following performs a full outer join between `df1` and `df2`. + assert isinstance(alias, basestring), "alias should be a string" + return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) + + @ignore_unicode_prefix + def join(self, other, joinExprs=None, joinType=None): + """Joins with another :class:`DataFrame`, using the given join expression. + + The following performs a full outer join between ``df1`` and ``df2``. :param other: Right side of the join - :param joinExprs: Join expression - :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. + :param joinExprs: a string for join column name, or a join expression (Column). + If joinExprs is a string indicating the name of the join column, + the column must exist on both sides, and this performs an inner equi-join. + :param joinType: str, default 'inner'. + One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + + >>> df.join(df2, 'name').select(df.name, df2.height).collect() + [Row(name=u'Bob', height=85)] """ if joinExprs is None: jdf = self._jdf.join(other._jdf) + elif isinstance(joinExprs, basestring): + jdf = self._jdf.join(other._jdf, joinExprs) else: assert isinstance(joinExprs, Column), "joinExprs should be Column" if joinType is None: @@ -496,13 +531,19 @@ def join(self, other, joinExprs=None, joinType=None): jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) return DataFrame(jdf, self.sql_ctx) - def sort(self, *cols): - """ Return a new :class:`DataFrame` sorted by the specified column(s). + @ignore_unicode_prefix + def sort(self, *cols, **kwargs): + """Returns a new :class:`DataFrame` sorted by the specified column(s). - :param cols: The columns or expressions used for sorting + :param cols: list of :class:`Column` or column names to sort by. + :param ascending: boolean or list of boolean (default True). + Sort ascending vs. descending. Specify list for multiple sort orders. + If a list is specified, length of the list must equal length of the `cols`. >>> df.sort(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.sort("age", ascending=False).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> df.orderBy(df.age.desc()).collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] >>> from pyspark.sql.functions import * @@ -510,18 +551,67 @@ def sort(self, *cols): [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] >>> df.orderBy(desc("age"), "name").collect() [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] + >>> df.orderBy(["age", "name"], ascending=[0, 1]).collect() + [Row(age=5, name=u'Bob'), Row(age=2, name=u'Alice')] """ if not cols: raise ValueError("should sort by at least one column") - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + jcols = [_to_java_column(c) for c in cols] + ascending = kwargs.get('ascending', True) + if isinstance(ascending, (bool, int)): + if not ascending: + jcols = [jc.desc() for jc in jcols] + elif isinstance(ascending, list): + jcols = [jc if asc else jc.desc() + for asc, jc in zip(ascending, jcols)] + else: + raise TypeError("ascending can only be boolean or list, but got %s" % type(ascending)) + + jdf = self._jdf.sort(self._jseq(jcols)) return DataFrame(jdf, self.sql_ctx) orderBy = sort + def _jseq(self, cols, converter=None): + """Return a JVM Seq of Columns from a list of Column or names""" + return _to_seq(self.sql_ctx._sc, cols, converter) + + def _jcols(self, *cols): + """Return a JVM Seq of Columns from a list of Column or column names + + If `cols` has only one list in it, cols[0] will be used as the list. + """ + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + return self._jseq(cols, _to_java_column) + + def describe(self, *cols): + """Computes statistics for numeric columns. + + This include count, mean, stddev, min, and max. If no columns are + given, this function computes statistics for all numerical columns. + + >>> df.describe().show() + +-------+---+ + |summary|age| + +-------+---+ + | count| 2| + | mean|3.5| + | stddev|1.5| + | min| 2| + | max| 5| + +-------+---+ + """ + jdf = self._jdf.describe(self._jseq(cols)) + return DataFrame(jdf, self.sql_ctx) + + @ignore_unicode_prefix def head(self, n=None): - """ Return the first `n` rows or the first row if n is None. + """ + Returns the first ``n`` rows as a list of :class:`Row`, + or the first :class:`Row` if ``n`` is ``None.`` >>> df.head() Row(age=2, name=u'Alice') @@ -533,16 +623,18 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + @ignore_unicode_prefix def first(self): - """ Return the first row. + """Returns the first row as a :class:`Row`. >>> df.first() Row(age=2, name=u'Alice') """ return self.head() + @ignore_unicode_prefix def __getitem__(self, item): - """ Return the column by given name + """Returns the column as a :class:`Column`. >>> df.select(df['age']).collect() [Row(age=2), Row(age=5)] @@ -550,30 +642,43 @@ def __getitem__(self, item): [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df[ df.age > 3 ].collect() [Row(age=5, name=u'Bob')] + >>> df[df[0] > 3].collect() + [Row(age=5, name=u'Bob')] """ if isinstance(item, basestring): + if item not in self.columns: + raise IndexError("no such column: %s" % item) jc = self._jdf.apply(item) return Column(jc) elif isinstance(item, Column): return self.filter(item) - elif isinstance(item, list): + elif isinstance(item, (list, tuple)): return self.select(*item) + elif isinstance(item, int): + jc = self._jdf.apply(self.columns[item]) + return Column(jc) else: - raise IndexError("unexpected index: %s" % item) + raise TypeError("unexpected item type: %s" % type(item)) def __getattr__(self, name): - """ Return the column by given name + """Returns the :class:`Column` denoted by ``name``. >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] """ - if name.startswith("__"): - raise AttributeError(name) + if name not in self.columns: + raise AttributeError( + "'%s' object has no attribute '%s'" % (self.__class__.__name__, name)) jc = self._jdf.apply(name) return Column(jc) + @ignore_unicode_prefix def select(self, *cols): - """ Selecting a set of expressions. + """Projects a set of expressions and returns a new :class:`DataFrame`. + + :param cols: list of column names (string) or expressions (:class:`Column`). + If one of the column names is '*', that column is expanded to include all columns + in the current DataFrame. >>> df.select('*').collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -582,28 +687,30 @@ def select(self, *cols): >>> df.select(df.name, (df.age + 10).alias('age')).collect() [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.select(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.select(self._jcols(*cols)) return DataFrame(jdf, self.sql_ctx) def selectExpr(self, *expr): - """ - Selects a set of SQL expressions. This is a variant of - `select` that accepts SQL expressions. + """Projects a set of SQL expressions and returns a new :class:`DataFrame`. + + This is a variant of :func:`select` that accepts SQL expressions. >>> df.selectExpr("age * 2", "abs(age)").collect() [Row((age * 2)=4, Abs(age)=2), Row((age * 2)=10, Abs(age)=5)] """ - jexpr = ListConverter().convert(expr, self._sc._gateway._gateway_client) - jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) + if len(expr) == 1 and isinstance(expr[0], list): + expr = expr[0] + jdf = self._jdf.selectExpr(self._jseq(expr)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def filter(self, condition): - """ Filtering rows using the given condition, which could be - :class:`Column` expression or string of SQL expression. + """Filters rows using the given condition. - where() is an alias for filter(). + :func:`where` is an alias for :func:`filter`. + + :param condition: a :class:`Column` of :class:`types.BooleanType` + or a string of SQL expression. >>> df.filter(df.age > 3).collect() [Row(age=5, name=u'Bob')] @@ -625,26 +732,34 @@ def filter(self, condition): where = filter + @ignore_unicode_prefix def groupBy(self, *cols): - """ Group the :class:`DataFrame` using the specified columns, + """Groups the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedData` for all the available aggregate functions. + :func:`groupby` is an alias for :func:`groupBy`. + + :param cols: list of columns to group by. + Each element should be a column name (string) or an expression (:class:`Column`). + >>> df.groupBy().avg().collect() [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] + >>> df.groupBy(['name', df.age]).count().collect() + [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)] """ - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.groupBy(self._jcols(*cols)) return GroupedData(jdf, self.sql_ctx) + groupby = groupBy + def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups - (shorthand for df.groupBy.agg()). + (shorthand for ``df.groupBy.agg()``). >>> df.agg({"age": "max"}).collect() [Row(MAX(age)=5)] @@ -678,16 +793,202 @@ def subtract(self, other): """ return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) + def dropna(self, how='any', thresh=None, subset=None): + """Returns a new :class:`DataFrame` omitting rows with null values. + + This is an alias for ``na.drop()``. + + :param how: 'any' or 'all'. + If 'any', drop a row if it contains any nulls. + If 'all', drop a row only if all its values are null. + :param thresh: int, default None + If specified, drop rows that have less than `thresh` non-null values. + This overwrites the `how` parameter. + :param subset: optional list of column names to consider. + + >>> df4.dropna().show() + +---+------+-----+ + |age|height| name| + +---+------+-----+ + | 10| 80|Alice| + +---+------+-----+ + + >>> df4.na.drop().show() + +---+------+-----+ + |age|height| name| + +---+------+-----+ + | 10| 80|Alice| + +---+------+-----+ + """ + if how is not None and how not in ['any', 'all']: + raise ValueError("how ('" + how + "') should be 'any' or 'all'") + + if subset is None: + subset = self.columns + elif isinstance(subset, basestring): + subset = [subset] + elif not isinstance(subset, (list, tuple)): + raise ValueError("subset should be a list or tuple of column names") + + if thresh is None: + thresh = len(subset) if how == 'any' else 1 + + return DataFrame(self._jdf.na().drop(thresh, self._jseq(subset)), self.sql_ctx) + + def fillna(self, value, subset=None): + """Replace null values, alias for ``na.fill()``. + + :param value: int, long, float, string, or dict. + Value to replace null values with. + If the value is a dict, then `subset` is ignored and `value` must be a mapping + from column name (string) to replacement value. The replacement value must be + an int, long, float, or string. + :param subset: optional list of column names to consider. + Columns specified in subset that do not have matching data type are ignored. + For example, if `value` is a string, and subset contains a non-string column, + then the non-string column is simply ignored. + + >>> df4.fillna(50).show() + +---+------+-----+ + |age|height| name| + +---+------+-----+ + | 10| 80|Alice| + | 5| 50| Bob| + | 50| 50| Tom| + | 50| 50| null| + +---+------+-----+ + + >>> df4.fillna({'age': 50, 'name': 'unknown'}).show() + +---+------+-------+ + |age|height| name| + +---+------+-------+ + | 10| 80| Alice| + | 5| null| Bob| + | 50| null| Tom| + | 50| null|unknown| + +---+------+-------+ + + >>> df4.na.fill({'age': 50, 'name': 'unknown'}).show() + +---+------+-------+ + |age|height| name| + +---+------+-------+ + | 10| 80| Alice| + | 5| null| Bob| + | 50| null| Tom| + | 50| null|unknown| + +---+------+-------+ + """ + if not isinstance(value, (float, int, long, basestring, dict)): + raise ValueError("value should be a float, int, long, string, or dict") + + if isinstance(value, (int, long)): + value = float(value) + + if isinstance(value, dict): + return DataFrame(self._jdf.na().fill(value), self.sql_ctx) + elif subset is None: + return DataFrame(self._jdf.na().fill(value), self.sql_ctx) + else: + if isinstance(subset, basestring): + subset = [subset] + elif not isinstance(subset, (list, tuple)): + raise ValueError("subset should be a list or tuple of column names") + + return DataFrame(self._jdf.na().fill(value, self._jseq(subset)), self.sql_ctx) + + def corr(self, col1, col2, method=None): + """ + Calculates the correlation of two columns of a DataFrame as a double value. Currently only + supports the Pearson Correlation Coefficient. + :func:`DataFrame.corr` and :func:`DataFrameStatFunctions.corr` are aliases. + + :param col1: The name of the first column + :param col2: The name of the second column + :param method: The correlation method. Currently only supports "pearson" + """ + if not isinstance(col1, str): + raise ValueError("col1 should be a string.") + if not isinstance(col2, str): + raise ValueError("col2 should be a string.") + if not method: + method = "pearson" + if not method == "pearson": + raise ValueError("Currently only the calculation of the Pearson Correlation " + + "coefficient is supported.") + return self._jdf.stat().corr(col1, col2, method) + + def cov(self, col1, col2): + """ + Calculate the sample covariance for the given columns, specified by their names, as a + double value. :func:`DataFrame.cov` and :func:`DataFrameStatFunctions.cov` are aliases. + + :param col1: The name of the first column + :param col2: The name of the second column + """ + if not isinstance(col1, str): + raise ValueError("col1 should be a string.") + if not isinstance(col2, str): + raise ValueError("col2 should be a string.") + return self._jdf.stat().cov(col1, col2) + + def crosstab(self, col1, col2): + """ + Computes a pair-wise frequency table of the given columns. Also known as a contingency + table. The number of distinct values for each column should be less than 1e4. The first + column of each row will be the distinct values of `col1` and the column names will be the + distinct values of `col2`. The name of the first column will be `$col1_$col2`. Pairs that + have no occurrences will have `null` as their counts. + :func:`DataFrame.crosstab` and :func:`DataFrameStatFunctions.crosstab` are aliases. + + :param col1: The name of the first column. Distinct items will make the first item of + each row. + :param col2: The name of the second column. Distinct items will make the column names + of the DataFrame. + """ + if not isinstance(col1, str): + raise ValueError("col1 should be a string.") + if not isinstance(col2, str): + raise ValueError("col2 should be a string.") + return DataFrame(self._jdf.stat().crosstab(col1, col2), self.sql_ctx) + + def freqItems(self, cols, support=None): + """ + Finding frequent items for columns, possibly with false positives. Using the + frequent element count algorithm described in + "http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou". + :func:`DataFrame.freqItems` and :func:`DataFrameStatFunctions.freqItems` are aliases. + + :param cols: Names of the columns to calculate frequent items for as a list or tuple of + strings. + :param support: The frequency with which to consider an item 'frequent'. Default is 1%. + The support must be greater than 1e-4. + """ + if isinstance(cols, tuple): + cols = list(cols) + if not isinstance(cols, list): + raise ValueError("cols must be a list or tuple of column names as strings.") + if not support: + support = 0.01 + return DataFrame(self._jdf.stat().freqItems(_to_seq(self._sc, cols), support), self.sql_ctx) + + @ignore_unicode_prefix def withColumn(self, colName, col): - """ Return a new :class:`DataFrame` by adding a column. + """Returns a new :class:`DataFrame` by adding a column. + + :param colName: string, name of the new column. + :param col: a :class:`Column` expression for the new column. >>> df.withColumn('age2', df.age + 2).collect() [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] """ return self.select('*', col.alias(colName)) + @ignore_unicode_prefix def withColumnRenamed(self, existing, new): - """ Rename an existing column to a new name + """REturns a new :class:`DataFrame` by renaming an existing column. + + :param existing: string, name of the existing column to rename. + :param col: string, new name of the column. >>> df.withColumnRenamed('age', 'age2').collect() [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] @@ -698,8 +999,9 @@ def withColumnRenamed(self, existing, new): return self.select(*cols) def toPandas(self): - """ - Collect all the rows and return a `pandas.DataFrame`. + """Returns the contents of this :class:`DataFrame` as Pandas ``pandas.DataFrame``. + + This is only available if Pandas is installed and available. >>> df.toPandas() # doctest: +SKIP age name @@ -712,8 +1014,7 @@ def toPandas(self): # Having SchemaRDD for backward compatibility (for docs) class SchemaRDD(DataFrame): - """ - SchemaRDD is deprecated, please use DataFrame + """SchemaRDD is deprecated, please use :class:`DataFrame`. """ @@ -729,10 +1030,8 @@ def _api(self): def df_varargs_api(f): def _api(self, *args): - jargs = ListConverter().convert(args, - self.sql_ctx._sc._gateway._gateway_client) name = f.__name__ - jdf = getattr(self._jdf, name)(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jargs)) + jdf = getattr(self._jdf, name)(_to_seq(self.sql_ctx._sc, args)) return DataFrame(jdf, self.sql_ctx) _api.__name__ = f.__name__ _api.__doc__ = f.__doc__ @@ -740,50 +1039,50 @@ def _api(self, *args): class GroupedData(object): - """ A set of methods for aggregations on a :class:`DataFrame`, - created by DataFrame.groupBy(). + created by :func:`DataFrame.groupBy`. """ def __init__(self, jdf, sql_ctx): self._jdf = jdf self.sql_ctx = sql_ctx + @ignore_unicode_prefix def agg(self, *exprs): - """ Compute aggregates by specifying a map from column name - to aggregate methods. + """Compute aggregates and returns the result as a :class:`DataFrame`. - The available aggregate methods are `avg`, `max`, `min`, - `sum`, `count`. + The available aggregate functions are `avg`, `max`, `min`, `sum`, `count`. - :param exprs: list or aggregate columns or a map from column - name to aggregate methods. + If ``exprs`` is a single :class:`dict` mapping from string to string, then the key + is the column to perform aggregation on, and the value is the aggregate function. + + Alternatively, ``exprs`` can also be a list of aggregate :class:`Column` expressions. + + :param exprs: a dict mapping from column name (string) to aggregate functions (string), + or a list of :class:`Column`. >>> gdf = df.groupBy(df.name) >>> gdf.agg({"*": "count"}).collect() - [Row(name=u'Bob', COUNT(1)=1), Row(name=u'Alice', COUNT(1)=1)] + [Row(name=u'Alice', COUNT(1)=1), Row(name=u'Bob', COUNT(1)=1)] >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age)=5), Row(MIN(age)=2)] + [Row(MIN(age)=2), Row(MIN(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): - jmap = MapConverter().convert(exprs[0], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(jmap) + jdf = self._jdf.agg(exprs[0]) else: # Columns assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - jcols = ListConverter().convert([c._jc for c in exprs[1:]], - self.sql_ctx._sc._gateway._gateway_client) - jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) + jdf = self._jdf.agg(exprs[0]._jc, + _to_seq(self.sql_ctx._sc, [c._jc for c in exprs[1:]])) return DataFrame(jdf, self.sql_ctx) @dfapi def count(self): - """ Count the number of rows for each group. + """Counts the number of records for each group. >>> df.groupBy(df.age).count().collect() [Row(age=2, count=1), Row(age=5, count=1)] @@ -791,8 +1090,11 @@ def count(self): @df_varargs_api def mean(self, *cols): - """Compute the average value for each numeric columns - for each group. This is an alias for `avg`. + """Computes average values for each numeric columns for each group. + + :func:`mean` is an alias for :func:`avg`. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().mean('age').collect() [Row(AVG(age)=3.5)] @@ -802,8 +1104,11 @@ def mean(self, *cols): @df_varargs_api def avg(self, *cols): - """Compute the average value for each numeric columns - for each group. + """Computes average values for each numeric columns for each group. + + :func:`mean` is an alias for :func:`avg`. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().avg('age').collect() [Row(AVG(age)=3.5)] @@ -813,8 +1118,7 @@ def avg(self, *cols): @df_varargs_api def max(self, *cols): - """Compute the max value for each numeric columns for - each group. + """Computes the max value for each numeric columns for each group. >>> df.groupBy().max('age').collect() [Row(MAX(age)=5)] @@ -824,8 +1128,9 @@ def max(self, *cols): @df_varargs_api def min(self, *cols): - """Compute the min value for each numeric column for - each group. + """Computes the min value for each numeric column for each group. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().min('age').collect() [Row(MIN(age)=2)] @@ -835,8 +1140,9 @@ def min(self, *cols): @df_varargs_api def sum(self, *cols): - """Compute the sum for each numeric columns for each - group. + """Compute the sum for each numeric columns for each group. + + :param cols: list of column names (string). Non-numeric columns are ignored. >>> df.groupBy().sum('age').collect() [Row(SUM(age)=7)] @@ -863,6 +1169,18 @@ def _to_java_column(col): return jcol +def _to_seq(sc, cols, converter=None): + """ + Convert a list of Column (or names) into a JVM Seq of Column. + + An optional `converter` could be used to convert items in `cols` + into JVM Column objects. + """ + if converter: + cols = [converter(c) for c in cols] + return sc._jvm.PythonUtils.toSeq(cols) + + def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): @@ -929,11 +1247,13 @@ def __init__(self, jc): __sub__ = _bin_op("minus") __mul__ = _bin_op("multiply") __div__ = _bin_op("divide") + __truediv__ = _bin_op("divide") __mod__ = _bin_op("mod") __radd__ = _bin_op("plus") __rsub__ = _reverse_op("minus") __rmul__ = _bin_op("multiply") __rdiv__ = _reverse_op("divide") + __rtruediv__ = _reverse_op("divide") __rmod__ = _reverse_op("mod") # logistic operators @@ -955,7 +1275,51 @@ def __init__(self, jc): # container operators __contains__ = _bin_op("contains") __getitem__ = _bin_op("getItem") - getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") + + def getItem(self, key): + """An expression that gets an item at position `ordinal` out of a list, + or gets an item by key out of a dict. + + >>> df = sc.parallelize([([1, 2], {"key": "value"})]).toDF(["l", "d"]) + >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() + +----+------+ + |l[0]|d[key]| + +----+------+ + | 1| value| + +----+------+ + >>> df.select(df.l[0], df.d["key"]).show() + +----+------+ + |l[0]|d[key]| + +----+------+ + | 1| value| + +----+------+ + """ + return self[key] + + def getField(self, name): + """An expression that gets a field by name in a StructField. + + >>> from pyspark.sql import Row + >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() + >>> df.select(df.r.getField("b")).show() + +---+ + |r.b| + +---+ + | b| + +---+ + >>> df.select(df.r.a).show() + +---+ + |r.a| + +---+ + | 1| + +---+ + """ + return Column(self._jc.getField(name)) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + return self.getField(item) # string methods rlike = _bin_op("rlike") @@ -963,6 +1327,7 @@ def __init__(self, jc): startswith = _bin_op("startsWith") endswith = _bin_op("endsWith") + @ignore_unicode_prefix def substr(self, startPos, length): """ Return a :class:`Column` which is a substring of the column @@ -985,6 +1350,23 @@ def substr(self, startPos, length): __getslice__ = substr + @ignore_unicode_prefix + def inSet(self, *cols): + """ A boolean expression that is evaluated to true if the value of this + expression is contained by the evaluated values of the arguments. + + >>> df[df.name.inSet("Bob", "Mike")].collect() + [Row(age=5, name=u'Bob')] + >>> df[df.age.inSet([1, 2, 3])].collect() + [Row(age=2, name=u'Alice')] + """ + if len(cols) == 1 and isinstance(cols[0], (list, set)): + cols = cols[0] + cols = [c._jc if isinstance(c, Column) else _create_column_from_literal(c) for c in cols] + sc = SparkContext._active_spark_context + jc = getattr(self._jc, "in")(_to_seq(sc, cols)) + return Column(jc) + # order asc = _unary_op("asc", "Returns a sort expression based on the" " ascending order of the given column name.") @@ -1002,6 +1384,7 @@ def alias(self, alias): """ return Column(getattr(self._jc, "as")(alias)) + @ignore_unicode_prefix def cast(self, dataType): """ Convert the column into type `dataType` @@ -1025,6 +1408,52 @@ def __repr__(self): return 'Column<%s>' % self._jc.toString().encode('utf8') +class DataFrameNaFunctions(object): + """Functionality for working with missing data in :class:`DataFrame`. + """ + + def __init__(self, df): + self.df = df + + def drop(self, how='any', thresh=None, subset=None): + return self.df.dropna(how=how, thresh=thresh, subset=subset) + + drop.__doc__ = DataFrame.dropna.__doc__ + + def fill(self, value, subset=None): + return self.df.fillna(value=value, subset=subset) + + fill.__doc__ = DataFrame.fillna.__doc__ + + +class DataFrameStatFunctions(object): + """Functionality for statistic functions with :class:`DataFrame`. + """ + + def __init__(self, df): + self.df = df + + def corr(self, col1, col2, method=None): + return self.df.corr(col1, col2, method) + + corr.__doc__ = DataFrame.corr.__doc__ + + def cov(self, col1, col2): + return self.df.cov(col1, col2) + + cov.__doc__ = DataFrame.cov.__doc__ + + def crosstab(self, col1, col2): + return self.df.crosstab(col1, col2) + + crosstab.__doc__ = DataFrame.crosstab.__doc__ + + def freqItems(self, cols, support=None): + return self.df.freqItems(cols, support) + + freqItems.__doc__ = DataFrame.freqItems.__doc__ + + def _test(): import doctest from pyspark.context import SparkContext @@ -1033,13 +1462,19 @@ def _test(): globs = pyspark.sql.dataframe.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) globs['df2'] = sc.parallelize([Row(name='Tom', height=80), Row(name='Bob', height=85)]).toDF() globs['df3'] = sc.parallelize([Row(name='Alice', age=2, height=80), Row(name='Bob', age=5, height=85)]).toDF() + + globs['df4'] = sc.parallelize([Row(name='Alice', age=10, height=80), + Row(name='Bob', age=5, height=None), + Row(name='Tom', age=None, height=None), + Row(name=None, age=None, height=None)]).toDF() + (failure_count, test_count) = doctest.testmod( pyspark.sql.dataframe, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5873f09ae3275..641220a264295 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -18,19 +18,26 @@ """ A collections of builtin functions """ +import sys -from itertools import imap - -from py4j.java_collections import ListConverter +if sys.version < "3": + from itertools import imap as map from pyspark import SparkContext -from pyspark.rdd import _prepare_for_python_RDD +from pyspark.rdd import _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.sql.types import StringType -from pyspark.sql.dataframe import Column, _to_java_column +from pyspark.sql.dataframe import Column, _to_java_column, _to_seq -__all__ = ['countDistinct', 'approxCountDistinct', 'udf'] +__all__ = [ + 'approxCountDistinct', + 'countDistinct', + 'monotonicallyIncreasingId', + 'rand', + 'randn', + 'sparkPartitionId', + 'udf'] def _create_function(name, doc=""): @@ -54,7 +61,7 @@ def _(col): 'upper': 'Converts a string expression to upper case.', 'lower': 'Converts a string expression to upper case.', 'sqrt': 'Computes the square root of the specified float value.', - 'abs': 'Computes the absolutle value.', + 'abs': 'Computes the absolute value.', 'max': 'Aggregate function: returns the maximum value of the expression in a group.', 'min': 'Aggregate function: returns the minimum value of the expression in a group.', @@ -67,7 +74,6 @@ def _(col): 'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.', } - for _name, _doc in _functions.items(): globals()[_name] = _create_function(_name, _doc) del _name, _doc @@ -75,23 +81,26 @@ def _(col): __all__.sort() -def countDistinct(col, *cols): - """ Return a new Column for distinct count of `col` or `cols` +def array(*cols): + """Creates a new array column. - >>> df.agg(countDistinct(df.age, df.name).alias('c')).collect() - [Row(c=2)] + :param cols: list of column names (string) or list of :class:`Column` expressions that have + the same data type. - >>> df.agg(countDistinct("age", "name").alias('c')).collect() - [Row(c=2)] + >>> df.select(array('age', 'age').alias("arr")).collect() + [Row(arr=[2, 2]), Row(arr=[5, 5])] + >>> df.select(array([df.age, df.age]).alias("arr")).collect() + [Row(arr=[2, 2]), Row(arr=[5, 5])] """ sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], sc._gateway._gateway_client) - jc = sc._jvm.functions.countDistinct(_to_java_column(col), sc._jvm.PythonUtils.toSeq(jcols)) + if len(cols) == 1 and isinstance(cols[0], (list, set)): + cols = cols[0] + jc = sc._jvm.functions.array(_to_seq(sc, cols, _to_java_column)) return Column(jc) def approxCountDistinct(col, rsd=None): - """ Return a new Column for approximate distinct count of `col` + """Returns a new :class:`Column` for approximate distinct count of ``col``. >>> df.agg(approxCountDistinct(df.age).alias('c')).collect() [Row(c=2)] @@ -104,6 +113,93 @@ def approxCountDistinct(col, rsd=None): return Column(jc) +def countDistinct(col, *cols): + """Returns a new :class:`Column` for distinct count of ``col`` or ``cols``. + + >>> df.agg(countDistinct(df.age, df.name).alias('c')).collect() + [Row(c=2)] + + >>> df.agg(countDistinct("age", "name").alias('c')).collect() + [Row(c=2)] + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.countDistinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column)) + return Column(jc) + + +def monotonicallyIncreasingId(): + """A column that generates monotonically increasing 64-bit integers. + + The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. + The current implementation puts the partition ID in the upper 31 bits, and the record number + within each partition in the lower 33 bits. The assumption is that the data frame has + less than 1 billion partitions, and each partition has less than 8 billion records. + + As an example, consider a [[DataFrame]] with two partitions, each with 3 records. + This expression would return the following IDs: + 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. + + >>> df0 = sc.parallelize(range(2), 2).mapPartitions(lambda x: [(1,), (2,), (3,)]).toDF(['col1']) + >>> df0.select(monotonicallyIncreasingId().alias('id')).collect() + [Row(id=0), Row(id=1), Row(id=2), Row(id=8589934592), Row(id=8589934593), Row(id=8589934594)] + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.monotonicallyIncreasingId()) + + +def rand(seed=None): + """Generates a random column with i.i.d. samples from U[0.0, 1.0]. + """ + sc = SparkContext._active_spark_context + if seed: + jc = sc._jvm.functions.rand(seed) + else: + jc = sc._jvm.functions.rand() + return Column(jc) + + +def randn(seed=None): + """Generates a column with i.i.d. samples from the standard normal distribution. + """ + sc = SparkContext._active_spark_context + if seed: + jc = sc._jvm.functions.randn(seed) + else: + jc = sc._jvm.functions.randn() + return Column(jc) + + +def sparkPartitionId(): + """A column for partition ID of the Spark task. + + Note that this is indeterministic because it depends on data partitioning and task scheduling. + + >>> df.repartition(1).select(sparkPartitionId().alias("pid")).collect() + [Row(pid=0), Row(pid=0)] + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.sparkPartitionId()) + + +@ignore_unicode_prefix +def struct(*cols): + """Creates a new struct column. + + :param cols: list of column names (string) or list of :class:`Column` expressions + that are named or aliased. + + >>> df.select(struct('age', 'name').alias("struct")).collect() + [Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))] + >>> df.select(struct([df.age, df.name]).alias("struct")).collect() + [Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))] + """ + sc = SparkContext._active_spark_context + if len(cols) == 1 and isinstance(cols[0], (list, set)): + cols = cols[0] + jc = sc._jvm.functions.struct(_to_seq(sc, cols, _to_java_column)) + return Column(jc) + + class UserDefinedFunction(object): """ User defined function in Python @@ -116,14 +212,15 @@ def __init__(self, func, returnType): def _create_judf(self): f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) sc = SparkContext._active_spark_context pickled_command, broadcast_vars, env, includes = _prepare_for_python_RDD(sc, command, self) ssql_ctx = sc._jvm.SQLContext(sc._jsc.sc()) jdt = ssql_ctx.parseDataType(self.returnType.json()) - judf = sc._jvm.UserDefinedPythonFunction(f.__name__, bytearray(pickled_command), env, + fname = f.__name__ if hasattr(f, '__name__') else f.__class__.__name__ + judf = sc._jvm.UserDefinedPythonFunction(fname, bytearray(pickled_command), env, includes, sc.pythonExec, broadcast_vars, sc._javaAccumulator, jdt) return judf @@ -135,14 +232,12 @@ def __del__(self): def __call__(self, *cols): sc = SparkContext._active_spark_context - jcols = ListConverter().convert([_to_java_column(c) for c in cols], - sc._gateway._gateway_client) - jc = self._judf.apply(sc._jvm.PythonUtils.toSeq(jcols)) + jc = self._judf.apply(_to_seq(sc, cols, _to_java_column)) return Column(jc) def udf(f, returnType=StringType()): - """Create a user defined function (UDF) + """Creates a :class:`Column` expression representing a user defined function (UDF). >>> from pyspark.sql.types import IntegerType >>> slen = udf(lambda s: len(s), IntegerType()) @@ -160,7 +255,7 @@ def _test(): globs = pyspark.sql.functions.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, diff --git a/python/pyspark/sql/mathfunctions.py b/python/pyspark/sql/mathfunctions.py new file mode 100644 index 0000000000000..7dbcab8694293 --- /dev/null +++ b/python/pyspark/sql/mathfunctions.py @@ -0,0 +1,101 @@ +# +# 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. +# + +""" +A collection of builtin math functions +""" + +from pyspark import SparkContext +from pyspark.sql.dataframe import Column + +__all__ = [] + + +def _create_unary_mathfunction(name, doc=""): + """ Create a unary mathfunction by name""" + def _(col): + sc = SparkContext._active_spark_context + jc = getattr(sc._jvm.mathfunctions, name)(col._jc if isinstance(col, Column) else col) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return _ + + +def _create_binary_mathfunction(name, doc=""): + """ Create a binary mathfunction by name""" + def _(col1, col2): + sc = SparkContext._active_spark_context + # users might write ints for simplicity. This would throw an error on the JVM side. + if type(col1) is int: + col1 = col1 * 1.0 + if type(col2) is int: + col2 = col2 * 1.0 + jc = getattr(sc._jvm.mathfunctions, name)(col1._jc if isinstance(col1, Column) else col1, + col2._jc if isinstance(col2, Column) else col2) + return Column(jc) + _.__name__ = name + _.__doc__ = doc + return _ + + +# math functions are found under another object therefore, they need to be handled separately +_mathfunctions = { + 'acos': 'Computes the cosine inverse of the given value; the returned angle is in the range' + + '0.0 through pi.', + 'asin': 'Computes the sine inverse of the given value; the returned angle is in the range' + + '-pi/2 through pi/2.', + 'atan': 'Computes the tangent inverse of the given value.', + 'cbrt': 'Computes the cube-root of the given value.', + 'ceil': 'Computes the ceiling of the given value.', + 'cos': 'Computes the cosine of the given value.', + 'cosh': 'Computes the hyperbolic cosine of the given value.', + 'exp': 'Computes the exponential of the given value.', + 'expm1': 'Computes the exponential of the given value minus one.', + 'floor': 'Computes the floor of the given value.', + 'log': 'Computes the natural logarithm of the given value.', + 'log10': 'Computes the logarithm of the given value in Base 10.', + 'log1p': 'Computes the natural logarithm of the given value plus one.', + 'rint': 'Returns the double value that is closest in value to the argument and' + + ' is equal to a mathematical integer.', + 'signum': 'Computes the signum of the given value.', + 'sin': 'Computes the sine of the given value.', + 'sinh': 'Computes the hyperbolic sine of the given value.', + 'tan': 'Computes the tangent of the given value.', + 'tanh': 'Computes the hyperbolic tangent of the given value.', + 'toDeg': 'Converts an angle measured in radians to an approximately equivalent angle ' + + 'measured in degrees.', + 'toRad': 'Converts an angle measured in degrees to an approximately equivalent angle ' + + 'measured in radians.' +} + +# math functions that take two arguments as input +_binary_mathfunctions = { + 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + + 'polar coordinates (r, theta).', + 'hypot': 'Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow.', + 'pow': 'Returns the value of the first argument raised to the power of the second argument.' +} + +for _name, _doc in _mathfunctions.items(): + globals()[_name] = _create_unary_mathfunction(_name, _doc) +for _name, _doc in _binary_mathfunctions.items(): + globals()[_name] = _create_binary_mathfunction(_name, _doc) +del _name, _doc +__all__ += _mathfunctions.keys() +__all__ += _binary_mathfunctions.keys() +__all__.sort() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 2720439416682..7ea6656d31c4e 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -25,6 +25,8 @@ import shutil import tempfile import pickle +import functools +import datetime import py4j @@ -41,6 +43,7 @@ from pyspark.sql.types import * from pyspark.sql.types import UserDefinedType, _infer_type from pyspark.tests import ReusedPySparkTestCase +from pyspark.sql.functions import UserDefinedFunction class ExamplePointUDT(UserDefinedType): @@ -106,7 +109,7 @@ def setUpClass(cls): os.unlink(cls.tempdir.name) cls.sqlCtx = SQLContext(cls.sc) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] - rdd = cls.sc.parallelize(cls.testData) + rdd = cls.sc.parallelize(cls.testData, 2) cls.df = rdd.toDF() @classmethod @@ -114,6 +117,35 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name, ignore_errors=True) + def test_udf_with_callable(self): + d = [Row(number=i, squared=i**2) for i in range(10)] + rdd = self.sc.parallelize(d) + data = self.sqlCtx.createDataFrame(rdd) + + class PlusFour: + def __call__(self, col): + if col is not None: + return col + 4 + + call = PlusFour() + pudf = UserDefinedFunction(call, LongType()) + res = data.select(pudf(data['number']).alias('plus_four')) + self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85) + + def test_udf_with_partial_function(self): + d = [Row(number=i, squared=i**2) for i in range(10)] + rdd = self.sc.parallelize(d) + data = self.sqlCtx.createDataFrame(rdd) + + def some_func(col, param): + if col is not None: + return col + param + + pfunc = functools.partial(some_func, param=4) + pudf = UserDefinedFunction(pfunc, LongType()) + res = data.select(pudf(data['number']).alias('plus_four')) + self.assertEqual(res.agg({'plus_four': 'sum'}).collect()[0][0], 85) + def test_udf(self): self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() @@ -126,13 +158,13 @@ def test_udf2(self): self.assertEqual(4, res[0]) def test_udf_with_array_type(self): - d = [Row(l=range(3), d={"key": range(5)})] + d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) self.sqlCtx.createDataFrame(rdd).registerTempTable("test") self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() - self.assertEqual(range(3), l1) + self.assertEqual(list(range(3)), l1) self.assertEqual(1, l2) def test_broadcast_in_udf(self): @@ -235,7 +267,7 @@ def test_infer_nested_schema(self): def test_apply_schema(self): from datetime import date, datetime - rdd = self.sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + rdd = self.sc.parallelize([(127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, (2,), [1, 2, 3], None)]) schema = StructType([ @@ -251,7 +283,7 @@ def test_apply_schema(self): StructField("struct1", StructType([StructField("b", ShortType(), False)]), False), StructField("list1", ArrayType(ByteType(), False), False), StructField("null1", DoubleType(), True)]) - df = self.sqlCtx.applySchema(rdd, schema) + df = self.sqlCtx.createDataFrame(rdd, schema) results = df.map(lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int1, x.float1, x.date1, x.time1, x.map1["a"], x.struct1.b, x.list1, x.null1)) r = (127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), @@ -271,14 +303,14 @@ def test_apply_schema(self): abstract = "byte1 short1 float1 time1 map1{} struct1(b) list1[]" schema = _parse_schema_abstract(abstract) typedSchema = _infer_schema_type(rdd.first(), schema) - df = self.sqlCtx.applySchema(rdd, typedSchema) + df = self.sqlCtx.createDataFrame(rdd, typedSchema) r = (127, -32768, 1.0, datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, Row(b=2), [1, 2, 3]) self.assertEqual(r, tuple(df.first())) def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] df = self.sc.parallelize(d).toDF() - k, v = df.head().m.items()[0] + k, v = list(df.head().m.items())[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -343,6 +375,13 @@ def test_column_select(self): self.assertEqual(self.testData, df.select(df.key, df.value).collect()) self.assertEqual([Row(value='1')], df.where(df.key == 1).select(df.value).collect()) + def test_freqItems(self): + vals = [Row(a=1, b=-2.0) if i % 2 == 0 else Row(a=i, b=i * 1.0) for i in range(100)] + df = self.sc.parallelize(vals).toDF() + items = df.stat.freqItems(("a", "b"), 0.4).collect()[0] + self.assertTrue(1 in items[0]) + self.assertTrue(-2.0 in items[1]) + def test_aggregator(self): df = self.df g = df.groupBy() @@ -355,6 +394,65 @@ def test_aggregator(self): self.assertTrue(95 < g.agg(functions.approxCountDistinct(df.key)).first()[0]) self.assertEqual(100, g.agg(functions.countDistinct(df.value)).first()[0]) + def test_corr(self): + import math + df = self.sc.parallelize([Row(a=i, b=math.sqrt(i)) for i in range(10)]).toDF() + corr = df.stat.corr("a", "b") + self.assertTrue(abs(corr - 0.95734012) < 1e-6) + + def test_cov(self): + df = self.sc.parallelize([Row(a=i, b=2 * i) for i in range(10)]).toDF() + cov = df.stat.cov("a", "b") + self.assertTrue(abs(cov - 55.0 / 3) < 1e-6) + + def test_crosstab(self): + df = self.sc.parallelize([Row(a=i % 3, b=i % 2) for i in range(1, 7)]).toDF() + ct = df.stat.crosstab("a", "b").collect() + ct = sorted(ct, key=lambda x: x[0]) + for i, row in enumerate(ct): + self.assertEqual(row[0], str(i)) + self.assertTrue(row[1], 1) + self.assertTrue(row[2], 1) + + def test_math_functions(self): + df = self.sc.parallelize([Row(a=i, b=2 * i) for i in range(10)]).toDF() + from pyspark.sql import mathfunctions as functions + import math + + def get_values(l): + return [j[0] for j in l] + + def assert_close(a, b): + c = get_values(b) + diff = [abs(v - c[k]) < 1e-6 for k, v in enumerate(a)] + return sum(diff) == len(a) + assert_close([math.cos(i) for i in range(10)], + df.select(functions.cos(df.a)).collect()) + assert_close([math.cos(i) for i in range(10)], + df.select(functions.cos("a")).collect()) + assert_close([math.sin(i) for i in range(10)], + df.select(functions.sin(df.a)).collect()) + assert_close([math.sin(i) for i in range(10)], + df.select(functions.sin(df['a'])).collect()) + assert_close([math.pow(i, 2 * i) for i in range(10)], + df.select(functions.pow(df.a, df.b)).collect()) + assert_close([math.pow(i, 2) for i in range(10)], + df.select(functions.pow(df.a, 2)).collect()) + assert_close([math.pow(i, 2) for i in range(10)], + df.select(functions.pow(df.a, 2.0)).collect()) + assert_close([math.hypot(i, 2 * i) for i in range(10)], + df.select(functions.hypot(df.a, df.b)).collect()) + + def test_rand_functions(self): + df = self.df + from pyspark.sql import functions + rnd = df.select('key', functions.rand()).collect() + for row in rnd: + assert row[1] >= 0.0 and row[1] <= 1.0, "got: %s" % row[1] + rndn = df.select('key', functions.randn(5)).collect() + for row in rndn: + assert row[1] >= -4.0 and row[1] <= 4.0, "got: %s" % row[1] + def test_save_and_load(self): df = self.df tmpPath = tempfile.mkdtemp() @@ -395,6 +493,24 @@ def test_help_command(self): pydoc.render_doc(df.foo) pydoc.render_doc(df.take(1)) + def test_access_column(self): + df = self.df + self.assertTrue(isinstance(df.key, Column)) + self.assertTrue(isinstance(df['key'], Column)) + self.assertTrue(isinstance(df[0], Column)) + self.assertRaises(IndexError, lambda: df[2]) + self.assertRaises(IndexError, lambda: df["bad_key"]) + self.assertRaises(TypeError, lambda: df[{}]) + + def test_access_nested_types(self): + df = self.sc.parallelize([Row(l=[1], r=Row(a=1, b="b"), d={"k": "v"})]).toDF() + self.assertEqual(1, df.select(df.l[0]).first()[0]) + self.assertEqual(1, df.select(df.l.getItem(0)).first()[0]) + self.assertEqual(1, df.select(df.r.a).first()[0]) + self.assertEqual("b", df.select(df.r.getField("b")).first()[0]) + self.assertEqual("v", df.select(df.d["k"]).first()[0]) + self.assertEqual("v", df.select(df.d.getItem("k")).first()[0]) + def test_infer_long_type(self): longrow = [Row(f1='a', f2=100000000000000)] df = self.sc.parallelize(longrow).toDF() @@ -415,6 +531,112 @@ def test_infer_long_type(self): self.assertEqual(_infer_type(2**61), LongType()) self.assertEqual(_infer_type(2**71), LongType()) + def test_filter_with_datetime(self): + time = datetime.datetime(2015, 4, 17, 23, 1, 2, 3000) + date = time.date() + row = Row(date=date, time=time) + df = self.sqlCtx.createDataFrame([row]) + self.assertEqual(1, df.filter(df.date == date).count()) + self.assertEqual(1, df.filter(df.time == time).count()) + self.assertEqual(0, df.filter(df.date > date).count()) + self.assertEqual(0, df.filter(df.time > time).count()) + + def test_dropna(self): + schema = StructType([ + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("height", DoubleType(), True)]) + + # shouldn't drop a non-null row + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, 80.1)], schema).dropna().count(), + 1) + + # dropping rows with a single null value + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna().count(), + 0) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(how='any').count(), + 0) + + # if how = 'all', only drop rows if all values are null + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(how='all').count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(None, None, None)], schema).dropna(how='all').count(), + 0) + + # how and subset + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna(how='any', subset=['name', 'age']).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, None)], schema).dropna(how='any', subset=['name', 'age']).count(), + 0) + + # threshold + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 80.1)], schema).dropna(thresh=2).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, None)], schema).dropna(thresh=2).count(), + 0) + + # threshold and subset + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna(thresh=2, subset=['name', 'age']).count(), + 1) + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', None, 180.9)], schema).dropna(thresh=2, subset=['name', 'age']).count(), + 0) + + # thresh should take precedence over how + self.assertEqual(self.sqlCtx.createDataFrame( + [(u'Alice', 50, None)], schema).dropna( + how='any', thresh=2, subset=['name', 'age']).count(), + 1) + + def test_fillna(self): + schema = StructType([ + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("height", DoubleType(), True)]) + + # fillna shouldn't change non-null values + row = self.sqlCtx.createDataFrame([(u'Alice', 10, 80.1)], schema).fillna(50).first() + self.assertEqual(row.age, 10) + + # fillna with int + row = self.sqlCtx.createDataFrame([(u'Alice', None, None)], schema).fillna(50).first() + self.assertEqual(row.age, 50) + self.assertEqual(row.height, 50.0) + + # fillna with double + row = self.sqlCtx.createDataFrame([(u'Alice', None, None)], schema).fillna(50.1).first() + self.assertEqual(row.age, 50) + self.assertEqual(row.height, 50.1) + + # fillna with string + row = self.sqlCtx.createDataFrame([(None, None, None)], schema).fillna("hello").first() + self.assertEqual(row.name, u"hello") + self.assertEqual(row.age, None) + + # fillna with subset specified for numeric cols + row = self.sqlCtx.createDataFrame( + [(None, None, None)], schema).fillna(50, subset=['name', 'age']).first() + self.assertEqual(row.name, None) + self.assertEqual(row.age, 50) + self.assertEqual(row.height, None) + + # fillna with subset specified for numeric cols + row = self.sqlCtx.createDataFrame( + [(None, None, None)], schema).fillna("haha", subset=['name', 'age']).first() + self.assertEqual(row.name, "haha") + self.assertEqual(row.age, None) + self.assertEqual(row.height, None) + class HiveContextSQLTests(ReusedPySparkTestCase): @@ -427,6 +649,9 @@ def setUpClass(cls): except py4j.protocol.Py4JError: cls.sqlCtx = None return + except TypeError: + cls.sqlCtx = None + return os.unlink(cls.tempdir.name) _scala_HiveContext =\ cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 1e597d64e03fe..944fa414b0c0e 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -31,7 +31,7 @@ class StatCounter(object): def __init__(self, values=[]): - self.n = 0L # Running count of our values + self.n = 0 # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") @@ -87,7 +87,7 @@ def copy(self): return copy.deepcopy(self) def count(self): - return self.n + return int(self.n) def mean(self): return self.mu diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 2c73083c9f9a8..ac5ba69e8dbbb 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -14,10 +14,12 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +from __future__ import print_function + import os import sys -from py4j.java_collections import ListConverter from py4j.java_gateway import java_import, JavaObject from pyspark import RDD, SparkConf @@ -157,7 +159,7 @@ def getOrCreate(cls, checkpointPath, setupFunc): try: jssc = gw.jvm.JavaStreamingContext(checkpointPath) except Exception: - print >>sys.stderr, "failed to load StreamingContext from checkpoint" + print("failed to load StreamingContext from checkpoint", file=sys.stderr) raise jsc = jssc.sparkContext() @@ -302,9 +304,7 @@ def queueStream(self, rdds, oneAtATime=True, default=None): rdds = [self._sc.parallelize(input) for input in rdds] self._check_serializers(rdds) - jrdds = ListConverter().convert([r._jrdd for r in rdds], - SparkContext._gateway._gateway_client) - queue = self._jvm.PythonDStream.toRDDQueue(jrdds) + queue = self._jvm.PythonDStream.toRDDQueue([r._jrdd for r in rdds]) if default: default = default._reserialize(rdds[0]._jrdd_deserializer) jdstream = self._jssc.queueStream(queue, oneAtATime, default._jrdd) @@ -319,8 +319,7 @@ def transform(self, dstreams, transformFunc): the transform function parameter will be the same as the order of corresponding DStreams in the list. """ - jdstreams = ListConverter().convert([d._jdstream for d in dstreams], - SparkContext._gateway._gateway_client) + jdstreams = [d._jdstream for d in dstreams] # change the final serializer to sc.serializer func = TransformFunction(self._sc, lambda t, *rdds: transformFunc(rdds).map(lambda x: x), @@ -343,6 +342,5 @@ def union(self, *dstreams): if len(set(s._slideDuration for s in dstreams)) > 1: raise ValueError("All DStreams should have same slide duration") first = dstreams[0] - jrest = ListConverter().convert([d._jdstream for d in dstreams[1:]], - SparkContext._gateway._gateway_client) + jrest = [d._jdstream for d in dstreams[1:]] return DStream(self._jssc.union(first._jdstream, jrest), self, first._jrdd_deserializer) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 3fa42444239f7..ff097985fae3e 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -15,11 +15,15 @@ # limitations under the License. # -from itertools import chain, ifilter, imap +import sys import operator import time +from itertools import chain from datetime import datetime +if sys.version < "3": + from itertools import imap as map, ifilter as filter + from py4j.protocol import Py4JJavaError from pyspark import RDD @@ -76,7 +80,7 @@ def filter(self, f): Return a new DStream containing only the elements that satisfy predicate. """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def flatMap(self, f, preservesPartitioning=False): @@ -85,7 +89,7 @@ def flatMap(self, f, preservesPartitioning=False): this DStream, and then flattening the results """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def map(self, f, preservesPartitioning=False): @@ -93,7 +97,7 @@ def map(self, f, preservesPartitioning=False): Return a new DStream by applying a function to each element of DStream. """ def func(iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitions(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -150,7 +154,7 @@ def foreachRDD(self, func): """ Apply a function to each RDD in this DStream. """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: old_func = func func = lambda t, rdd: old_func(rdd) jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) @@ -165,14 +169,14 @@ def pprint(self, num=10): """ def takeAndPrint(time, rdd): taken = rdd.take(num + 1) - print "-------------------------------------------" - print "Time: %s" % time - print "-------------------------------------------" + print("-------------------------------------------") + print("Time: %s" % time) + print("-------------------------------------------") for record in taken[:num]: - print record + print(record) if len(taken) > num: - print "..." - print + print("...") + print() self.foreachRDD(takeAndPrint) @@ -181,7 +185,7 @@ def mapValues(self, f): Return a new DStream by applying a map function to the value of each key-value pairs in this DStream without changing the key. """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def flatMapValues(self, f): @@ -189,7 +193,7 @@ def flatMapValues(self, f): Return a new DStream by applying a flatmap function to the value of each key-value pairs in this DStream without changing the key. """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def glom(self): @@ -286,10 +290,10 @@ def transform(self, func): `func` can have one argument of `rdd`, or have two arguments of (`time`, `rdd`) """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: oldfunc = func func = lambda t, rdd: oldfunc(rdd) - assert func.func_code.co_argcount == 2, "func should take one or two arguments" + assert func.__code__.co_argcount == 2, "func should take one or two arguments" return TransformedDStream(self, func) def transformWith(self, func, other, keepSerializer=False): @@ -300,10 +304,10 @@ def transformWith(self, func, other, keepSerializer=False): `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three arguments of (`time`, `rdd_a`, `rdd_b`) """ - if func.func_code.co_argcount == 2: + if func.__code__.co_argcount == 2: oldfunc = func func = lambda t, a, b: oldfunc(a, b) - assert func.func_code.co_argcount == 3, "func should take two or three arguments" + assert func.__code__.co_argcount == 3, "func should take two or three arguments" jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer, other._jrdd_deserializer) dstream = self._sc._jvm.PythonTransformed2DStream(self._jdstream.dstream(), other._jdstream.dstream(), jfunc) @@ -460,7 +464,7 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio keyed = self.map(lambda x: (1, x)) reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1) - return reduced.map(lambda (k, v): v) + return reduced.map(lambda kv: kv[1]) def countByWindow(self, windowDuration, slideDuration): """ @@ -489,7 +493,7 @@ def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=Non keyed = self.map(lambda x: (x, 1)) counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub, windowDuration, slideDuration, numPartitions) - return counted.filter(lambda (k, v): v > 0).count() + return counted.filter(lambda kv: kv[1] > 0).count() def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None): """ @@ -548,7 +552,8 @@ def reduceFunc(t, a, b): def invReduceFunc(t, a, b): b = b.reduceByKey(func, numPartitions) joined = a.leftOuterJoin(b, numPartitions) - return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1) + return joined.mapValues(lambda kv: invFunc(kv[0], kv[1]) + if kv[1] is not None else kv[0]) jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer) if invReduceFunc: @@ -579,9 +584,9 @@ def reduceFunc(t, a, b): g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) else: g = a.cogroup(b.partitionBy(numPartitions), numPartitions) - g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) - state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) - return state.filter(lambda (k, v): v is not None) + g = g.mapValues(lambda ab: (list(ab[1]), list(ab[0])[0] if len(ab[0]) else None)) + state = g.mapValues(lambda vs_s: updateFunc(vs_s[0], vs_s[1])) + return state.filter(lambda k_v: k_v[1] is not None) jreduceFunc = TransformFunction(self._sc, reduceFunc, self._sc.serializer, self._jrdd_deserializer) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index f083ed149effb..e278b29003f69 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -15,14 +15,14 @@ # limitations under the License. # -from py4j.java_collections import MapConverter -from py4j.java_gateway import java_import, Py4JError, Py4JJavaError +from py4j.java_gateway import Py4JJavaError +from pyspark.rdd import RDD from pyspark.storagelevel import StorageLevel from pyspark.serializers import PairDeserializer, NoOpSerializer from pyspark.streaming import DStream -__all__ = ['KafkaUtils', 'utf8_decoder'] +__all__ = ['Broker', 'KafkaUtils', 'OffsetRange', 'TopicAndPartition', 'utf8_decoder'] def utf8_decoder(s): @@ -57,8 +57,6 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, }) if not isinstance(topics, dict): raise TypeError("topics should be dict") - jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client) - jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client) jlevel = ssc._sc._getJavaStorageLevel(storageLevel) try: @@ -66,11 +64,108 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") helper = helperClass.newInstance() - jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) - except Py4JJavaError, e: + jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel) + except Py4JJavaError as e: # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): - print """ + KafkaUtils._printErrorMsg(ssc.sparkContext) + raise e + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) + + @staticmethod + def createDirectStream(ssc, topics, kafkaParams, fromOffsets={}, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder): + """ + .. note:: Experimental + + Create an input stream that directly pulls messages from a Kafka Broker and specific offset. + + This is not a receiver based Kafka input stream, it directly pulls the message from Kafka + in each batch duration and processed without storing. + + This does not use Zookeeper to store offsets. The consumed offsets are tracked + by the stream itself. For interoperability with Kafka monitoring tools that depend on + Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + You can access the offsets used in each batch from the generated RDDs (see + + To recover from driver failures, you have to enable checkpointing in the StreamingContext. + The information on consumed offset can be recovered from the checkpoint. + See the programming guide for details (constraints, etc.). + + :param ssc: StreamingContext object. + :param topics: list of topic_name to consume. + :param kafkaParams: Additional params for Kafka. + :param fromOffsets: Per-topic/partition Kafka offsets defining the (inclusive) starting + point of the stream. + :param keyDecoder: A function used to decode key (default is utf8_decoder). + :param valueDecoder: A function used to decode value (default is utf8_decoder). + :return: A DStream object + """ + if not isinstance(topics, list): + raise TypeError("topics should be list") + if not isinstance(kafkaParams, dict): + raise TypeError("kafkaParams should be dict") + + try: + helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ + .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") + helper = helperClass.newInstance() + + jfromOffsets = dict([(k._jTopicAndPartition(helper), + v) for (k, v) in fromOffsets.items()]) + jstream = helper.createDirectStream(ssc._jssc, kafkaParams, set(topics), jfromOffsets) + except Py4JJavaError as e: + if 'ClassNotFoundException' in str(e.java_exception): + KafkaUtils._printErrorMsg(ssc.sparkContext) + raise e + + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) + + @staticmethod + def createRDD(sc, kafkaParams, offsetRanges, leaders={}, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder): + """ + .. note:: Experimental + + Create a RDD from Kafka using offset ranges for each topic and partition. + :param sc: SparkContext object + :param kafkaParams: Additional params for Kafka + :param offsetRanges: list of offsetRange to specify topic:partition:[start, end) to consume + :param leaders: Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty + map, in which case leaders will be looked up on the driver. + :param keyDecoder: A function used to decode key (default is utf8_decoder) + :param valueDecoder: A function used to decode value (default is utf8_decoder) + :return: A RDD object + """ + if not isinstance(kafkaParams, dict): + raise TypeError("kafkaParams should be dict") + if not isinstance(offsetRanges, list): + raise TypeError("offsetRanges should be list") + + try: + helperClass = sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ + .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") + helper = helperClass.newInstance() + joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges] + jleaders = dict([(k._jTopicAndPartition(helper), + v._jBroker(helper)) for (k, v) in leaders.items()]) + jrdd = helper.createRDD(sc._jsc, kafkaParams, joffsetRanges, jleaders) + except Py4JJavaError as e: + if 'ClassNotFoundException' in str(e.java_exception): + KafkaUtils._printErrorMsg(sc) + raise e + + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + rdd = RDD(jrdd, sc, ser) + return rdd.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) + + @staticmethod + def _printErrorMsg(sc): + print(""" ________________________________________________________________________________________________ Spark Streaming's Kafka libraries not found in class path. Try one of the following. @@ -88,8 +183,63 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, ________________________________________________________________________________________________ -""" % (ssc.sparkContext.version, ssc.sparkContext.version) - raise e - ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) - stream = DStream(jstream, ssc, ser) - return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) +""" % (sc.version, sc.version)) + + +class OffsetRange(object): + """ + Represents a range of offsets from a single Kafka TopicAndPartition. + """ + + def __init__(self, topic, partition, fromOffset, untilOffset): + """ + Create a OffsetRange to represent range of offsets + :param topic: Kafka topic name. + :param partition: Kafka partition id. + :param fromOffset: Inclusive starting offset. + :param untilOffset: Exclusive ending offset. + """ + self._topic = topic + self._partition = partition + self._fromOffset = fromOffset + self._untilOffset = untilOffset + + def _jOffsetRange(self, helper): + return helper.createOffsetRange(self._topic, self._partition, self._fromOffset, + self._untilOffset) + + +class TopicAndPartition(object): + """ + Represents a specific top and partition for Kafka. + """ + + def __init__(self, topic, partition): + """ + Create a Python TopicAndPartition to map to the Java related object + :param topic: Kafka topic name. + :param partition: Kafka partition id. + """ + self._topic = topic + self._partition = partition + + def _jTopicAndPartition(self, helper): + return helper.createTopicAndPartition(self._topic, self._partition) + + +class Broker(object): + """ + Represent the host and port info for a Kafka broker. + """ + + def __init__(self, host, port): + """ + Create a Python Broker to map to the Java related object. + :param host: Broker's hostname. + :param port: Broker's port. + """ + self._host = host + self._port = port + + def _jBroker(self, helper): + return helper.createBroker(self._host, self._port) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 608f8e26473a6..33ea8c9293d74 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -16,39 +16,57 @@ # import os +import sys from itertools import chain import time import operator -import unittest import tempfile +import random import struct +from functools import reduce + +if sys.version_info[:2] <= (2, 6): + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) +else: + import unittest from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext +from pyspark.streaming.kafka import Broker, KafkaUtils, OffsetRange, TopicAndPartition class PySparkStreamingTestCase(unittest.TestCase): - timeout = 10 # seconds - duration = 1 + timeout = 4 # seconds + duration = .2 - def setUp(self): - class_name = self.__class__.__name__ + @classmethod + def setUpClass(cls): + class_name = cls.__name__ conf = SparkConf().set("spark.default.parallelism", 1) - self.sc = SparkContext(appName=class_name, conf=conf) - self.sc.setCheckpointDir("/tmp") - # TODO: decrease duration to speed up tests + cls.sc = SparkContext(appName=class_name, conf=conf) + cls.sc.setCheckpointDir("/tmp") + + @classmethod + def tearDownClass(cls): + cls.sc.stop() + + def setUp(self): self.ssc = StreamingContext(self.sc, self.duration) def tearDown(self): - self.ssc.stop() + self.ssc.stop(False) def wait_for(self, result, n): start_time = time.time() while len(result) < n and time.time() - start_time < self.timeout: time.sleep(0.01) if len(result) < n: - print "timeout after", self.timeout + print("timeout after", self.timeout) def _take(self, dstream, n): """ @@ -128,7 +146,7 @@ def test_map(self): def func(dstream): return dstream.map(str) - expected = map(lambda x: map(str, x), input) + expected = [list(map(str, x)) for x in input] self._test_func(input, func, expected) def test_flatMap(self): @@ -137,8 +155,8 @@ def test_flatMap(self): def func(dstream): return dstream.flatMap(lambda x: (x, x * 2)) - expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), - input) + expected = [list(chain.from_iterable((map(lambda y: [y, y * 2], x)))) + for x in input] self._test_func(input, func, expected) def test_filter(self): @@ -147,7 +165,7 @@ def test_filter(self): def func(dstream): return dstream.filter(lambda x: x % 2 == 0) - expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input) + expected = [[y for y in x if y % 2 == 0] for x in input] self._test_func(input, func, expected) def test_count(self): @@ -156,7 +174,7 @@ def test_count(self): def func(dstream): return dstream.count() - expected = map(lambda x: [len(x)], input) + expected = [[len(x)] for x in input] self._test_func(input, func, expected) def test_reduce(self): @@ -165,7 +183,7 @@ def test_reduce(self): def func(dstream): return dstream.reduce(operator.add) - expected = map(lambda x: [reduce(operator.add, x)], input) + expected = [[reduce(operator.add, x)] for x in input] self._test_func(input, func, expected) def test_reduceByKey(self): @@ -182,27 +200,27 @@ def func(dstream): def test_mapValues(self): """Basic operation test for DStream.mapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 2), (3, 3)], + [(0, 4), (1, 1), (2, 2), (3, 3)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.mapValues(lambda x: x + 10) expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], - [("", 14), (1, 11), (2, 12), (3, 13)], + [(0, 14), (1, 11), (2, 12), (3, 13)], [(1, 11), (2, 11), (3, 11), (4, 11)]] self._test_func(input, func, expected, sort=True) def test_flatMapValues(self): """Basic operation test for DStream.flatMapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 1), (3, 1)], + [(0, 4), (1, 1), (2, 1), (3, 1)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.flatMapValues(lambda x: (x, x + 10)) expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), ("c", 1), ("c", 11), ("d", 1), ("d", 11)], - [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(0, 4), (0, 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] self._test_func(input, func, expected) @@ -230,7 +248,7 @@ def f(iterator): def test_countByValue(self): """Basic operation test for DStream.countByValue.""" - input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + input = [list(range(1, 5)) * 2, list(range(5, 7)) + list(range(5, 9)), ["a", "a", "b", ""]] def func(dstream): return dstream.countByValue() @@ -282,7 +300,7 @@ def test_union(self): def func(d1, d2): return d1.union(d2) - expected = [range(6), range(6), range(6)] + expected = [list(range(6)), list(range(6)), list(range(6))] self._test_func(input1, func, expected, input2=input2) def test_cogroup(self): @@ -361,13 +379,13 @@ def func(dstream): class WindowFunctionTests(PySparkStreamingTestCase): - timeout = 20 + timeout = 5 def test_window(self): input = [range(1), range(2), range(3), range(4), range(5)] def func(dstream): - return dstream.window(3, 1).count() + return dstream.window(.6, .2).count() expected = [[1], [3], [6], [9], [12], [9], [5]] self._test_func(input, func, expected) @@ -376,7 +394,7 @@ def test_count_by_window(self): input = [range(1), range(2), range(3), range(4), range(5)] def func(dstream): - return dstream.countByWindow(3, 1) + return dstream.countByWindow(.6, .2) expected = [[1], [3], [6], [9], [12], [9], [5]] self._test_func(input, func, expected) @@ -385,7 +403,7 @@ def test_count_by_window_large(self): input = [range(1), range(2), range(3), range(4), range(5), range(6)] def func(dstream): - return dstream.countByWindow(5, 1) + return dstream.countByWindow(1, .2) expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]] self._test_func(input, func, expected) @@ -394,7 +412,7 @@ def test_count_by_value_and_window(self): input = [range(1), range(2), range(3), range(4), range(5), range(6)] def func(dstream): - return dstream.countByValueAndWindow(5, 1) + return dstream.countByValueAndWindow(1, .2) expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]] self._test_func(input, func, expected) @@ -403,7 +421,7 @@ def test_group_by_key_and_window(self): input = [[('a', i)] for i in range(5)] def func(dstream): - return dstream.groupByKeyAndWindow(3, 1).mapValues(list) + return dstream.groupByKeyAndWindow(.6, .2).mapValues(list) expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])], [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]] @@ -421,7 +439,7 @@ class StreamingContextTests(PySparkStreamingTestCase): duration = 0.1 def _add_input_stream(self): - inputs = map(lambda x: range(1, x), range(101)) + inputs = [range(1, x) for x in range(101)] stream = self.ssc.queueStream(inputs) self._collect(stream, 1, block=False) @@ -434,11 +452,11 @@ def test_stop_only_streaming_context(self): def test_stop_multiple_times(self): self._add_input_stream() self.ssc.start() - self.ssc.stop() - self.ssc.stop() + self.ssc.stop(False) + self.ssc.stop(False) def test_queue_stream(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) result = self._collect(dstream, 3) self.assertEqual(input, result) @@ -454,13 +472,13 @@ def test_text_file_stream(self): with open(os.path.join(d, name), "w") as f: f.writelines(["%d\n" % i for i in range(10)]) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], result) + self.assertEqual([list(range(10)), list(range(10))], result) def test_binary_records_stream(self): d = tempfile.mkdtemp() self.ssc = StreamingContext(self.sc, self.duration) dstream = self.ssc.binaryRecordsStream(d, 10).map( - lambda v: struct.unpack("10b", str(v))) + lambda v: struct.unpack("10b", bytes(v))) result = self._collect(dstream, 2, block=False) self.ssc.start() for name in ('a', 'b'): @@ -468,10 +486,10 @@ def test_binary_records_stream(self): with open(os.path.join(d, name), "wb") as f: f.write(bytearray(range(10))) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], map(lambda v: list(v[0]), result)) + self.assertEqual([list(range(10)), list(range(10))], [list(v[0]) for v in result]) def test_union(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) dstream2 = self.ssc.queueStream(input) dstream3 = self.ssc.union(dstream, dstream2) @@ -493,10 +511,7 @@ def func(rdds): self.assertEqual([2, 3, 1], self._take(dstream, 3)) -class CheckpointTests(PySparkStreamingTestCase): - - def setUp(self): - pass +class CheckpointTests(unittest.TestCase): def test_get_or_create(self): inputd = tempfile.mkdtemp() @@ -516,12 +531,12 @@ def setup(): return ssc cpd = tempfile.mkdtemp("test_streaming_cps") - self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc = StreamingContext.getOrCreate(cpd, setup) ssc.start() def check_output(n): while not os.listdir(outputd): - time.sleep(0.1) + time.sleep(0.01) time.sleep(1) # make sure mtime is larger than the previous one with open(os.path.join(inputd, str(n)), 'w') as f: f.writelines(["%d\n" % i for i in range(10)]) @@ -551,10 +566,120 @@ def check_output(n): ssc.stop(True, True) time.sleep(1) - self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc = StreamingContext.getOrCreate(cpd, setup) ssc.start() check_output(3) + ssc.stop(True, True) + + +class KafkaStreamTests(PySparkStreamingTestCase): + timeout = 20 # seconds + duration = 1 + def setUp(self): + super(KafkaStreamTests, self).setUp() + + kafkaTestUtilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ + .loadClass("org.apache.spark.streaming.kafka.KafkaTestUtils") + self._kafkaTestUtils = kafkaTestUtilsClz.newInstance() + self._kafkaTestUtils.setup() + + def tearDown(self): + if self._kafkaTestUtils is not None: + self._kafkaTestUtils.teardown() + self._kafkaTestUtils = None + + super(KafkaStreamTests, self).tearDown() + + def _randomTopic(self): + return "topic-%d" % random.randint(0, 10000) + + def _validateStreamResult(self, sendData, stream): + result = {} + for i in chain.from_iterable(self._collect(stream.map(lambda x: x[1]), + sum(sendData.values()))): + result[i] = result.get(i, 0) + 1 + + self.assertEqual(sendData, result) + + def _validateRddResult(self, sendData, rdd): + result = {} + for i in rdd.map(lambda x: x[1]).collect(): + result[i] = result.get(i, 0) + 1 + self.assertEqual(sendData, result) + + def test_kafka_stream(self): + """Test the Python Kafka stream API.""" + topic = self._randomTopic() + sendData = {"a": 3, "b": 5, "c": 10} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) + + stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), + "test-streaming-consumer", {topic: 1}, + {"auto.offset.reset": "smallest"}) + self._validateStreamResult(sendData, stream) + + def test_kafka_direct_stream(self): + """Test the Python direct Kafka stream API.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) + self._validateStreamResult(sendData, stream) + + @unittest.skipIf(sys.version >= "3", "long type not support") + def test_kafka_direct_stream_from_offset(self): + """Test the Python direct Kafka stream API with start offset specified.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + fromOffsets = {TopicAndPartition(topic, 0): long(0)} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) + self._validateStreamResult(sendData, stream) + + @unittest.skipIf(sys.version >= "3", "long type not support") + def test_kafka_rdd(self): + """Test the Python direct Kafka RDD API.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) + self._validateRddResult(sendData, rdd) + + @unittest.skipIf(sys.version >= "3", "long type not support") + def test_kafka_rdd_with_leaders(self): + """Test the Python direct Kafka RDD API with leaders.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + address = self._kafkaTestUtils.brokerAddress().split(":") + leaders = {TopicAndPartition(topic, 0): Broker(address[0], int(address[1]))} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) + self._validateRddResult(sendData, rdd) if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index 86ee5aa04f252..34291f30a5652 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -91,9 +91,9 @@ def dumps(self, id): except Exception: traceback.print_exc() - def loads(self, bytes): + def loads(self, data): try: - f, deserializers = self.serializer.loads(str(bytes)) + f, deserializers = self.serializer.loads(bytes(data)) return TransformFunction(self.ctx, f, *deserializers) except Exception: traceback.print_exc() @@ -116,7 +116,7 @@ def rddToFileName(prefix, suffix, timestamp): """ if isinstance(timestamp, datetime): seconds = time.mktime(timestamp.timetuple()) - timestamp = long(seconds * 1000) + timestamp.microsecond / 1000 + timestamp = int(seconds * 1000) + timestamp.microsecond // 1000 if suffix is None: return prefix + "-" + str(timestamp) else: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index dd8d3b1c53733..ea63a396da5b8 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,8 +19,8 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ + from array import array -from fileinput import input from glob import glob import os import re @@ -34,6 +34,8 @@ import threading import hashlib +from py4j.protocol import Py4JJavaError + if sys.version_info[:2] <= (2, 6): try: import unittest2 as unittest @@ -42,6 +44,14 @@ sys.exit(1) else: import unittest + if sys.version_info[0] >= 3: + xrange = range + basestring = str + +if sys.version >= "3": + from io import StringIO +else: + from StringIO import StringIO from pyspark.conf import SparkConf @@ -49,7 +59,9 @@ from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer, \ + PairDeserializer, CartesianDeserializer, AutoBatchedSerializer, AutoSerializer, \ + FlattenedValuesSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark import shuffle from pyspark.profiler import BasicProfiler @@ -76,9 +88,9 @@ class MergerTests(unittest.TestCase): def setUp(self): - self.N = 1 << 14 + self.N = 1 << 12 self.l = [i for i in xrange(self.N)] - self.data = zip(self.l, self.l) + self.data = list(zip(self.l, self.l)) self.agg = Aggregator(lambda x: [x], lambda x, y: x.append(y) or x, lambda x, y: x.extend(y) or x) @@ -86,84 +98,110 @@ def setUp(self): def test_in_memory(self): m = InMemoryMerger(self.agg) m.mergeValues(self.data) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = InMemoryMerger(self.agg) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + m.mergeCombiners(map(lambda x_y: (x_y[0], [x_y[1]]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_small_dataset(self): m = ExternalMerger(self.agg, 1000) m.mergeValues(self.data) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 1000) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data)) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 10) + m = ExternalMerger(self.agg, 20) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 10) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10, partitions=3) - m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + m = ExternalMerger(self.agg, 5, partitions=3) + m.mergeCombiners(map(lambda k_v: (k_v[0], [str(k_v[1])]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), + self.assertEqual(sum(len(v) for k, v in m.items()), self.N * 10) m._cleanup() + def test_group_by_key(self): + + def gen_data(N, step): + for i in range(1, N + 1, step): + for j in range(i): + yield (i, [j]) + + def gen_gs(N, step=1): + return shuffle.GroupByKey(gen_data(N, step)) + + self.assertEqual(1, len(list(gen_gs(1)))) + self.assertEqual(2, len(list(gen_gs(2)))) + self.assertEqual(100, len(list(gen_gs(100)))) + self.assertEqual(list(range(1, 101)), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(list(range(k)) == list(vs) for k, vs in gen_gs(100))) + + for k, vs in gen_gs(50002, 10000): + self.assertEqual(k, len(vs)) + self.assertEqual(list(range(k)), list(vs)) + + ser = PickleSerializer() + l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) + for k, vs in l: + self.assertEqual(k, len(vs)) + self.assertEqual(list(range(k)), list(vs)) + class SorterTests(unittest.TestCase): def test_in_memory_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1024) - self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) def test_external_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1) - self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) self.assertGreater(shuffle.DiskBytesSpilled, 0) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") sc = SparkContext(conf=conf) - l = range(10240) + l = list(range(10240)) random.shuffle(l) - rdd = sc.parallelize(l, 10) - self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + rdd = sc.parallelize(l, 4) + self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect()) sc.stop() @@ -171,11 +209,11 @@ class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): from collections import namedtuple - from cPickle import dumps, loads + from pickle import dumps, loads P = namedtuple("P", "x y") p1 = P(1, 3) p2 = loads(dumps(p1, 2)) - self.assertEquals(p1, p2) + self.assertEqual(p1, p2) def test_itemgetter(self): from operator import itemgetter @@ -217,7 +255,7 @@ def test_pickling_file_handles(self): ser = CloudPickleSerializer() out1 = sys.stderr out2 = ser.loads(ser.dumps(out1)) - self.assertEquals(out1, out2) + self.assertEqual(out1, out2) def test_func_globals(self): @@ -234,19 +272,48 @@ def __reduce__(self): def foo(): sys.exit(0) - self.assertTrue("exit" in foo.func_code.co_names) + self.assertTrue("exit" in foo.__code__.co_names) ser.dumps(foo) def test_compressed_serializer(self): ser = CompressedSerializer(PickleSerializer()) - from StringIO import StringIO + try: + from StringIO import StringIO + except ImportError: + from io import BytesIO as StringIO io = StringIO() ser.dump_stream(["abc", u"123", range(5)], io) io.seek(0) self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io))) ser.dump_stream(range(1000), io) io.seek(0) - self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) + self.assertEqual(["abc", u"123", range(5)] + list(range(1000)), list(ser.load_stream(io))) + io.close() + + def test_hash_serializer(self): + hash(NoOpSerializer()) + hash(UTF8Deserializer()) + hash(PickleSerializer()) + hash(MarshalSerializer()) + hash(AutoSerializer()) + hash(BatchedSerializer(PickleSerializer())) + hash(AutoBatchedSerializer(MarshalSerializer())) + hash(PairDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CartesianDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CompressedSerializer(PickleSerializer())) + hash(FlattenedValuesSerializer(PickleSerializer())) + + +class QuietTest(object): + def __init__(self, sc): + self.log4j = sc._jvm.org.apache.log4j + + def __enter__(self): + self.old_level = self.log4j.LogManager.getRootLogger().getLevel() + self.log4j.LogManager.getRootLogger().setLevel(self.log4j.Level.FATAL) + + def __exit__(self, exc_type, exc_val, exc_tb): + self.log4j.LogManager.getRootLogger().setLevel(self.old_level) class PySparkTestCase(unittest.TestCase): @@ -311,7 +378,7 @@ def test_checkpoint_and_restore(self): self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(), flatMappedRDD._jrdd_deserializer) - self.assertEquals([1, 2, 3, 4], recovered.collect()) + self.assertEqual([1, 2, 3, 4], recovered.collect()) class AddFileTests(PySparkTestCase): @@ -320,16 +387,11 @@ def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that # this job fails due to `userlibrary` not being on the Python path: # disable logging in log4j temporarily - log4j = self.sc._jvm.org.apache.log4j - old_level = log4j.LogManager.getRootLogger().getLevel() - log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) - def func(x): from userlibrary import UserClass return UserClass().hello() - self.assertRaises(Exception, - self.sc.parallelize(range(2)).map(func).first) - log4j.LogManager.getRootLogger().setLevel(old_level) + with QuietTest(self.sc): + self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) # Add the file, so the job should now succeed: path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") @@ -343,7 +405,7 @@ def test_add_file_locally(self): download_path = SparkFiles.get("hello.txt") self.assertNotEqual(path, download_path) with open(download_path) as test_file: - self.assertEquals("Hello World!\n", test_file.readline()) + self.assertEqual("Hello World!\n", test_file.readline()) def test_add_py_file_locally(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -352,7 +414,7 @@ def func(): from userlibrary import UserClass self.assertRaises(ImportError, func) path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") - self.sc.addFile(path) + self.sc.addPyFile(path) from userlibrary import UserClass self.assertEqual("Hello World!", UserClass().hello()) @@ -362,7 +424,7 @@ def test_add_egg_file_locally(self): def func(): from userlib import UserClass self.assertRaises(ImportError, func) - path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg") + path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip") self.sc.addPyFile(path) from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) @@ -398,8 +460,9 @@ def test_save_as_textfile_with_unicode(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode("utf-8")) def test_save_as_textfile_with_utf8(self): x = u"\u00A1Hola, mundo!" @@ -407,19 +470,20 @@ def test_save_as_textfile_with_utf8(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode('utf8')) def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) rdd2 = self.sc.parallelize([3, 4]) cart = rdd1.cartesian(rdd2) - result = cart.map(lambda (x, y): x + y).collect() + result = cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect() def test_transforming_pickle_file(self): # Regression test for SPARK-2601 - data = self.sc.parallelize(["Hello", "World!"]) + data = self.sc.parallelize([u"Hello", u"World!"]) tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsPickleFile(tempFile.name) @@ -432,19 +496,20 @@ def test_cartesian_on_textfile(self): a = self.sc.textFile(path) result = a.cartesian(a).collect() (x, y) = result[0] - self.assertEqual("Hello World!", x.strip()) - self.assertEqual("Hello World!", y.strip()) + self.assertEqual(u"Hello World!", x.strip()) + self.assertEqual(u"Hello World!", y.strip()) def test_deleting_input_files(self): # Regression test for SPARK-1025 tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) filtered_data = data.filter(lambda x: True) self.assertEqual(1, filtered_data.count()) os.unlink(tempFile.name) - self.assertRaises(Exception, lambda: filtered_data.count()) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: filtered_data.count()) def test_sampling_default_seed(self): # Test for SPARK-3995 (default seed setting) @@ -481,21 +546,21 @@ def test_namedtuple_in_rdd(self): jon = Person(1, "Jon", "Doe") jane = Person(2, "Jane", "Doe") theDoes = self.sc.parallelize([jon, jane]) - self.assertEquals([jon, jane], theDoes.collect()) + self.assertEqual([jon, jane], theDoes.collect()) def test_large_broadcast(self): - N = 100000 + N = 10000 data = [[float(i) for i in range(300)] for i in range(N)] - bdata = self.sc.broadcast(data) # 270MB + bdata = self.sc.broadcast(data) # 27MB m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() - self.assertEquals(N, m) + self.assertEqual(N, m) def test_multiple_broadcasts(self): N = 1 << 21 b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM - r = range(1 << 15) + r = list(range(1 << 15)) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -506,7 +571,7 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -517,14 +582,12 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) def test_large_closure(self): - N = 1000000 + N = 200000 data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) - self.assertEquals(N, rdd.first()) - self.assertTrue(rdd._broadcast is not None) - rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) - self.assertEqual(1, rdd.first()) - self.assertTrue(rdd._broadcast is None) + self.assertEqual(N, rdd.first()) + # regression test for SPARK-6886 + self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) @@ -554,24 +617,25 @@ def test_zip_with_different_number_of_items(self): # different number of partitions b = self.sc.parallelize(range(100, 106), 3) self.assertRaises(ValueError, lambda: a.zip(b)) - # different number of batched items in JVM - b = self.sc.parallelize(range(100, 104), 2) - self.assertRaises(Exception, lambda: a.zip(b).count()) - # different number of items in one pair - b = self.sc.parallelize(range(100, 106), 2) - self.assertRaises(Exception, lambda: a.zip(b).count()) - # same total number of items, but different distributions - a = self.sc.parallelize([2, 3], 2).flatMap(range) - b = self.sc.parallelize([3, 2], 2).flatMap(range) - self.assertEquals(a.count(), b.count()) - self.assertRaises(Exception, lambda: a.zip(b).count()) + with QuietTest(self.sc): + # different number of batched items in JVM + b = self.sc.parallelize(range(100, 104), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # different number of items in one pair + b = self.sc.parallelize(range(100, 106), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # same total number of items, but different distributions + a = self.sc.parallelize([2, 3], 2).flatMap(range) + b = self.sc.parallelize([3, 2], 2).flatMap(range) + self.assertEqual(a.count(), b.count()) + self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): rdd = self.sc.parallelize(range(1000)) - self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050) rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7) self.assertTrue(18 < rdd.countApproxDistinct() < 22) @@ -585,59 +649,59 @@ def test_count_approx_distinct(self): def test_histogram(self): # empty rdd = self.sc.parallelize([]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) self.assertRaises(ValueError, lambda: rdd.histogram(1)) # out of range rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1]) # in range with one bucket rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals([4], rdd.histogram([0, 10])[1]) - self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([4], rdd.histogram([0, 10])[1]) + self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1]) # in range with one bucket exact match - self.assertEquals([4], rdd.histogram([1, 4])[1]) + self.assertEqual([4], rdd.histogram([1, 4])[1]) # out of range with two buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1]) # out of range with two uneven buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) # in range with two buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two bucket and None rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two uneven buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1]) # mixed range with two uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) - self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1]) # mixed range with four uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # mixed range with uneven buckets and NaN rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, None, float('nan')]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # out of range with infinite buckets rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) - self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + self.assertEqual([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) # invalid buckets self.assertRaises(ValueError, lambda: rdd.histogram([])) @@ -647,25 +711,25 @@ def test_histogram(self): # without buckets rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + self.assertEqual(([1, 4], [4]), rdd.histogram(1)) # without buckets single element rdd = self.sc.parallelize([1]) - self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + self.assertEqual(([1, 1], [1]), rdd.histogram(1)) # without bucket no range rdd = self.sc.parallelize([1] * 4) - self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + self.assertEqual(([1, 1], [4]), rdd.histogram(1)) # without buckets basic two rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) # without buckets with more requested than elements rdd = self.sc.parallelize([1, 2]) buckets = [1 + 0.2 * i for i in range(6)] hist = [1, 0, 0, 0, 1] - self.assertEquals((buckets, hist), rdd.histogram(5)) + self.assertEqual((buckets, hist), rdd.histogram(5)) # invalid RDDs rdd = self.sc.parallelize([1, float('inf')]) @@ -675,15 +739,8 @@ def test_histogram(self): # string rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) - self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) - self.assertRaises(TypeError, lambda: rdd.histogram(2)) - - # mixed RDD - rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) - self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) - self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) def test_repartitionAndSortWithinPartitions(self): @@ -691,16 +748,31 @@ def test_repartitionAndSortWithinPartitions(self): repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) partitions = repartitioned.glom().collect() - self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) - self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)]) def test_distinct(self): rdd = self.sc.parallelize((1, 2, 3)*10, 10) - self.assertEquals(rdd.getNumPartitions(), 10) - self.assertEquals(rdd.distinct().count(), 3) + self.assertEqual(rdd.getNumPartitions(), 10) + self.assertEqual(rdd.distinct().count(), 3) result = rdd.distinct(5) - self.assertEquals(result.getNumPartitions(), 5) - self.assertEquals(result.count(), 3) + self.assertEqual(result.getNumPartitions(), 5) + self.assertEqual(result.count(), 3) + + def test_external_group_by_key(self): + self.sc._conf.set("spark.python.worker.memory", "1m") + N = 200001 + kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) + gkv = kv.groupByKey().cache() + self.assertEqual(3, gkv.count()) + filtered = gkv.filter(lambda kv: kv[0] == 1) + self.assertEqual(1, filtered.count()) + self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect()) + self.assertEqual([(N // 3, N // 3)], + filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) + result = filtered.collect()[0][1] + self.assertEqual(N // 3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList)) def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -725,7 +797,7 @@ def test_null_in_rdd(self): rdd = RDD(jrdd, self.sc, UTF8Deserializer()) self.assertEqual([u"a", None, u"b"], rdd.collect()) rdd = RDD(jrdd, self.sc, NoOpSerializer()) - self.assertEqual(["a", None, "b"], rdd.collect()) + self.assertEqual([b"a", None, b"b"], rdd.collect()) def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 @@ -752,9 +824,9 @@ def test_narrow_dependency_in_join(self): self.assertEqual(rdd.getNumPartitions() + 2, parted.union(rdd).getNumPartitions()) self.assertEqual(rdd.getNumPartitions() + 2, rdd.union(parted).getNumPartitions()) - self.sc.setJobGroup("test1", "test", True) tracker = self.sc.statusTracker() + self.sc.setJobGroup("test1", "test", True) d = sorted(parted.join(parted).collect()) self.assertEqual(10, len(d)) self.assertEqual((0, (0, 0)), d[0]) @@ -771,14 +843,14 @@ def test_narrow_dependency_in_join(self): self.sc.setJobGroup("test3", "test", True) d = sorted(parted.cogroup(parted).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test3")[0] self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) self.sc.setJobGroup("test4", "test", True) d = sorted(parted.cogroup(rdd).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) @@ -787,6 +859,17 @@ def test_take_on_jrdd(self): rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) rdd._jrdd.first() + def test_sortByKey_uses_all_partitions_not_only_first_and_last(self): + # Regression test for SPARK-5969 + seq = [(i * 59 % 101, i) for i in range(101)] # unsorted sequence + rdd = self.sc.parallelize(seq) + for ascending in [True, False]: + sort = rdd.sortByKey(ascending=ascending, numPartitions=5) + self.assertEqual(sort.collect(), sorted(seq, reverse=not ascending)) + sizes = sort.glom().map(len).collect() + for size in sizes: + self.assertGreater(size, 0) + class ProfilerTests(PySparkTestCase): @@ -808,7 +891,12 @@ def test_profiler(self): func_names = [func_name for fname, n, func_name in stat_list] self.assertTrue("heavy_foo" in func_names) + old_stdout = sys.stdout + sys.stdout = io = StringIO() self.sc.show_profiles() + self.assertTrue("heavy_foo" in io.getvalue()) + sys.stdout = old_stdout + d = tempfile.gettempdir() self.sc.dump_profiles(d) self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) @@ -832,7 +920,7 @@ def show(self, id): def do_computation(self): def heavy_foo(x): - for i in range(1 << 20): + for i in range(1 << 18): x = 1 rdd = self.sc.parallelize(range(100)) @@ -853,6 +941,7 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/", @@ -901,15 +990,16 @@ def test_sequencefiles(self): en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] self.assertEqual(nulls, en) - maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) + maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect() em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] - self.assertEqual(maps, em) + for v in maps: + self.assertTrue(v in em) # arrays get pickled to tuples by default tuples = sorted(self.sc.sequenceFile( @@ -1036,8 +1126,8 @@ def test_converters(self): def test_binary_files(self): path = os.path.join(self.tempdir.name, "binaryfiles") os.mkdir(path) - data = "short binary data" - with open(os.path.join(path, "part-0000"), 'w') as f: + data = b"short binary data" + with open(os.path.join(path, "part-0000"), 'wb') as f: f.write(data) [(p, d)] = self.sc.binaryFiles(path).collect() self.assertTrue(p.endswith("part-0000")) @@ -1050,7 +1140,7 @@ def test_binary_records(self): for i in range(100): f.write('%04d' % i) result = self.sc.binaryRecords(path, 4).map(int).collect() - self.assertEqual(range(100), result) + self.assertEqual(list(range(100)), result) class OutputFormatTests(ReusedPySparkTestCase): @@ -1062,6 +1152,7 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.tempdir.name, ignore_errors=True) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] @@ -1102,8 +1193,9 @@ def test_sequencefiles(self): (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") - maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) - self.assertEqual(maps, em) + maps = self.sc.sequenceFile(basepath + "/sfmap/").collect() + for v in maps: + self.assertTrue(v, em) def test_oldhadoop(self): basepath = self.tempdir.name @@ -1115,12 +1207,13 @@ def test_oldhadoop(self): "org.apache.hadoop.mapred.SequenceFileOutputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable") - result = sorted(self.sc.hadoopFile( + result = self.sc.hadoopFile( basepath + "/oldhadoop/", "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) - self.assertEqual(result, dict_data) + "org.apache.hadoop.io.MapWritable").collect() + for v in result: + self.assertTrue(v, dict_data) conf = { "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", @@ -1130,12 +1223,13 @@ def test_oldhadoop(self): } self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) input_conf = {"mapred.input.dir": basepath + "/olddataset/"} - old_dataset = sorted(self.sc.hadoopRDD( + result = self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - conf=input_conf).collect()) - self.assertEqual(old_dataset, dict_data) + conf=input_conf).collect() + for v in result: + self.assertTrue(v, dict_data) def test_newhadoop(self): basepath = self.tempdir.name @@ -1170,6 +1264,7 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) + @unittest.skipIf(sys.version >= "3", "serialize of array") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays @@ -1250,7 +1345,7 @@ def test_reserialization(self): basepath = self.tempdir.name x = range(1, 5) y = range(1001, 1005) - data = zip(x, y) + data = list(zip(x, y)) rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) @@ -1301,7 +1396,7 @@ def connect(self, port): sock = socket(AF_INET, SOCK_STREAM) sock.connect(('127.0.0.1', port)) # send a split index of -1 to shutdown the worker - sock.send("\xFF\xFF\xFF\xFF") + sock.send(b"\xFF\xFF\xFF\xFF") sock.close() return True @@ -1341,8 +1436,7 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) -class WorkerTests(PySparkTestCase): - +class WorkerTests(ReusedPySparkTestCase): def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() @@ -1357,7 +1451,10 @@ def sleep(x): # start job in background thread def run(): - self.sc.parallelize(range(1)).foreach(sleep) + try: + self.sc.parallelize(range(1), 1).foreach(sleep) + except Exception: + pass import threading t = threading.Thread(target=run) t.daemon = True @@ -1366,7 +1463,8 @@ def run(): daemon_pid, worker_pid = 0, 0 while True: if os.path.exists(path): - data = open(path).read().split(' ') + with open(path) as f: + data = f.read().split(' ') daemon_pid, worker_pid = map(int, data) break time.sleep(0.1) @@ -1397,18 +1495,20 @@ def test_after_exception(self): def raise_exception(_): raise Exception() rdd = self.sc.parallelize(range(100), 1) - self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) self.assertEqual(100, rdd.map(str).count()) def test_after_jvm_exception(self): tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name, 1) filtered_data = data.filter(lambda x: True) self.assertEqual(1, filtered_data.count()) os.unlink(tempFile.name) - self.assertRaises(Exception, lambda: filtered_data.count()) + with QuietTest(self.sc): + self.assertRaises(Exception, lambda: filtered_data.count()) rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) @@ -1441,6 +1541,17 @@ def count(): self.assertTrue(not t.isAlive()) self.assertEqual(100000, rdd.count()) + def test_with_different_versions_of_python(self): + rdd = self.sc.parallelize(range(10)) + rdd.count() + version = sys.version_info + sys.version_info = (2, 0, 0) + try: + with QuietTest(self.sc): + self.assertRaises(Py4JJavaError, lambda: rdd.count()) + finally: + sys.version_info = version + class SparkSubmitTests(unittest.TestCase): @@ -1510,12 +1621,12 @@ def test_single_script(self): |from pyspark import SparkContext | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + |print(sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) def test_script_with_local_functions(self): """Submit and test a single script file calling a global function""" @@ -1526,12 +1637,12 @@ def test_script_with_local_functions(self): | return x * 3 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[3, 6, 9]", out) + self.assertIn("[3, 6, 9]", out.decode('utf-8')) def test_module_dependency(self): """Submit and test a script with a dependency on another module""" @@ -1540,7 +1651,7 @@ def test_module_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1550,7 +1661,7 @@ def test_module_dependency(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_module_dependency_on_cluster(self): """Submit and test a script with a dependency on another module on a cluster""" @@ -1559,7 +1670,7 @@ def test_module_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1570,7 +1681,7 @@ def test_module_dependency_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency(self): """Submit and test a script with a dependency on a Spark Package""" @@ -1579,14 +1690,14 @@ def test_package_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", "file:" + self.programDir, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency_on_cluster(self): """Submit and test a script with a dependency on a Spark Package on a cluster""" @@ -1595,7 +1706,7 @@ def test_package_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", @@ -1603,7 +1714,7 @@ def test_package_dependency_on_cluster(self): "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_single_script_on_cluster(self): """Submit and test a single script on a cluster""" @@ -1614,7 +1725,7 @@ def test_single_script_on_cluster(self): | return x * 2 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) # this will fail if you have different spark.executor.memory # in conf/spark-defaults.conf @@ -1623,7 +1734,7 @@ def test_single_script_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) class ContextTests(unittest.TestCase): @@ -1661,9 +1772,14 @@ def test_with_stop(self): def test_progress_api(self): with SparkContext() as sc: sc.setJobGroup('test_progress_api', '', True) - rdd = sc.parallelize(range(10)).map(lambda x: time.sleep(100)) - t = threading.Thread(target=rdd.collect) + + def run(): + try: + rdd.count() + except Exception: + pass + t = threading.Thread(target=run) t.daemon = True t.start() # wait for scheduler to start @@ -1698,7 +1814,7 @@ class SciPyTests(PySparkTestCase): def test_serialize(self): from scipy.special import gammaln x = range(1, 5) - expected = map(gammaln, x) + expected = list(map(gammaln, x)) observed = self.sc.parallelize(x).map(gammaln).collect() self.assertEqual(expected, observed) @@ -1719,11 +1835,11 @@ def test_statcounter_array(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") if not _have_numpy: - print "NOTE: Skipping NumPy tests as it does not seem to be installed" + print("NOTE: Skipping NumPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") if not _have_numpy: - print "NOTE: NumPy tests were skipped as it does not seem to be installed" + print("NOTE: NumPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8a93c320ec5d3..fbdaf3a5814cd 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -18,6 +18,7 @@ """ Worker that receives input from Piped RDD. """ +from __future__ import print_function import os import sys import time @@ -37,9 +38,9 @@ def report_times(outfile, boot, init, finish): write_int(SpecialLengths.TIMING_DATA, outfile) - write_long(1000 * boot, outfile) - write_long(1000 * init, outfile) - write_long(1000 * finish, outfile) + write_long(int(1000 * boot), outfile) + write_long(int(1000 * init), outfile) + write_long(int(1000 * finish), outfile) def add_path(path): @@ -72,6 +73,9 @@ def main(infile, outfile): for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) add_path(os.path.join(spark_files_dir, filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) @@ -88,7 +92,11 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, profiler, deserializer, serializer) = command + (func, profiler, deserializer, serializer), version = command + if version != sys.version_info[:2]: + raise Exception(("Python in worker has different version %s than that in " + + "driver %s, PySpark cannot run with different minor versions") % + (sys.version_info[:2], version)) init_time = time.time() def process(): @@ -102,14 +110,14 @@ def process(): except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) + write_with_length(traceback.format_exc().encode("utf-8"), outfile) except IOError: # JVM close the socket pass except Exception: # Write the error to stderr if it happened while serializing - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() + print("PySpark worker failed with exception:", file=sys.stderr) + print(traceback.format_exc(), file=sys.stderr) exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) diff --git a/python/run-tests b/python/run-tests index b7630c356cfae..0e0eee3564e7c 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,11 +21,14 @@ # Figure out where the Spark framework is installed FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" +. "$FWDIR"/bin/load-spark-env.sh + # CD into the python directory to find things on the right path cd "$FWDIR/python" FAILED=0 LOG_FILE=unit-tests.log +START=$(date +"%s") rm -f $LOG_FILE @@ -33,8 +36,8 @@ rm -f $LOG_FILE rm -rf metastore warehouse function run_test() { - echo "Running test: $1" | tee -a $LOG_FILE - + echo -en "Running test: $1 ... " | tee -a $LOG_FILE + start=$(date +"%s") SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 > $LOG_FILE 2>&1 FAILED=$((PIPESTATUS[0]||$FAILED)) @@ -46,6 +49,9 @@ function run_test() { echo "Had test failures; see logs." echo -en "\033[0m" # No color exit -1 + else + now=$(date +"%s") + echo "ok ($(($now - $start))s)" fi } @@ -57,14 +63,14 @@ function run_core_tests() { PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py" run_test "pyspark/serializers.py" - run_test "pyspark/profiler.py" + run_test "pyspark/profiler.py" run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" } function run_sql_tests() { echo "Run sql tests ..." - run_test "pyspark/sql/types.py" + run_test "pyspark/sql/_types.py" run_test "pyspark/sql/context.py" run_test "pyspark/sql/dataframe.py" run_test "pyspark/sql/functions.py" @@ -77,6 +83,7 @@ function run_mllib_tests() { run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/evaluation.py" run_test "pyspark/mllib/feature.py" + run_test "pyspark/mllib/fpm.py" run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/rand.py" run_test "pyspark/mllib/recommendation.py" @@ -91,11 +98,27 @@ function run_ml_tests() { echo "Run ml tests ..." run_test "pyspark/ml/feature.py" run_test "pyspark/ml/classification.py" + run_test "pyspark/ml/tuning.py" run_test "pyspark/ml/tests.py" } function run_streaming_tests() { echo "Run streaming tests ..." + + KAFKA_ASSEMBLY_DIR="$FWDIR"/external/kafka-assembly + JAR_PATH="${KAFKA_ASSEMBLY_DIR}/target/scala-${SPARK_SCALA_VERSION}" + for f in "${JAR_PATH}"/spark-streaming-kafka-assembly-*.jar; do + if [[ ! -e "$f" ]]; then + echo "Failed to find Spark Streaming Kafka assembly jar in $KAFKA_ASSEMBLY_DIR" 1>&2 + echo "You need to build Spark with " \ + "'build/sbt assembly/assembly streaming-kafka-assembly/assembly' or" \ + "'build/mvn package' before running this program" 1>&2 + exit 1 + fi + KAFKA_ASSEMBLY_JAR="$f" + done + + export PYSPARK_SUBMIT_ARGS="--jars ${KAFKA_ASSEMBLY_JAR} pyspark-shell" run_test "pyspark/streaming/util.py" run_test "pyspark/streaming/tests.py" } @@ -118,6 +141,19 @@ run_mllib_tests run_ml_tests run_streaming_tests +# Try to test with Python 3 +if [ $(which python3.4) ]; then + export PYSPARK_PYTHON="python3.4" + echo "Testing with Python3.4 version:" + $PYSPARK_PYTHON --version + + run_core_tests + run_sql_tests + run_mllib_tests + run_ml_tests + run_streaming_tests +fi + # Try to test with PyPy if [ $(which pypy) ]; then export PYSPARK_PYTHON="pypy" @@ -130,9 +166,8 @@ if [ $(which pypy) ]; then fi if [[ $FAILED == 0 ]]; then - echo -en "\033[32m" # Green - echo "Tests passed." - echo -en "\033[0m" # No color + now=$(date +"%s") + echo -e "\033[32mTests passed \033[0min $(($now - $START)) seconds" fi # TODO: in the long-run, it would be nice to use a test runner like `nose`. diff --git a/python/test_support/userlib-0.1-py2.7.egg b/python/test_support/userlib-0.1-py2.7.egg deleted file mode 100644 index 1674c9cb2227e..0000000000000 Binary files a/python/test_support/userlib-0.1-py2.7.egg and /dev/null differ diff --git a/python/test_support/userlib-0.1.zip b/python/test_support/userlib-0.1.zip new file mode 100644 index 0000000000000..496e1349aa967 Binary files /dev/null and b/python/test_support/userlib-0.1.zip differ diff --git a/repl/pom.xml b/repl/pom.xml index edfa1c7f2c29c..03053b4c3b287 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -84,6 +84,11 @@ scalacheck_${scala.binary.version} test + + org.mockito + mockito-all + test + diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 1bb62c84abddc..1cb910f376060 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -1129,7 +1129,7 @@ class SparkIMain(@BeanProperty val factory: ScriptEngineFactory, initialSettings def apply(line: String): Result = debugging(s"""parse("$line")""") { var isIncomplete = false - currentRun.reporting.withIncompleteHandler((_, _) => isIncomplete = true) { + currentRun.parsing.withIncompleteHandler((_, _) => isIncomplete = true) { reporter.reset() val trees = newUnitParser(line).parseStats() if (reporter.hasErrors) Error diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 9805609120005..004941d5f50ae 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -17,9 +17,10 @@ package org.apache.spark.repl -import java.io.{ByteArrayOutputStream, InputStream, FileNotFoundException} -import java.net.{URI, URL, URLEncoder} -import java.util.concurrent.{Executors, ExecutorService} +import java.io.{IOException, ByteArrayOutputStream, InputStream} +import java.net.{HttpURLConnection, URI, URL, URLEncoder} + +import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} @@ -43,6 +44,9 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader val parentLoader = new ParentClassLoader(parent) + // Allows HTTP connect and read timeouts to be controlled for testing / debugging purposes + private[repl] var httpUrlConnectionTimeoutMillis: Int = -1 + // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { if (Set("http", "https", "ftp").contains(uri.getScheme)) { @@ -71,30 +75,66 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader } } + private def getClassFileInputStreamFromHttpServer(pathInDirectory: String): InputStream = { + val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { + val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) + val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) + newuri.toURL + } else { + new URL(classUri + "/" + urlEncode(pathInDirectory)) + } + val connection: HttpURLConnection = Utils.setupSecureURLConnection(url.openConnection(), + SparkEnv.get.securityManager).asInstanceOf[HttpURLConnection] + // Set the connection timeouts (for testing purposes) + if (httpUrlConnectionTimeoutMillis != -1) { + connection.setConnectTimeout(httpUrlConnectionTimeoutMillis) + connection.setReadTimeout(httpUrlConnectionTimeoutMillis) + } + connection.connect() + try { + if (connection.getResponseCode != 200) { + // Close the error stream so that the connection is eligible for re-use + try { + connection.getErrorStream.close() + } catch { + case ioe: IOException => + logError("Exception while closing error stream", ioe) + } + throw new ClassNotFoundException(s"Class file not found at URL $url") + } else { + connection.getInputStream + } + } catch { + case NonFatal(e) if !e.isInstanceOf[ClassNotFoundException] => + connection.disconnect() + throw e + } + } + + private def getClassFileInputStreamFromFileSystem(pathInDirectory: String): InputStream = { + val path = new Path(directory, pathInDirectory) + if (fileSystem.exists(path)) { + fileSystem.open(path) + } else { + throw new ClassNotFoundException(s"Class file not found at path $path") + } + } + def findClassLocally(name: String): Option[Class[_]] = { + val pathInDirectory = name.replace('.', '/') + ".class" + var inputStream: InputStream = null try { - val pathInDirectory = name.replace('.', '/') + ".class" - val inputStream = { + inputStream = { if (fileSystem != null) { - fileSystem.open(new Path(directory, pathInDirectory)) + getClassFileInputStreamFromFileSystem(pathInDirectory) } else { - val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { - val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) - val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) - newuri.toURL - } else { - new URL(classUri + "/" + urlEncode(pathInDirectory)) - } - - Utils.setupSecureURLConnection(url.openConnection(), SparkEnv.get.securityManager) - .getInputStream + getClassFileInputStreamFromHttpServer(pathInDirectory) } } val bytes = readAndTransformClass(name, inputStream) - inputStream.close() Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: FileNotFoundException => + case e: ClassNotFoundException => // We did not find the class logDebug(s"Did not load class $name from REPL class server at $uri", e) None @@ -102,6 +142,15 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader // Something bad happened while checking if the class exists logError(s"Failed to check existence of class $name on REPL class server at $uri", e) None + } finally { + if (inputStream != null) { + try { + inputStream.close() + } catch { + case e: Exception => + logError("Exception while closing inputStream", e) + } + } } } diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index e7e4a4113174a..e2ee9c963a4da 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 6a79e76a34db8..c709cde740748 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -20,13 +20,25 @@ package org.apache.spark.repl import java.io.File import java.net.{URL, URLClassLoader} +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.scalatest.concurrent.Interruptor +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.mock.MockitoSugar +import org.mockito.Mockito._ -import org.apache.spark.{SparkConf, TestUtils} +import org.apache.spark._ import org.apache.spark.util.Utils -class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { +class ExecutorClassLoaderSuite + extends FunSuite + with BeforeAndAfterAll + with MockitoSugar + with Logging { val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") @@ -34,6 +46,7 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { var tempDir2: File = _ var url1: String = _ var urls2: Array[URL] = _ + var classServer: HttpServer = _ override def beforeAll() { super.beforeAll() @@ -47,8 +60,12 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { override def afterAll() { super.afterAll() + if (classServer != null) { + classServer.stop() + } Utils.deleteRecursively(tempDir1) Utils.deleteRecursively(tempDir2) + SparkEnv.set(null) } test("child first") { @@ -83,4 +100,53 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { } } + test("failing to fetch classes from HTTP server should not leak resources (SPARK-6209)") { + // This is a regression test for SPARK-6209, a bug where each failed attempt to load a class + // from the driver's class server would leak a HTTP connection, causing the class server's + // thread / connection pool to be exhausted. + val conf = new SparkConf() + val securityManager = new SecurityManager(conf) + classServer = new HttpServer(conf, tempDir1, securityManager) + classServer.start() + // ExecutorClassLoader uses SparkEnv's SecurityManager, so we need to mock this + val mockEnv = mock[SparkEnv] + when(mockEnv.securityManager).thenReturn(securityManager) + SparkEnv.set(mockEnv) + // Create an ExecutorClassLoader that's configured to load classes from the HTTP server + val parentLoader = new URLClassLoader(Array.empty, null) + val classLoader = new ExecutorClassLoader(conf, classServer.uri, parentLoader, false) + classLoader.httpUrlConnectionTimeoutMillis = 500 + // Check that this class loader can actually load classes that exist + val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + // Try to perform a full GC now, since GC during the test might mask resource leaks + System.gc() + // When the original bug occurs, the test thread becomes blocked in a classloading call + // and does not respond to interrupts. Therefore, use a custom ScalaTest interruptor to + // shut down the HTTP server when the test times out + val interruptor: Interruptor = new Interruptor { + override def apply(thread: Thread): Unit = { + classServer.stop() + classServer = null + thread.interrupt() + } + } + def tryAndFailToLoadABunchOfClasses(): Unit = { + // The number of trials here should be much larger than Jetty's thread / connection limit + // in order to expose thread or connection leaks + for (i <- 1 to 1000) { + if (Thread.currentThread().isInterrupted) { + throw new InterruptedException() + } + // Incorporate the iteration number into the class name in order to avoid any response + // caching that might be added in the future + intercept[ClassNotFoundException] { + classLoader.loadClass(s"ReplFakeClassDoesNotExist$i").newInstance() + } + } + } + failAfter(10 seconds)(tryAndFailToLoadABunchOfClasses())(interruptor) + } + } diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 92e76a3fe6ca2..de762acc8fa0e 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -29,7 +29,7 @@ # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. ## -usage="Usage: spark-daemon.sh [--config ] (start|stop) " +usage="Usage: spark-daemon.sh [--config ] (start|stop|status) " # if no args specified, show usage if [ $# -le 1 ]; then @@ -129,7 +129,7 @@ run_command() { if [ -f "$pid" ]; then TARGET_ID="$(cat "$pid")" - if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then echo "$command running as process $TARGET_ID. Stop it first." exit 1 fi @@ -163,7 +163,7 @@ run_command() { echo "$newpid" > "$pid" sleep 2 # Check if the process has died; in that case we'll tail the log so the user can see - if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then + if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then echo "failed to launch $command:" tail -2 "$log" | sed 's/^/ /' echo "full log in $log" @@ -195,6 +195,23 @@ case $option in fi ;; + (status) + + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo $command is running. + exit 0 + else + echo $pid file is present but $command not running + exit 1 + fi + else + echo $command not running. + exit 2 + fi + ;; + (*) echo $usage exit 1 diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh new file mode 100755 index 0000000000000..ef1fc573d5c65 --- /dev/null +++ b/sbin/start-mesos-dispatcher.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash + +# +# 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. +# +# Starts the Mesos Cluster Dispatcher on the machine this script is executed on. +# The Mesos Cluster Dispatcher is responsible for launching the Mesos framework and +# Rest server to handle driver requests for Mesos cluster mode. +# Only one cluster dispatcher is needed per Mesos cluster. + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then + SPARK_MESOS_DISPATCHER_PORT=7077 +fi + +if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then + SPARK_MESOS_DISPATCHER_HOST=`hostname` +fi + + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.mesos.MesosClusterDispatcher 1 --host $SPARK_MESOS_DISPATCHER_HOST --port $SPARK_MESOS_DISPATCHER_PORT "$@" diff --git a/sbin/start-shuffle-service.sh b/sbin/start-shuffle-service.sh new file mode 100755 index 0000000000000..4fddcf7f95d40 --- /dev/null +++ b/sbin/start-shuffle-service.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Starts the external shuffle server on the machine this script is executed on. +# +# Usage: start-shuffle-server.sh +# +# Use the SPARK_SHUFFLE_OPTS environment variable to set shuffle server configuration. +# + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +exec "$sbin"/spark-daemon.sh start org.apache.spark.deploy.ExternalShuffleService 1 diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 2fc35309f4ca5..4c919ff76a8f5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -17,10 +17,69 @@ # limitations under the License. # -# Usage: start-slave.sh -# where is like "spark://localhost:7077" +# Starts a slave on the machine this script is executed on. +# +# Environment Variables +# +# SPARK_WORKER_INSTANCES The number of worker instances to run on this +# slave. Default is 1. +# SPARK_WORKER_PORT The base port number for the first worker. If set, +# subsequent workers will increment this number. If +# unset, Spark will find a valid port number, but +# with no guarantee of a predictable pattern. +# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first +# worker. Subsequent workers will increment this +# number. Default is 8081. + +usage="Usage: start-slave.sh where is like spark://localhost:7077" + +if [ $# -lt 1 ]; then + echo $usage + echo Called as start-slave.sh $* + exit 1 +fi sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +# Determine desired worker port +if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 +fi + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$SPARK_WORKER_PORT" = "" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) + fi + WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) + + "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker $WORKER_NUM \ + --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" +} + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi + diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 76316a3067c93..24d6268815ed3 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,6 +17,8 @@ # limitations under the License. # +# Starts a slave instance on each machine specified in the conf/slaves file. + sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" @@ -57,13 +59,4 @@ if [ "$START_TACHYON" == "true" ]; then fi # Launch the slaves -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" -else - if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then - SPARK_WORKER_WEBUI_PORT=8081 - fi - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" diff --git a/sbin/stop-mesos-dispatcher.sh b/sbin/stop-mesos-dispatcher.sh new file mode 100755 index 0000000000000..cb65d95b5e524 --- /dev/null +++ b/sbin/stop-mesos-dispatcher.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# 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. +# +# Stop the Mesos Cluster dispatcher on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +. "$sbin/spark-config.sh" + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.mesos.MesosClusterDispatcher 1 + diff --git a/sbin/stop-shuffle-service.sh b/sbin/stop-shuffle-service.sh new file mode 100755 index 0000000000000..4cb6891ae27fa --- /dev/null +++ b/sbin/stop-shuffle-service.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Stops the external shuffle service on the machine this script is executed on. + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.ExternalShuffleService 1 diff --git a/sbin/stop-slave.sh b/sbin/stop-slave.sh new file mode 100755 index 0000000000000..3d1da5b254f2a --- /dev/null +++ b/sbin/stop-slave.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# A shell script to stop all workers on a single slave +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this slave. Default is 1. + +# Usage: stop-slave.sh +# Stops all slaves on this worker machine + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index 7c2201100ef97..54c9bd46803a9 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -29,10 +29,4 @@ if [ -e "$sbin"/../tachyon/bin/tachyon ]; then "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon killAll tachyon.worker.Worker fi -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/stop-slave.sh diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 0ff521706c71a..7168d5b2a8e26 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -137,9 +137,9 @@ - + - + diff --git a/sql/README.md b/sql/README.md index fbb3200a3a4b4..46aec7cef7984 100644 --- a/sql/README.md +++ b/sql/README.md @@ -12,7 +12,10 @@ Spark SQL is broken up into four subprojects: Other dependencies for developers --------------------------------- -In order to create new hive test cases , you will need to set several environmental variables. +In order to create new hive test cases (i.e. a test suite based on `HiveComparisonTest`), +you will need to setup your development environment based on the following instructions. + +If you are working with Hive 0.12.0, you will need to set several environmental variables as follows. ``` export HIVE_HOME="/hive/build/dist" @@ -20,6 +23,24 @@ export HIVE_DEV_HOME="/hive/" export HADOOP_HOME="/hadoop-1.0.4" ``` +If you are working with Hive 0.13.1, the following steps are needed: + +1. Download Hive's [0.13.1](https://hive.apache.org/downloads.html) and set `HIVE_HOME` with `export HIVE_HOME=""`. Please do not set `HIVE_DEV_HOME` (See [SPARK-4119](https://issues.apache.org/jira/browse/SPARK-4119)). +2. Set `HADOOP_HOME` with `export HADOOP_HOME=""` +3. Download all Hive 0.13.1a jars (Hive jars actually used by Spark) from [here](http://mvnrepository.com/artifact/org.spark-project.hive) and replace corresponding original 0.13.1 jars in `$HIVE_HOME/lib`. +4. Download [Kryo 2.21 jar](http://mvnrepository.com/artifact/com.esotericsoftware.kryo/kryo/2.21) (Note: 2.22 jar does not work) and [Javolution 5.5.1 jar](http://mvnrepository.com/artifact/javolution/javolution/5.5.1) to `$HIVE_HOME/lib`. +5. This step is optional. But, when generating golden answer files, if a Hive query fails and you find that Hive tries to talk to HDFS or you find weird runtime NPEs, set the following in your test suite... + +``` +val testTempDir = Utils.createTempDir() +// We have to use kryo to let Hive correctly serialize some plans. +sql("set hive.plan.serialization.format=kryo") +// Explicitly set fs to local fs. +sql(s"set fs.default.name=file://$testTempDir/") +// Ask Hive to run jobs in-process as a single map and reduce task. +sql("set mapred.job.tracker=local") +``` + Using the console ================= An interactive scala console can be invoked by running `build/sbt hive/console`. @@ -56,6 +77,6 @@ res2: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,v You can also build further queries on top of these `DataFrames` using the query DSL. ``` -scala> query.where('key > 30).select(avg('key)).collect() +scala> query.where(query("key") > 30).select(avg(query("key"))).collect() res3: Array[org.apache.spark.sql.Row] = Array([274.79025423728814]) ``` diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 3dea2ee76542f..5c322d032d474 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -50,6 +50,11 @@ spark-core_${scala.binary.version} ${project.version} + + org.apache.spark + spark-unsafe_${scala.binary.version} + ${project.version} + org.scalacheck scalacheck_${scala.binary.version} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java new file mode 100644 index 0000000000000..299ff3728a6d9 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java @@ -0,0 +1,259 @@ +/* + * 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.sql.catalyst.expressions; + +import java.util.Arrays; +import java.util.Iterator; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.map.BytesToBytesMap; +import org.apache.spark.unsafe.memory.MemoryLocation; +import org.apache.spark.unsafe.memory.TaskMemoryManager; + +/** + * Unsafe-based HashMap for performing aggregations where the aggregated values are fixed-width. + * + * This map supports a maximum of 2 billion keys. + */ +public final class UnsafeFixedWidthAggregationMap { + + /** + * An empty aggregation buffer, encoded in UnsafeRow format. When inserting a new key into the + * map, we copy this buffer and use it as the value. + */ + private final long[] emptyAggregationBuffer; + + private final StructType aggregationBufferSchema; + + private final StructType groupingKeySchema; + + /** + * Encodes grouping keys as UnsafeRows. + */ + private final UnsafeRowConverter groupingKeyToUnsafeRowConverter; + + /** + * A hashmap which maps from opaque bytearray keys to bytearray values. + */ + private final BytesToBytesMap map; + + /** + * Re-used pointer to the current aggregation buffer + */ + private final UnsafeRow currentAggregationBuffer = new UnsafeRow(); + + /** + * Scratch space that is used when encoding grouping keys into UnsafeRow format. + * + * By default, this is a 1MB array, but it will grow as necessary in case larger keys are + * encountered. + */ + private long[] groupingKeyConversionScratchSpace = new long[1024 / 8]; + + private final boolean enablePerfMetrics; + + /** + * @return true if UnsafeFixedWidthAggregationMap supports grouping keys with the given schema, + * false otherwise. + */ + public static boolean supportsGroupKeySchema(StructType schema) { + for (StructField field: schema.fields()) { + if (!UnsafeRow.readableFieldTypes.contains(field.dataType())) { + return false; + } + } + return true; + } + + /** + * @return true if UnsafeFixedWidthAggregationMap supports aggregation buffers with the given + * schema, false otherwise. + */ + public static boolean supportsAggregationBufferSchema(StructType schema) { + for (StructField field: schema.fields()) { + if (!UnsafeRow.settableFieldTypes.contains(field.dataType())) { + return false; + } + } + return true; + } + + /** + * Create a new UnsafeFixedWidthAggregationMap. + * + * @param emptyAggregationBuffer the default value for new keys (a "zero" of the agg. function) + * @param aggregationBufferSchema the schema of the aggregation buffer, used for row conversion. + * @param groupingKeySchema the schema of the grouping key, used for row conversion. + * @param memoryManager the memory manager used to allocate our Unsafe memory structures. + * @param initialCapacity the initial capacity of the map (a sizing hint to avoid re-hashing). + * @param enablePerfMetrics if true, performance metrics will be recorded (has minor perf impact) + */ + public UnsafeFixedWidthAggregationMap( + Row emptyAggregationBuffer, + StructType aggregationBufferSchema, + StructType groupingKeySchema, + TaskMemoryManager memoryManager, + int initialCapacity, + boolean enablePerfMetrics) { + this.emptyAggregationBuffer = + convertToUnsafeRow(emptyAggregationBuffer, aggregationBufferSchema); + this.aggregationBufferSchema = aggregationBufferSchema; + this.groupingKeyToUnsafeRowConverter = new UnsafeRowConverter(groupingKeySchema); + this.groupingKeySchema = groupingKeySchema; + this.map = new BytesToBytesMap(memoryManager, initialCapacity, enablePerfMetrics); + this.enablePerfMetrics = enablePerfMetrics; + } + + /** + * Convert a Java object row into an UnsafeRow, allocating it into a new long array. + */ + private static long[] convertToUnsafeRow(Row javaRow, StructType schema) { + final UnsafeRowConverter converter = new UnsafeRowConverter(schema); + final long[] unsafeRow = new long[converter.getSizeRequirement(javaRow)]; + final long writtenLength = + converter.writeRow(javaRow, unsafeRow, PlatformDependent.LONG_ARRAY_OFFSET); + assert (writtenLength == unsafeRow.length): "Size requirement calculation was wrong!"; + return unsafeRow; + } + + /** + * Return the aggregation buffer for the current group. For efficiency, all calls to this method + * return the same object. + */ + public UnsafeRow getAggregationBuffer(Row groupingKey) { + final int groupingKeySize = groupingKeyToUnsafeRowConverter.getSizeRequirement(groupingKey); + // Make sure that the buffer is large enough to hold the key. If it's not, grow it: + if (groupingKeySize > groupingKeyConversionScratchSpace.length) { + // This new array will be initially zero, so there's no need to zero it out here + groupingKeyConversionScratchSpace = new long[groupingKeySize]; + } else { + // Zero out the buffer that's used to hold the current row. This is necessary in order + // to ensure that rows hash properly, since garbage data from the previous row could + // otherwise end up as padding in this row. As a performance optimization, we only zero out + // the portion of the buffer that we'll actually write to. + Arrays.fill(groupingKeyConversionScratchSpace, 0, groupingKeySize, 0); + } + final long actualGroupingKeySize = groupingKeyToUnsafeRowConverter.writeRow( + groupingKey, + groupingKeyConversionScratchSpace, + PlatformDependent.LONG_ARRAY_OFFSET); + assert (groupingKeySize == actualGroupingKeySize) : "Size requirement calculation was wrong!"; + + // Probe our map using the serialized key + final BytesToBytesMap.Location loc = map.lookup( + groupingKeyConversionScratchSpace, + PlatformDependent.LONG_ARRAY_OFFSET, + groupingKeySize); + if (!loc.isDefined()) { + // This is the first time that we've seen this grouping key, so we'll insert a copy of the + // empty aggregation buffer into the map: + loc.putNewKey( + groupingKeyConversionScratchSpace, + PlatformDependent.LONG_ARRAY_OFFSET, + groupingKeySize, + emptyAggregationBuffer, + PlatformDependent.LONG_ARRAY_OFFSET, + emptyAggregationBuffer.length + ); + } + + // Reset the pointer to point to the value that we just stored or looked up: + final MemoryLocation address = loc.getValueAddress(); + currentAggregationBuffer.pointTo( + address.getBaseObject(), + address.getBaseOffset(), + aggregationBufferSchema.length(), + aggregationBufferSchema + ); + return currentAggregationBuffer; + } + + /** + * Mutable pair object returned by {@link UnsafeFixedWidthAggregationMap#iterator()}. + */ + public static class MapEntry { + private MapEntry() { }; + public final UnsafeRow key = new UnsafeRow(); + public final UnsafeRow value = new UnsafeRow(); + } + + /** + * Returns an iterator over the keys and values in this map. + * + * For efficiency, each call returns the same object. + */ + public Iterator iterator() { + return new Iterator() { + + private final MapEntry entry = new MapEntry(); + private final Iterator mapLocationIterator = map.iterator(); + + @Override + public boolean hasNext() { + return mapLocationIterator.hasNext(); + } + + @Override + public MapEntry next() { + final BytesToBytesMap.Location loc = mapLocationIterator.next(); + final MemoryLocation keyAddress = loc.getKeyAddress(); + final MemoryLocation valueAddress = loc.getValueAddress(); + entry.key.pointTo( + keyAddress.getBaseObject(), + keyAddress.getBaseOffset(), + groupingKeySchema.length(), + groupingKeySchema + ); + entry.value.pointTo( + valueAddress.getBaseObject(), + valueAddress.getBaseOffset(), + aggregationBufferSchema.length(), + aggregationBufferSchema + ); + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Free the unsafe memory associated with this map. + */ + public void free() { + map.free(); + } + + @SuppressWarnings("UseOfSystemOutOrSystemErr") + public void printPerfMetrics() { + if (!enablePerfMetrics) { + throw new IllegalStateException("Perf metrics not enabled"); + } + System.out.println("Average probes per lookup: " + map.getAverageProbesPerLookup()); + System.out.println("Number of hash collisions: " + map.getNumHashCollisions()); + System.out.println("Time spent resizing (ns): " + map.getTimeSpentResizingNs()); + System.out.println("Total memory consumption (bytes): " + map.getTotalMemoryConsumption()); + } + +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java new file mode 100644 index 0000000000000..bb546b3086b33 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -0,0 +1,435 @@ +/* + * 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.sql.catalyst.expressions; + +import scala.collection.Map; +import scala.collection.Seq; +import scala.collection.mutable.ArraySeq; + +import javax.annotation.Nullable; +import java.math.BigDecimal; +import java.sql.Date; +import java.util.*; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.DataType; +import static org.apache.spark.sql.types.DataTypes.*; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.UTF8String; +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.bitset.BitSetMethods; + +/** + * An Unsafe implementation of Row which is backed by raw memory instead of Java objects. + * + * Each tuple has three parts: [null bit set] [values] [variable length portion] + * + * The bit set is used for null tracking and is aligned to 8-byte word boundaries. It stores + * one bit per field. + * + * In the `values` region, we store one 8-byte word per field. For fields that hold fixed-length + * primitive types, such as long, double, or int, we store the value directly in the word. For + * fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the + * base address of the row) that points to the beginning of the variable-length field. + * + * Instances of `UnsafeRow` act as pointers to row data stored in this format. + */ +public final class UnsafeRow implements MutableRow { + + private Object baseObject; + private long baseOffset; + + Object getBaseObject() { return baseObject; } + long getBaseOffset() { return baseOffset; } + + /** The number of fields in this row, used for calculating the bitset width (and in assertions) */ + private int numFields; + + /** The width of the null tracking bit set, in bytes */ + private int bitSetWidthInBytes; + /** + * This optional schema is required if you want to call generic get() and set() methods on + * this UnsafeRow, but is optional if callers will only use type-specific getTYPE() and setTYPE() + * methods. This should be removed after the planned InternalRow / Row split; right now, it's only + * needed by the generic get() method, which is only called internally by code that accesses + * UTF8String-typed columns. + */ + @Nullable + private StructType schema; + + private long getFieldOffset(int ordinal) { + return baseOffset + bitSetWidthInBytes + ordinal * 8L; + } + + public static int calculateBitSetWidthInBytes(int numFields) { + return ((numFields / 64) + (numFields % 64 == 0 ? 0 : 1)) * 8; + } + + /** + * Field types that can be updated in place in UnsafeRows (e.g. we support set() for these types) + */ + public static final Set settableFieldTypes; + + /** + * Fields types can be read(but not set (e.g. set() will throw UnsupportedOperationException). + */ + public static final Set readableFieldTypes; + + static { + settableFieldTypes = Collections.unmodifiableSet( + new HashSet( + Arrays.asList(new DataType[] { + NullType, + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType + }))); + + // We support get() on a superset of the types for which we support set(): + final Set _readableFieldTypes = new HashSet( + Arrays.asList(new DataType[]{ + StringType + })); + _readableFieldTypes.addAll(settableFieldTypes); + readableFieldTypes = Collections.unmodifiableSet(_readableFieldTypes); + } + + /** + * Construct a new UnsafeRow. The resulting row won't be usable until `pointTo()` has been called, + * since the value returned by this constructor is equivalent to a null pointer. + */ + public UnsafeRow() { } + + /** + * Update this UnsafeRow to point to different backing data. + * + * @param baseObject the base object + * @param baseOffset the offset within the base object + * @param numFields the number of fields in this row + * @param schema an optional schema; this is necessary if you want to call generic get() or set() + * methods on this row, but is optional if the caller will only use type-specific + * getTYPE() and setTYPE() methods. + */ + public void pointTo( + Object baseObject, + long baseOffset, + int numFields, + @Nullable StructType schema) { + assert numFields >= 0 : "numFields should >= 0"; + assert schema == null || schema.fields().length == numFields; + this.bitSetWidthInBytes = calculateBitSetWidthInBytes(numFields); + this.baseObject = baseObject; + this.baseOffset = baseOffset; + this.numFields = numFields; + this.schema = schema; + } + + private void assertIndexIsValid(int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + assert index < numFields : "index (" + index + ") should < " + numFields; + } + + @Override + public void setNullAt(int i) { + assertIndexIsValid(i); + BitSetMethods.set(baseObject, baseOffset, i); + // To preserve row equality, zero out the value when setting the column to null. + // Since this row does does not currently support updates to variable-length values, we don't + // have to worry about zeroing out that data. + PlatformDependent.UNSAFE.putLong(baseObject, getFieldOffset(i), 0); + } + + private void setNotNullAt(int i) { + assertIndexIsValid(i); + BitSetMethods.unset(baseObject, baseOffset, i); + } + + @Override + public void update(int ordinal, Object value) { + throw new UnsupportedOperationException(); + } + + @Override + public void setInt(int ordinal, int value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putInt(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setLong(int ordinal, long value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putLong(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setDouble(int ordinal, double value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putDouble(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setBoolean(int ordinal, boolean value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putBoolean(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setShort(int ordinal, short value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putShort(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setByte(int ordinal, byte value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putByte(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setFloat(int ordinal, float value) { + assertIndexIsValid(ordinal); + setNotNullAt(ordinal); + PlatformDependent.UNSAFE.putFloat(baseObject, getFieldOffset(ordinal), value); + } + + @Override + public void setString(int ordinal, String value) { + throw new UnsupportedOperationException(); + } + + @Override + public int size() { + return numFields; + } + + @Override + public int length() { + return size(); + } + + @Override + public StructType schema() { + return schema; + } + + @Override + public Object apply(int i) { + return get(i); + } + + @Override + public Object get(int i) { + assertIndexIsValid(i); + assert (schema != null) : "Schema must be defined when calling generic get() method"; + final DataType dataType = schema.fields()[i].dataType(); + // UnsafeRow is only designed to be invoked by internal code, which only invokes this generic + // get() method when trying to access UTF8String-typed columns. If we refactor the codebase to + // separate the internal and external row interfaces, then internal code can fetch strings via + // a new getUTF8String() method and we'll be able to remove this method. + if (isNullAt(i)) { + return null; + } else if (dataType == StringType) { + return getUTF8String(i); + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public boolean isNullAt(int i) { + assertIndexIsValid(i); + return BitSetMethods.isSet(baseObject, baseOffset, i); + } + + @Override + public boolean getBoolean(int i) { + assertIndexIsValid(i); + return PlatformDependent.UNSAFE.getBoolean(baseObject, getFieldOffset(i)); + } + + @Override + public byte getByte(int i) { + assertIndexIsValid(i); + return PlatformDependent.UNSAFE.getByte(baseObject, getFieldOffset(i)); + } + + @Override + public short getShort(int i) { + assertIndexIsValid(i); + return PlatformDependent.UNSAFE.getShort(baseObject, getFieldOffset(i)); + } + + @Override + public int getInt(int i) { + assertIndexIsValid(i); + return PlatformDependent.UNSAFE.getInt(baseObject, getFieldOffset(i)); + } + + @Override + public long getLong(int i) { + assertIndexIsValid(i); + return PlatformDependent.UNSAFE.getLong(baseObject, getFieldOffset(i)); + } + + @Override + public float getFloat(int i) { + assertIndexIsValid(i); + if (isNullAt(i)) { + return Float.NaN; + } else { + return PlatformDependent.UNSAFE.getFloat(baseObject, getFieldOffset(i)); + } + } + + @Override + public double getDouble(int i) { + assertIndexIsValid(i); + if (isNullAt(i)) { + return Float.NaN; + } else { + return PlatformDependent.UNSAFE.getDouble(baseObject, getFieldOffset(i)); + } + } + + public UTF8String getUTF8String(int i) { + assertIndexIsValid(i); + final UTF8String str = new UTF8String(); + final long offsetToStringSize = getLong(i); + final int stringSizeInBytes = + (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + offsetToStringSize); + final byte[] strBytes = new byte[stringSizeInBytes]; + PlatformDependent.copyMemory( + baseObject, + baseOffset + offsetToStringSize + 8, // The `+ 8` is to skip past the size to get the data + strBytes, + PlatformDependent.BYTE_ARRAY_OFFSET, + stringSizeInBytes + ); + str.set(strBytes); + return str; + } + + @Override + public String getString(int i) { + return getUTF8String(i).toString(); + } + + @Override + public BigDecimal getDecimal(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public Date getDate(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public Seq getSeq(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public List getList(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public Map getMap(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public scala.collection.immutable.Map getValuesMap(Seq fieldNames) { + throw new UnsupportedOperationException(); + } + + @Override + public java.util.Map getJavaMap(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public Row getStruct(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public T getAs(int i) { + throw new UnsupportedOperationException(); + } + + @Override + public T getAs(String fieldName) { + throw new UnsupportedOperationException(); + } + + @Override + public int fieldIndex(String name) { + throw new UnsupportedOperationException(); + } + + @Override + public Row copy() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean anyNull() { + return BitSetMethods.anySet(baseObject, baseOffset, bitSetWidthInBytes); + } + + @Override + public Seq toSeq() { + final ArraySeq values = new ArraySeq(numFields); + for (int fieldNumber = 0; fieldNumber < numFields; fieldNumber++) { + values.update(fieldNumber, get(fieldNumber)); + } + return values; + } + + @Override + public String toString() { + return mkString("[", ",", "]"); + } + + @Override + public String mkString() { + return toSeq().mkString(); + } + + @Override + public String mkString(String sep) { + return toSeq().mkString(sep); + } + + @Override + public String mkString(String start, String sep, String end) { + return toSeq().mkString(start, sep, end); + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 15add84878ecf..f9992185a4563 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -30,6 +30,12 @@ class AnalysisException protected[sql] ( val startPosition: Option[Int] = None) extends Exception with Serializable { + def withPosition(line: Option[Int], startPosition: Option[Int]): AnalysisException = { + val newException = new AnalysisException(message, line, startPosition) + newException.setStackTrace(getStackTrace) + newException + } + override def getMessage: String = { val lineAnnotation = line.map(l => s" line $l").getOrElse("") val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index d794f034f5578..4190b7ffe1c8f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.{StructType, DateUtils} +import org.apache.spark.sql.types.StructType object Row { /** @@ -257,6 +257,7 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ + // TODO(davies): This is not the right default implementation, we use Int as Date internally def getDate(i: Int): java.sql.Date = apply(i).asInstanceOf[java.sql.Date] /** @@ -305,6 +306,38 @@ trait Row extends Serializable { */ def getAs[T](i: Int): T = apply(i).asInstanceOf[T] + /** + * Returns the value of a given fieldName. + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + * @throws ClassCastException when data type does not match. + */ + def getAs[T](fieldName: String): T = getAs[T](fieldIndex(fieldName)) + + /** + * Returns the index of a given field name. + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + */ + def fieldIndex(name: String): Int = { + throw new UnsupportedOperationException("fieldIndex on a Row without schema is undefined.") + } + + /** + * Returns a Map(name -> value) for the requested fieldNames + * + * @throws UnsupportedOperationException when schema is not defined. + * @throws IllegalArgumentException when fieldName do not exist. + * @throws ClassCastException when data type does not match. + */ + def getValuesMap[T](fieldNames: Seq[String]): Map[String, T] = { + fieldNames.map { name => + name -> getAs[T](name) + }.toMap + } + override def toString(): String = s"[${this.mkString(",")}]" /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala index 366be00473d1c..2eb3e167baad5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/AbstractSparkSQLParser.scala @@ -25,14 +25,10 @@ import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.plans.logical._ -private[sql] object KeywordNormalizer { - def apply(str: String) = str.toLowerCase() -} - private[sql] abstract class AbstractSparkSQLParser extends StandardTokenParsers with PackratParsers { - def apply(input: String): LogicalPlan = { + def parse(input: String): LogicalPlan = { // Initialize the Keywords. lexical.initialize(reservedWords) phrase(start)(new lexical.Scanner(input)) match { @@ -42,7 +38,7 @@ private[sql] abstract class AbstractSparkSQLParser } protected case class Keyword(str: String) { - def normalize = KeywordNormalizer(str) + def normalize: String = lexical.normalizeKeyword(str) def parser: Parser[String] = normalize } @@ -81,7 +77,7 @@ private[sql] abstract class AbstractSparkSQLParser class SqlLexical extends StdLexical { case class FloatLit(chars: String) extends Token { - override def toString = chars + override def toString: String = chars } /* This is a work around to support the lazy setting */ @@ -90,13 +86,16 @@ class SqlLexical extends StdLexical { reserved ++= keywords } + /* Normal the keyword string */ + def normalizeKeyword(str: String): String = str.toLowerCase + delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~", "<=>" ) protected override def processIdent(name: String) = { - val token = KeywordNormalizer(name) + val token = normalizeKeyword(name) if (reserved contains token) Keyword(token) else Identifier(name) } @@ -120,7 +119,7 @@ class SqlLexical extends StdLexical { | failure("illegal character") ) - override def identChar = letter | elem('_') + override def identChar: Parser[Elem] = letter | elem('_') override def whitespace: Parser[Any] = ( whitespaceChar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala new file mode 100644 index 0000000000000..a13e2f36a1a1f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -0,0 +1,352 @@ +/* + * 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.sql.catalyst + +import java.lang.{Iterable => JavaIterable} +import java.util.{Map => JavaMap} + +import scala.collection.mutable.HashMap + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +/** + * Functions to convert Scala types to Catalyst types and vice versa. + */ +object CatalystTypeConverters { + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + + /** + * Converts Scala objects to catalyst rows / types. This method is slow, and for batch + * conversion you should be using converter produced by createToCatalystConverter. + * Note: This is always called after schemaFor has been called. + * This ordering is important for UDT registration. + */ + def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (obj, udt: UserDefinedType[_]) => + udt.serialize(obj) + + case (o: Option[_], _) => + o.map(convertToCatalyst(_, dataType)).orNull + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToCatalyst(_, arrayType.elementType)) + + case (jit: JavaIterable[_], arrayType: ArrayType) => { + val iter = jit.iterator + var listOfItems: List[Any] = List() + while (iter.hasNext) { + val item = iter.next() + listOfItems :+= convertToCatalyst(item, arrayType.elementType) + } + listOfItems + } + + case (s: Array[_], arrayType: ArrayType) => + s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) + } + + case (jmap: JavaMap[_, _], mapType: MapType) => + val iter = jmap.entrySet.iterator + var listOfEntries: List[(Any, Any)] = List() + while (iter.hasNext) { + val entry = iter.next() + listOfEntries :+= (convertToCatalyst(entry.getKey, mapType.keyType), + convertToCatalyst(entry.getValue, mapType.valueType)) + } + listOfEntries.toMap + + case (p: Product, structType: StructType) => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = convertToCatalyst(iter.next(), structType.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case (d: String, _) => + UTF8String(d) + + case (d: BigDecimal, _) => + Decimal(d) + + case (d: java.math.BigDecimal, _) => + Decimal(d) + + case (d: java.sql.Date, _) => + DateUtils.fromJavaDate(d) + + case (r: Row, structType: StructType) => + val converters = structType.fields.map { + f => (item: Any) => convertToCatalyst(item, f.dataType) + } + convertRowWithConverters(r, structType, converters) + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Scala objects to the specified catalyst type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToCatalystConverter(dataType: DataType): Any => Any = { + def extractOption(item: Any): Any = item match { + case opt: Option[_] => opt.orNull + case other => other + } + + dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item) => extractOption(item) match { + case null => null + case other => udt.serialize(other) + } + + case arrayType: ArrayType => + val elementConverter = createToCatalystConverter(arrayType.elementType) + (item: Any) => { + extractOption(item) match { + case a: Array[_] => a.toSeq.map(elementConverter) + case s: Seq[_] => s.map(elementConverter) + case i: JavaIterable[_] => { + val iter = i.iterator + var convertedIterable: List[Any] = List() + while (iter.hasNext) { + val item = iter.next() + convertedIterable :+= elementConverter(item) + } + convertedIterable + } + case null => null + } + } + + case mapType: MapType => + val keyConverter = createToCatalystConverter(mapType.keyType) + val valueConverter = createToCatalystConverter(mapType.valueType) + (item: Any) => { + extractOption(item) match { + case m: Map[_, _] => + m.map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + + case jmap: JavaMap[_, _] => + val iter = jmap.entrySet.iterator + val convertedMap: HashMap[Any, Any] = HashMap() + while (iter.hasNext) { + val entry = iter.next() + convertedMap(keyConverter(entry.getKey)) = valueConverter(entry.getValue) + } + convertedMap + + case null => null + } + } + + case structType: StructType => + val converters = structType.fields.map(f => createToCatalystConverter(f.dataType)) + (item: Any) => { + extractOption(item) match { + case r: Row => + convertRowWithConverters(r, structType, converters) + + case p: Product => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = converters(idx)(iter.next()) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case null => + null + } + } + + case dateType: DateType => (item: Any) => extractOption(item) match { + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case other => other + } + + case dataType: StringType => (item: Any) => extractOption(item) match { + case s: String => UTF8String(s) + case other => other + } + + case _ => + (item: Any) => extractOption(item) match { + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case other => other + } + } + } + + /** + * Converts Scala objects to catalyst rows / types. + * + * Note: This should be called before do evaluation on Row + * (It does not support UDT) + * This is used to create an RDD or test results with correct types for Catalyst. + */ + def convertToCatalyst(a: Any): Any = a match { + case s: String => UTF8String(s) + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case seq: Seq[Any] => seq.map(convertToCatalyst) + case r: Row => Row(r.toSeq.map(convertToCatalyst): _*) + case arr: Array[Any] => arr.toSeq.map(convertToCatalyst).toArray + case m: Map[Any, Any] => + m.map { case (k, v) => (convertToCatalyst(k), convertToCatalyst(v)) }.toMap + case other => other + } + + /** + * Converts Catalyst types used internally in rows to standard Scala types + * This method is slow, and for batch conversion you should be using converter + * produced by createToScalaConverter. + */ + def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (d, udt: UserDefinedType[_]) => + udt.deserialize(d) + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToScala(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) + } + + case (r: Row, s: StructType) => + convertRowToScala(r, s) + + case (d: Decimal, _: DecimalType) => + d.toJavaBigDecimal + + case (i: Int, DateType) => + DateUtils.toJavaDate(i) + + case (s: UTF8String, StringType) => + s.toString() + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Catalyst types to Scala type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToScalaConverter(dataType: DataType): Any => Any = dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item: Any) => if (item == null) null else udt.deserialize(item) + + case arrayType: ArrayType => + val elementConverter = createToScalaConverter(arrayType.elementType) + (item: Any) => if (item == null) null else item.asInstanceOf[Seq[_]].map(elementConverter) + + case mapType: MapType => + val keyConverter = createToScalaConverter(mapType.keyType) + val valueConverter = createToScalaConverter(mapType.valueType) + (item: Any) => if (item == null) { + null + } else { + item.asInstanceOf[Map[_, _]].map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + } + + case s: StructType => + val converters = s.fields.map(f => createToScalaConverter(f.dataType)) + (item: Any) => { + if (item == null) { + null + } else { + convertRowWithConverters(item.asInstanceOf[Row], s, converters) + } + } + + case _: DecimalType => + (item: Any) => item match { + case d: Decimal => d.toJavaBigDecimal + case other => other + } + + case DateType => + (item: Any) => item match { + case i: Int => DateUtils.toJavaDate(i) + case other => other + } + + case StringType => + (item: Any) => item match { + case s: UTF8String => s.toString() + case other => other + } + + case other => + (item: Any) => item + } + + def convertRowToScala(r: Row, schema: StructType): Row = { + val ar = new Array[Any](r.size) + var idx = 0 + while (idx < r.size) { + ar(idx) = convertToScala(r(idx), schema.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } + + /** + * Converts a row by applying the provided set of converter functions. It is used for both + * toScala and toCatalyst conversions. + */ + private[sql] def convertRowWithConverters( + row: Row, + schema: StructType, + converters: Array[Any => Any]): Row = { + val ar = new Array[Any](row.size) + var idx = 0 + while (idx < row.size) { + ar(idx) = converters(idx)(row(idx)) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala new file mode 100644 index 0000000000000..977003493d471 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/Dialect.scala @@ -0,0 +1,33 @@ +/* + * 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.sql.catalyst + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * Root class of SQL Parser Dialect, and we don't guarantee the binary + * compatibility for the future release, let's keep it as the internal + * interface for advanced user. + * + */ +@DeveloperApi +abstract class Dialect { + // this is the main function that will be implemented by sql parser. + def parse(sqlText: String): LogicalPlan +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d6126c24fc50d..c52965507c715 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp - import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -46,56 +44,6 @@ trait ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) - /** - * Converts Scala objects to catalyst rows / types. - * Note: This is always called after schemaFor has been called. - * This ordering is important for UDT registration. - */ - def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (obj, udt: UserDefinedType[_]) => udt.serialize(obj) - case (o: Option[_], _) => o.map(convertToCatalyst(_, dataType)).orNull - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, arrayType.elementType)) - case (s: Array[_], arrayType: ArrayType) => if (arrayType.elementType.isPrimitive) { - s.toSeq - } else { - s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) - } - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) - } - case (p: Product, structType: StructType) => - new GenericRow( - p.productIterator.toSeq.zip(structType.fields).map { case (elem, field) => - convertToCatalyst(elem, field.dataType) - }.toArray) - case (d: BigDecimal, _) => Decimal(d) - case (d: java.math.BigDecimal, _) => Decimal(d) - case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) - case (other, _) => other - } - - /** Converts Catalyst types used internally in rows to standard Scala types */ - def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (d, udt: UserDefinedType[_]) => udt.deserialize(d) - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToScala(_, arrayType.elementType)) - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) - } - case (r: Row, s: StructType) => convertRowToScala(r, s) - case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal - case (i: Int, DateType) => DateUtils.toJavaDate(i) - case (other, _) => other - } - - def convertRowToScala(r: Row, schema: StructType): Row = { - // TODO: This is very slow!!! - new GenericRowWithSchema( - r.toSeq.zip(schema.fields.map(_.dataType)) - .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray, schema) - } - /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => @@ -160,7 +108,7 @@ trait ScalaReflection { StructField(p.name.toString, dataType, nullable) }), nullable = true) case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) @@ -179,24 +127,27 @@ trait ScalaReflection { case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + case other => + throw new UnsupportedOperationException(s"Schema for type $other is not supported") } } def typeOfObject: PartialFunction[Any, DataType] = { // The data type can be determined without ambiguity. - case obj: BooleanType.JvmType => BooleanType - case obj: BinaryType.JvmType => BinaryType - case obj: StringType.JvmType => StringType - case obj: ByteType.JvmType => ByteType - case obj: ShortType.JvmType => ShortType - case obj: IntegerType.JvmType => IntegerType - case obj: LongType.JvmType => LongType - case obj: FloatType.JvmType => FloatType - case obj: DoubleType.JvmType => DoubleType + case obj: Boolean => BooleanType + case obj: Array[Byte] => BinaryType + case obj: String => StringType + case obj: UTF8String => StringType + case obj: Byte => ByteType + case obj: Short => ShortType + case obj: Int => IntegerType + case obj: Long => LongType + case obj: Float => FloatType + case obj: Double => DoubleType case obj: java.sql.Date => DateType case obj: java.math.BigDecimal => DecimalType.Unlimited case obj: Decimal => DecimalType.Unlimited - case obj: TimestampType.JvmType => TimestampType + case obj: java.sql.Timestamp => TimestampType case null => NullType // For other cases, there is no obvious mapping from the type of the given object to a // Catalyst data type. A user should provide his/her specific rules diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 54ab13ca352d2..1d3a2dc0d9bb0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.types._ * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ -class SqlParser extends AbstractSparkSQLParser { +class SqlParser extends AbstractSparkSQLParser with DataTypeParser { def parseExpression(input: String): Expression = { // Initialize the Keywords. @@ -61,11 +61,8 @@ class SqlParser extends AbstractSparkSQLParser { protected val CAST = Keyword("CAST") protected val COALESCE = Keyword("COALESCE") protected val COUNT = Keyword("COUNT") - protected val DATE = Keyword("DATE") - protected val DECIMAL = Keyword("DECIMAL") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") - protected val DOUBLE = Keyword("DOUBLE") protected val ELSE = Keyword("ELSE") protected val END = Keyword("END") protected val EXCEPT = Keyword("EXCEPT") @@ -78,7 +75,6 @@ class SqlParser extends AbstractSparkSQLParser { protected val IF = Keyword("IF") protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") - protected val INT = Keyword("INT") protected val INSERT = Keyword("INSERT") protected val INTERSECT = Keyword("INTERSECT") protected val INTO = Keyword("INTO") @@ -105,18 +101,17 @@ class SqlParser extends AbstractSparkSQLParser { protected val SELECT = Keyword("SELECT") protected val SEMI = Keyword("SEMI") protected val SQRT = Keyword("SQRT") - protected val STRING = Keyword("STRING") protected val SUBSTR = Keyword("SUBSTR") protected val SUBSTRING = Keyword("SUBSTRING") protected val SUM = Keyword("SUM") protected val TABLE = Keyword("TABLE") protected val THEN = Keyword("THEN") - protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") protected val UNION = Keyword("UNION") protected val UPPER = Keyword("UPPER") protected val WHEN = Keyword("WHEN") protected val WHERE = Keyword("WHERE") + protected val WITH = Keyword("WITH") protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { exprs.zipWithIndex.map { @@ -126,13 +121,14 @@ class SqlParser extends AbstractSparkSQLParser { } protected lazy val start: Parser[LogicalPlan] = - ( (select | ("(" ~> select <~ ")")) * - ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } - | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } - | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} - | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } - ) - | insert + start1 | insert | cte + + protected lazy val start1: Parser[LogicalPlan] = + (select | ("(" ~> select <~ ")")) * + ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } + | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } + | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} + | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) protected lazy val select: Parser[LogicalPlan] = @@ -145,7 +141,7 @@ class SqlParser extends AbstractSparkSQLParser { sortType.? ~ (LIMIT ~> expression).? ^^ { case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => - val base = r.getOrElse(NoRelation) + val base = r.getOrElse(OneRowRelation) val withFilter = f.map(Filter(_, base)).getOrElse(base) val withProjection = g .map(Aggregate(_, assignAliases(p), withFilter)) @@ -159,7 +155,12 @@ class SqlParser extends AbstractSparkSQLParser { protected lazy val insert: Parser[LogicalPlan] = INSERT ~> (OVERWRITE ^^^ true | INTO ^^^ false) ~ (TABLE ~> relation) ~ select ^^ { - case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o, false) + } + + protected lazy val cte: Parser[LogicalPlan] = + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start1 <~ ")"), ",") ~ (start1 | insert) ^^ { + case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) } protected lazy val projection: Parser[Expression] = @@ -315,18 +316,20 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val cast: Parser[Expression] = - CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { case exp ~ t => Cast(exp, t) } + CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { + case exp ~ t => Cast(exp, t) + } protected lazy val literal: Parser[Literal] = ( numericLiteral | booleanLiteral - | stringLit ^^ {case s => Literal(s, StringType) } - | NULL ^^^ Literal(null, NullType) + | stringLit ^^ {case s => Literal.create(s, StringType) } + | NULL ^^^ Literal.create(null, NullType) ) protected lazy val booleanLiteral: Parser[Literal] = - ( TRUE ^^^ Literal(true, BooleanType) - | FALSE ^^^ Literal(false, BooleanType) + ( TRUE ^^^ Literal.create(true, BooleanType) + | FALSE ^^^ Literal.create(false, BooleanType) ) protected lazy val numericLiteral: Parser[Literal] = @@ -362,6 +365,7 @@ class SqlParser extends AbstractSparkSQLParser { protected lazy val baseExpression: Parser[Expression] = ( "*" ^^^ UnresolvedStar(None) + | ident <~ "." ~ "*" ^^ { case tableName => UnresolvedStar(Option(tableName)) } | primary ) @@ -378,28 +382,13 @@ class SqlParser extends AbstractSparkSQLParser { | "(" ~> expression <~ ")" | function | dotExpressionHeader - | ident ^^ UnresolvedAttribute + | ident ^^ {case i => UnresolvedAttribute.quoted(i)} | signedPrimary | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { - case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) - } - - protected lazy val dataType: Parser[DataType] = - ( STRING ^^^ StringType - | TIMESTAMP ^^^ TimestampType - | DOUBLE ^^^ DoubleType - | fixedDecimalType - | DECIMAL ^^^ DecimalType.Unlimited - | DATE ^^^ DateType - | INT ^^^ IntegerType - ) - - protected lazy val fixedDecimalType: Parser[DataType] = - (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + case i1 ~ i2 ~ rest => UnresolvedAttribute(Seq(i1, i2) ++ rest) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 92d3db077c5e1..7b543b6c2aa42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -27,7 +28,7 @@ import org.apache.spark.sql.types._ /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing - * when all relations are already filled in and the analyser needs only to resolve attribute + * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) @@ -37,11 +38,12 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true * [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and * a [[FunctionRegistry]]. */ -class Analyzer(catalog: Catalog, - registry: FunctionRegistry, - caseSensitive: Boolean, - maxIterations: Int = 100) - extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { +class Analyzer( + catalog: Catalog, + registry: FunctionRegistry, + caseSensitive: Boolean, + maxIterations: Int = 100) + extends RuleExecutor[LogicalPlan] with HiveTypeCoercion with CheckAnalysis { val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution @@ -58,15 +60,15 @@ class Analyzer(catalog: Catalog, ResolveReferences :: ResolveGroupingAnalytics :: ResolveSortReferences :: + ResolveGenerate :: ImplicitGenerate :: ResolveFunctions :: + ExtractWindowExpressions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: TrimGroupingAliases :: typeCoercionRules ++ - extendedResolutionRules : _*), - Batch("Remove SubQueries", fixedPoint, - EliminateSubQueries) + extendedResolutionRules : _*) ) /** @@ -141,10 +143,10 @@ class Analyzer(catalog: Catalog, case x: Expression if nonSelectedGroupExprSet.contains(x) => // if the input attribute in the Invalid Grouping Expression set of for this group // replace it with constant null - Literal(null, expr.dataType) + Literal.create(null, expr.dataType) case x if x == g.gid => // replace the groupingId with concrete value (the bit mask) - Literal(bitmask, IntegerType) + Literal.create(bitmask, IntegerType) }) result += GroupExpression(substitution) @@ -170,21 +172,36 @@ class Analyzer(catalog: Catalog, * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation) = { + def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { try { - catalog.lookupRelation(u.tableIdentifier, u.alias) + // In hive, if there is same table name in database and CTE definition, + // hive will use the table in database, not the CTE one. + // Taking into account the reasonableness and the implementation complexity, + // here use the CTE definition first, check table name only and ignore database name + cteRelations.get(u.tableIdentifier.last) + .map(relation => u.alias.map(Subquery(_, relation)).getOrElse(relation)) + .getOrElse(catalog.lookupRelation(u.tableIdentifier, u.alias)) } catch { case _: NoSuchTableException => - u.failAnalysis(s"no such table ${u.tableIdentifier}") + u.failAnalysis(s"no such table ${u.tableName}") } } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _) => - i.copy( - table = EliminateSubQueries(getTable(u))) - case u: UnresolvedRelation => - getTable(u) + def apply(plan: LogicalPlan): LogicalPlan = { + val (realPlan, cteRelations) = plan match { + // TODO allow subquery to define CTE + // Add cte table to a temp relation map,drop `with` plan and keep its child + case With(child, relations) => (child, relations) + case other => (other, Map.empty[String, LogicalPlan]) + } + + realPlan transform { + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => + i.copy( + table = EliminateSubQueries(getTable(u, cteRelations))) + case u: UnresolvedRelation => + getTable(u, cteRelations) + } } } @@ -214,6 +231,12 @@ class Analyzer(catalog: Catalog, case o => o :: Nil } Alias(c.copy(children = expandedArgs), name)() :: Nil + case Alias(c @ CreateStruct(args), name) if containsStar(args) => + val expandedArgs = args.flatMap { + case s: Star => s.expand(child.output, resolver) + case o => o :: Nil + } + Alias(c.copy(children = expandedArgs), name)() :: Nil case o => o :: Nil }, child) @@ -254,7 +277,15 @@ class Analyzer(catalog: Catalog, case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) - }.head // Only handle first case found, others will be fixed on the next pass. + }.headOption.getOrElse { // Only handle first case, others will be fixed on the next pass. + sys.error( + s""" + |Failure when resolving conflicting references in Join: + |$plan + | + |Conflicting attributes: ${conflictingAttributes.mkString(",")} + """.stripMargin) + } val attributeRewrites = AttributeMap(oldRelation.output.zip(newRelation.output)) val newRight = right transformUp { @@ -269,17 +300,19 @@ class Analyzer(catalog: Catalog, case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressionsUp { - case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) && + case u @ UnresolvedAttribute(nameParts) if nameParts.length == 1 && + resolver(nameParts(0), VirtualColumn.groupingIdName) && q.isInstanceOf[GroupingAnalytics] => // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics q.asInstanceOf[GroupingAnalytics].gid - case u @ UnresolvedAttribute(name) => + case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolveChildren(name, resolver).getOrElse(u) + val result = + withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => - resolveGetField(child, fieldName) + GetField(child, fieldName, resolver) } } @@ -299,36 +332,6 @@ class Analyzer(catalog: Catalog, */ protected def containsStar(exprs: Seq[Expression]): Boolean = exprs.exists(_.collect { case _: Star => true }.nonEmpty) - - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - protected def resolveGetField(expr: Expression, fieldName: String): Expression = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } } /** @@ -341,19 +344,16 @@ class Analyzer(catalog: Catalog, def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case s @ Sort(ordering, global, p @ Project(projectList, child)) if !s.resolved && p.resolved => - val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) - val resolved = unresolved.flatMap(child.resolve(_, resolver)) - val requiredAttributes = - AttributeSet(resolved.flatMap(_.collect { case a: Attribute => a })) + val (resolvedOrdering, missing) = resolveAndFindMissing(ordering, p, child) - val missingInProject = requiredAttributes -- p.output - if (missingInProject.nonEmpty) { + // If this rule was not a no-op, return the transformed plan, otherwise return the original. + if (missing.nonEmpty) { // Add missing attributes and then project them away after the sort. - Project(projectList.map(_.toAttribute), - Sort(ordering, global, - Project(projectList ++ missingInProject, child))) + Project(p.output, + Sort(resolvedOrdering, global, + Project(projectList ++ missing, child))) } else { - logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") + logDebug(s"Failed to find $missing in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } case s @ Sort(ordering, global, a @ Aggregate(grouping, aggs, child)) @@ -365,18 +365,54 @@ class Analyzer(catalog: Catalog, grouping.collect { case ne: NamedExpression => ne.toAttribute } ) - logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve(_, resolver)) - val missingInAggs = resolved.filterNot(a.outputSet.contains) - logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") - if (missingInAggs.nonEmpty) { + val (resolvedOrdering, missing) = resolveAndFindMissing(ordering, a, groupingRelation) + + if (missing.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. Project(a.output, - Sort(ordering, global, Aggregate(grouping, aggs ++ missingInAggs, child))) + Sort(resolvedOrdering, global, + Aggregate(grouping, aggs ++ missing, child))) } else { s // Nothing we can do here. Return original plan. } } + + /** + * Given a child and a grandchild that are present beneath a sort operator, returns + * a resolved sort ordering and a list of attributes that are missing from the child + * but are present in the grandchild. + */ + def resolveAndFindMissing( + ordering: Seq[SortOrder], + child: LogicalPlan, + grandchild: LogicalPlan): (Seq[SortOrder], Seq[Attribute]) = { + // Find any attributes that remain unresolved in the sort. + val unresolved: Seq[Seq[String]] = + ordering.flatMap(_.collect { case UnresolvedAttribute(nameParts) => nameParts }) + + // Create a map from name, to resolved attributes, when the desired name can be found + // prior to the projection. + val resolved: Map[Seq[String], NamedExpression] = + unresolved.flatMap(u => grandchild.resolve(u, resolver).map(a => u -> a)).toMap + + // Construct a set that contains all of the attributes that we need to evaluate the + // ordering. + val requiredAttributes = AttributeSet(resolved.values) + + // Figure out which ones are missing from the projection, so that we can add them and + // remove them after the sort. + val missingInProject = requiredAttributes -- child.output + + // Now that we have all the attributes we need, reconstruct a resolved ordering. + // It is important to do it here, instead of waiting for the standard resolved as adding + // attributes to the project below can actually introduce ambiquity that was not present + // before. + val resolvedOrdering = ordering.map(_ transform { + case u @ UnresolvedAttribute(name) => resolved.getOrElse(name, u) + }).asInstanceOf[Seq[SortOrder]] + + (resolvedOrdering, missingInProject.toSeq) + } } /** @@ -441,8 +477,256 @@ class Analyzer(catalog: Catalog, */ object ImplicitGenerate extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Project(Seq(Alias(g: Generator, _)), child) => - Generate(g, join = false, outer = false, None, child) + case Project(Seq(Alias(g: Generator, name)), child) => + Generate(g, join = false, outer = false, + qualifier = None, UnresolvedAttribute(name) :: Nil, child) + case Project(Seq(MultiAlias(g: Generator, names)), child) => + Generate(g, join = false, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), child) + } + } + + /** + * Resolve the Generate, if the output names specified, we will take them, otherwise + * we will try to provide the default names, which follow the same rule with Hive. + */ + object ResolveGenerate extends Rule[LogicalPlan] { + // Construct the output attributes for the generator, + // The output attribute names can be either specified or + // auto generated. + private def makeGeneratorOutput( + generator: Generator, + generatorOutput: Seq[Attribute]): Seq[Attribute] = { + val elementTypes = generator.elementTypes + + if (generatorOutput.length == elementTypes.length) { + generatorOutput.zip(elementTypes).map { + case (a, (t, nullable)) if !a.resolved => + AttributeReference(a.name, t, nullable)() + case (a, _) => a + } + } else if (generatorOutput.length == 0) { + elementTypes.zipWithIndex.map { + // keep the default column names as Hive does _c0, _c1, _cN + case ((t, nullable), i) => AttributeReference(s"_c$i", t, nullable)() + } + } else { + throw new AnalysisException( + s""" + |The number of aliases supplied in the AS clause does not match + |the number of columns output by the UDTF expected + |${elementTypes.size} aliases but got ${generatorOutput.size} + """.stripMargin) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p: Generate if !p.child.resolved || !p.generator.resolved => p + case p: Generate if p.resolved == false => + // if the generator output names are not specified, we will use the default ones. + Generate( + p.generator, + join = p.join, + outer = p.outer, + p.qualifier, + makeGeneratorOutput(p.generator, p.generatorOutput), p.child) + } + } + + /** + * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and + * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] + * operators for every distinct [[WindowSpecDefinition]]. + * + * This rule handles three cases: + * - A [[Project]] having [[WindowExpression]]s in its projectList; + * - An [[Aggregate]] having [[WindowExpression]]s in its aggregateExpressions. + * - An [[Filter]]->[[Aggregate]] pattern representing GROUP BY with a HAVING + * clause and the [[Aggregate]] has [[WindowExpression]]s in its aggregateExpressions. + * Note: If there is a GROUP BY clause in the query, aggregations and corresponding + * filters (expressions in the HAVING clause) should be evaluated before any + * [[WindowExpression]]. If a query has SELECT DISTINCT, the DISTINCT part should be + * evaluated after all [[WindowExpression]]s. + * + * For every case, the transformation works as follows: + * 1. For a list of [[Expression]]s (a projectList or an aggregateExpressions), partitions + * it two lists of [[Expression]]s, one for all [[WindowExpression]]s and another for + * all regular expressions. + * 2. For all [[WindowExpression]]s, groups them based on their [[WindowSpecDefinition]]s. + * 3. For every distinct [[WindowSpecDefinition]], creates a [[Window]] operator and inserts + * it into the plan tree. + */ + object ExtractWindowExpressions extends Rule[LogicalPlan] { + def hasWindowFunction(projectList: Seq[NamedExpression]): Boolean = + projectList.exists(hasWindowFunction) + + def hasWindowFunction(expr: NamedExpression): Boolean = { + expr.find { + case window: WindowExpression => true + case _ => false + }.isDefined + } + + /** + * From a Seq of [[NamedExpression]]s, extract window expressions and + * other regular expressions. + */ + def extract( + expressions: Seq[NamedExpression]): (Seq[NamedExpression], Seq[NamedExpression]) = { + // First, we simple partition the input expressions to two part, one having + // WindowExpressions and another one without WindowExpressions. + val (windowExpressions, regularExpressions) = expressions.partition(hasWindowFunction) + + // Then, we need to extract those regular expressions used in the WindowExpression. + // For example, when we have col1 - Sum(col2 + col3) OVER (PARTITION BY col4 ORDER BY col5), + // we need to make sure that col1 to col5 are all projected from the child of the Window + // operator. + val extractedExprBuffer = new ArrayBuffer[NamedExpression]() + def extractExpr(expr: Expression): Expression = expr match { + case ne: NamedExpression => + // If a named expression is not in regularExpressions, add extract it and replace it + // with an AttributeReference. + val missingExpr = + AttributeSet(Seq(expr)) -- (regularExpressions ++ extractedExprBuffer) + if (missingExpr.nonEmpty) { + extractedExprBuffer += ne + } + ne.toAttribute + case e: Expression if e.foldable => + e // No need to create an attribute reference if it will be evaluated as a Literal. + case e: Expression => + // For other expressions, we extract it and replace it with an AttributeReference (with + // an interal column name, e.g. "_w0"). + val withName = Alias(e, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + } + + // Now, we extract expressions from windowExpressions by using extractExpr. + val newWindowExpressions = windowExpressions.map { + _.transform { + // Extracts children expressions of a WindowFunction (input parameters of + // a WindowFunction). + case wf : WindowFunction => + val newChildren = wf.children.map(extractExpr(_)) + wf.withNewChildren(newChildren) + + // Extracts expressions from the partition spec and order spec. + case wsc @ WindowSpecDefinition(partitionSpec, orderSpec, _) => + val newPartitionSpec = partitionSpec.map(extractExpr(_)) + val newOrderSpec = orderSpec.map { so => + val newChild = extractExpr(so.child) + so.copy(child = newChild) + } + wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec) + + // Extracts AggregateExpression. For example, for SUM(x) - Sum(y) OVER (...), + // we need to extract SUM(x). + case agg: AggregateExpression => + val withName = Alias(agg, s"_w${extractedExprBuffer.length}")() + extractedExprBuffer += withName + withName.toAttribute + }.asInstanceOf[NamedExpression] + } + + (newWindowExpressions, regularExpressions ++ extractedExprBuffer) + } + + /** + * Adds operators for Window Expressions. Every Window operator handles a single Window Spec. + */ + def addWindow(windowExpressions: Seq[NamedExpression], child: LogicalPlan): LogicalPlan = { + // First, we group window expressions based on their Window Spec. + val groupedWindowExpression = windowExpressions.groupBy { expr => + val windowExpression = expr.find { + case window: WindowExpression => true + case other => false + }.map(_.asInstanceOf[WindowExpression].windowSpec) + windowExpression.getOrElse( + failAnalysis(s"$windowExpressions does not have any WindowExpression.")) + }.toSeq + + // For every Window Spec, we add a Window operator and set currentChild as the child of it. + var currentChild = child + var i = 0 + while (i < groupedWindowExpression.size) { + val (windowSpec, windowExpressions) = groupedWindowExpression(i) + // Set currentChild to the newly created Window operator. + currentChild = Window(currentChild.output, windowExpressions, windowSpec, currentChild) + + // Move to next WindowExpression. + i += 1 + } + + // We return the top operator. + currentChild + } + + // We have to use transformDown at here to make sure the rule of + // "Aggregate with Having clause" will be triggered. + def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) => + child.transform { + case plan => plan.transformExpressions { + case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) => + val errorMessage = + s"Window specification $windowName is not defined in the WINDOW clause." + val windowSpecDefinition = + windowDefinitions + .get(windowName) + .getOrElse(failAnalysis(errorMessage)) + WindowExpression(c, windowSpecDefinition) + } + } + + // Aggregate with Having clause. This rule works with an unresolved Aggregate because + // a resolved Aggregate will not have Window Functions. + case f @ Filter(condition, a @ Aggregate(groupingExprs, aggregateExprs, child)) + if child.resolved && + hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add a Filter operator for conditions in the Having clause. + val withFilter = Filter(condition, withAggregate) + val withWindow = addWindow(windowExpressions, withFilter) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + case p: LogicalPlan if !p.childrenResolved => p + + // Aggregate without Having clause. + case a @ Aggregate(groupingExprs, aggregateExprs, child) + if hasWindowFunction(aggregateExprs) && + !a.expressions.exists(!_.resolved) => + val (windowExpressions, aggregateExpressions) = extract(aggregateExprs) + // Create an Aggregate operator to evaluate aggregation functions. + val withAggregate = Aggregate(groupingExprs, aggregateExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withAggregate) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = aggregateExprs.map (_.toAttribute) + Project(finalProjectList, withWindow) + + // We only extract Window Expressions after all expressions of the Project + // have been resolved. + case p @ Project(projectList, child) + if hasWindowFunction(projectList) && !p.expressions.exists(!_.resolved) => + val (windowExpressions, regularExpressions) = extract(projectList) + // We add a project to get all needed expressions for window expressions from the child + // of the original Project operator. + val withProject = Project(regularExpressions, child) + // Add Window operators. + val withWindow = addWindow(windowExpressions, withProject) + + // Finally, generate output columns according to the original projectList. + val finalProjectList = projectList.map (_.toAttribute) + Project(finalProjectList, withWindow) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 9e6e2912e0622..18c24b651921a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -22,11 +22,13 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} /** - * Thrown by a catalog when a table cannot be found. The analzyer will rethrow the exception + * Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception * as an AnalysisException with the correct position information. */ class NoSuchTableException extends Exception +class NoSuchDatabaseException extends Exception + /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ @@ -86,12 +88,12 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { tables += ((getDbTableName(tableIdent), plan)) } - override def unregisterTable(tableIdentifier: Seq[String]) = { + override def unregisterTable(tableIdentifier: Seq[String]): Unit = { val tableIdent = processTableIdentifier(tableIdentifier) tables -= getDbTableName(tableIdent) } - override def unregisterAllTables() = { + override def unregisterAllTables(): Unit = { tables.clear() } @@ -147,8 +149,8 @@ trait OverrideCatalog extends Catalog { } abstract override def lookupRelation( - tableIdentifier: Seq[String], - alias: Option[String] = None): LogicalPlan = { + tableIdentifier: Seq[String], + alias: Option[String] = None): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val overriddenTable = overrides.get(getDBTable(tableIdent)) val tableWithQualifers = overriddenTable.map(r => Subquery(tableIdent.last, r)) @@ -201,19 +203,19 @@ trait OverrideCatalog extends Catalog { /** * A trivial catalog that returns an error when a relation is requested. Used for testing when all - * relations are already filled in and the analyser needs only to resolve attribute references. + * relations are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { - val caseSensitive: Boolean = true + override val caseSensitive: Boolean = true - def tableExists(tableIdentifier: Seq[String]): Boolean = { + override def tableExists(tableIdentifier: Seq[String]): Boolean = { throw new UnsupportedOperationException } - def lookupRelation( - tableIdentifier: Seq[String], - alias: Option[String] = None) = { + override def lookupRelation( + tableIdentifier: Seq[String], + alias: Option[String] = None): LogicalPlan = { throw new UnsupportedOperationException } @@ -221,11 +223,11 @@ object EmptyCatalog extends Catalog { throw new UnsupportedOperationException } - def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { + override def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { throw new UnsupportedOperationException } - def unregisterTable(tableIdentifier: Seq[String]): Unit = { + override def unregisterTable(tableIdentifier: Seq[String]): Unit = { throw new UnsupportedOperationException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 4e8fc892f3eea..c8288c6767004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.types._ /** * Throws user facing errors when passed invalid queries that fail to analyze. */ -class CheckAnalysis { +trait CheckAnalysis { + self: Analyzer => /** * Override to provide additional checks for correct analysis. @@ -33,17 +34,31 @@ class CheckAnalysis { */ val extendedCheckRules: Seq[LogicalPlan => Unit] = Nil - def failAnalysis(msg: String) = { + protected def failAnalysis(msg: String): Nothing = { throw new AnalysisException(msg) } - def apply(plan: LogicalPlan): Unit = { + def containsMultipleGenerators(exprs: Seq[Expression]): Boolean = { + exprs.flatMap(_.collect { + case e: Generator => true + }).length >= 1 + } + + def checkAnalysis(plan: LogicalPlan): Unit = { // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. plan.foreachUp { case operator: LogicalPlan => operator transformExpressionsUp { case a: Attribute if !a.resolved => + if (operator.childrenResolved) { + a match { + case UnresolvedAttribute(nameParts) => + // Throw errors for specific problems with get field. + operator.resolveChildren(nameParts, resolver, throwErrors = true) + } + } + val from = operator.inputSet.map(_.name).mkString(", ") a.failAnalysis(s"cannot resolve '${a.prettyString}' given input columns $from") @@ -55,6 +70,11 @@ class CheckAnalysis { failAnalysis( s"invalid expression ${b.prettyString} " + s"between ${b.left.simpleString} and ${b.right.simpleString}") + + case w @ WindowExpression(windowFunction, windowSpec) if windowSpec.validate.nonEmpty => + // The window spec is not valid. + val reason = windowSpec.validate.get + failAnalysis(s"Window specification $windowSpec is not valid because $reason") } operator match { @@ -63,7 +83,7 @@ class CheckAnalysis { s"filter expression '${f.condition.prettyString}' " + s"of type ${f.condition.dataType.simpleString} is not a boolean.") - case aggregatePlan@Aggregate(groupingExprs, aggregateExprs, child) => + case Aggregate(groupingExprs, aggregateExprs, child) => def checkValidAggregateExpression(expr: Expression): Unit = expr match { case _: AggregateExpression => // OK case e: Attribute if !groupingExprs.contains(e) => @@ -85,18 +105,28 @@ class CheckAnalysis { cleaned.foreach(checkValidAggregateExpression) - case o if o.children.nonEmpty && - !o.references.filter(_.name != "grouping__id").subsetOf(o.inputSet) => - val missingAttributes = (o.references -- o.inputSet).map(_.prettyString).mkString(",") - val input = o.inputSet.map(_.prettyString).mkString(",") + case _ => // Fallbacks to the following checks + } + + operator match { + case o if o.children.nonEmpty && o.missingInput.nonEmpty => + val missingAttributes = o.missingInput.mkString(",") + val input = o.inputSet.mkString(",") - failAnalysis(s"resolved attributes $missingAttributes missing from $input") + failAnalysis( + s"resolved attribute(s) $missingAttributes missing from $input " + + s"in operator ${operator.simpleString}") - // Catch all case o if !o.resolved => failAnalysis( s"unresolved operator ${operator.simpleString}") + case p @ Project(exprs, _) if containsMultipleGenerators(exprs) => + failAnalysis( + s"""Only a single table generating function is allowed in a SELECT clause, found: + | ${exprs.map(_.prettyString).mkString(",")}""".stripMargin) + + case _ => // Analysis successful! } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 9f334f6d42ad1..16ca5bcd57a72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -35,7 +35,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry { val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) - def registerFunction(name: String, builder: FunctionBuilder) = { + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { functionBuilders.put(name, builder) } @@ -47,7 +47,7 @@ trait OverrideFunctionRegistry extends FunctionRegistry { class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistry { val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) - def registerFunction(name: String, builder: FunctionBuilder) = { + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { functionBuilders.put(name, builder) } @@ -57,17 +57,19 @@ class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistr } /** - * A trivial catalog that returns an error when a function is requested. Used for testing when all - * functions are already filled in and the analyser needs only to resolve attribute references. + * A trivial catalog that returns an error when a function is requested. Used for testing when all + * functions are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { - def registerFunction(name: String, builder: FunctionBuilder) = ??? + override def registerFunction(name: String, builder: FunctionBuilder): Unit = { + throw new UnsupportedOperationException + } - def lookupFunction(name: String, children: Seq[Expression]): Expression = { + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } - def caseSensitive: Boolean = ??? + override def caseSensitive: Boolean = throw new UnsupportedOperationException } /** @@ -76,7 +78,7 @@ object EmptyFunctionRegistry extends FunctionRegistry { * TODO move this into util folder? */ object StringKeyHashMap { - def apply[T](caseSensitive: Boolean) = caseSensitive match { + def apply[T](caseSensitive: Boolean): StringKeyHashMap[T] = caseSensitive match { case false => new StringKeyHashMap[T](_.toLowerCase) case true => new StringKeyHashMap[T](identity) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 34ef7d28cc7f2..831fb4fe95fe7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -78,6 +78,8 @@ trait HiveTypeCoercion { FunctionArgumentConversion :: CaseWhenCoercion :: Division :: + PropagateTypes :: + ExpectedInputConversion :: Nil /** @@ -114,7 +116,7 @@ trait HiveTypeCoercion { * the appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal("NaN", StringType) + val stringNaN = Literal("NaN") def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { @@ -237,37 +239,43 @@ trait HiveTypeCoercion { a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) // we should cast all timestamp/date/string compare into string compare - case p: BinaryPredicate if p.left.dataType == StringType - && p.right.dataType == DateType => + case p: BinaryComparison if p.left.dataType == StringType && + p.right.dataType == DateType => p.makeCopy(Array(p.left, Cast(p.right, StringType))) - case p: BinaryPredicate if p.left.dataType == DateType - && p.right.dataType == StringType => + case p: BinaryComparison if p.left.dataType == DateType && + p.right.dataType == StringType => p.makeCopy(Array(Cast(p.left, StringType), p.right)) - case p: BinaryPredicate if p.left.dataType == StringType - && p.right.dataType == TimestampType => + case p: BinaryComparison if p.left.dataType == StringType && + p.right.dataType == TimestampType => p.makeCopy(Array(p.left, Cast(p.right, StringType))) - case p: BinaryPredicate if p.left.dataType == TimestampType - && p.right.dataType == StringType => + case p: BinaryComparison if p.left.dataType == TimestampType && + p.right.dataType == StringType => p.makeCopy(Array(Cast(p.left, StringType), p.right)) - case p: BinaryPredicate if p.left.dataType == TimestampType - && p.right.dataType == DateType => + case p: BinaryComparison if p.left.dataType == TimestampType && + p.right.dataType == DateType => p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) - case p: BinaryPredicate if p.left.dataType == DateType - && p.right.dataType == TimestampType => + case p: BinaryComparison if p.left.dataType == DateType && + p.right.dataType == TimestampType => p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) - case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => + case p: BinaryComparison if p.left.dataType == StringType && + p.right.dataType != StringType => p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) - case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => + case p: BinaryComparison if p.left.dataType != StringType && + p.right.dataType == StringType => p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) - case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == StringType) => + case i @ In(a, b) if a.dataType == DateType && + b.forall(_.dataType == StringType) => i.makeCopy(Array(Cast(a, StringType), b)) - case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == StringType) => + case i @ In(a, b) if a.dataType == TimestampType && + b.forall(_.dataType == StringType) => i.makeCopy(Array(Cast(a, StringType), b)) - case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == TimestampType) => + case i @ In(a, b) if a.dataType == DateType && + b.forall(_.dataType == TimestampType) => i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) - case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == DateType) => + case i @ In(a, b) if a.dataType == TimestampType && + b.forall(_.dataType == DateType) => i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) case Sum(e) if e.dataType == StringType => @@ -284,6 +292,7 @@ trait HiveTypeCoercion { * Calculates and propagates precision for fixed-precision decimals. Hive has a number of * rules for this based on the SQL standard and MS SQL: * https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf + * https://msdn.microsoft.com/en-us/library/ms190476.aspx * * In particular, if we have expressions e1 and e2 with precision/scale p1/s2 and p2/s2 * respectively, then the following operations have the following precision / scale: @@ -295,6 +304,7 @@ trait HiveTypeCoercion { * e1 * e2 p1 + p2 + 1 s1 + s2 * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) + * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) * sum(e1) p1 + 10 s1 * avg(e1) p1 + 4 s1 + 4 * @@ -310,7 +320,12 @@ trait HiveTypeCoercion { * - SHORT gets turned into DECIMAL(5, 0) * - INT gets turned into DECIMAL(10, 0) * - LONG gets turned into DECIMAL(20, 0) - * - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, + * - FLOAT and DOUBLE + * 1. Union operation: + * FLOAT gets turned into DECIMAL(7, 7), DOUBLE gets turned into DECIMAL(15, 15) (this is the + * same as Hive) + * 2. Other operation: + * FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, * but note that unlimited decimals are considered bigger than doubles in WidenTypes) */ // scalastyle:on @@ -327,76 +342,127 @@ trait HiveTypeCoercion { def isFloat(t: DataType): Boolean = t == FloatType || t == DoubleType - def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - // Skip nodes whose children have not been resolved yet - case e if !e.childrenResolved => e + // Conversion rules for float and double into fixed-precision decimals + val floatTypeToFixed: Map[DataType, DecimalType] = Map( + FloatType -> DecimalType(7, 7), + DoubleType -> DecimalType(15, 15) + ) - case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Add(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - ) - - case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Subtract(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - ) - - case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Multiply(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(p1 + p2 + 1, s1 + s2) - ) - - case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Divide(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) - ) - - case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - Cast( - Remainder(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), - DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) - ) - - case LessThan(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case GreaterThan(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), - e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => - GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) - - // Promote integers inside a binary expression with fixed-precision decimals to decimals, - // and fixed-precision decimals in an expression with floats / doubles to doubles - case b: BinaryExpression if b.left.dataType != b.right.dataType => - (b.left.dataType, b.right.dataType) match { - case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => - b.makeCopy(Array(Cast(b.left, intTypeToFixed(t)), b.right)) - case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => - b.makeCopy(Array(b.left, Cast(b.right, intTypeToFixed(t)))) - case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(b.left, Cast(b.right, DoubleType))) - case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(b.left, DoubleType), b.right)) - case _ => - b + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // fix decimal precision for union + case u @ Union(left, right) if u.childrenResolved && !u.resolved => + val castedInput = left.output.zip(right.output).map { + case (l, r) if l.dataType != r.dataType => + (l.dataType, r.dataType) match { + case (DecimalType.Fixed(p1, s1), DecimalType.Fixed(p2, s2)) => + // Union decimals with precision/scale p1/s2 and p2/s2 will be promoted to + // DecimalType(max(s1, s2) + max(p1-s1, p2-s2), max(s1, s2)) + val fixedType = DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2), max(s1, s2)) + (Alias(Cast(l, fixedType), l.name)(), Alias(Cast(r, fixedType), r.name)()) + case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => + (Alias(Cast(l, intTypeToFixed(t)), l.name)(), r) + case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => + (l, Alias(Cast(r, intTypeToFixed(t)), r.name)()) + case (t, DecimalType.Fixed(p, s)) if floatTypeToFixed.contains(t) => + (Alias(Cast(l, floatTypeToFixed(t)), l.name)(), r) + case (DecimalType.Fixed(p, s), t) if floatTypeToFixed.contains(t) => + (l, Alias(Cast(r, floatTypeToFixed(t)), r.name)()) + case _ => (l, r) + } + case other => other } - // TODO: MaxOf, MinOf, etc might want other rules + val (castedLeft, castedRight) = castedInput.unzip + + val newLeft = + if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { + Project(castedLeft, left) + } else { + left + } + + val newRight = + if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { + Project(castedRight, right) + } else { + right + } + + Union(newLeft, newRight) + + // fix decimal precision for expressions + case q => q.transformExpressions { + // Skip nodes whose children have not been resolved yet + case e if !e.childrenResolved => e - // SUM and AVERAGE are handled by the implementations of those expressions + case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Add(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Subtract(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Multiply(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 + p2 + 1, s1 + s2) + ) + + case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Divide(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) + ) + + case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Remainder(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) + ) + + case LessThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + // Promote integers inside a binary expression with fixed-precision decimals to decimals, + // and fixed-precision decimals in an expression with floats / doubles to doubles + case b: BinaryExpression if b.left.dataType != b.right.dataType => + (b.left.dataType, b.right.dataType) match { + case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => + b.makeCopy(Array(Cast(b.left, intTypeToFixed(t)), b.right)) + case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => + b.makeCopy(Array(b.left, Cast(b.right, intTypeToFixed(t)))) + case (t, DecimalType.Fixed(p, s)) if isFloat(t) => + b.makeCopy(Array(b.left, Cast(b.right, DoubleType))) + case (DecimalType.Fixed(p, s), t) if isFloat(t) => + b.makeCopy(Array(Cast(b.left, DoubleType), b.right)) + case _ => + b + } + + // TODO: MaxOf, MinOf, etc might want other rules + + // SUM and AVERAGE are handled by the implementations of those expressions + } } + } /** @@ -421,8 +487,8 @@ trait HiveTypeCoercion { // No need to change the EqualNullSafe operators, too case e: EqualNullSafe => e // Otherwise turn them to Byte types so that there exists and ordering. - case p: BinaryComparison - if p.left.dataType == BooleanType && p.right.dataType == BooleanType => + case p: BinaryComparison if p.left.dataType == BooleanType && + p.right.dataType == BooleanType => p.makeCopy(Array(Cast(p.left, ByteType), Cast(p.right, ByteType))) } } @@ -580,4 +646,22 @@ trait HiveTypeCoercion { } } + /** + * Casts types according to the expected input types for Expressions that have the trait + * `ExpectsInputTypes`. + */ + object ExpectedInputConversion extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + case e: ExpectsInputTypes if e.children.map(_.dataType) != e.expectedChildTypes => + val newC = (e.children, e.children.map(_.dataType), e.expectedChildTypes).zipped.map { + case (child, actual, expected) => + if (actual == expected) child else Cast(child, expected) + } + e.withNewChildren(newC) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 894c3500cf533..35b74024a4cab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -30,5 +30,5 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * of itself with globally unique expression ids. */ trait MultiInstanceRelation { - def newInstance(): this.type + def newInstance(): LogicalPlan } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index e95f19e69ed43..7731336d247db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -38,8 +38,16 @@ package object analysis { implicit class AnalysisErrorAt(t: TreeNode[_]) { /** Fails the analysis at the point where a specific tree node was parsed. */ - def failAnalysis(msg: String) = { + def failAnalysis(msg: String): Nothing = { throw new AnalysisException(msg, t.origin.line, t.origin.startPosition) } } + + /** Catches any AnalysisExceptions thrown by `f` and attaches `t`'s position if any. */ + def withPosition[A](t: TreeNode[_])(f: => A): A = { + try f catch { + case a: AnalysisException => + throw a.withPosition(t.origin.line, t.origin.startPosition) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a7cd4124e56f3..3f567e3e8b2a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.types.DataType /** * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully @@ -36,24 +37,34 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str case class UnresolvedRelation( tableIdentifier: Seq[String], alias: Option[String] = None) extends LeafNode { - override def output = Nil + + /** Returns a `.` separated name for this relation. */ + def tableName: String = tableIdentifier.mkString(".") + + override def output: Seq[Attribute] = Nil + override lazy val resolved = false } /** * Holds the name of an attribute that has yet to be resolved. */ -case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { - override def exprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") +case class UnresolvedAttribute(nameParts: Seq[String]) + extends Attribute with trees.LeafNode[Expression] { + + def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this - override def withNullability(newNullability: Boolean) = this - override def withQualifiers(newQualifiers: Seq[String]) = this - override def withName(newName: String) = UnresolvedAttribute(name) + override def newInstance(): UnresolvedAttribute = this + override def withNullability(newNullability: Boolean): UnresolvedAttribute = this + override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this + override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -62,17 +73,22 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } +object UnresolvedAttribute { + def apply(name: String): UnresolvedAttribute = new UnresolvedAttribute(name.split("\\.")) + def quoted(name: String): UnresolvedAttribute = new UnresolvedAttribute(Seq(name)) +} + case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { - override def dataType = throw new UnresolvedException(this, "dataType") - override def foldable = throw new UnresolvedException(this, "foldable") - override def nullable = throw new UnresolvedException(this, "nullable") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"'$name(${children.mkString(",")})" + override def toString: String = s"'$name(${children.mkString(",")})" } /** @@ -82,17 +98,17 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E trait Star extends Attribute with trees.LeafNode[Expression] { self: Product => - override def name = throw new UnresolvedException(this, "name") - override def exprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def name: String = throw new UnresolvedException(this, "name") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this - override def withNullability(newNullability: Boolean) = this - override def withQualifiers(newQualifiers: Seq[String]) = this - override def withName(newName: String) = this + override def newInstance(): Star = this + override def withNullability(newNullability: Boolean): Star = this + override def withQualifiers(newQualifiers: Seq[String]): Star = this + override def withName(newName: String): Star = this // Star gets expanded at runtime so we never evaluate a Star. override def eval(input: Row = null): EvaluatedType = @@ -125,7 +141,7 @@ case class UnresolvedStar(table: Option[String]) extends Star { } } - override def toString = table.map(_ + ".").getOrElse("") + "*" + override def toString: String = table.map(_ + ".").getOrElse("") + "*" } /** @@ -140,25 +156,25 @@ case class UnresolvedStar(table: Option[String]) extends Star { case class MultiAlias(child: Expression, names: Seq[String]) extends Attribute with trees.UnaryNode[Expression] { - override def name = throw new UnresolvedException(this, "name") + override def name: String = throw new UnresolvedException(this, "name") - override def exprId = throw new UnresolvedException(this, "exprId") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") - override def dataType = throw new UnresolvedException(this, "dataType") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") - override def nullable = throw new UnresolvedException(this, "nullable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") - override def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - override def newInstance() = this + override def newInstance(): MultiAlias = this - override def withNullability(newNullability: Boolean) = this + override def withNullability(newNullability: Boolean): MultiAlias = this - override def withQualifiers(newQualifiers: Seq[String]) = this + override def withQualifiers(newQualifiers: Seq[String]): MultiAlias = this - override def withName(newName: String) = this + override def withName(newName: String): MultiAlias = this override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") @@ -175,17 +191,17 @@ case class MultiAlias(child: Expression, names: Seq[String]) */ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star { override def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = expressions - override def toString = expressions.mkString("ResolvedStar(", ", ", ")") + override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") } case class UnresolvedGetField(child: Expression, fieldName: String) extends UnaryExpression { - override def dataType = throw new UnresolvedException(this, "dataType") - override def foldable = throw new UnresolvedException(this, "foldable") - override def nullable = throw new UnresolvedException(this, "nullable") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"$child.$fieldName" + override def toString: String = s"$child.$fieldName" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 51a09ac0e1249..fa6cc7a1a36cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} @@ -61,60 +61,60 @@ package object dsl { trait ImplicitOperators { def expr: Expression - def unary_- = UnaryMinus(expr) - def unary_! = Not(expr) - def unary_~ = BitwiseNot(expr) - - def + (other: Expression) = Add(expr, other) - def - (other: Expression) = Subtract(expr, other) - def * (other: Expression) = Multiply(expr, other) - def / (other: Expression) = Divide(expr, other) - def % (other: Expression) = Remainder(expr, other) - def & (other: Expression) = BitwiseAnd(expr, other) - def | (other: Expression) = BitwiseOr(expr, other) - def ^ (other: Expression) = BitwiseXor(expr, other) - - def && (other: Expression) = And(expr, other) - def || (other: Expression) = Or(expr, other) - - def < (other: Expression) = LessThan(expr, other) - def <= (other: Expression) = LessThanOrEqual(expr, other) - def > (other: Expression) = GreaterThan(expr, other) - def >= (other: Expression) = GreaterThanOrEqual(expr, other) - def === (other: Expression) = EqualTo(expr, other) - def <=> (other: Expression) = EqualNullSafe(expr, other) - def !== (other: Expression) = Not(EqualTo(expr, other)) - - def in(list: Expression*) = In(expr, list) - - def like(other: Expression) = Like(expr, other) - def rlike(other: Expression) = RLike(expr, other) - def contains(other: Expression) = Contains(expr, other) - def startsWith(other: Expression) = StartsWith(expr, other) - def endsWith(other: Expression) = EndsWith(expr, other) - def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + def unary_- : Expression= UnaryMinus(expr) + def unary_! : Predicate = Not(expr) + def unary_~ : Expression = BitwiseNot(expr) + + def + (other: Expression): Expression = Add(expr, other) + def - (other: Expression): Expression = Subtract(expr, other) + def * (other: Expression): Expression = Multiply(expr, other) + def / (other: Expression): Expression = Divide(expr, other) + def % (other: Expression): Expression = Remainder(expr, other) + def & (other: Expression): Expression = BitwiseAnd(expr, other) + def | (other: Expression): Expression = BitwiseOr(expr, other) + def ^ (other: Expression): Expression = BitwiseXor(expr, other) + + def && (other: Expression): Predicate = And(expr, other) + def || (other: Expression): Predicate = Or(expr, other) + + def < (other: Expression): Predicate = LessThan(expr, other) + def <= (other: Expression): Predicate = LessThanOrEqual(expr, other) + def > (other: Expression): Predicate = GreaterThan(expr, other) + def >= (other: Expression): Predicate = GreaterThanOrEqual(expr, other) + def === (other: Expression): Predicate = EqualTo(expr, other) + def <=> (other: Expression): Predicate = EqualNullSafe(expr, other) + def !== (other: Expression): Predicate = Not(EqualTo(expr, other)) + + def in(list: Expression*): Expression = In(expr, list) + + def like(other: Expression): Expression = Like(expr, other) + def rlike(other: Expression): Expression = RLike(expr, other) + def contains(other: Expression): Expression = Contains(expr, other) + def startsWith(other: Expression): Expression = StartsWith(expr, other) + def endsWith(other: Expression): Expression = EndsWith(expr, other) + def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression = Substring(expr, pos, len) - def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)): Expression = Substring(expr, pos, len) - def isNull = IsNull(expr) - def isNotNull = IsNotNull(expr) + def isNull: Predicate = IsNull(expr) + def isNotNull: Predicate = IsNotNull(expr) - def getItem(ordinal: Expression) = GetItem(expr, ordinal) - def getField(fieldName: String) = UnresolvedGetField(expr, fieldName) + def getItem(ordinal: Expression): Expression = GetItem(expr, ordinal) + def getField(fieldName: String): UnresolvedGetField = UnresolvedGetField(expr, fieldName) - def cast(to: DataType) = Cast(expr, to) + def cast(to: DataType): Expression = Cast(expr, to) - def asc = SortOrder(expr, Ascending) - def desc = SortOrder(expr, Descending) + def asc: SortOrder = SortOrder(expr, Ascending) + def desc: SortOrder = SortOrder(expr, Descending) - def as(alias: String) = Alias(expr, alias)() - def as(alias: Symbol) = Alias(expr, alias.name)() + def as(alias: String): NamedExpression = Alias(expr, alias)() + def as(alias: Symbol): NamedExpression = Alias(expr, alias.name)() } trait ExpressionConversions { implicit class DslExpression(e: Expression) extends ImplicitOperators { - def expr = e + def expr: Expression = e } implicit def booleanToLiteral(b: Boolean): Literal = Literal(b) @@ -144,94 +144,100 @@ package object dsl { } } - def sum(e: Expression) = Sum(e) - def sumDistinct(e: Expression) = SumDistinct(e) - def count(e: Expression) = Count(e) - def countDistinct(e: Expression*) = CountDistinct(e) - def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) - def avg(e: Expression) = Average(e) - def first(e: Expression) = First(e) - def last(e: Expression) = Last(e) - def min(e: Expression) = Min(e) - def max(e: Expression) = Max(e) - def upper(e: Expression) = Upper(e) - def lower(e: Expression) = Lower(e) - def sqrt(e: Expression) = Sqrt(e) - def abs(e: Expression) = Abs(e) - - implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } + def sum(e: Expression): Expression = Sum(e) + def sumDistinct(e: Expression): Expression = SumDistinct(e) + def count(e: Expression): Expression = Count(e) + def countDistinct(e: Expression*): Expression = CountDistinct(e) + def approxCountDistinct(e: Expression, rsd: Double = 0.05): Expression = + ApproxCountDistinct(e, rsd) + def avg(e: Expression): Expression = Average(e) + def first(e: Expression): Expression = First(e) + def last(e: Expression): Expression = Last(e) + def min(e: Expression): Expression = Min(e) + def max(e: Expression): Expression = Max(e) + def upper(e: Expression): Expression = Upper(e) + def lower(e: Expression): Expression = Lower(e) + def sqrt(e: Expression): Expression = Sqrt(e) + def abs(e: Expression): Expression = Abs(e) + + implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s: String = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { override def expr: Expression = Literal(s) - def attr = analysis.UnresolvedAttribute(s) + def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s) } abstract class ImplicitAttribute extends ImplicitOperators { def s: String - def expr = attr - def attr = analysis.UnresolvedAttribute(s) + def expr: UnresolvedAttribute = attr + def attr: UnresolvedAttribute = analysis.UnresolvedAttribute(s) /** Creates a new AttributeReference of type boolean */ - def boolean = AttributeReference(s, BooleanType, nullable = true)() + def boolean: AttributeReference = AttributeReference(s, BooleanType, nullable = true)() /** Creates a new AttributeReference of type byte */ - def byte = AttributeReference(s, ByteType, nullable = true)() + def byte: AttributeReference = AttributeReference(s, ByteType, nullable = true)() /** Creates a new AttributeReference of type short */ - def short = AttributeReference(s, ShortType, nullable = true)() + def short: AttributeReference = AttributeReference(s, ShortType, nullable = true)() /** Creates a new AttributeReference of type int */ - def int = AttributeReference(s, IntegerType, nullable = true)() + def int: AttributeReference = AttributeReference(s, IntegerType, nullable = true)() /** Creates a new AttributeReference of type long */ - def long = AttributeReference(s, LongType, nullable = true)() + def long: AttributeReference = AttributeReference(s, LongType, nullable = true)() /** Creates a new AttributeReference of type float */ - def float = AttributeReference(s, FloatType, nullable = true)() + def float: AttributeReference = AttributeReference(s, FloatType, nullable = true)() /** Creates a new AttributeReference of type double */ - def double = AttributeReference(s, DoubleType, nullable = true)() + def double: AttributeReference = AttributeReference(s, DoubleType, nullable = true)() /** Creates a new AttributeReference of type string */ - def string = AttributeReference(s, StringType, nullable = true)() + def string: AttributeReference = AttributeReference(s, StringType, nullable = true)() /** Creates a new AttributeReference of type date */ - def date = AttributeReference(s, DateType, nullable = true)() + def date: AttributeReference = AttributeReference(s, DateType, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal = AttributeReference(s, DecimalType.Unlimited, nullable = true)() + def decimal: AttributeReference = + AttributeReference(s, DecimalType.Unlimited, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal(precision: Int, scale: Int) = + def decimal(precision: Int, scale: Int): AttributeReference = AttributeReference(s, DecimalType(precision, scale), nullable = true)() /** Creates a new AttributeReference of type timestamp */ - def timestamp = AttributeReference(s, TimestampType, nullable = true)() + def timestamp: AttributeReference = AttributeReference(s, TimestampType, nullable = true)() /** Creates a new AttributeReference of type binary */ - def binary = AttributeReference(s, BinaryType, nullable = true)() + def binary: AttributeReference = AttributeReference(s, BinaryType, nullable = true)() /** Creates a new AttributeReference of type array */ - def array(dataType: DataType) = AttributeReference(s, ArrayType(dataType), nullable = true)() + def array(dataType: DataType): AttributeReference = + AttributeReference(s, ArrayType(dataType), nullable = true)() /** Creates a new AttributeReference of type map */ def map(keyType: DataType, valueType: DataType): AttributeReference = map(MapType(keyType, valueType)) - def map(mapType: MapType) = AttributeReference(s, mapType, nullable = true)() + + def map(mapType: MapType): AttributeReference = + AttributeReference(s, mapType, nullable = true)() /** Creates a new AttributeReference of type struct */ def struct(fields: StructField*): AttributeReference = struct(StructType(fields)) - def struct(structType: StructType) = AttributeReference(s, structType, nullable = true)() + def struct(structType: StructType): AttributeReference = + AttributeReference(s, structType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { - def notNull = a.withNullability(false) - def nullable = a.withNullability(true) + def notNull: AttributeReference = a.withNullability(false) + def nullable: AttributeReference = a.withNullability(true) // Protobuf terminology - def required = a.withNullability(false) + def required: AttributeReference = a.withNullability(false) - def at(ordinal: Int) = BoundReference(ordinal, a.dataType, a.nullable) + def at(ordinal: Int): BoundReference = BoundReference(ordinal, a.dataType, a.nullable) } } @@ -241,23 +247,23 @@ package object dsl { abstract class LogicalPlanFunctions { def logicalPlan: LogicalPlan - def select(exprs: NamedExpression*) = Project(exprs, logicalPlan) + def select(exprs: NamedExpression*): LogicalPlan = Project(exprs, logicalPlan) - def where(condition: Expression) = Filter(condition, logicalPlan) + def where(condition: Expression): LogicalPlan = Filter(condition, logicalPlan) - def limit(limitExpr: Expression) = Limit(limitExpr, logicalPlan) + def limit(limitExpr: Expression): LogicalPlan = Limit(limitExpr, logicalPlan) def join( otherPlan: LogicalPlan, joinType: JoinType = Inner, - condition: Option[Expression] = None) = + condition: Option[Expression] = None): LogicalPlan = Join(logicalPlan, otherPlan, joinType, condition) - def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, true, logicalPlan) + def orderBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, true, logicalPlan) - def sortBy(sortExprs: SortOrder*) = Sort(sortExprs, false, logicalPlan) + def sortBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, false, logicalPlan) - def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { + def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): LogicalPlan = { val aliasedExprs = aggregateExprs.map { case ne: NamedExpression => ne case e => Alias(e, e.toString)() @@ -265,41 +271,38 @@ package object dsl { Aggregate(groupingExprs, aliasedExprs, logicalPlan) } - def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan) + def subquery(alias: Symbol): LogicalPlan = Subquery(alias.name, logicalPlan) - def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan) + def unionAll(otherPlan: LogicalPlan): LogicalPlan = Union(logicalPlan, otherPlan) - def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) = + def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean): LogicalPlan = Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan) - def sample( - fraction: Double, - withReplacement: Boolean = true, - seed: Int = (math.random * 1000).toInt) = - Sample(fraction, withReplacement, seed, logicalPlan) - + // TODO specify the output column names def generate( generator: Generator, join: Boolean = false, outer: Boolean = false, - alias: Option[String] = None) = - Generate(generator, join, outer, None, logicalPlan) + alias: Option[String] = None): LogicalPlan = + Generate(generator, join = join, outer = outer, alias, Nil, logicalPlan) - def insertInto(tableName: String, overwrite: Boolean = false) = + def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( - analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) + analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite, false) - def analyze = analysis.SimpleAnalyzer(logicalPlan) + def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer.execute(logicalPlan)) } object plans { // scalastyle:ignore implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions { - def writeToFile(path: String) = WriteToFile(path, logicalPlan) + def writeToFile(path: String): LogicalPlan = WriteToFile(path, logicalPlan) } } case class ScalaUdfBuilder[T: TypeTag](f: AnyRef) { - def call(args: Expression*) = ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args) + def call(args: Expression*): ScalaUdf = { + ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args) + } } // scalastyle:off diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala index bdeb660b1ecb7..0fd4f9b374ee0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -38,6 +38,8 @@ package object errors { } } + class DialectException(msg: String, cause: Throwable) extends Exception(msg, cause) + /** * Wraps any exceptions that are thrown while executing `f` in a * [[catalyst.errors.TreeNodeException TreeNodeException]], attaching the provided `tree`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index 82e760b6c6916..96a11e352ec50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -23,7 +23,9 @@ package org.apache.spark.sql.catalyst.expressions * of the name, or the expected nullability). */ object AttributeMap { - def apply[A](kvs: Seq[(Attribute, A)]) = new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) + def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = { + new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) + } } class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index adaeab0b5c027..5345696570b41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -19,27 +19,27 @@ package org.apache.spark.sql.catalyst.expressions protected class AttributeEquals(val a: Attribute) { - override def hashCode() = a match { + override def hashCode(): Int = a match { case ar: AttributeReference => ar.exprId.hashCode() case a => a.hashCode() } - override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match { + override def equals(other: Any): Boolean = (a, other.asInstanceOf[AttributeEquals].a) match { case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId case (a1, a2) => a1 == a2 } } object AttributeSet { - def apply(a: Attribute) = - new AttributeSet(Set(new AttributeEquals(a))) + def apply(a: Attribute): AttributeSet = new AttributeSet(Set(new AttributeEquals(a))) /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ - def apply(baseSet: Seq[Expression]) = + def apply(baseSet: Iterable[Expression]): AttributeSet = { new AttributeSet( baseSet .flatMap(_.references) .map(new AttributeEquals(_)).toSet) + } } /** @@ -57,8 +57,9 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) extends Traversable[Attribute] with Serializable { /** Returns true if the members of this AttributeSet and other are the same. */ - override def equals(other: Any) = other match { - case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + override def equals(other: Any): Boolean = other match { + case otherSet: AttributeSet => + otherSet.size == baseSet.size && baseSet.map(_.a).forall(otherSet.contains) case _ => false } @@ -81,32 +82,34 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in * `other`. */ - def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + def subsetOf(other: AttributeSet): Boolean = baseSet.subsetOf(other.baseSet) /** * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found * in `other`. */ - def --(other: Traversable[NamedExpression]) = + def --(other: Traversable[NamedExpression]): AttributeSet = new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) /** * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found * in `other`. */ - def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + def ++(other: AttributeSet): AttributeSet = new AttributeSet(baseSet ++ other.baseSet) /** * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to * true. */ - override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + override def filter(f: Attribute => Boolean): AttributeSet = + new AttributeSet(baseSet.filter(ae => f(ae.a))) /** * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in * `this` and `other`. */ - def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + def intersect(other: AttributeSet): AttributeSet = + new AttributeSet(baseSet.intersect(other.baseSet)) override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 76a9f08dea85f..c6217f07c452d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -28,13 +28,21 @@ import org.apache.spark.sql.catalyst.trees * the layout of intermediate tuples, BindReferences should be run after all such transformations. */ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) - extends Expression with trees.LeafNode[Expression] { + extends NamedExpression with trees.LeafNode[Expression] { type EvaluatedType = Any - override def toString = s"input[$ordinal]" + override def toString: String = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) + + override def name: String = s"i[$ordinal]" + + override def toAttribute: Attribute = throw new UnsupportedOperationException + + override def qualifiers: Seq[String] = throw new UnsupportedOperationException + + override def exprId: ExprId = throw new UnsupportedOperationException } object BindReferences extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index b1bc858478ee1..adf941ab2a45f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -21,7 +21,6 @@ import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ /** Cast the child expression to the target data type. */ @@ -29,9 +28,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w override lazy val resolved = childrenResolved && resolve(child.dataType, dataType) - override def foldable = child.foldable + override def foldable: Boolean = child.foldable - override def nullable = forceNullable(child.dataType, dataType) || child.nullable + override def nullable: Boolean = forceNullable(child.dataType, dataType) || child.nullable private[this] def forceNullable(from: DataType, to: DataType) = (from, to) match { case (StringType, _: NumericType) => true @@ -103,7 +102,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } - override def toString = s"CAST($child, $dataType)" + override def toString: String = s"CAST($child, $dataType)" type EvaluatedType = Any @@ -112,21 +111,21 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { - case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) - case DateType => buildCast[Int](_, d => DateUtils.toString(d)) - case TimestampType => buildCast[Timestamp](_, timestampToString) - case _ => buildCast[Any](_, _.toString) + case BinaryType => buildCast[Array[Byte]](_, UTF8String(_)) + case DateType => buildCast[Int](_, d => UTF8String(DateUtils.toString(d))) + case TimestampType => buildCast[Timestamp](_, t => UTF8String(timestampToString(t))) + case _ => buildCast[Any](_, o => UTF8String(o.toString)) } // BinaryConverter private[this] def castToBinary(from: DataType): Any => Any = from match { - case StringType => buildCast[String](_, _.getBytes("UTF-8")) + case StringType => buildCast[UTF8String](_, _.getBytes) } // UDFToBoolean private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, _.length() != 0) + buildCast[UTF8String](_, _.length() != 0) case TimestampType => buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => @@ -151,8 +150,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => { + buildCast[UTF8String](_, utfs => { // Throw away extra if more than 9 decimal places + val s = utfs.toString val periodIdx = s.indexOf(".") var n = s if (periodIdx != -1 && n.length() - periodIdx > 9) { @@ -227,8 +227,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => - try DateUtils.fromJavaDate(Date.valueOf(s)) + buildCast[UTF8String](_, s => + try DateUtils.fromJavaDate(Date.valueOf(s.toString)) catch { case _: java.lang.IllegalArgumentException => null } ) case TimestampType => @@ -245,7 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toLong catch { + buildCast[UTF8String](_, s => try s.toString.toLong catch { case _: NumberFormatException => null }) case BooleanType => @@ -261,7 +261,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // IntConverter private[this] def castToInt(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toInt catch { + buildCast[UTF8String](_, s => try s.toString.toInt catch { case _: NumberFormatException => null }) case BooleanType => @@ -277,7 +277,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ShortConverter private[this] def castToShort(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toShort catch { + buildCast[UTF8String](_, s => try s.toString.toShort catch { case _: NumberFormatException => null }) case BooleanType => @@ -293,7 +293,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ByteConverter private[this] def castToByte(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toByte catch { + buildCast[UTF8String](_, s => try s.toString.toByte catch { case _: NumberFormatException => null }) case BooleanType => @@ -323,7 +323,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => - buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { + buildCast[UTF8String](_, s => try { + changePrecision(Decimal(s.toString.toDouble), target) + } catch { case _: NumberFormatException => null }) case BooleanType => @@ -348,7 +350,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DoubleConverter private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toDouble catch { + buildCast[UTF8String](_, s => try s.toString.toDouble catch { case _: NumberFormatException => null }) case BooleanType => @@ -364,7 +366,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // FloatConverter private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toFloat catch { + buildCast[UTF8String](_, s => try s.toString.toFloat catch { case _: NumberFormatException => null }) case BooleanType => @@ -394,10 +396,17 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w val casts = from.fields.zip(to.fields).map { case (fromField, toField) => cast(fromField.dataType, toField.dataType) } - // TODO: This is very slow! - buildCast[Row](_, row => Row(row.toSeq.zip(casts).map { - case (v, cast) => if (v == null) null else cast(v) - }: _*)) + // TODO: Could be faster? + val newRow = new GenericMutableRow(from.fields.size) + buildCast[Row](_, row => { + var i = 0 + while (i < row.length) { + val v = row(i) + newRow.update(i, if (v == null) null else casts(i)(v)) + i += 1 + } + newRow.copy() + }) } private[this] def cast(from: DataType, to: DataType): Any => Any = to match { @@ -430,14 +439,14 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w object Cast { // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { - override def initialValue() = { + override def initialValue(): SimpleDateFormat = { new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } } // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { - override def initialValue() = { + override def initialValue(): SimpleDateFormat = { new SimpleDateFormat("yyyy-MM-dd") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 6ad39b8372cfb..4fd1bc4dd642d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types._ @@ -65,7 +64,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns true if all the children of this expression have been resolved to a specific schema * and false if any still contains any unresolved placeholders. */ - def childrenResolved = !children.exists(!_.resolved) + def childrenResolved: Boolean = !children.exists(!_.resolved) /** * Returns a string representation of this expression that does not have developer centric @@ -84,9 +83,11 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express def symbol: String - override def foldable = left.foldable && right.foldable + override def foldable: Boolean = left.foldable && right.foldable - override def toString = s"($left $symbol $right)" + override def nullable: Boolean = left.nullable || right.nullable + + override def toString: String = s"($left $symbol $right)" } abstract class LeafExpression extends Expression with trees.LeafNode[Expression] { @@ -104,8 +105,18 @@ abstract class UnaryExpression extends Expression with trees.UnaryNode[Expressio case class GroupExpression(children: Seq[Expression]) extends Expression { self: Product => type EvaluatedType = Seq[Any] - override def eval(input: Row): EvaluatedType = ??? - override def nullable = false - override def foldable = false - override def dataType = ??? + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = false + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException +} + +/** + * Expressions that require a specific `DataType` as input should implement this trait + * so that the proper type conversions can be performed in the analyzer. + */ +trait ExpectsInputTypes { + + def expectedChildTypes: Seq[DataType] + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index db5d897ee569f..8cae548279eb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -40,7 +40,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { new GenericRow(outputArray) } - override def toString = s"Row => [${exprArray.mkString(",")}]" + override def toString: String = s"Row => [${exprArray.mkString(",")}]" } /** @@ -107,12 +107,12 @@ class JoinedRow extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -142,7 +142,7 @@ class JoinedRow extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -153,7 +153,7 @@ class JoinedRow extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -207,12 +207,12 @@ class JoinedRow2 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -242,7 +242,7 @@ class JoinedRow2 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -253,7 +253,7 @@ class JoinedRow2 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -301,12 +301,12 @@ class JoinedRow3 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -336,7 +336,7 @@ class JoinedRow3 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -347,7 +347,7 @@ class JoinedRow3 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -395,12 +395,12 @@ class JoinedRow4 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -430,7 +430,7 @@ class JoinedRow4 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -441,7 +441,7 @@ class JoinedRow4 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" @@ -489,12 +489,12 @@ class JoinedRow5 extends Row { override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq - override def length = row1.length + row2.length + override def length: Int = row1.length + row2.length - override def apply(i: Int) = + override def apply(i: Int): Any = if (i < row1.length) row1(i) else row2(i - row1.length) - override def isNullAt(i: Int) = + override def isNullAt(i: Int): Boolean = if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) override def getInt(i: Int): Int = @@ -524,7 +524,7 @@ class JoinedRow5 extends Row { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy() = { + override def copy(): Row = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -535,7 +535,101 @@ class JoinedRow5 extends Row { new GenericRow(copiedValues) } - override def toString() = { + override def toString: String = { + // Make sure toString never throws NullPointerException. + if ((row1 eq null) && (row2 eq null)) { + "[ empty row ]" + } else if (row1 eq null) { + row2.mkString("[", ",", "]") + } else if (row2 eq null) { + row1.mkString("[", ",", "]") + } else { + mkString("[", ",", "]") + } + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow6 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + override def toSeq: Seq[Any] = row1.toSeq ++ row2.toSeq + + override def length: Int = row1.length + row2.length + + override def apply(i: Int): Any = + if (i < row1.length) row1(i) else row2(i - row1.length) + + override def isNullAt(i: Int): Boolean = + if (i < row1.length) row1.isNullAt(i) else row2.isNullAt(i - row1.length) + + override def getInt(i: Int): Int = + if (i < row1.length) row1.getInt(i) else row2.getInt(i - row1.length) + + override def getLong(i: Int): Long = + if (i < row1.length) row1.getLong(i) else row2.getLong(i - row1.length) + + override def getDouble(i: Int): Double = + if (i < row1.length) row1.getDouble(i) else row2.getDouble(i - row1.length) + + override def getBoolean(i: Int): Boolean = + if (i < row1.length) row1.getBoolean(i) else row2.getBoolean(i - row1.length) + + override def getShort(i: Int): Short = + if (i < row1.length) row1.getShort(i) else row2.getShort(i - row1.length) + + override def getByte(i: Int): Byte = + if (i < row1.length) row1.getByte(i) else row2.getByte(i - row1.length) + + override def getFloat(i: Int): Float = + if (i < row1.length) row1.getFloat(i) else row2.getFloat(i - row1.length) + + override def getString(i: Int): String = + if (i < row1.length) row1.getString(i) else row2.getString(i - row1.length) + + override def getAs[T](i: Int): T = + if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) + + override def copy(): Row = { + val totalSize = row1.length + row2.length + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString: String = { // Make sure toString never throws NullPointerException. if ((row1 eq null) && (row2 eq null)) { "[ empty row ]" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 8a36c6810790d..9a77ca624ebe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types.DataType /** @@ -29,9 +29,9 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def nullable = true + override def nullable: Boolean = true - override def toString = s"scalaUDF(${children.mkString(",")})" + override def toString: String = s"scalaUDF(${children.mkString(",")})" // scalastyle:off @@ -39,363 +39,924 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val evals = (0 to x - 1).map(x => s" ScalaReflection.convertToScala(children($x).eval(input), children($x).dataType)").reduce(_ + ",\n " + _) - - s""" - case $x => - function.asInstanceOf[($anys) => Any]( - $evals) - """ + val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) + lazy val converters = (0 to x - 1).map(x => s"lazy val converter$x = CatalystTypeConverters.createToScalaConverter(child$x.dataType)").reduce(_ + "\n " + _) + val evals = (0 to x - 1).map(x => s"converter$x(child$x.eval(input))").reduce(_ + ",\n " + _) + + s"""case $x => + val func = function.asInstanceOf[($anys) => Any] + $childs + $converters + (input: Row) => { + func( + $evals) + } + """ }.foreach(println) */ - - override def eval(input: Row): Any = { - val result = children.size match { - case 0 => function.asInstanceOf[() => Any]() - case 1 => - function.asInstanceOf[(Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType)) - - - case 2 => - function.asInstanceOf[(Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType)) - - - case 3 => - function.asInstanceOf[(Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType)) - - - case 4 => - function.asInstanceOf[(Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType)) - - - case 5 => - function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType)) - - - case 6 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType)) - - - case 7 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType)) - - - case 8 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType)) - - - case 9 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType)) - - - case 10 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType)) - - - case 11 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType)) - - - case 12 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType)) - - - case 13 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType)) - - - case 14 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType)) - - - case 15 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType)) - - - case 16 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType)) - - - case 17 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType)) - - - case 18 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType)) - - - case 19 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType)) - - - case 20 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType)) - - - case 21 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), - ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType)) - - - case 22 => - function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), - ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), - ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), - ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), - ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), - ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), - ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), - ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), - ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), - ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), - ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), - ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), - ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), - ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), - ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), - ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), - ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), - ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), - ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), - ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), - ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType), - ScalaReflection.convertToScala(children(21).eval(input), children(21).dataType)) - - } - // scalastyle:on - - ScalaReflection.convertToCatalyst(result, dataType) + + val f = children.size match { + case 0 => + val func = function.asInstanceOf[() => Any] + (input: Row) => { + func() + } + + case 1 => + val func = function.asInstanceOf[(Any) => Any] + val child0 = children(0) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + (input: Row) => { + func( + converter0(child0.eval(input))) + } + + case 2 => + val func = function.asInstanceOf[(Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input))) + } + + case 3 => + val func = function.asInstanceOf[(Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input))) + } + + case 4 => + val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input))) + } + + case 5 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input))) + } + + case 6 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input))) + } + + case 7 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input))) + } + + case 8 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input))) + } + + case 9 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input))) + } + + case 10 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input))) + } + + case 11 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input))) + } + + case 12 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input))) + } + + case 13 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input))) + } + + case 14 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input))) + } + + case 15 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input))) + } + + case 16 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input))) + } + + case 17 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input))) + } + + case 18 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input))) + } + + case 19 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input))) + } + + case 20 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input))) + } + + case 21 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + val child20 = children(20) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input))) + } + + case 22 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + val child0 = children(0) + val child1 = children(1) + val child2 = children(2) + val child3 = children(3) + val child4 = children(4) + val child5 = children(5) + val child6 = children(6) + val child7 = children(7) + val child8 = children(8) + val child9 = children(9) + val child10 = children(10) + val child11 = children(11) + val child12 = children(12) + val child13 = children(13) + val child14 = children(14) + val child15 = children(15) + val child16 = children(16) + val child17 = children(17) + val child18 = children(18) + val child19 = children(19) + val child20 = children(20) + val child21 = children(21) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) + lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) + (input: Row) => { + func( + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input)), + converter21(child21.eval(input))) + } } + + // scalastyle:on + + override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType) + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d00b2ac09745c..83074eb1e6310 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.types.DataType abstract sealed class SortDirection case object Ascending extends SortDirection @@ -31,12 +32,12 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def dataType = child.dataType - override def nullable = child.nullable + override def dataType: DataType = child.dataType + override def nullable: Boolean = child.nullable // SortOrder itself is never evaluated. override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") - override def toString = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" + override def toString: String = s"$child ${if (direction == Ascending) "ASC" else "DESC"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 21d714c9a8c3b..aa4099e4d7bf9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -62,126 +62,126 @@ abstract class MutableValue extends Serializable { var isNull: Boolean = true def boxed: Any def update(v: Any) - def copy(): this.type + def copy(): MutableValue } final class MutableInt extends MutableValue { var value: Int = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Int] + value = v.asInstanceOf[Int] } - def copy() = { + override def copy(): MutableInt = { val newCopy = new MutableInt newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableInt] } } final class MutableFloat extends MutableValue { var value: Float = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Float] + value = v.asInstanceOf[Float] } - def copy() = { + override def copy(): MutableFloat = { val newCopy = new MutableFloat newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableFloat] } } final class MutableBoolean extends MutableValue { var value: Boolean = false - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Boolean] + value = v.asInstanceOf[Boolean] } - def copy() = { + override def copy(): MutableBoolean = { val newCopy = new MutableBoolean newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableBoolean] } } final class MutableDouble extends MutableValue { var value: Double = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Double] + value = v.asInstanceOf[Double] } - def copy() = { + override def copy(): MutableDouble = { val newCopy = new MutableDouble newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableDouble] } } final class MutableShort extends MutableValue { var value: Short = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Short] } - def copy() = { + override def copy(): MutableShort = { val newCopy = new MutableShort newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableShort] } } final class MutableLong extends MutableValue { var value: Long = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Long] } - def copy() = { + override def copy(): MutableLong = { val newCopy = new MutableLong newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableLong] } } final class MutableByte extends MutableValue { var value: Byte = 0 - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = value = { isNull = false v.asInstanceOf[Byte] } - def copy() = { + override def copy(): MutableByte = { val newCopy = new MutableByte newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableByte] } } final class MutableAny extends MutableValue { var value: Any = _ - def boxed = if (isNull) null else value - def update(v: Any) = value = { + override def boxed: Any = if (isNull) null else value + override def update(v: Any): Unit = { isNull = false - v.asInstanceOf[Any] + value = v.asInstanceOf[Any] } - def copy() = { + override def copy(): MutableAny = { val newCopy = new MutableAny newCopy.isNull = isNull newCopy.value = value - newCopy.asInstanceOf[this.type] + newCopy.asInstanceOf[MutableAny] } } @@ -202,6 +202,7 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR case DoubleType => new MutableDouble case BooleanType => new MutableBoolean case LongType => new MutableLong + case DateType => new MutableInt // We use INT for DATE internally case _ => new MutableAny }.toArray) @@ -230,13 +231,17 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new GenericRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = { - if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + override def update(ordinal: Int, value: Any) { + if (value == null) { + setNullAt(ordinal) + } else { + values(ordinal).update(value) + } } - override def setString(ordinal: Int, value: String) = update(ordinal, value) + override def setString(ordinal: Int, value: String): Unit = update(ordinal, UTF8String(value)) - override def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int): String = apply(ordinal).toString override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala new file mode 100644 index 0000000000000..5b2c8572784bd --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala @@ -0,0 +1,223 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.PlatformDependent +import org.apache.spark.unsafe.array.ByteArrayMethods + +/** + * Converts Rows into UnsafeRow format. This class is NOT thread-safe. + * + * @param fieldTypes the data types of the row's columns. + */ +class UnsafeRowConverter(fieldTypes: Array[DataType]) { + + def this(schema: StructType) { + this(schema.fields.map(_.dataType)) + } + + /** Re-used pointer to the unsafe row being written */ + private[this] val unsafeRow = new UnsafeRow() + + /** Functions for encoding each column */ + private[this] val writers: Array[UnsafeColumnWriter] = { + fieldTypes.map(t => UnsafeColumnWriter.forType(t)) + } + + /** The size, in bytes, of the fixed-length portion of the row, including the null bitmap */ + private[this] val fixedLengthSize: Int = + (8 * fieldTypes.length) + UnsafeRow.calculateBitSetWidthInBytes(fieldTypes.length) + + /** + * Compute the amount of space, in bytes, required to encode the given row. + */ + def getSizeRequirement(row: Row): Int = { + var fieldNumber = 0 + var variableLengthFieldSize: Int = 0 + while (fieldNumber < writers.length) { + if (!row.isNullAt(fieldNumber)) { + variableLengthFieldSize += writers(fieldNumber).getSize(row, fieldNumber) + } + fieldNumber += 1 + } + fixedLengthSize + variableLengthFieldSize + } + + /** + * Convert the given row into UnsafeRow format. + * + * @param row the row to convert + * @param baseObject the base object of the destination address + * @param baseOffset the base offset of the destination address + * @return the number of bytes written. This should be equal to `getSizeRequirement(row)`. + */ + def writeRow(row: Row, baseObject: Object, baseOffset: Long): Long = { + unsafeRow.pointTo(baseObject, baseOffset, writers.length, null) + var fieldNumber = 0 + var appendCursor: Int = fixedLengthSize + while (fieldNumber < writers.length) { + if (row.isNullAt(fieldNumber)) { + unsafeRow.setNullAt(fieldNumber) + } else { + appendCursor += writers(fieldNumber).write(row, unsafeRow, fieldNumber, appendCursor) + } + fieldNumber += 1 + } + appendCursor + } + +} + +/** + * Function for writing a column into an UnsafeRow. + */ +private abstract class UnsafeColumnWriter { + /** + * Write a value into an UnsafeRow. + * + * @param source the row being converted + * @param target a pointer to the converted unsafe row + * @param column the column to write + * @param appendCursor the offset from the start of the unsafe row to the end of the row; + * used for calculating where variable-length data should be written + * @return the number of variable-length bytes written + */ + def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int + + /** + * Return the number of bytes that are needed to write this variable-length value. + */ + def getSize(source: Row, column: Int): Int +} + +private object UnsafeColumnWriter { + + def forType(dataType: DataType): UnsafeColumnWriter = { + dataType match { + case NullType => NullUnsafeColumnWriter + case BooleanType => BooleanUnsafeColumnWriter + case ByteType => ByteUnsafeColumnWriter + case ShortType => ShortUnsafeColumnWriter + case IntegerType => IntUnsafeColumnWriter + case LongType => LongUnsafeColumnWriter + case FloatType => FloatUnsafeColumnWriter + case DoubleType => DoubleUnsafeColumnWriter + case StringType => StringUnsafeColumnWriter + case t => + throw new UnsupportedOperationException(s"Do not know how to write columns of type $t") + } + } +} + +// ------------------------------------------------------------------------------------------------ + +private object NullUnsafeColumnWriter extends NullUnsafeColumnWriter +private object BooleanUnsafeColumnWriter extends BooleanUnsafeColumnWriter +private object ByteUnsafeColumnWriter extends ByteUnsafeColumnWriter +private object ShortUnsafeColumnWriter extends ShortUnsafeColumnWriter +private object IntUnsafeColumnWriter extends IntUnsafeColumnWriter +private object LongUnsafeColumnWriter extends LongUnsafeColumnWriter +private object FloatUnsafeColumnWriter extends FloatUnsafeColumnWriter +private object DoubleUnsafeColumnWriter extends DoubleUnsafeColumnWriter +private object StringUnsafeColumnWriter extends StringUnsafeColumnWriter + +private abstract class PrimitiveUnsafeColumnWriter extends UnsafeColumnWriter { + // Primitives don't write to the variable-length region: + def getSize(sourceRow: Row, column: Int): Int = 0 +} + +private class NullUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setNullAt(column) + 0 + } +} + +private class BooleanUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setBoolean(column, source.getBoolean(column)) + 0 + } +} + +private class ByteUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setByte(column, source.getByte(column)) + 0 + } +} + +private class ShortUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setShort(column, source.getShort(column)) + 0 + } +} + +private class IntUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setInt(column, source.getInt(column)) + 0 + } +} + +private class LongUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setLong(column, source.getLong(column)) + 0 + } +} + +private class FloatUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setFloat(column, source.getFloat(column)) + 0 + } +} + +private class DoubleUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter { + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + target.setDouble(column, source.getDouble(column)) + 0 + } +} + +private class StringUnsafeColumnWriter private() extends UnsafeColumnWriter { + def getSize(source: Row, column: Int): Int = { + val numBytes = source.get(column).asInstanceOf[UTF8String].getBytes.length + 8 + ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes) + } + + override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = { + val value = source.get(column).asInstanceOf[UTF8String] + val baseObject = target.getBaseObject + val baseOffset = target.getBaseOffset + val numBytes = value.getBytes.length + PlatformDependent.UNSAFE.putLong(baseObject, baseOffset + appendCursor, numBytes) + PlatformDependent.copyMemory( + value.getBytes, + PlatformDependent.BYTE_ARRAY_OFFSET, + baseObject, + baseOffset + appendCursor + 8, + numBytes + ) + target.setLong(column, appendCursor) + 8 + ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 735b7488fdcbd..f3830c6d3bcf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -79,27 +79,29 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def nullable = base.nullable - override def dataType = base.dataType + override def nullable: Boolean = base.nullable + override def dataType: DataType = base.dataType def update(input: Row): Unit // Do we really need this? - override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + override def newInstance(): AggregateFunction = { + makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + } } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"MIN($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"MIN($child)" override def asPartial: SplitEvaluation = { val partialMin = Alias(Min(child), "PartialMin")() SplitEvaluation(Min(partialMin.toAttribute), partialMin :: Nil) } - override def newInstance() = new MinFunction(child, this) + override def newInstance(): MinFunction = new MinFunction(child, this) } case class MinFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -121,16 +123,16 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"MAX($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"MAX($child)" override def asPartial: SplitEvaluation = { val partialMax = Alias(Max(child), "PartialMax")() SplitEvaluation(Max(partialMax.toAttribute), partialMax :: Nil) } - override def newInstance() = new MaxFunction(child, this) + override def newInstance(): MaxFunction = new MaxFunction(child, this) } case class MaxFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -152,29 +154,29 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"COUNT($child)" + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"COUNT($child)" override def asPartial: SplitEvaluation = { val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Coalesce(Seq(Sum(partialCount.toAttribute), Literal(0L))), partialCount :: Nil) } - override def newInstance() = new CountFunction(child, this) + override def newInstance(): CountFunction = new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate { def this() = this(null) - override def children = expressions + override def children: Seq[Expression] = expressions - override def nullable = false - override def dataType = LongType - override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" - override def newInstance() = new CountDistinctFunction(expressions, this) + override def nullable: Boolean = false + override def dataType: DataType = LongType + override def toString: String = s"COUNT(DISTINCT ${expressions.mkString(",")})" + override def newInstance(): CountDistinctFunction = new CountDistinctFunction(expressions, this) - override def asPartial = { + override def asPartial: SplitEvaluation = { val partialSet = Alias(CollectHashSet(expressions), "partialSets")() SplitEvaluation( CombineSetsAndCount(partialSet.toAttribute), @@ -185,11 +187,12 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression { def this() = this(null) - override def children = expressions - override def nullable = false - override def dataType = ArrayType(expressions.head.dataType) - override def toString = s"AddToHashSet(${expressions.mkString(",")})" - override def newInstance() = new CollectHashSetFunction(expressions, this) + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(expressions.head.dataType) + override def toString: String = s"AddToHashSet(${expressions.mkString(",")})" + override def newInstance(): CollectHashSetFunction = + new CollectHashSetFunction(expressions, this) } case class CollectHashSetFunction( @@ -219,11 +222,13 @@ case class CollectHashSetFunction( case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression { def this() = this(null) - override def children = inputSet :: Nil - override def nullable = false - override def dataType = LongType - override def toString = s"CombineAndCount($inputSet)" - override def newInstance() = new CombineSetsAndCountFunction(inputSet, this) + override def children: Seq[Expression] = inputSet :: Nil + override def nullable: Boolean = false + override def dataType: DataType = LongType + override def toString: String = s"CombineAndCount($inputSet)" + override def newInstance(): CombineSetsAndCountFunction = { + new CombineSetsAndCountFunction(inputSet, this) + } } case class CombineSetsAndCountFunction( @@ -246,30 +251,51 @@ case class CombineSetsAndCountFunction( override def eval(input: Row): Any = seen.size.toLong } +/** The data type of ApproxCountDistinctPartition since its output is a HyperLogLog object. */ +private[sql] case object HyperLogLogUDT extends UserDefinedType[HyperLogLog] { + + override def sqlType: DataType = BinaryType + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def serialize(obj: Any): Array[Byte] = + obj.asInstanceOf[HyperLogLog].getBytes + + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def deserialize(datum: Any): HyperLogLog = + HyperLogLog.Builder.build(datum.asInstanceOf[Array[Byte]]) + + override def userClass: Class[HyperLogLog] = classOf[HyperLogLog] +} + case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = child.dataType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" - override def newInstance() = new ApproxCountDistinctPartitionFunction(child, this, relativeSD) + override def nullable: Boolean = false + override def dataType: DataType = HyperLogLogUDT + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance(): ApproxCountDistinctPartitionFunction = { + new ApproxCountDistinctPartitionFunction(child, this, relativeSD) + } } case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" - override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD) + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance(): ApproxCountDistinctMergeFunction = { + new ApproxCountDistinctMergeFunction(child, this, relativeSD) + } } case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = LongType - override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def nullable: Boolean = false + override def dataType: LongType.type = LongType + override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" override def asPartial: SplitEvaluation = { val partialCount = @@ -280,14 +306,14 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) partialCount :: Nil) } - override def newInstance() = new CountDistinctFunction(child :: Nil, this) + override def newInstance(): CountDistinctFunction = new CountDistinctFunction(child :: Nil, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true + override def nullable: Boolean = true - override def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 4, scale + 4) // Add 4 digits after decimal point, like Hive case DecimalType.Unlimited => @@ -296,11 +322,11 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN DoubleType } - override def toString = s"AVG($child)" + override def toString: String = s"AVG($child)" override def asPartial: SplitEvaluation = { child.dataType match { - case DecimalType.Fixed(_, _) => + case DecimalType.Fixed(_, _) | DecimalType.Unlimited => // Turn the child to unlimited decimals for calculation, before going back to fixed val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() @@ -323,14 +349,14 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } } - override def newInstance() = new AverageFunction(child, this) + override def newInstance(): AverageFunction = new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true + override def nullable: Boolean = true - override def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive case DecimalType.Unlimited => @@ -339,33 +365,57 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ child.dataType } - override def toString = s"SUM($child)" + override def toString: String = s"SUM($child)" override def asPartial: SplitEvaluation = { child.dataType match { case DecimalType.Fixed(_, _) => val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() SplitEvaluation( - Cast(Sum(partialSum.toAttribute), dataType), + Cast(CombineSum(partialSum.toAttribute), dataType), partialSum :: Nil) case _ => val partialSum = Alias(Sum(child), "PartialSum")() SplitEvaluation( - Sum(partialSum.toAttribute), + CombineSum(partialSum.toAttribute), partialSum :: Nil) } } - override def newInstance() = new SumFunction(child, this) + override def newInstance(): SumFunction = new SumFunction(child, this) +} + +/** + * Sum should satisfy 3 cases: + * 1) sum of all null values = zero + * 2) sum for table column with no data = null + * 3) sum of column with null and not null values = sum of not null values + * Require separate CombineSum Expression and function as it has to distinguish "No data" case + * versus "data equals null" case, while aggregating results and at each partial expression.i.e., + * Combining PartitionLevel InputData + * <-- null + * Zero <-- Zero <-- null + * + * <-- null <-- no data + * null <-- null <-- no data + */ +case class CombineSum(child: Expression) extends AggregateExpression { + def this() = this(null) + + override def children: Seq[Expression] = child :: Nil + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"CombineSum($child)" + override def newInstance(): CombineSumFunction = new CombineSumFunction(child, this) } case class SumDistinct(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { def this() = this(null) - override def nullable = true - override def dataType = child.dataType match { + override def nullable: Boolean = true + override def dataType: DataType = child.dataType match { case DecimalType.Fixed(precision, scale) => DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive case DecimalType.Unlimited => @@ -373,10 +423,10 @@ case class SumDistinct(child: Expression) case _ => child.dataType } - override def toString = s"SUM(DISTINCT ${child})" - override def newInstance() = new SumDistinctFunction(child, this) + override def toString: String = s"SUM(DISTINCT $child)" + override def newInstance(): SumDistinctFunction = new SumDistinctFunction(child, this) - override def asPartial = { + override def asPartial: SplitEvaluation = { val partialSet = Alias(CollectHashSet(child :: Nil), "partialSets")() SplitEvaluation( CombineSetsAndSum(partialSet.toAttribute, this), @@ -387,11 +437,13 @@ case class SumDistinct(child: Expression) case class CombineSetsAndSum(inputSet: Expression, base: Expression) extends AggregateExpression { def this() = this(null, null) - override def children = inputSet :: Nil - override def nullable = true - override def dataType = base.dataType - override def toString = s"CombineAndSum($inputSet)" - override def newInstance() = new CombineSetsAndSumFunction(inputSet, this) + override def children: Seq[Expression] = inputSet :: Nil + override def nullable: Boolean = true + override def dataType: DataType = base.dataType + override def toString: String = s"CombineAndSum($inputSet)" + override def newInstance(): CombineSetsAndSumFunction = { + new CombineSetsAndSumFunction(inputSet, this) + } } case class CombineSetsAndSumFunction( @@ -425,9 +477,9 @@ case class CombineSetsAndSumFunction( } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def nullable = true - override def dataType = child.dataType - override def toString = s"FIRST($child)" + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"FIRST($child)" override def asPartial: SplitEvaluation = { val partialFirst = Alias(First(child), "PartialFirst")() @@ -435,14 +487,14 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance() = new FirstFunction(child, this) + override def newInstance(): FirstFunction = new FirstFunction(child, this) } case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references - override def nullable = true - override def dataType = child.dataType - override def toString = s"LAST($child)" + override def references: AttributeSet = child.references + override def nullable: Boolean = true + override def dataType: DataType = child.dataType + override def toString: String = s"LAST($child)" override def asPartial: SplitEvaluation = { val partialLast = Alias(Last(child), "PartialLast")() @@ -450,7 +502,7 @@ case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode Last(partialLast.toAttribute), partialLast :: Nil) } - override def newInstance() = new LastFunction(child, this) + override def newInstance(): LastFunction = new LastFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) @@ -471,7 +523,8 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) private var count: Long = _ private val sum = MutableLiteral(zero.eval(null), calcType) - private def addFunction(value: Any) = Add(sum, Cast(Literal(value, expr.dataType), calcType)) + private def addFunction(value: Any) = Add(sum, + Cast(Literal.create(value, expr.dataType), calcType)) override def eval(input: Row): Any = { if (count == 0L) { @@ -565,7 +618,8 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr private val sum = MutableLiteral(null, calcType) - private val addFunction = Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum)) + private val addFunction = + Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) override def update(input: Row): Unit = { sum.update(addFunction, input) @@ -580,6 +634,43 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr } } +case class CombineSumFunction(expr: Expression, base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + private val calcType = + expr.dataType match { + case DecimalType.Fixed(_, _) => + DecimalType.Unlimited + case _ => + expr.dataType + } + + private val zero = Cast(Literal(0), calcType) + + private val sum = MutableLiteral(null, calcType) + + private val addFunction = + Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) + + override def update(input: Row): Unit = { + val result = expr.eval(input) + // partial sum result can be null only when no input rows present + if(result != null) { + sum.update(addFunction, input) + } + } + + override def eval(input: Row): Any = { + expr.dataType match { + case DecimalType.Fixed(_, _) => + Cast(sum, dataType).eval(null) + case _ => sum.eval(null) + } + } +} + case class SumDistinctFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -651,6 +742,7 @@ case class LastFunction(expr: Expression, base: AggregateExpression) extends Agg result = input } - override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row]) - else null + override def eval(input: Row): Any = { + if (result != null) expr.eval(result.asInstanceOf[Row]) else null + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 00b0d3c683fe2..c7a37ad966df6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"-$child" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"-$child" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -47,10 +47,10 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { case class Sqrt(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = DoubleType - override def foldable = child.foldable - def nullable = true - override def toString = s"SQRT($child)" + override def dataType: DataType = DoubleType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = true + override def toString: String = s"SQRT($child)" lazy val numeric = child.dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -74,14 +74,12 @@ abstract class BinaryArithmetic extends BinaryExpression { type EvaluatedType = Any - def nullable = left.nullable || right.nullable - override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType && !DecimalType.isFixed(left.dataType) - def dataType = { + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") @@ -108,7 +106,7 @@ abstract class BinaryArithmetic extends BinaryExpression { } case class Add(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "+" + override def symbol: String = "+" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -131,7 +129,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic { } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "-" + override def symbol: String = "-" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -154,7 +152,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "*" + override def symbol: String = "*" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] @@ -177,9 +175,9 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "/" + override def symbol: String = "/" - override def nullable = true + override def nullable: Boolean = true lazy val div: (Any, Any) => Any = dataType match { case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div @@ -203,9 +201,9 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic } case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "%" + override def symbol: String = "%" - override def nullable = true + override def nullable: Boolean = true lazy val integral = dataType match { case i: IntegralType => i.integral.asInstanceOf[Integral[Any]] @@ -232,7 +230,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet * A function that calculates bitwise and(&) of two numbers. */ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "&" + override def symbol: String = "&" lazy val and: (Any, Any) => Any = dataType match { case ByteType => @@ -253,7 +251,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme * A function that calculates bitwise or(|) of two numbers. */ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "|" + override def symbol: String = "|" lazy val or: (Any, Any) => Any = dataType match { case ByteType => @@ -274,7 +272,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet * A function that calculates bitwise xor(^) of two numbers. */ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { - def symbol = "^" + override def symbol: String = "^" lazy val xor: (Any, Any) => Any = dataType match { case ByteType => @@ -297,10 +295,10 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme case class BitwiseNot(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"~$child" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"~$child" lazy val not: (Any) => Any = dataType match { case ByteType => @@ -327,17 +325,17 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any - override def foldable = left.foldable && right.foldable + override def foldable: Boolean = left.foldable && right.foldable - override def nullable = left.nullable && right.nullable + override def nullable: Boolean = left.nullable && right.nullable - override def children = left :: right :: Nil + override def children: Seq[Expression] = left :: right :: Nil override lazy val resolved = left.resolved && right.resolved && left.dataType == right.dataType - override def dataType = { + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") @@ -346,7 +344,7 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { } lazy val ordering = left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } @@ -366,7 +364,52 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { } } - override def toString = s"MaxOf($left, $right)" + override def toString: String = s"MaxOf($left, $right)" +} + +case class MinOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def foldable: Boolean = left.foldable && right.foldable + + override def nullable: Boolean = left.nullable && right.nullable + + override def children: Seq[Expression] = left :: right :: Nil + + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + lazy val ordering = left.dataType match { + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + val evalE2 = right.eval(input) + if (evalE1 == null) { + evalE2 + } else if (evalE2 == null) { + evalE1 + } else { + if (ordering.compare(evalE1, evalE2) < 0) { + evalE1 + } else { + evalE2 + } + } + } + + override def toString: String = s"MinOf($left, $right)" } /** @@ -375,10 +418,10 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { case class Abs(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"Abs($child)" + override def dataType: DataType = child.dataType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"Abs($child)" lazy val numeric = dataType match { case n: NumericType => n.numeric.asInstanceOf[Numeric[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index e48b8cde20eda..d17af0e7ff87e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -91,18 +91,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val startTime = System.nanoTime() val result = create(in) val endTime = System.nanoTime() - def timeMs = (endTime - startTime).toDouble / 1000000 + def timeMs: Double = (endTime - startTime).toDouble / 1000000 logInfo(s"Code generated expression $in in $timeMs ms") result } }) /** Generates the requested evaluator binding the given expression(s) to the inputSchema. */ - def apply(expressions: InType, inputSchema: Seq[Attribute]): OutType = - apply(bind(expressions, inputSchema)) + def generate(expressions: InType, inputSchema: Seq[Attribute]): OutType = + generate(bind(expressions, inputSchema)) /** Generates the requested evaluator given already bound expression(s). */ - def apply(expressions: InType): OutType = cache.get(canonicalize(expressions)) + def generate(expressions: InType): OutType = cache.get(canonicalize(expressions)) /** * Returns a term name that is unique within this instance of a `CodeGenerator`. @@ -216,10 +216,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val $primitiveTerm: ${termForType(dataType)} = $value """.children - case expressions.Literal(value: String, dataType) => + case expressions.Literal(value: UTF8String, dataType) => q""" val $nullTerm = ${value == null} - val $primitiveTerm: ${termForType(dataType)} = $value + val $primitiveTerm: ${termForType(dataType)} = + org.apache.spark.sql.types.UTF8String(${value.getBytes}) """.children case expressions.Literal(value: Int, dataType) => @@ -243,11 +244,14 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children case Cast(child @ DateType(), StringType) => - child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + child.castOrNull(c => + q"""org.apache.spark.sql.types.UTF8String( + org.apache.spark.sql.types.DateUtils.toString($c))""", + StringType) case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) @@ -272,9 +276,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - ${eval.primitiveTerm}.toString + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.toString) """.children + case EqualTo(e1 @ BinaryType(), e2 @ BinaryType()) => + (e1, e2).evaluateAs (BooleanType) { + case (eval1, eval2) => + q""" + java.util.Arrays.equals($eval1.asInstanceOf[Array[Byte]], + $eval2.asInstanceOf[Array[Byte]]) + """ + } + case EqualTo(e1, e2) => (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } @@ -464,7 +477,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val itemEval = expressionEvaluator(item) val setEval = expressionEvaluator(set) - val ArrayType(elementType, _) = set.dataType + val elementType = set.dataType.asInstanceOf[OpenHashSetUDT].elementType itemEval.code ++ setEval.code ++ q""" @@ -482,7 +495,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val leftEval = expressionEvaluator(left) val rightEval = expressionEvaluator(right) - val ArrayType(elementType, _) = left.dataType + val elementType = left.dataType.asInstanceOf[OpenHashSetUDT].elementType leftEval.code ++ rightEval.code ++ q""" @@ -524,6 +537,30 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case MinOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + if (${eval1.primitiveTerm} < ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + case UnscaledValue(child) => val childEval = expressionEvaluator(child) @@ -573,7 +610,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val localLogger = log val localLoggerTree = reify { localLogger } q""" - $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + $localLoggerTree.debug( + ${e.toString} + ": " + (if ($nullTerm) "null" else $primitiveTerm.toString)) """ :: Nil } else { Nil @@ -584,7 +622,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { dataType match { - case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" + case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]" + case dt: DataType if isNativeType(dt) => q"$inputRow.${accessorForType(dt)}($ordinal)" case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" } } @@ -595,7 +634,9 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin ordinal: Int, value: TermName) = { dataType match { - case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" + case StringType => q"$destinationRow.update($ordinal, $value)" + case dt: DataType if isNativeType(dt) => + q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" case _ => q"$destinationRow.update($ordinal, $value)" } } @@ -618,24 +659,36 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case DoubleType => "Double" case FloatType => "Float" case BooleanType => "Boolean" - case StringType => "String" + case StringType => "org.apache.spark.sql.types.UTF8String" } protected def defaultPrimitive(dt: DataType) = dt match { case BooleanType => ru.Literal(Constant(false)) case FloatType => ru.Literal(Constant(-1.0.toFloat)) - case StringType => ru.Literal(Constant("")) + case StringType => q"""org.apache.spark.sql.types.UTF8String("")""" case ShortType => ru.Literal(Constant(-1.toShort)) case LongType => ru.Literal(Constant(-1L)) case ByteType => ru.Literal(Constant(-1.toByte)) case DoubleType => ru.Literal(Constant(-1.toDouble)) case DecimalType() => q"org.apache.spark.sql.types.Decimal(-1)" case IntegerType => ru.Literal(Constant(-1)) + case DateType => ru.Literal(Constant(-1)) case _ => ru.Literal(Constant(null)) } protected def termForType(dt: DataType) = dt match { - case n: NativeType => n.tag + case n: AtomicType => n.tag case _ => typeTag[Any] } + + /** + * List of data types that have special accessors and setters in [[Row]]. + */ + protected val nativeTypes = + Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + + /** + * Returns true if the data type has a special accessor and setter in [[Row]]. + */ + protected def isNativeType(dt: DataType) = nativeTypes.contains(dt) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index a419fd7ecb39b..840260703ab74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -30,7 +30,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu val mutableRowName = newTermName("mutableRow") protected def canonicalize(in: Seq[Expression]): Seq[Expression] = - in.map(ExpressionCanonicalizer(_)) + in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = in.map(BindReferences.bindReference(_, inputSchema)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 0db29eb404bd1..b129c0d898bb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NumericType} +import org.apache.spark.sql.types.{BinaryType, StringType, NumericType} /** * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of @@ -30,7 +30,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit import scala.reflect.runtime.universe._ protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = - in.map(ExpressionCanonicalizer(_).asInstanceOf[SortOrder]) + in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = in.map(BindReferences.bindReference(_, inputSchema)) @@ -43,6 +43,18 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit val evalB = expressionEvaluator(order.child) val compare = order.child.dataType match { + case BinaryType => + q""" + val x = ${if (order.direction == Ascending) evalA.primitiveTerm else evalB.primitiveTerm} + val y = ${if (order.direction != Ascending) evalB.primitiveTerm else evalA.primitiveTerm} + var i = 0 + while (i < x.length && i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + i = i+1 + } + return x.length - y.length + """ case _: NumericType => q""" val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 2a0935c790cf3..40e163024360e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -26,7 +26,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { import scala.reflect.runtime.{universe => ru} import scala.reflect.runtime.universe._ - protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer(in) + protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 69397a73a8880..584f938445c8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -31,7 +31,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { import scala.reflect.runtime.universe._ protected def canonicalize(in: Seq[Expression]): Seq[Expression] = - in.map(ExpressionCanonicalizer(_)) + in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = in.map(BindReferences.bindReference(_, inputSchema)) @@ -109,38 +109,56 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { q"override def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" } - val specificAccessorFunctions = NativeType.all.map { dataType => + val specificAccessorFunctions = nativeTypes.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // getString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) return $elementName" :: Nil case _ => Nil } - - q""" - override def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + // Row() need this interface to compile + case StringType => + q""" + override def getString(i: Int): String = { + $accessorFailure + }""" + case other => + q""" + override def ${accessorForType(dataType)}(i: Int): ${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } } - val specificMutatorFunctions = NativeType.all.map { dataType => + val specificMutatorFunctions = nativeTypes.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // setString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } - - q""" - override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + case StringType => + // MutableRow() need this interface to compile + q""" + override def setString(i: Int, value: String) { + $accessorFailure + }""" + case other => + q""" + override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}) { + ..$ifStatements; + $accessorFailure + }""" + } } val hashValues = expressions.zipWithIndex.map { case (e,i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 68051a2a2007e..fc1f69655963d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types._ /** @@ -27,12 +29,12 @@ import org.apache.spark.sql.types._ case class GetItem(child: Expression, ordinal: Expression) extends Expression { type EvaluatedType = Any - val children = child :: ordinal :: Nil + val children: Seq[Expression] = child :: ordinal :: Nil /** `Null` is returned for invalid ordinals. */ - override def nullable = true - override def foldable = child.foldable && ordinal.foldable + override def nullable: Boolean = true + override def foldable: Boolean = child.foldable && ordinal.foldable - def dataType = child.dataType match { + override def dataType: DataType = child.dataType match { case ArrayType(dt, _) => dt case MapType(_, vt, _) => vt } @@ -40,7 +42,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { childrenResolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - override def toString = s"$child[$ordinal]" + override def toString: String = s"$child[$ordinal]" override def eval(input: Row): Any = { val value = child.eval(input) @@ -75,19 +77,54 @@ trait GetField extends UnaryExpression { self: Product => type EvaluatedType = Any - override def foldable = child.foldable - override def toString = s"$child.${field.name}" + override def foldable: Boolean = child.foldable + override def toString: String = s"$child.${field.name}" def field: StructField } +object GetField { + /** + * Returns the resolved `GetField`, and report error if no desired field or over one + * desired fields are found. + */ + def apply( + expr: Expression, + fieldName: String, + resolver: Resolver): GetField = { + def findField(fields: Array[StructField]): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + throw new AnalysisException( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } + expr.dataType match { + case StructType(fields) => + val ordinal = findField(fields) + StructGetField(expr, fields(ordinal), ordinal) + case ArrayType(StructType(fields), containsNull) => + val ordinal = findField(fields) + ArrayGetField(expr, fields(ordinal), ordinal, containsNull) + case otherType => + throw new AnalysisException(s"GetField is not valid on fields of type $otherType") + } + } +} + /** * Returns the value of fields in the Struct `child`. */ case class StructGetField(child: Expression, field: StructField, ordinal: Int) extends GetField { - def dataType = field.dataType - override def nullable = child.nullable || field.nullable + override def dataType: DataType = field.dataType + override def nullable: Boolean = child.nullable || field.nullable override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Row] @@ -101,8 +138,8 @@ case class StructGetField(child: Expression, field: StructField, ordinal: Int) e case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, containsNull: Boolean) extends GetField { - def dataType = ArrayType(field.dataType, containsNull) - override def nullable = child.nullable + override def dataType: DataType = ArrayType(field.dataType, containsNull) + override def nullable: Boolean = child.nullable override def eval(input: Row): Any = { val baseValue = child.eval(input).asInstanceOf[Seq[Row]] @@ -120,7 +157,7 @@ case class ArrayGetField(child: Expression, field: StructField, ordinal: Int, co case class CreateArray(children: Seq[Expression]) extends Expression { override type EvaluatedType = Any - override def foldable = !children.exists(!_.foldable) + override def foldable: Boolean = children.forall(_.foldable) lazy val childTypes = children.map(_.dataType).distinct @@ -140,5 +177,32 @@ case class CreateArray(children: Seq[Expression]) extends Expression { children.map(_.eval(input)) } - override def toString = s"Array(${children.mkString(",")})" + override def toString: String = s"Array(${children.mkString(",")})" +} + +/** + * Returns a Row containing the evaluation of all children expressions. + * TODO: [[CreateStruct]] does not support codegen. + */ +case class CreateStruct(children: Seq[NamedExpression]) extends Expression { + override type EvaluatedType = Row + + override def foldable: Boolean = children.forall(_.foldable) + + override lazy val resolved: Boolean = childrenResolved + + override lazy val dataType: StructType = { + assert(resolved, + s"CreateStruct contains unresolvable children: ${children.filterNot(_.resolved)}.") + val fields = children.map { child => + StructField(child.name, child.dataType, child.nullable, child.metadata) + } + StructType(fields) + } + + override def nullable: Boolean = false + + override def eval(input: Row): EvaluatedType = { + Row(children.map(_.eval(input)): _*) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index 83d8c1d42bca4..adb94df7d1c7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -24,9 +24,9 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { override type EvaluatedType = Any override def dataType: DataType = LongType - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"UnscaledValue($child)" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"UnscaledValue($child)" override def eval(input: Row): Any = { val childResult = child.eval(input) @@ -43,9 +43,9 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un override type EvaluatedType = Decimal override def dataType: DataType = DecimalType(precision, scale) - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"MakeDecimal($child,$precision,$scale)" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"MakeDecimal($child,$precision,$scale)" override def eval(input: Row): Decimal = { val childResult = child.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 0983d274def3f..9a6cb048af5ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.types._ /** @@ -42,83 +42,57 @@ abstract class Generator extends Expression { override type EvaluatedType = TraversableOnce[Row] - override lazy val dataType = - ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))) + // TODO ideally we should return the type of ArrayType(StructType), + // however, we don't keep the output field names in the Generator. + override def dataType: DataType = throw new UnsupportedOperationException - override def nullable = false + override def nullable: Boolean = false /** - * Should be overridden by specific generators. Called only once for each instance to ensure - * that rule application does not change the output schema of a generator. + * The output element data types in structure of Seq[(DataType, Nullable)] + * TODO we probably need to add more information like metadata etc. */ - protected def makeOutput(): Seq[Attribute] - - private var _output: Seq[Attribute] = null - - def output: Seq[Attribute] = { - if (_output == null) { - _output = makeOutput() - } - _output - } + def elementTypes: Seq[(DataType, Boolean)] /** Should be implemented by child classes to perform specific Generators. */ override def eval(input: Row): TraversableOnce[Row] - - /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ - override def makeCopy(newArgs: Array[AnyRef]): this.type = { - val copy = super.makeCopy(newArgs) - copy._output = _output - copy - } } /** * A generator that produces its output using the provided lambda function. */ case class UserDefinedGenerator( - schema: Seq[Attribute], + elementTypes: Seq[(DataType, Boolean)], function: Row => TraversableOnce[Row], children: Seq[Expression]) - extends Generator{ - - override protected def makeOutput(): Seq[Attribute] = schema + extends Generator { override def eval(input: Row): TraversableOnce[Row] = { + // TODO(davies): improve this + // Convert the objects into Scala Type before calling function, we need schema to support UDT + val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true))) val inputRow = new InterpretedProjection(children) - function(inputRow(input)) + function(CatalystTypeConverters.convertToScala(inputRow(input), inputSchema).asInstanceOf[Row]) } - override def toString = s"UserDefinedGenerator(${children.mkString(",")})" + override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" } /** * Given an input array produces a sequence of rows for each value in the array. */ -case class Explode(attributeNames: Seq[String], child: Expression) +case class Explode(child: Expression) extends Generator with trees.UnaryNode[Expression] { override lazy val resolved = child.resolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - private lazy val elementTypes = child.dataType match { + override def elementTypes: Seq[(DataType, Boolean)] = child.dataType match { case ArrayType(et, containsNull) => (et, containsNull) :: Nil case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil } - // TODO: Move this pattern into Generator. - protected def makeOutput() = - if (attributeNames.size == elementTypes.size) { - attributeNames.zip(elementTypes).map { - case (n, (t, nullable)) => AttributeReference(n, t, nullable)() - } - } else { - elementTypes.zipWithIndex.map { - case ((t, nullable), i) => AttributeReference(s"c_$i", t, nullable)() - } - } - override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_, _) => @@ -130,5 +104,5 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - override def toString() = s"explode($child)" + override def toString: String = s"explode($child)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 9ff66563c8164..18cba4cc46707 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types._ object Literal { @@ -29,7 +30,7 @@ object Literal { case f: Float => Literal(f, FloatType) case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) - case s: String => Literal(s, StringType) + case s: String => Literal(UTF8String(s), StringType) case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) @@ -41,6 +42,10 @@ object Literal { case _ => throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } + + def create(v: Any, dataType: DataType): Literal = { + Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) + } } /** @@ -62,16 +67,18 @@ object IntegerLiteral { } } -case class Literal(value: Any, dataType: DataType) extends LeafExpression { - - override def foldable = true - def nullable = value == null +/** + * In order to do type checking, use Literal.create() instead of constructor + */ +case class Literal protected (value: Any, dataType: DataType) extends LeafExpression { + override def foldable: Boolean = true + override def nullable: Boolean = value == null - override def toString = if (value != null) value.toString else "null" + override def toString: String = if (value != null) value.toString else "null" type EvaluatedType = Any - override def eval(input: Row):Any = value + override def eval(input: Row): Any = value } // TODO: Specialize @@ -79,9 +86,9 @@ case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean extends LeafExpression { type EvaluatedType = Any - def update(expression: Expression, input: Row) = { + def update(expression: Expression, input: Row): Unit = { value = expression.eval(input) } - override def eval(input: Row) = value + override def eval(input: Row): Any = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala new file mode 100644 index 0000000000000..fcc06d3aa1036 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala @@ -0,0 +1,93 @@ +/* + * 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.sql.catalyst.expressions.mathfuncs + +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, BinaryExpression, Expression, Row} +import org.apache.spark.sql.types._ + +/** + * A binary expression specifically for math functions that take two `Double`s as input and returns + * a `Double`. + * @param f The math function. + * @param name The short name of the function + */ +abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) + extends BinaryExpression with Serializable with ExpectsInputTypes { self: Product => + type EvaluatedType = Any + override def symbol: String = null + override def expectedChildTypes: Seq[DataType] = Seq(DoubleType, DoubleType) + + override def nullable: Boolean = left.nullable || right.nullable + override def toString: String = s"$name($left, $right)" + + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType && + !DecimalType.isFixed(left.dataType) + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + val result = f(evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double]) + if (result.isNaN) null else result + } + } + } +} + +case class Atan2( + left: Expression, + right: Expression) extends BinaryMathExpression(math.atan2, "ATAN2") { + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if (evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + // With codegen, the values returned by -0.0 and 0.0 are different. Handled with +0.0 + val result = math.atan2(evalE1.asInstanceOf[Double] + 0.0, + evalE2.asInstanceOf[Double] + 0.0) + if (result.isNaN) null else result + } + } + } +} + +case class Hypot( + left: Expression, + right: Expression) extends BinaryMathExpression(math.hypot, "HYPOT") + +case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala new file mode 100644 index 0000000000000..dc68469e060cb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala @@ -0,0 +1,92 @@ +/* + * 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.sql.catalyst.expressions.mathfuncs + +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, Row, UnaryExpression} +import org.apache.spark.sql.types._ + +/** + * A unary expression specifically for math functions. Math Functions expect a specific type of + * input format, therefore these functions extend `ExpectsInputTypes`. + * @param name The short name of the function + */ +abstract class MathematicalExpression(f: Double => Double, name: String) + extends UnaryExpression with Serializable with ExpectsInputTypes { + self: Product => + type EvaluatedType = Any + + override def expectedChildTypes: Seq[DataType] = Seq(DoubleType) + override def dataType: DataType = DoubleType + override def foldable: Boolean = child.foldable + override def nullable: Boolean = true + override def toString: String = s"$name($child)" + + override def eval(input: Row): Any = { + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + val result = f(evalE.asInstanceOf[Double]) + if (result.isNaN) null else result + } + } +} + +case class Acos(child: Expression) extends MathematicalExpression(math.acos, "ACOS") + +case class Asin(child: Expression) extends MathematicalExpression(math.asin, "ASIN") + +case class Atan(child: Expression) extends MathematicalExpression(math.atan, "ATAN") + +case class Cbrt(child: Expression) extends MathematicalExpression(math.cbrt, "CBRT") + +case class Ceil(child: Expression) extends MathematicalExpression(math.ceil, "CEIL") + +case class Cos(child: Expression) extends MathematicalExpression(math.cos, "COS") + +case class Cosh(child: Expression) extends MathematicalExpression(math.cosh, "COSH") + +case class Exp(child: Expression) extends MathematicalExpression(math.exp, "EXP") + +case class Expm1(child: Expression) extends MathematicalExpression(math.expm1, "EXPM1") + +case class Floor(child: Expression) extends MathematicalExpression(math.floor, "FLOOR") + +case class Log(child: Expression) extends MathematicalExpression(math.log, "LOG") + +case class Log10(child: Expression) extends MathematicalExpression(math.log10, "LOG10") + +case class Log1p(child: Expression) extends MathematicalExpression(math.log1p, "LOG1P") + +case class Rint(child: Expression) extends MathematicalExpression(math.rint, "ROUND") + +case class Signum(child: Expression) extends MathematicalExpression(math.signum, "SIGNUM") + +case class Sin(child: Expression) extends MathematicalExpression(math.sin, "SIN") + +case class Sinh(child: Expression) extends MathematicalExpression(math.sinh, "SINH") + +case class Tan(child: Expression) extends MathematicalExpression(math.tan, "TAN") + +case class Tanh(child: Expression) extends MathematicalExpression(math.tanh, "TANH") + +case class ToDegrees(child: Expression) + extends MathematicalExpression(math.toDegrees, "DEGREES") + +case class ToRadians(child: Expression) + extends MathematicalExpression(math.toRadians, "RADIANS") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 17f7f9fe51376..afcb2ce8b9cb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees.LeafNode import org.apache.spark.sql.types._ object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() - def newExprId = ExprId(curId.getAndIncrement()) + def newExprId: ExprId = ExprId(curId.getAndIncrement()) def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType) } @@ -41,6 +42,13 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId + /** + * Returns a dot separated fully qualified name for this attribute. Given that there can be + * multiple qualifiers, it is possible that there are other possible way to refer to this + * attribute. + */ + def qualifiedName: String = (qualifiers.headOption.toSeq :+ name).mkString(".") + /** * All possible qualifiers for the expression. * @@ -72,13 +80,13 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => - override def references = AttributeSet(this) + override def references: AttributeSet = AttributeSet(this) def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute - def toAttribute = this + def toAttribute: Attribute = this def newInstance(): Attribute } @@ -95,25 +103,32 @@ abstract class Attribute extends NamedExpression { * @param name the name to be associated with the result of computing [[child]]. * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this * alias. Auto-assigned if left blank. + * @param explicitMetadata Explicit metadata associated with this alias that overwrites child's. */ -case class Alias(child: Expression, name: String) - (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) +case class Alias(child: Expression, name: String)( + val exprId: ExprId = NamedExpression.newExprId, + val qualifiers: Seq[String] = Nil, + val explicitMetadata: Option[Metadata] = None) extends NamedExpression with trees.UnaryNode[Expression] { override type EvaluatedType = Any + // Alias(Generator, xx) need to be transformed into Generate(generator, ...) + override lazy val resolved = childrenResolved && !child.isInstanceOf[Generator] - override def eval(input: Row) = child.eval(input) + override def eval(input: Row): Any = child.eval(input) - override def dataType = child.dataType - override def nullable = child.nullable + override def dataType: DataType = child.dataType + override def nullable: Boolean = child.nullable override def metadata: Metadata = { - child match { - case named: NamedExpression => named.metadata - case _ => Metadata.empty + explicitMetadata.getOrElse { + child match { + case named: NamedExpression => named.metadata + case _ => Metadata.empty + } } } - override def toAttribute = { + override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifiers) } else { @@ -123,11 +138,14 @@ case class Alias(child: Expression, name: String) override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" - override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + override protected final def otherCopyArgs: Seq[AnyRef] = { + exprId :: qualifiers :: explicitMetadata :: Nil + } override def equals(other: Any): Boolean = other match { case a: Alias => - name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers + name == a.name && exprId == a.exprId && child == a.child && qualifiers == a.qualifiers && + explicitMetadata == a.explicitMetadata case _ => false } } @@ -153,7 +171,7 @@ case class AttributeReference( val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType case _ => false } @@ -167,7 +185,7 @@ case class AttributeReference( h } - override def newInstance() = + override def newInstance(): AttributeReference = AttributeReference(name, dataType, nullable, metadata)(qualifiers = qualifiers) /** @@ -192,7 +210,7 @@ case class AttributeReference( /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ - override def withQualifiers(newQualifiers: Seq[String]) = { + override def withQualifiers(newQualifiers: Seq[String]): AttributeReference = { if (newQualifiers.toSet == qualifiers.toSet) { this } else { @@ -214,20 +232,22 @@ case class AttributeReference( case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { type EvaluatedType = Any - override def toString = name - - override def withNullability(newNullability: Boolean): Attribute = ??? - override def newInstance(): Attribute = ??? - override def withQualifiers(newQualifiers: Seq[String]): Attribute = ??? - override def withName(newName: String): Attribute = ??? - override def qualifiers: Seq[String] = ??? - override def exprId: ExprId = ??? - override def eval(input: Row): EvaluatedType = ??? - override def nullable: Boolean = ??? + override def toString: String = name + + override def withNullability(newNullability: Boolean): Attribute = + throw new UnsupportedOperationException + override def newInstance(): Attribute = throw new UnsupportedOperationException + override def withQualifiers(newQualifiers: Seq[String]): Attribute = + throw new UnsupportedOperationException + override def withName(newName: String): Attribute = throw new UnsupportedOperationException + override def qualifiers: Seq[String] = throw new UnsupportedOperationException + override def exprId: ExprId = throw new UnsupportedOperationException + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = throw new UnsupportedOperationException override def dataType: DataType = NullType } object VirtualColumn { - val groupingIdName = "grouping__id" - def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)() + val groupingIdName: String = "grouping__id" + def newGroupingId: AttributeReference = AttributeReference(groupingIdName, IntegerType, false)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 08b982bc671e7..f9161cf34f0c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -19,22 +19,23 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.types.DataType case class Coalesce(children: Seq[Expression]) extends Expression { type EvaluatedType = Any /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ - def nullable = !children.exists(!_.nullable) + override def nullable: Boolean = !children.exists(!_.nullable) // Coalesce is foldable if all children are foldable. - override def foldable = !children.exists(!_.foldable) + override def foldable: Boolean = !children.exists(!_.foldable) // Only resolved if all the children are of the same type. override lazy val resolved = childrenResolved && (children.map(_.dataType).distinct.size == 1) - override def toString = s"Coalesce(${children.mkString(",")})" + override def toString: String = s"Coalesce(${children.mkString(",")})" - def dataType = if (resolved) { + override def dataType: DataType = if (resolved) { children.head.dataType } else { val childTypes = children.map(c => s"$c: ${c.dataType}").mkString(", ") @@ -54,22 +55,45 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - override def foldable = child.foldable - def nullable = false + override def foldable: Boolean = child.foldable + override def nullable: Boolean = false override def eval(input: Row): Any = { child.eval(input) == null } - override def toString = s"IS NULL $child" + override def toString: String = s"IS NULL $child" } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - override def foldable = child.foldable - def nullable = false - override def toString = s"IS NOT NULL $child" + override def foldable: Boolean = child.foldable + override def nullable: Boolean = false + override def toString: String = s"IS NOT NULL $child" override def eval(input: Row): Any = { child.eval(input) != null } } + +/** + * A predicate that is evaluated to be true if there are at least `n` non-null values. + */ +case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate { + override def nullable: Boolean = false + override def foldable: Boolean = false + override def toString: String = s"AtLeastNNulls(n, ${children.mkString(",")})" + + private[this] val childrenArray = children.toArray + + override def eval(input: Row): Boolean = { + var numNonNulls = 0 + var i = 0 + while (i < childrenArray.length && numNonNulls < n) { + if (childrenArray(i).eval(input) != null) { + numNonNulls += 1 + } + i += 1 + } + numNonNulls >= n + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 0024ef92c0452..26c38c56c04f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.{BinaryType, BooleanType, NativeType} +import org.apache.spark.sql.types.{DataType, BinaryType, BooleanType, AtomicType} object InterpretedPredicate { - def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = - apply(BindReferences.bindReference(expression, inputSchema)) + def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = + create(BindReferences.bindReference(expression, inputSchema)) - def apply(expression: Expression): (Row => Boolean) = { + def create(expression: Expression): (Row => Boolean) = { (r: Row) => expression.eval(r).asInstanceOf[Boolean] } } @@ -34,7 +34,7 @@ object InterpretedPredicate { trait Predicate extends Expression { self: Product => - def dataType = BooleanType + override def dataType: DataType = BooleanType type EvaluatedType = Any } @@ -70,15 +70,13 @@ trait PredicateHelper { expr.references.subsetOf(plan.outputSet) } -abstract class BinaryPredicate extends BinaryExpression with Predicate { - self: Product => - def nullable = left.nullable || right.nullable -} -case class Not(child: Expression) extends UnaryExpression with Predicate { - override def foldable = child.foldable - def nullable = child.nullable - override def toString = s"NOT $child" +case class Not(child: Expression) extends UnaryExpression with Predicate with ExpectsInputTypes { + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def toString: String = s"NOT $child" + + override def expectedChildTypes: Seq[DataType] = Seq(BooleanType) override def eval(input: Row): Any = { child.eval(input) match { @@ -92,10 +90,10 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { * Evaluates to `true` if `list` contains `value`. */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { - def children = value +: list + override def children: Seq[Expression] = value +: list - def nullable = true // TODO: Figure out correct nullability semantics of IN. - override def toString = s"$value IN ${list.mkString("(", ",", ")")}" + override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN. + override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}" override def eval(input: Row): Any = { val evaluatedValue = value.eval(input) @@ -110,18 +108,22 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { case class InSet(value: Expression, hset: Set[Any]) extends Predicate { - def children = value :: Nil + override def children: Seq[Expression] = value :: Nil - def nullable = true // TODO: Figure out correct nullability semantics of IN. - override def toString = s"$value INSET ${hset.mkString("(", ",", ")")}" + override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN. + override def toString: String = s"$value INSET ${hset.mkString("(", ",", ")")}" override def eval(input: Row): Any = { hset.contains(value.eval(input)) } } -case class And(left: Expression, right: Expression) extends BinaryPredicate { - def symbol = "&&" +case class And(left: Expression, right: Expression) + extends BinaryExpression with Predicate with ExpectsInputTypes { + + override def expectedChildTypes: Seq[DataType] = Seq(BooleanType, BooleanType) + + override def symbol: String = "&&" override def eval(input: Row): Any = { val l = left.eval(input) @@ -142,8 +144,12 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { } } -case class Or(left: Expression, right: Expression) extends BinaryPredicate { - def symbol = "||" +case class Or(left: Expression, right: Expression) + extends BinaryExpression with Predicate with ExpectsInputTypes { + + override def expectedChildTypes: Seq[DataType] = Seq(BooleanType, BooleanType) + + override def symbol: String = "||" override def eval(input: Row): Any = { val l = left.eval(input) @@ -164,12 +170,13 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { } } -abstract class BinaryComparison extends BinaryPredicate { +abstract class BinaryComparison extends BinaryExpression with Predicate { self: Product => } case class EqualTo(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "=" + override def symbol: String = "=" + override def eval(input: Row): Any = { val l = left.eval(input) if (l == null) { @@ -178,15 +185,16 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison val r = right.eval(input) if (r == null) null else if (left.dataType != BinaryType) l == r - else BinaryType.ordering.compare( - l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) == 0 + else java.util.Arrays.equals(l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) } } } case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<=>" - override def nullable = false + override def symbol: String = "<=>" + + override def nullable: Boolean = false + override def eval(input: Row): Any = { val l = left.eval(input) val r = right.eval(input) @@ -201,22 +209,22 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<" + override def symbol: String = "<" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -230,22 +238,22 @@ case class LessThan(left: Expression, right: Expression) extends BinaryCompariso } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { - def symbol = "<=" + override def symbol: String = "<=" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -259,15 +267,15 @@ case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryCo } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { - def symbol = ">" + override def symbol: String = ">" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } @@ -288,22 +296,22 @@ case class GreaterThan(left: Expression, right: Expression) extends BinaryCompar } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { - def symbol = ">=" + override def symbol: String = ">=" - lazy val ordering = { + lazy val ordering: Ordering[Any] = { if (left.dataType != right.dataType) { throw new TreeNodeException(this, s"Types do not match ${left.dataType} != ${right.dataType}") } left.dataType match { - case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]] case other => sys.error(s"Type $other does not support ordered operations") } } override def eval(input: Row): Any = { val evalE1 = left.eval(input) - if(evalE1 == null) { + if (evalE1 == null) { null } else { val evalE2 = right.eval(input) @@ -317,13 +325,13 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) - extends Expression { + extends Expression { - def children = predicate :: trueValue :: falseValue :: Nil - override def nullable = trueValue.nullable || falseValue.nullable + override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil + override def nullable: Boolean = trueValue.nullable || falseValue.nullable override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType - def dataType = { + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException( this, @@ -342,7 +350,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } } - override def toString = s"if ($predicate) $trueValue else $falseValue" + override def toString: String = s"if ($predicate) $trueValue else $falseValue" } // scalastyle:off @@ -362,9 +370,10 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi // scalastyle:on case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any - def children = branches - def dataType = { + override def children: Seq[Expression] = branches + + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") } @@ -379,12 +388,12 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { @transient private[this] lazy val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) - override def nullable = { + override def nullable: Boolean = { // If no value is nullable and no elseValue is provided, the whole statement defaults to null. values.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true)) } - override lazy val resolved = { + override lazy val resolved: Boolean = { if (!childrenResolved) { false } else { @@ -415,7 +424,7 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { res } - override def toString = { + override def toString: String = { "CASE" + branches.sliding(2, 2).map { case Seq(cond, value) => s" WHEN $cond THEN $value" case Seq(elseValue) => s" ELSE $elseValue" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala new file mode 100644 index 0000000000000..66d7c8b07cce8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala @@ -0,0 +1,56 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.TaskContext +import org.apache.spark.sql.types.{DataType, DoubleType} +import org.apache.spark.util.Utils +import org.apache.spark.util.random.XORShiftRandom + +/** + * A Random distribution generating expression. + * TODO: This can be made generic to generate any type of random distribution, or any type of + * StructType. + * + * Since this expression is stateful, it cannot be a case object. + */ +abstract class RDG(seed: Long) extends LeafExpression with Serializable { + self: Product => + + /** + * Record ID within each partition. By being transient, the Random Number Generator is + * reset every time we serialize and deserialize it. + */ + @transient protected lazy val rng = new XORShiftRandom(seed + TaskContext.get().partitionId()) + + override type EvaluatedType = Double + + override def nullable: Boolean = false + + override def dataType: DataType = DoubleType +} + +/** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ +case class Rand(seed: Long = Utils.random.nextLong()) extends RDG(seed) { + override def eval(input: Row): Double = rng.nextDouble() +} + +/** Generate a random column with i.i.d. gaussian random distribution. */ +case class Randn(seed: Long = Utils.random.nextLong()) extends RDG(seed) { + override def eval(input: Row): Double = rng.nextGaussian() +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index f03d6f71a9fae..5fd892c42e69c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{StructType, NativeType} - +import org.apache.spark.sql.types.{UTF8String, DataType, StructType, AtomicType} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -37,6 +36,7 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) + // TODO(davies): add setDate() and setDecimal() } /** @@ -44,8 +44,8 @@ trait MutableRow extends Row { */ object EmptyRow extends Row { override def apply(i: Int): Any = throw new UnsupportedOperationException - override def toSeq = Seq.empty - override def length = 0 + override def toSeq: Seq[Any] = Seq.empty + override def length: Int = 0 override def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException override def getInt(i: Int): Int = throw new UnsupportedOperationException override def getLong(i: Int): Long = throw new UnsupportedOperationException @@ -56,7 +56,7 @@ object EmptyRow extends Row { override def getByte(i: Int): Byte = throw new UnsupportedOperationException override def getString(i: Int): String = throw new UnsupportedOperationException override def getAs[T](i: Int): T = throw new UnsupportedOperationException - def copy() = this + override def copy(): Row = this } /** @@ -66,17 +66,17 @@ object EmptyRow extends Row { */ class GenericRow(protected[sql] val values: Array[Any]) extends Row { /** No-arg constructor for serialization. */ - def this() = this(null) + protected def this() = this(null) def this(size: Int) = this(new Array[Any](size)) - override def toSeq = values.toSeq + override def toSeq: Seq[Any] = values.toSeq - override def length = values.length + override def length: Int = values.length - override def apply(i: Int) = values(i) + override def apply(i: Int): Any = values(i) - override def isNullAt(i: Int) = values(i) == null + override def isNullAt(i: Int): Boolean = values(i) == null override def getInt(i: Int): Int = { if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") @@ -114,9 +114,15 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { } override def getString(i: Int): String = { - values(i).asInstanceOf[String] + values(i) match { + case null => null + case s: String => s + case utf8: UTF8String => utf8.toString + } } + // TODO(davies): add getDate and getDecimal + // Custom hashCode function that matches the efficient code generated version. override def hashCode: Int = { var result: Int = 37 @@ -167,16 +173,21 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { case _ => false } - def copy() = this + override def copy(): Row = this } class GenericRowWithSchema(values: Array[Any], override val schema: StructType) extends GenericRow(values) { + + /** No-arg constructor for serialization. */ + protected def this() = this(null, null) + + override def fieldIndex(name: String): Int = schema.fieldIndex(name) } class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { /** No-arg constructor for serialization. */ - def this() = this(null) + protected def this() = this(null) def this(size: Int) = this(new Array[Any](size)) @@ -186,15 +197,14 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { override def setFloat(ordinal: Int, value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int, value: Int): Unit = { values(ordinal) = value } override def setLong(ordinal: Int, value: Long): Unit = { values(ordinal) = value } - override def setString(ordinal: Int, value: String): Unit = { values(ordinal) = value } - + override def setString(ordinal: Int, value: String) { values(ordinal) = UTF8String(value)} override def setNullAt(i: Int): Unit = { values(i) = null } override def setShort(ordinal: Int, value: Short): Unit = { values(ordinal) = value } override def update(ordinal: Int, value: Any): Unit = { values(ordinal) = value } - override def copy() = new GenericRow(values.clone()) + override def copy(): Row = new GenericRow(values.clone()) } @@ -217,10 +227,11 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return if (order.direction == Ascending) 1 else -1 } else { val comparison = order.dataType match { - case n: NativeType if order.direction == Ascending => + case n: AtomicType if order.direction == Ascending => n.ordering.asInstanceOf[Ordering[Any]].compare(left, right) - case n: NativeType if order.direction == Descending => + case n: AtomicType if order.direction == Descending => n.ordering.asInstanceOf[Ordering[Any]].reverse.compare(left, right) + case other => sys.error(s"Type $other does not support ordered operations") } if (comparison != 0) return comparison } @@ -229,3 +240,10 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return 0 } } + +object RowOrdering { + def forSchema(dataTypes: Seq[DataType]): RowOrdering = + new RowOrdering(dataTypes.zipWithIndex.map { + case(dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + }) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index 3a5bdca1f07c3..4c44182278207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -20,23 +20,48 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet +/** The data type for expressions returning an OpenHashSet as the result. */ +private[sql] class OpenHashSetUDT( + val elementType: DataType) extends UserDefinedType[OpenHashSet[Any]] { + + override def sqlType: DataType = ArrayType(elementType) + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def serialize(obj: Any): Seq[Any] = { + obj.asInstanceOf[OpenHashSet[Any]].iterator.toSeq + } + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def deserialize(datum: Any): OpenHashSet[Any] = { + val iterator = datum.asInstanceOf[Seq[Any]].iterator + val set = new OpenHashSet[Any] + while(iterator.hasNext) { + set.add(iterator.next()) + } + + set + } + + override def userClass: Class[OpenHashSet[Any]] = classOf[OpenHashSet[Any]] + + private[spark] override def asNullable: OpenHashSetUDT = this +} + /** * Creates a new set of the specified type */ case class NewSet(elementType: DataType) extends LeafExpression { type EvaluatedType = Any - def nullable = false + override def nullable: Boolean = false - // We are currently only using these Expressions internally for aggregation. However, if we ever - // expose these to users we'll want to create a proper type instead of hijacking ArrayType. - def dataType = ArrayType(elementType) + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(elementType) - def eval(input: Row): Any = { + override def eval(input: Row): Any = { new OpenHashSet[Any]() } - override def toString = s"new Set($dataType)" + override def toString: String = s"new Set($dataType)" } /** @@ -46,12 +71,13 @@ case class NewSet(elementType: DataType) extends LeafExpression { case class AddItemToSet(item: Expression, set: Expression) extends Expression { type EvaluatedType = Any - def children = item :: set :: Nil + override def children: Seq[Expression] = item :: set :: Nil + + override def nullable: Boolean = set.nullable - def nullable = set.nullable + override def dataType: OpenHashSetUDT = set.dataType.asInstanceOf[OpenHashSetUDT] - def dataType = set.dataType - def eval(input: Row): Any = { + override def eval(input: Row): Any = { val itemEval = item.eval(input) val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] @@ -67,7 +93,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { } } - override def toString = s"$set += $item" + override def toString: String = s"$set += $item" } /** @@ -77,13 +103,13 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { case class CombineSets(left: Expression, right: Expression) extends BinaryExpression { type EvaluatedType = Any - def nullable = left.nullable || right.nullable + override def nullable: Boolean = left.nullable || right.nullable - def dataType = left.dataType + override def dataType: OpenHashSetUDT = left.dataType.asInstanceOf[OpenHashSetUDT] - def symbol = "++=" + override def symbol: String = "++=" - def eval(input: Row): Any = { + override def eval(input: Row): Any = { val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]] if(leftEval != null) { val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]] @@ -109,16 +135,16 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres case class CountSet(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def nullable = child.nullable + override def nullable: Boolean = child.nullable - def dataType = LongType + override def dataType: DataType = LongType - def eval(input: Row): Any = { + override def eval(input: Row): Any = { val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]] if (childEval != null) { childEval.size.toLong } } - override def toString = s"$child.count()" + override def toString: String = s"$child.count()" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index f85ee0a9bb6d8..d6f23df30ffb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,13 +19,10 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import scala.collection.IndexedSeqOptimized - - import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType} +import org.apache.spark.sql.types._ -trait StringRegexExpression { +trait StringRegexExpression extends ExpectsInputTypes { self: BinaryExpression => type EvaluatedType = Any @@ -33,8 +30,9 @@ trait StringRegexExpression { def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - def nullable: Boolean = left.nullable || right.nullable - def dataType: DataType = BooleanType + override def nullable: Boolean = left.nullable || right.nullable + override def dataType: DataType = BooleanType + override def expectedChildTypes: Seq[DataType] = Seq(StringType, StringType) // try cache the pattern for Literal private lazy val cache: Pattern = right match { @@ -60,49 +58,28 @@ trait StringRegexExpression { if(r == null) { null } else { - val regex = pattern(r.asInstanceOf[String]) + val regex = pattern(r.asInstanceOf[UTF8String].toString()) if(regex == null) { null } else { - matches(regex, l.asInstanceOf[String]) + matches(regex, l.asInstanceOf[UTF8String].toString()) } } } } } -trait CaseConversionExpression { - self: UnaryExpression => - - type EvaluatedType = Any - - def convert(v: String): String - - override def foldable: Boolean = child.foldable - def nullable: Boolean = child.nullable - def dataType: DataType = StringType - - override def eval(input: Row): Any = { - val evaluated = child.eval(input) - if (evaluated == null) { - null - } else { - convert(evaluated.toString) - } - } -} - /** * Simple RegEx pattern matching function */ case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - def symbol = "LIKE" + override def symbol: String = "LIKE" // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character - override def escape(v: String) = + override def escape(v: String): String = if (!v.isEmpty) { "(?s)" + (' ' +: v.init).zip(v).flatMap { case (prev, '\\') => "" @@ -129,19 +106,41 @@ case class Like(left: Expression, right: Expression) case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - def symbol = "RLIKE" + override def symbol: String = "RLIKE" override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } +trait CaseConversionExpression extends ExpectsInputTypes { + self: UnaryExpression => + + type EvaluatedType = Any + + def convert(v: UTF8String): UTF8String + + override def foldable: Boolean = child.foldable + override def nullable: Boolean = child.nullable + override def dataType: DataType = StringType + override def expectedChildTypes: Seq[DataType] = Seq(StringType) + + override def eval(input: Row): Any = { + val evaluated = child.eval(input) + if (evaluated == null) { + null + } else { + convert(evaluated.asInstanceOf[UTF8String]) + } + } +} + /** * A function that converts the characters of a string to uppercase. */ case class Upper(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toUpperCase() + override def convert(v: UTF8String): UTF8String = v.toUpperCase() - override def toString() = s"Upper($child)" + override def toString: String = s"Upper($child)" } /** @@ -149,85 +148,89 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE */ case class Lower(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toLowerCase() + override def convert(v: UTF8String): UTF8String = v.toLowerCase() - override def toString() = s"Lower($child)" + override def toString: String = s"Lower($child)" } /** A base trait for functions that compare two strings, returning a boolean. */ trait StringComparison { self: BinaryExpression => - type EvaluatedType = Any + def compare(l: UTF8String, r: UTF8String): Boolean - def nullable: Boolean = left.nullable || right.nullable - override def dataType: DataType = BooleanType + override type EvaluatedType = Any - def compare(l: String, r: String): Boolean + override def nullable: Boolean = left.nullable || right.nullable override def eval(input: Row): Any = { - val leftEval = left.eval(input).asInstanceOf[String] + val leftEval = left.eval(input) if(leftEval == null) { null } else { - val rightEval = right.eval(input).asInstanceOf[String] - if (rightEval == null) null else compare(leftEval, rightEval) + val rightEval = right.eval(input) + if (rightEval == null) null + else compare(leftEval.asInstanceOf[UTF8String], rightEval.asInstanceOf[UTF8String]) } } - def symbol: String = nodeName + override def symbol: String = nodeName - override def toString() = s"$nodeName($left, $right)" + override def toString: String = s"$nodeName($left, $right)" } /** * A function that returns true if the string `left` contains the string `right`. */ case class Contains(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { - override def compare(l: String, r: String) = l.contains(r) + extends BinaryExpression with Predicate with StringComparison with ExpectsInputTypes { + override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) + override def expectedChildTypes: Seq[DataType] = Seq(StringType, StringType) } /** * A function that returns true if the string `left` starts with the string `right`. */ case class StartsWith(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { - def compare(l: String, r: String) = l.startsWith(r) + extends BinaryExpression with Predicate with StringComparison with ExpectsInputTypes { + override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) + override def expectedChildTypes: Seq[DataType] = Seq(StringType, StringType) } /** * A function that returns true if the string `left` ends with the string `right`. */ case class EndsWith(left: Expression, right: Expression) - extends BinaryExpression with StringComparison { - def compare(l: String, r: String) = l.endsWith(r) + extends BinaryExpression with Predicate with StringComparison with ExpectsInputTypes { + override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) + override def expectedChildTypes: Seq[DataType] = Seq(StringType, StringType) } /** * A function that takes a substring of its first argument starting at a given position. * Defined for String and Binary types. */ -case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression { +case class Substring(str: Expression, pos: Expression, len: Expression) + extends Expression with ExpectsInputTypes { type EvaluatedType = Any - override def foldable = str.foldable && pos.foldable && len.foldable + override def foldable: Boolean = str.foldable && pos.foldable && len.foldable - def nullable: Boolean = str.nullable || pos.nullable || len.nullable - def dataType: DataType = { + override def nullable: Boolean = str.nullable || pos.nullable || len.nullable + override def dataType: DataType = { if (!resolved) { throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved") } if (str.dataType == BinaryType) str.dataType else StringType } - override def children = str :: pos :: len :: Nil + override def expectedChildTypes: Seq[DataType] = Seq(StringType, IntegerType, IntegerType) + + override def children: Seq[Expression] = str :: pos :: len :: Nil @inline - def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) - (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { - val len = str.length + def slicePos(startPos: Int, sliceLen: Int, length: () => Int): (Int, Int) = { // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and // negative indices for start positions. If a start index i is greater than 0, it // refers to element i-1 in the sequence. If a start index i is less than 0, it refers @@ -236,7 +239,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends val start = startPos match { case pos if pos > 0 => pos - 1 - case neg if neg < 0 => len + neg + case neg if neg < 0 => length() + neg case _ => 0 } @@ -245,12 +248,11 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends case x => start + x } - str.slice(start, end) + (start, end) } override def eval(input: Row): Any = { val string = str.eval(input) - val po = pos.eval(input) val ln = len.eval(input) @@ -258,16 +260,20 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends null } else { val start = po.asInstanceOf[Int] - val length = ln.asInstanceOf[Int] - + val length = ln.asInstanceOf[Int] string match { - case ba: Array[Byte] => slice(ba, start, length) - case other => slice(other.toString, start, length) + case ba: Array[Byte] => + val (st, end) = slicePos(start, length, () => ba.length) + ba.slice(st, end) + case s: UTF8String => + val (st, end) = slicePos(start, length, () => s.length()) + s.slice(st, end) } } } - override def toString = len match { + override def toString: String = len match { + // TODO: This is broken because max is not an integer value. case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)" case _ => s"SUBSTR($str, $pos, $len)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala new file mode 100644 index 0000000000000..099d67ca7fee3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -0,0 +1,340 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.{NumericType, DataType} + +/** + * The trait of the Window Specification (specified in the OVER clause or WINDOW clause) for + * Window Functions. + */ +sealed trait WindowSpec + +/** + * The specification for a window function. + * @param partitionSpec It defines the way that input rows are partitioned. + * @param orderSpec It defines the ordering of rows in a partition. + * @param frameSpecification It defines the window frame in a partition. + */ +case class WindowSpecDefinition( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + frameSpecification: WindowFrame) extends Expression with WindowSpec { + + def validate: Option[String] = frameSpecification match { + case UnspecifiedFrame => + Some("Found a UnspecifiedFrame. It should be converted to a SpecifiedWindowFrame " + + "during analysis. Please file a bug report.") + case frame: SpecifiedWindowFrame => frame.validate.orElse { + def checkValueBasedBoundaryForRangeFrame(): Option[String] = { + if (orderSpec.length > 1) { + // It is not allowed to have a value-based PRECEDING and FOLLOWING + // as the boundary of a Range Window Frame. + Some("This Range Window Frame only accepts at most one ORDER BY expression.") + } else if (orderSpec.nonEmpty && !orderSpec.head.dataType.isInstanceOf[NumericType]) { + Some("The data type of the expression in the ORDER BY clause should be a numeric type.") + } else { + None + } + } + + (frame.frameType, frame.frameStart, frame.frameEnd) match { + case (RangeFrame, vp: ValuePreceding, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, vf: ValueFollowing, _) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vp: ValuePreceding) => checkValueBasedBoundaryForRangeFrame() + case (RangeFrame, _, vf: ValueFollowing) => checkValueBasedBoundaryForRangeFrame() + case (_, _, _) => None + } + } + } + + type EvaluatedType = Any + + override def children: Seq[Expression] = partitionSpec ++ orderSpec + + override lazy val resolved: Boolean = + childrenResolved && frameSpecification.isInstanceOf[SpecifiedWindowFrame] + + + override def toString: String = simpleString + + override def eval(input: Row): EvaluatedType = throw new UnsupportedOperationException + override def nullable: Boolean = true + override def foldable: Boolean = false + override def dataType: DataType = throw new UnsupportedOperationException +} + +/** + * A Window specification reference that refers to the [[WindowSpecDefinition]] defined + * under the name `name`. + */ +case class WindowSpecReference(name: String) extends WindowSpec + +/** + * The trait used to represent the type of a Window Frame. + */ +sealed trait FrameType + +/** + * RowFrame treats rows in a partition individually. When a [[ValuePreceding]] + * or a [[ValueFollowing]] is used as its [[FrameBoundary]], the value is considered + * as a physical offset. + * For example, `ROW BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a 3-row frame, + * from the row precedes the current row to the row follows the current row. + */ +case object RowFrame extends FrameType + +/** + * RangeFrame treats rows in a partition as groups of peers. + * All rows having the same `ORDER BY` ordering are considered as peers. + * When a [[ValuePreceding]] or a [[ValueFollowing]] is used as its [[FrameBoundary]], + * the value is considered as a logical offset. + * For example, assuming the value of the current row's `ORDER BY` expression `expr` is `v`, + * `RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING` represents a frame containing rows whose values + * `expr` are in the range of [v-1, v+1]. + * + * If `ORDER BY` clause is not defined, all rows in the partition is considered as peers + * of the current row. + */ +case object RangeFrame extends FrameType + +/** + * The trait used to represent the type of a Window Frame Boundary. + */ +sealed trait FrameBoundary { + def notFollows(other: FrameBoundary): Boolean +} + +/** UNBOUNDED PRECEDING boundary. */ +case object UnboundedPreceding extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => true + case vp: ValuePreceding => true + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED PRECEDING" +} + +/** PRECEDING boundary. */ +case class ValuePreceding(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case ValuePreceding(anotherValue) => value >= anotherValue + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = s"$value PRECEDING" +} + +/** CURRENT ROW boundary. */ +case object CurrentRow extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => true + case vf: ValueFollowing => true + case UnboundedFollowing => true + } + + override def toString: String = "CURRENT ROW" +} + +/** FOLLOWING boundary. */ +case class ValueFollowing(value: Int) extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case ValueFollowing(anotherValue) => value <= anotherValue + case UnboundedFollowing => true + } + + override def toString: String = s"$value FOLLOWING" +} + +/** UNBOUNDED FOLLOWING boundary. */ +case object UnboundedFollowing extends FrameBoundary { + def notFollows(other: FrameBoundary): Boolean = other match { + case UnboundedPreceding => false + case vp: ValuePreceding => false + case CurrentRow => false + case vf: ValueFollowing => false + case UnboundedFollowing => true + } + + override def toString: String = "UNBOUNDED FOLLOWING" +} + +/** + * The trait used to represent the a Window Frame. + */ +sealed trait WindowFrame + +/** Used as a place holder when a frame specification is not defined. */ +case object UnspecifiedFrame extends WindowFrame + +/** A specified Window Frame. */ +case class SpecifiedWindowFrame( + frameType: FrameType, + frameStart: FrameBoundary, + frameEnd: FrameBoundary) extends WindowFrame { + + /** If this WindowFrame is valid or not. */ + def validate: Option[String] = (frameType, frameStart, frameEnd) match { + case (_, UnboundedFollowing, _) => + Some(s"$UnboundedFollowing is not allowed as the start of a Window Frame.") + case (_, _, UnboundedPreceding) => + Some(s"$UnboundedPreceding is not allowed as the end of a Window Frame.") + // case (RowFrame, start, end) => ??? RowFrame specific rule + // case (RangeFrame, start, end) => ??? RangeFrame specific rule + case (_, start, end) => + if (start.notFollows(end)) { + None + } else { + val reason = + s"The end of this Window Frame $end is smaller than the start of " + + s"this Window Frame $start." + Some(reason) + } + } + + override def toString: String = frameType match { + case RowFrame => s"ROWS BETWEEN $frameStart AND $frameEnd" + case RangeFrame => s"RANGE BETWEEN $frameStart AND $frameEnd" + } +} + +object SpecifiedWindowFrame { + /** + * + * @param hasOrderSpecification If the window spec has order by expressions. + * @param acceptWindowFrame If the window function accepts user-specified frame. + * @return + */ + def defaultWindowFrame( + hasOrderSpecification: Boolean, + acceptWindowFrame: Boolean): SpecifiedWindowFrame = { + if (hasOrderSpecification && acceptWindowFrame) { + // If order spec is defined and the window function supports user specified window frames, + // the default frame is RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. + SpecifiedWindowFrame(RangeFrame, UnboundedPreceding, CurrentRow) + } else { + // Otherwise, the default frame is + // ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, UnboundedFollowing) + } + } +} + +/** + * Every window function needs to maintain a output buffer for its output. + * It should expect that for a n-row window frame, it will be called n times + * to retrieve value corresponding with these n rows. + */ +trait WindowFunction extends Expression { + self: Product => + + def init(): Unit + + def reset(): Unit + + def prepareInputParameters(input: Row): AnyRef + + def update(input: AnyRef): Unit + + def batchUpdate(inputs: Array[AnyRef]): Unit + + def evaluate(): Unit + + def get(index: Int): Any + + def newInstance(): WindowFunction +} + +case class UnresolvedWindowFunction( + name: String, + children: Seq[Expression]) + extends Expression with WindowFunction { + + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + override def init(): Unit = + throw new UnresolvedException(this, "init") + override def reset(): Unit = + throw new UnresolvedException(this, "reset") + override def prepareInputParameters(input: Row): AnyRef = + throw new UnresolvedException(this, "prepareInputParameters") + override def update(input: AnyRef): Unit = + throw new UnresolvedException(this, "update") + override def batchUpdate(inputs: Array[AnyRef]): Unit = + throw new UnresolvedException(this, "batchUpdate") + override def evaluate(): Unit = + throw new UnresolvedException(this, "evaluate") + override def get(index: Int): Any = + throw new UnresolvedException(this, "get") + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def toString: String = s"'$name(${children.mkString(",")})" + + override def newInstance(): WindowFunction = + throw new UnresolvedException(this, "newInstance") +} + +case class UnresolvedWindowExpression( + child: UnresolvedWindowFunction, + windowSpec: WindowSpecReference) extends UnaryExpression { + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def foldable: Boolean = throw new UnresolvedException(this, "foldable") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") +} + +case class WindowExpression( + windowFunction: WindowFunction, + windowSpec: WindowSpecDefinition) extends Expression { + override type EvaluatedType = Any + + override def children: Seq[Expression] = + windowFunction :: windowSpec :: Nil + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def dataType: DataType = windowFunction.dataType + override def foldable: Boolean = windowFunction.foldable + override def nullable: Boolean = windowFunction.nullable + + override def toString: String = s"$windowFunction $windowSpec" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1a75fcf3545bd..709f7d672d931 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.immutable.HashSet +import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.FullOuter @@ -32,7 +33,16 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] object DefaultOptimizer extends Optimizer { val batches = - Batch("Combine Limits", FixedPoint(100), + // SubQueries are only needed for analysis and can be removed before execution. + Batch("Remove SubQueries", FixedPoint(100), + EliminateSubQueries) :: + Batch("Operator Reordering", FixedPoint(100), + UnionPushdown, + CombineFilters, + PushPredicateThroughProject, + PushPredicateThroughJoin, + PushPredicateThroughGenerate, + ColumnPruning, CombineLimits) :: Batch("ConstantFolding", FixedPoint(100), NullPropagation, @@ -45,13 +55,6 @@ object DefaultOptimizer extends Optimizer { OptimizeIn) :: Batch("Decimal Optimizations", FixedPoint(100), DecimalAggregates) :: - Batch("Filter Pushdown", FixedPoint(100), - UnionPushdown, - CombineFilters, - PushPredicateThroughProject, - PushPredicateThroughJoin, - PushPredicateThroughGenerate, - ColumnPruning) :: Batch("LocalRelation", FixedPoint(100), ConvertToLocalRelation) :: Nil } @@ -137,7 +140,7 @@ object ColumnPruning extends Rule[LogicalPlan] { condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ - def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) + def pruneJoinChild(c: LogicalPlan): LogicalPlan = prunedChild(c, allReferences) Project(projectList, Join(pruneJoinChild(left), pruneJoinChild(right), joinType, condition)) @@ -167,6 +170,9 @@ object ColumnPruning extends Rule[LogicalPlan] { Project(substitutedProjection, child) + case Project(projectList, Limit(exp, child)) => + Limit(exp, Project(projectList, child)) + // Eliminate no-op Projects case Project(projectList, child) if child.output == projectList => child } @@ -194,14 +200,19 @@ object LikeSimplification extends Rule[LogicalPlan] { val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => - StartsWith(l, Literal(pattern)) - case Like(l, Literal(endsWith(pattern), StringType)) => - EndsWith(l, Literal(pattern)) - case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => - Contains(l, Literal(pattern)) - case Like(l, Literal(equalTo(pattern), StringType)) => - EqualTo(l, Literal(pattern)) + case Like(l, Literal(utf, StringType)) => + utf.toString match { + case startsWith(pattern) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case endsWith(pattern) => + EndsWith(l, Literal(pattern)) + case contains(pattern) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + case equalTo(pattern) => + EqualTo(l, Literal(pattern)) + case _ => + Like(l, Literal.create(utf, StringType)) + } } } @@ -214,12 +225,12 @@ object NullPropagation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case e @ Count(Literal(null, _)) => Cast(Literal(0L), e.dataType) - case e @ IsNull(c) if !c.nullable => Literal(false, BooleanType) - case e @ IsNotNull(c) if !c.nullable => Literal(true, BooleanType) - case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) - case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) - case e @ StructGetField(Literal(null, _), _, _) => Literal(null, e.dataType) - case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal(null, e.dataType) + case e @ IsNull(c) if !c.nullable => Literal.create(false, BooleanType) + case e @ IsNotNull(c) if !c.nullable => Literal.create(true, BooleanType) + case e @ GetItem(Literal(null, _), _) => Literal.create(null, e.dataType) + case e @ GetItem(_, Literal(null, _)) => Literal.create(null, e.dataType) + case e @ StructGetField(Literal(null, _), _, _) => Literal.create(null, e.dataType) + case e @ ArrayGetField(Literal(null, _), _, _, _) => Literal.create(null, e.dataType) case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) case e @ Count(expr) if !expr.nullable => Count(Literal(1)) @@ -231,36 +242,36 @@ object NullPropagation extends Rule[LogicalPlan] { case _ => true } if (newChildren.length == 0) { - Literal(null, e.dataType) + Literal.create(null, e.dataType) } else if (newChildren.length == 1) { newChildren(0) } else { Coalesce(newChildren) } - case e @ Substring(Literal(null, _), _, _) => Literal(null, e.dataType) - case e @ Substring(_, Literal(null, _), _) => Literal(null, e.dataType) - case e @ Substring(_, _, Literal(null, _)) => Literal(null, e.dataType) + case e @ Substring(Literal(null, _), _, _) => Literal.create(null, e.dataType) + case e @ Substring(_, Literal(null, _), _) => Literal.create(null, e.dataType) + case e @ Substring(_, _, Literal(null, _)) => Literal.create(null, e.dataType) // Put exceptional cases above if any case e: BinaryArithmetic => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: BinaryComparison => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: StringRegexExpression => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } case e: StringComparison => e.children match { - case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) - case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e } } @@ -280,13 +291,13 @@ object ConstantFolding extends Rule[LogicalPlan] { case l: Literal => l // Fold expressions that are foldable. - case e if e.foldable => Literal(e.eval(null), e.dataType) + case e if e.foldable => Literal.create(e.eval(null), e.dataType) // Fold "literal in (item1, item2, ..., literal, ...)" into true directly. case In(Literal(v, _), list) if list.exists { case Literal(candidate, _) if candidate == v => true case _ => false - } => Literal(true, BooleanType) + } => Literal.create(true, BooleanType) } } } @@ -473,16 +484,16 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { object PushPredicateThroughGenerate extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter @ Filter(condition, - generate @ Generate(generator, join, outer, alias, grandChild)) => + case filter @ Filter(condition, g: Generate) => // Predicates that reference attributes produced by the `Generate` operator cannot // be pushed below the operator. val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { - conjunct => conjunct.references subsetOf grandChild.outputSet + conjunct => conjunct.references subsetOf g.child.outputSet } if (pushDown.nonEmpty) { val pushDownPredicate = pushDown.reduce(And) - val withPushdown = generate.copy(child = Filter(pushDownPredicate, grandChild)) + val withPushdown = Generate(g.generator, join = g.join, outer = g.outer, + g.qualifier, g.generatorOutput, Filter(pushDownPredicate, g.child)) stayUp.reduceOption(And).map(Filter(_, withPushdown)).getOrElse(withPushdown) } else { filter @@ -560,7 +571,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) joinType match { - case Inner => + case _ @ (Inner | LeftSemi) => // push down the single side only join filter for both sides sub queries val newLeft = leftJoinConditions. reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) @@ -568,7 +579,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = commonJoinCondition.reduceLeftOption(And) - Join(newLeft, newRight, Inner, newJoinCond) + Join(newLeft, newRight, joinType, newJoinCond) case RightOuter => // push down the left side only join filter for left side sub query val newLeft = leftJoinConditions. @@ -577,14 +588,14 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newJoinCond = (rightJoinConditions ++ commonJoinCondition).reduceLeftOption(And) Join(newLeft, newRight, RightOuter, newJoinCond) - case _ @ (LeftOuter | LeftSemi) => + case LeftOuter => // push down the right side only join filter for right sub query val newLeft = left val newRight = rightJoinConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) - Join(newLeft, newRight, joinType, newJoinCond) + Join(newLeft, newRight, LeftOuter, newJoinCond) case FullOuter => f } } @@ -643,7 +654,7 @@ object DecimalAggregates extends Rule[LogicalPlan] { case Average(e @ DecimalType.Expression(prec, scale)) if prec + 4 <= MAX_DOUBLE_DIGITS => Cast( - Divide(Average(UnscaledValue(e)), Literal(math.pow(10.0, scale), DoubleType)), + Divide(Average(UnscaledValue(e)), Literal.create(math.pow(10.0, scale), DoubleType)), DecimalType(prec + 4, scale + 4)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index b4c445b3badf1..4574934d910db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -91,16 +91,18 @@ object PhysicalOperation extends PredicateHelper { (None, Nil, other, Map.empty) } - def collectAliases(fields: Seq[Expression]) = fields.collect { - case a @ Alias(child, _) => a.toAttribute.asInstanceOf[Attribute] -> child + def collectAliases(fields: Seq[Expression]): Map[Attribute, Expression] = fields.collect { + case a @ Alias(child, _) => a.toAttribute -> child }.toMap - def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = expr.transform { - case a @ Alias(ref: AttributeReference, name) => - aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a) + def substitute(aliases: Map[Attribute, Expression])(expr: Expression): Expression = { + expr.transform { + case a @ Alias(ref: AttributeReference, name) => + aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a) - case a: AttributeReference => - aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a) + case a: AttributeReference => + aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 17a88e07de15f..7967189cacb24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} +import org.apache.spark.sql.catalyst.expressions.{VirtualColumn, Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType} @@ -47,8 +47,12 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy * Attributes that are referenced by expressions but not provided by this nodes children. * Subclasses should override this method if they produce attributes internally as it is used by * assertions designed to prevent the construction of invalid plans. + * + * Note that virtual columns should be excluded. Currently, we only support the grouping ID + * virtual column. */ - def missingInput: AttributeSet = references -- inputSet + def missingInput: AttributeSet = + (references -- inputSet).filter(_.name != VirtualColumn.groupingIdName) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. @@ -67,7 +71,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpressionDown(e: Expression) = { + @inline def transformExpressionDown(e: Expression): Expression = { val newE = e.transformDown(rule) if (newE.fastEquals(e)) { e @@ -81,6 +85,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionDown(e) case Some(e: Expression) => Some(transformExpressionDown(e)) case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionDown(e) case other => other @@ -99,7 +104,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = { var changed = false - @inline def transformExpressionUp(e: Expression) = { + @inline def transformExpressionUp(e: Expression): Expression = { val newE = e.transformUp(rule) if (newE.fastEquals(e)) { e @@ -113,6 +118,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case e: Expression => transformExpressionUp(e) case Some(e: Expression) => Some(transformExpressionUp(e)) case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case seq: Traversable[_] => seq.map { case e: Expression => transformExpressionUp(e) case other => other @@ -144,7 +150,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - def schema: StructType = StructType.fromAttributes(output) + lazy val schema: StructType = StructType.fromAttributes(output) /** Returns the output schema in the tree format. */ def schemaString: String = schema.treeString @@ -159,5 +165,5 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" - override def simpleString = statePrefix + super.simpleString + override def simpleString: String = statePrefix + super.simpleString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index bb79dc340553b..e3e070f0ff307 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, analysis} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.types.{DataTypeConversions, StructType, StructField} +import org.apache.spark.sql.types.{StructType, StructField} object LocalRelation { def apply(output: Attribute*): LocalRelation = new LocalRelation(output) @@ -31,7 +31,8 @@ object LocalRelation { def fromProduct(output: Seq[Attribute], data: Seq[Product]): LocalRelation = { val schema = StructType.fromAttributes(output) - LocalRelation(output, data.map(row => DataTypeConversions.productToRow(row, schema))) + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + LocalRelation(output, data.map(converter(_).asInstanceOf[Row])) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 8c4f09b58a4f2..ae4620a4e5abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, Resolver} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, EliminateSubQueries, Resolver} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode @@ -73,12 +73,16 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * can do better should override this function. */ def sameResult(plan: LogicalPlan): Boolean = { - plan.getClass == this.getClass && - plan.children.size == children.size && { - logDebug(s"[${cleanArgs.mkString(", ")}] == [${plan.cleanArgs.mkString(", ")}]") - cleanArgs == plan.cleanArgs + val cleanLeft = EliminateSubQueries(this) + val cleanRight = EliminateSubQueries(plan) + + cleanLeft.getClass == cleanRight.getClass && + cleanLeft.children.size == cleanRight.children.size && { + logDebug( + s"[${cleanRight.cleanArgs.mkString(", ")}] == [${cleanLeft.cleanArgs.mkString(", ")}]") + cleanRight.cleanArgs == cleanLeft.cleanArgs } && - (plan.children, children).zipped.forall(_ sameResult _) + (cleanLeft.children, cleanRight.children).zipped.forall(_ sameResult _) } /** Args that have cleaned such that differences in expression id should not affect equality */ @@ -105,16 +109,22 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolveChildren(name: String, resolver: Resolver): Option[NamedExpression] = - resolve(name, children.flatMap(_.output), resolver) + def resolveChildren( + nameParts: Seq[String], + resolver: Resolver, + throwErrors: Boolean = false): Option[NamedExpression] = + resolve(nameParts, children.flatMap(_.output), resolver, throwErrors) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this * LogicalPlan. The attribute is expressed as string in the following form: * `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String, resolver: Resolver): Option[NamedExpression] = - resolve(name, output, resolver) + def resolve( + nameParts: Seq[String], + resolver: Resolver, + throwErrors: Boolean = false): Option[NamedExpression] = + resolve(nameParts, output, resolver, throwErrors) /** * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. @@ -124,7 +134,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsTableColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { assert(nameParts.length > 1) @@ -144,7 +154,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { if (resolver(attribute.name, nameParts.head)) { @@ -156,11 +166,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** Performs attribute resolution given a name and a sequence of possible attributes. */ protected def resolve( - name: String, + nameParts: Seq[String], input: Seq[Attribute], - resolver: Resolver): Option[NamedExpression] = { - - val parts = name.split("\\.") + resolver: Resolver, + throwErrors: Boolean): Option[NamedExpression] = { // A sequence of possible candidate matches. // Each candidate is a tuple. The first element is a resolved attribute, followed by a list @@ -170,9 +179,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // and the second element will be List("c"). var candidates: Seq[(Attribute, List[String])] = { // If the name has 2 or more parts, try to resolve it as `table.column` first. - if (parts.length > 1) { + if (nameParts.length > 1) { input.flatMap { option => - resolveAsTableColumn(parts, resolver, option) + resolveAsTableColumn(nameParts, resolver, option) } } else { Seq.empty @@ -182,24 +191,30 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // If none of attributes match `table.column` pattern, we try to resolve it as a column. if (candidates.isEmpty) { candidates = input.flatMap { candidate => - resolveAsColumn(parts, resolver, candidate) + resolveAsColumn(nameParts, resolver, candidate) } } + def name = UnresolvedAttribute(nameParts).name + candidates.distinct match { // One match, no nested fields, use it. case Seq((a, Nil)) => Some(a) // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - // The foldLeft adds UnresolvedGetField for every remaining parts of the name, - // and aliased it with the last part of the name. - // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. - // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias - // the final expression as "c". - val fieldExprs = nestedFields.foldLeft(a: Expression)(UnresolvedGetField) - val aliasName = nestedFields.last - Some(Alias(fieldExprs, aliasName)()) + try { + // The foldLeft adds GetFields for every remaining parts of the identifier, + // and aliases it with the last part of the identifier. + // For example, consider "a.b.c", where "a" is resolved to an existing attribute. + // Then this will add GetField("c", GetField("b", a)), and alias + // the final expression as "c". + val fieldExprs = nestedFields.foldLeft(a: Expression)(GetField(_, _, resolver)) + val aliasName = nestedFields.last + Some(Alias(fieldExprs, aliasName)()) + } catch { + case a: AnalysisException if !throwErrors => None + } // No matches. case Seq() => @@ -208,8 +223,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // More than one match. case ambiguousReferences => + val referenceNames = ambiguousReferences.map(_._1).mkString(", ") throw new AnalysisException( - s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") + s"Reference '$name' is ambiguous, could be: $referenceNames.") } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 1e7b449d75b80..ba0abb2df596c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -22,16 +22,17 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { - def output = projectList.map(_.toAttribute) + override def output: Seq[Attribute] = projectList.map(_.toAttribute) override lazy val resolved: Boolean = { - val containsAggregatesOrGenerators = projectList.exists ( _.collect { + val hasSpecialExpressions = projectList.exists ( _.collect { case agg: AggregateExpression => agg case generator: Generator => generator + case window: WindowExpression => window }.nonEmpty ) - !expressions.exists(!_.resolved) && childrenResolved && !containsAggregatesOrGenerators + !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } } @@ -40,47 +41,56 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with * their output. + * @param generator the generator expression * @param join when true, each output row is implicitly joined with the input tuple that produced * it. * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. - * @param alias when set, this string is applied to the schema of the output of the transformation - * as a qualifier. + * @param qualifier Qualifier for the attributes of generator(UDTF) + * @param generatorOutput The output schema of the Generator. + * @param child Children logical plan node */ case class Generate( generator: Generator, join: Boolean, outer: Boolean, - alias: Option[String], + qualifier: Option[String], + generatorOutput: Seq[Attribute], child: LogicalPlan) extends UnaryNode { - protected def generatorOutput: Seq[Attribute] = { - val output = alias - .map(a => generator.output.map(_.withQualifiers(a :: Nil))) - .getOrElse(generator.output) - if (join && outer) { - output.map(_.withNullability(true)) - } else { - output - } + override lazy val resolved: Boolean = { + generator.resolved && + childrenResolved && + generator.elementTypes.length == generatorOutput.length && + !generatorOutput.exists(!_.resolved) } - override def output = - if (join) child.output ++ generatorOutput else generatorOutput + // we don't want the gOutput to be taken as part of the expressions + // as that will cause exceptions like unresolved attributes etc. + override def expressions: Seq[Expression] = generator :: Nil + + def output: Seq[Attribute] = { + val qualified = qualifier.map(q => + // prepend the new qualifier to the existed one + generatorOutput.map(a => a.withQualifiers(q +: a.qualifiers)) + ).getOrElse(generatorOutput) + + if (join) child.output ++ qualified else qualified + } } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { // TODO: These aren't really the same attributes as nullability etc might change. - override def output = left.output + override def output: Seq[Attribute] = left.output - override lazy val resolved = + override lazy val resolved: Boolean = childrenResolved && - !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } + left.output.zip(right.output).forall { case (l,r) => l.dataType == r.dataType } override def statistics: Statistics = { val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes @@ -94,7 +104,7 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def output = { + override def output: Seq[Attribute] = { joinType match { case LeftSemi => left.output @@ -109,7 +119,7 @@ case class Join( } } - def selfJoinResolved = left.outputSet.intersect(right.outputSet).isEmpty + private def selfJoinResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty // Joins are only resolved if they don't introduce ambiguious expression ids. override lazy val resolved: Boolean = { @@ -118,20 +128,22 @@ case class Join( } case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - def output = left.output + override def output: Seq[Attribute] = left.output } case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { - override def children = child :: Nil - override def output = child.output + override def children: Seq[LogicalPlan] = child :: Nil + override def output: Seq[Attribute] = child.output - override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + assert(overwrite || !ifNotExists) + override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) } @@ -143,14 +155,32 @@ case class CreateTableAsSelect[T]( child: LogicalPlan, allowExisting: Boolean, desc: Option[T] = None) extends UnaryNode { - override def output = Seq.empty[Attribute] - override lazy val resolved = databaseName != None && childrenResolved + override def output: Seq[Attribute] = Seq.empty[Attribute] + override lazy val resolved: Boolean = databaseName != None && childrenResolved +} + +/** + * A container for holding named common table expressions (CTEs) and a query plan. + * This operator will be removed during analysis and the relations will be substituted into child. + * @param child The final query of this CTE. + * @param cteRelations Queries that this CTE defined, + * key is the alias of the CTE definition, + * value is the CTE definition. + */ +case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} + +case class WithWindowDefinition( + windowDefinitions: Map[String, WindowSpecDefinition], + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } /** @@ -163,7 +193,7 @@ case class Sort( order: Seq[SortOrder], global: Boolean, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Aggregate( @@ -172,7 +202,26 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { - override def output = aggregateExpressions.map(_.toAttribute) + override lazy val resolved: Boolean = { + val hasWindowExpressions = aggregateExpressions.exists ( _.collect { + case window: WindowExpression => window + }.nonEmpty + ) + + !expressions.exists(!_.resolved) && childrenResolved && !hasWindowExpressions + } + + override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) +} + +case class Window( + projectList: Seq[Attribute], + windowExpressions: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: LogicalPlan) extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpressions).map(_.toAttribute) } /** @@ -199,7 +248,7 @@ trait GroupingAnalytics extends UnaryNode { def groupByExprs: Seq[Expression] def aggregations: Seq[NamedExpression] - override def output = aggregations.map(_.toAttribute) + override def output: Seq[Attribute] = aggregations.map(_.toAttribute) } /** @@ -264,7 +313,7 @@ case class Rollup( gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output override lazy val statistics: Statistics = { val limit = limitExpr.eval(null).asInstanceOf[Int] @@ -274,23 +323,60 @@ case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { - override def output = child.output.map(_.withQualifiers(alias :: Nil)) + override def output: Seq[Attribute] = child.output.map(_.withQualifiers(alias :: Nil)) } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) - extends UnaryNode { +/** + * Sample the dataset. + * + * @param lowerBound Lower-bound of the sampling probability (usually 0.0) + * @param upperBound Upper-bound of the sampling probability. The expected fraction sampled + * will be ub - lb. + * @param withReplacement Whether to sample with replacement. + * @param seed the random seed + * @param child the LogicalPlan + */ +case class Sample( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, + child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class Distinct(child: LogicalPlan) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output +} + +/** + * Return a new RDD that has exactly `numPartitions` partitions. Differs from + * [[RepartitionByExpression]] as this method is called directly by DataFrame's, because the user + * asked for `coalesce` or `repartition`. [[RepartitionByExpression]] is used when the consumer + * of the output requires some specific ordering or distribution of the data. + */ +case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) + extends UnaryNode { + override def output: Seq[Attribute] = child.output } -case object NoRelation extends LeafNode { - override def output = Nil +/** + * A relation with one row. This is used in "SELECT ..." without a from clause. + */ +case object OneRowRelation extends LeafNode { + override def output: Seq[Attribute] = Nil + + /** + * Computes [[Statistics]] for this plan. The default implementation assumes the output + * cardinality is the product of of all child plan's cardinality, i.e. applies in the case + * of cartesian joins. + * + * [[LeafNode]]s must override this. + */ + override def statistics: Statistics = Statistics(sizeInBytes = 1) } case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - override def output = left.output + override def output: Seq[Attribute] = left.output } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 72b0c5c8e7a26..63df2c1ee72ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} /** * Performs a physical redistribution of the data. Used when the consumer of the query @@ -26,14 +26,17 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} abstract class RedistributeData extends UnaryNode { self: Product => - def output = child.output + override def output: Seq[Attribute] = child.output } case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) - extends RedistributeData { -} - -case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) - extends RedistributeData { -} + extends RedistributeData +/** + * This method repartitions data using [[Expression]]s, and receives information about the + * number of partitions during execution. Used when a specific ordering or distribution is + * expected by the consumer of the query result. Use [[Repartition]] for RDD-like + * `coalesce` and `repartition`. + */ +case class RepartitionByExpression(partitionExpressions: Seq[Expression], child: LogicalPlan) + extends RedistributeData diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 3c3d7a3119064..fb4217a44807b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder} -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{DataType, IntegerType} /** * Specifies how tuples that share common expressions will be distributed when a query is executed @@ -72,7 +72,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "a single partition.") // TODO: This is not really valid... - def clustering = ordering.map(_.child).toSet + def clustering: Set[Expression] = ordering.map(_.child).toSet } sealed trait Partitioning { @@ -94,6 +94,9 @@ sealed trait Partitioning { * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean + + /** Returns the expressions that are used to key the partitioning. */ + def keyExpressions: Seq[Expression] } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { @@ -106,6 +109,8 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case UnknownPartitioning(_) => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object SinglePartition extends Partitioning { @@ -113,10 +118,12 @@ case object SinglePartition extends Partitioning { override def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object BroadcastPartitioning extends Partitioning { @@ -124,10 +131,12 @@ case object BroadcastPartitioning extends Partitioning { override def satisfies(required: Distribution): Boolean = true - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } /** @@ -139,9 +148,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - override def children = expressions - override def nullable = false - override def dataType = IntegerType + override def children: Seq[Expression] = expressions + override def nullable: Boolean = false + override def dataType: DataType = IntegerType private[this] lazy val clusteringSet = expressions.toSet @@ -152,12 +161,14 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case BroadcastPartitioning => true case h: HashPartitioning if h == this => true case _ => false } + override def keyExpressions: Seq[Expression] = expressions + override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } @@ -178,9 +189,9 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - override def children = ordering - override def nullable = false - override def dataType = IntegerType + override def children: Seq[SortOrder] = ordering + override def nullable: Boolean = false + override def dataType: DataType = IntegerType private[this] lazy val clusteringSet = ordering.map(_.child).toSet @@ -194,12 +205,14 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } - override def compatibleWith(other: Partitioning) = other match { + override def compatibleWith(other: Partitioning): Boolean = other match { case BroadcastPartitioning => true case r: RangePartitioning if r == this => true case _ => false } + override def keyExpressions: Seq[Expression] = ordering.map(_.child) + override def eval(input: Row): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index c441f0bf24d85..3f9858b0c4a43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -45,7 +45,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { * Executes the batches of rules defined by the subclass. The batches are executed serially * using the defined execution strategy. Within each batch, rules are also executed serially. */ - def apply(plan: TreeType): TreeType = { + def execute(plan: TreeType): TreeType = { var curPlan = plan batches.foreach { batch => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index f84ffe4e176cc..4b93f7d31b808 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.types.DataType /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) @@ -35,12 +36,12 @@ object CurrentOrigin { override def initialValue: Origin = Origin() } - def get = value.get() - def set(o: Origin) = value.set(o) + def get: Origin = value.get() + def set(o: Origin): Unit = value.set(o) - def reset() = value.set(Origin()) + def reset(): Unit = value.set(Origin()) - def setPosition(line: Int, start: Int) = { + def setPosition(line: Int, start: Int): Unit = { value.set( value.get.copy(line = Some(line), startPosition = Some(start))) } @@ -56,7 +57,7 @@ object CurrentOrigin { abstract class TreeNode[BaseType <: TreeNode[BaseType]] { self: BaseType with Product => - val origin = CurrentOrigin.get + val origin: Origin = CurrentOrigin.get /** Returns a Seq of the children of this node */ def children: Seq[BaseType] @@ -70,6 +71,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { this.eq(other) || this == other } + /** + * Find the first [[TreeNode]] that satisfies the condition specified by `f`. + * The condition is recursively applied to this node and all of its children (pre-order). + */ + def find(f: BaseType => Boolean): Option[BaseType] = f(this) match { + case true => Some(this) + case false => children.foldLeft(None: Option[BaseType]) { (l, r) => l.orElse(r.find(f)) } + } + /** * Runs the given function on this node and then recursively on [[children]]. * @param f the function to be applied to each node in the tree. @@ -84,7 +94,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param f the function to be applied to each node in the tree. */ def foreachUp(f: BaseType => Unit): Unit = { - children.foreach(_.foreach(f)) + children.foreach(_.foreachUp(f)) f(this) } @@ -150,6 +160,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { val remainingNewChildren = newChildren.toBuffer val remainingOldChildren = children.toBuffer val newArgs = productIterator.map { + // This rule is used to handle children is a input argument. + case s: Seq[_] => s.map { + case arg: TreeNode[_] if children contains arg => + val newChild = remainingNewChildren.remove(0) + val oldChild = remainingOldChildren.remove(0) + if (newChild fastEquals oldChild) { + oldChild + } else { + changed = true + newChild + } + case nonChild: AnyRef => nonChild + case null => null + } case arg: TreeNode[_] if children contains arg => val newChild = remainingNewChildren.remove(0) val oldChild = remainingOldChildren.remove(0) @@ -220,6 +244,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { Some(arg) } case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case args: Traversable[_] => args.map { case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformDown(rule) @@ -276,6 +301,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { Some(arg) } case m: Map[_,_] => m + case d: DataType => d // Avoid unpacking Structs case args: Traversable[_] => args.map { case arg: TreeNode[_] if children contains arg => val newChild = arg.asInstanceOf[BaseType].transformUp(rule) @@ -307,10 +333,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param newArgs the new product arguments. */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { + val defaultCtor = + getClass.getConstructors + .find(_.getParameterTypes.size != 0) + .headOption + .getOrElse(sys.error(s"No valid constructor for $nodeName")) + try { CurrentOrigin.withOrigin(origin) { // Skip no-arg constructors that are just there for kryo. - val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head if (otherCopyArgs.isEmpty) { defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] } else { @@ -320,18 +351,24 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { } catch { case e: java.lang.IllegalArgumentException => throw new TreeNodeException( - this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName? " - + s"Exception message: ${e.getMessage}.") + this, + s""" + |Failed to copy node. + |Is otherCopyArgs specified correctly for $nodeName. + |Exception message: ${e.getMessage} + |ctor: $defaultCtor? + |args: ${newArgs.mkString(", ")} + """.stripMargin) } } /** Returns the name of this type of TreeNode. Defaults to the class name. */ - def nodeName = getClass.getSimpleName + def nodeName: String = getClass.getSimpleName /** * The arguments that should be included in the arg string. Defaults to the `productIterator`. */ - protected def stringArgs = productIterator + protected def stringArgs: Iterator[Any] = productIterator /** Returns a string representing the arguments to this node, minus any children */ def argString: String = productIterator.flatMap { @@ -343,18 +380,18 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { }.mkString(", ") /** String representation of this node without any children */ - def simpleString = s"$nodeName $argString".trim + def simpleString: String = s"$nodeName $argString".trim override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ - def treeString = generateTreeString(0, new StringBuilder).toString + def treeString: String = generateTreeString(0, new StringBuilder).toString /** * Returns a string representation of the nodes in this tree, where each operator is numbered. * The numbers can be used with [[trees.TreeNode.apply apply]] to easily access specific subtrees. */ - def numberedTreeString = + def numberedTreeString: String = treeString.split("\n").zipWithIndex.map { case (line, i) => f"$i%02d $line" }.mkString("\n") /** @@ -406,14 +443,14 @@ trait BinaryNode[BaseType <: TreeNode[BaseType]] { def left: BaseType def right: BaseType - def children = Seq(left, right) + def children: Seq[BaseType] = Seq(left, right) } /** * A [[TreeNode]] with no children. */ trait LeafNode[BaseType <: TreeNode[BaseType]] { - def children = Nil + def children: Seq[BaseType] = Nil } /** @@ -421,6 +458,5 @@ trait LeafNode[BaseType <: TreeNode[BaseType]] { */ trait UnaryNode[BaseType <: TreeNode[BaseType]] { def child: BaseType - def children = child :: Nil + def children: Seq[BaseType] = child :: Nil } - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 79a8e06d4b4d4..ea6aa1850db4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -41,11 +41,11 @@ package object trees extends Logging { * A [[TreeNode]] companion for reference equality for Hash based Collection. */ class TreeNodeRef(val obj: TreeNode[_]) { - override def equals(o: Any) = o match { + override def equals(o: Any): Boolean = o match { case that: TreeNodeRef => that.obj.eq(obj) case _ => false } - override def hashCode = if (obj == null) 0 else obj.hashCode + override def hashCode: Int = if (obj == null) 0 else obj.hashCode } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index feed50f9a2a2d..9d613a940ee86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -17,13 +17,32 @@ package org.apache.spark.sql.catalyst -import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} +import java.io._ import org.apache.spark.util.Utils package object util { - def fileToString(file: File, encoding: String = "UTF-8") = { + /** Silences output to stderr or stdout for the duration of f */ + def quietly[A](f: => A): A = { + val origErr = System.err + val origOut = System.out + try { + System.setErr(new PrintStream(new OutputStream { + def write(b: Int) = {} + })) + System.setOut(new PrintStream(new OutputStream { + def write(b: Int) = {} + })) + + f + } finally { + System.setErr(origErr) + System.setOut(origOut) + } + } + + def fileToString(file: File, encoding: String = "UTF-8"): String = { val inStream = new FileInputStream(file) val outStream = new ByteArrayOutputStream try { @@ -42,10 +61,9 @@ package object util { new String(outStream.toByteArray, encoding) } - def resourceToString( - resource:String, - encoding: String = "UTF-8", - classLoader: ClassLoader = Utils.getSparkClassLoader) = { + def resourceToBytes( + resource: String, + classLoader: ClassLoader = Utils.getSparkClassLoader): Array[Byte] = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { @@ -61,7 +79,14 @@ package object util { finally { inStream.close() } - new String(outStream.toByteArray, encoding) + outStream.toByteArray + } + + def resourceToString( + resource:String, + encoding: String = "UTF-8", + classLoader: ClassLoader = Utils.getSparkClassLoader): String = { + new String(resourceToBytes(resource, classLoader), encoding) } def stringToFile(file: File, str: String): File = { @@ -93,7 +118,7 @@ package object util { new String(out.toByteArray) } - def stringOrNull(a: AnyRef) = if (a == null) null else a.toString + def stringOrNull(a: AnyRef): String = if (a == null) null else a.toString def benchmark[A](f: => A): A = { val startTime = System.nanoTime() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala new file mode 100644 index 0000000000000..b116163faccad --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -0,0 +1,74 @@ +/* + * 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.sql.types + +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.DeveloperApi + + +object ArrayType { + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, containsNull = true) +} + + +/** + * :: DeveloperApi :: + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * Please use [[DataTypes.createArrayType()]] to create a specific instance. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * + * @param elementType The data type of values. + * @param containsNull Indicates if values have `null` values + * + * @group dataType + */ +@DeveloperApi +case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null, false) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append( + s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") + DataType.buildFormattedString(elementType, s"$prefix |", builder) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("elementType" -> elementType.jsonValue) ~ + ("containsNull" -> containsNull) + + /** + * The default size of a value of the ArrayType is 100 * the default size of the element type. + * (We assume that there are 100 elements). + */ + override def defaultSize: Int = 100 * elementType.defaultSize + + override def simpleString: String = s"array<${elementType.simpleString}>" + + private[spark] override def asNullable: ArrayType = + ArrayType(elementType.asNullable, containsNull = true) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala new file mode 100644 index 0000000000000..a581a9e9468ef --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -0,0 +1,63 @@ +/* + * 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.sql.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Array[Byte]` values. + * Please use the singleton [[DataTypes.BinaryType]]. + * + * @group dataType + */ +@DeveloperApi +class BinaryType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + + private[sql] type InternalType = Array[Byte] + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = new Ordering[InternalType] { + def compare(x: Array[Byte], y: Array[Byte]): Int = { + for (i <- 0 until x.length; if i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + } + x.length - y.length + } + } + + /** + * The default size of a value of the BinaryType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + private[spark] override def asNullable: BinaryType = this +} + + +case object BinaryType extends BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala new file mode 100644 index 0000000000000..a7f228cefa57a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -0,0 +1,51 @@ +/* + * 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.sql.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. + * + *@group dataType + */ +@DeveloperApi +class BooleanType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the BooleanType is 1 byte. + */ + override def defaultSize: Int = 1 + + private[spark] override def asNullable: BooleanType = this +} + + +case object BooleanType extends BooleanType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala new file mode 100644 index 0000000000000..4d8685796ec76 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. + * + * @group dataType + */ +@DeveloperApi +class ByteType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Byte]] + private[sql] val integral = implicitly[Integral[Byte]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the ByteType is 1 byte. + */ + override def defaultSize: Int = 1 + + override def simpleString: String = "tinyint" + + private[spark] override def asNullable: ByteType = this +} + +case object ByteType extends ByteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala new file mode 100644 index 0000000000000..0992a7c311ee2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -0,0 +1,385 @@ +/* + * 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.sql.types + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.{TypeTag, runtimeMirror} +import scala.util.parsing.combinator.RegexParsers + +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.util.Utils + + +/** + * :: DeveloperApi :: + * The base type of all Spark SQL data types. + * + * @group dataType + */ +@DeveloperApi +abstract class DataType { + /** + * Enables matching against DataType for expressions: + * {{{ + * case Cast(child @ BinaryType(), StringType) => + * ... + * }}} + */ + private[sql] def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType == this => true + case _ => false + } + + /** + * The default size of a value of this data type, used internally for size estimation. + */ + def defaultSize: Int + + /** Name of the type used in JSON serialization. */ + def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase + + private[sql] def jsonValue: JValue = typeName + + /** The compact JSON representation of this data type. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this data type. */ + def prettyJson: String = pretty(render(jsonValue)) + + /** Readable string representation for the type. */ + def simpleString: String = typeName + + /** + * Check if `this` and `other` are the same data type when ignoring nullability + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def sameType(other: DataType): Boolean = + DataType.equalsIgnoreNullability(this, other) + + /** + * Returns the same data type but set all nullability fields are true + * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). + */ + private[spark] def asNullable: DataType +} + + +/** + * An internal type used to represent everything that is not null, UDTs, arrays, structs, and maps. + */ +protected[sql] abstract class AtomicType extends DataType { + private[sql] type InternalType + @transient private[sql] val tag: TypeTag[InternalType] + private[sql] val ordering: Ordering[InternalType] + + @transient private[sql] val classTag = ScalaReflectionLock.synchronized { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[InternalType](mirror.runtimeClass(tag.tpe)) + } +} + + +/** + * :: DeveloperApi :: + * Numeric data types. + * + * @group dataType + */ +abstract class NumericType extends AtomicType { + // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for + // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a + // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets + // desugared by the compiler into an argument to the objects constructor. This means there is no + // longer an no argument constructor and thus the JVM cannot serialize the object anymore. + private[sql] val numeric: Numeric[InternalType] +} + + +private[sql] object NumericType { + /** + * Enables matching against NumericType for expressions: + * {{{ + * case Cast(child @ NumericType(), StringType) => + * ... + * }}} + */ + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] +} + + +private[sql] object IntegralType { + /** + * Enables matching against IntegralType for expressions: + * {{{ + * case Cast(child @ IntegralType(), StringType) => + * ... + * }}} + */ + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[IntegralType] => true + case _ => false + } +} + + +private[sql] abstract class IntegralType extends NumericType { + private[sql] val integral: Integral[InternalType] +} + + +private[sql] object FractionalType { + /** + * Enables matching against FractionalType for expressions: + * {{{ + * case Cast(child @ FractionalType(), StringType) => + * ... + * }}} + */ + def unapply(a: Expression): Boolean = a match { + case e: Expression if e.dataType.isInstanceOf[FractionalType] => true + case _ => false + } +} + + +private[sql] abstract class FractionalType extends NumericType { + private[sql] val fractional: Fractional[InternalType] + private[sql] val asIntegral: Integral[InternalType] +} + + +object DataType { + + def fromJson(json: String): DataType = parseDataType(parse(json)) + + @deprecated("Use DataType.fromJson instead", "1.2.0") + def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) + + private val nonDecimalNameToType = { + Seq(NullType, DateType, TimestampType, BinaryType, + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + .map(t => t.typeName -> t).toMap + } + + /** Given the string representation of a type, return its DataType */ + private def nameToType(name: String): DataType = { + val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r + name match { + case "decimal" => DecimalType.Unlimited + case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) + case other => nonDecimalNameToType(other) + } + } + + private object JSortedObject { + def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { + case JObject(seq) => Some(seq.toList.sortBy(_._1)) + case _ => None + } + } + + // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. + private def parseDataType(json: JValue): DataType = json match { + case JString(name) => + nameToType(name) + + case JSortedObject( + ("containsNull", JBool(n)), + ("elementType", t: JValue), + ("type", JString("array"))) => + ArrayType(parseDataType(t), n) + + case JSortedObject( + ("keyType", k: JValue), + ("type", JString("map")), + ("valueContainsNull", JBool(n)), + ("valueType", v: JValue)) => + MapType(parseDataType(k), parseDataType(v), n) + + case JSortedObject( + ("fields", JArray(fields)), + ("type", JString("struct"))) => + StructType(fields.map(parseStructField)) + + case JSortedObject( + ("class", JString(udtClass)), + ("pyClass", _), + ("sqlType", _), + ("type", JString("udt"))) => + Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] + } + + private def parseStructField(json: JValue): StructField = json match { + case JSortedObject( + ("metadata", metadata: JObject), + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) + // Support reading schema when 'metadata' is missing. + case JSortedObject( + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable) + } + + private object CaseClassStringParser extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + ( "StringType" ^^^ StringType + | "FloatType" ^^^ FloatType + | "IntegerType" ^^^ IntegerType + | "ByteType" ^^^ ByteType + | "ShortType" ^^^ ShortType + | "DoubleType" ^^^ DoubleType + | "LongType" ^^^ LongType + | "BinaryType" ^^^ BinaryType + | "BooleanType" ^^^ BooleanType + | "DateType" ^^^ DateType + | "DecimalType()" ^^^ DecimalType.Unlimited + | fixedDecimalType + | "TimestampType" ^^^ TimestampType + ) + + protected lazy val fixedDecimalType: Parser[DataType] = + ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { + case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + } + + protected lazy val arrayType: Parser[DataType] = + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } + + protected lazy val mapType: Parser[DataType] = + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) + } + + protected lazy val structField: Parser[StructField] = + ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { + case name ~ tpe ~ nullable => + StructField(name, tpe, nullable = nullable) + } + + protected lazy val boolVal: Parser[Boolean] = + ( "true" ^^^ true + | "false" ^^^ false + ) + + protected lazy val structType: Parser[DataType] = + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { + case fields => StructType(fields) + } + + protected lazy val dataType: Parser[DataType] = + ( arrayType + | mapType + | structType + | primitiveType + ) + + /** + * Parses a string representation of a DataType. + * + * TODO: Generate parser as pickler... + */ + def apply(asString: String): DataType = parseAll(dataType, asString) match { + case Success(result, _) => result + case failure: NoSuccess => + throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") + } + } + + protected[types] def buildFormattedString( + dataType: DataType, + prefix: String, + builder: StringBuilder): Unit = { + dataType match { + case array: ArrayType => + array.buildFormattedString(prefix, builder) + case struct: StructType => + struct.buildFormattedString(prefix, builder) + case map: MapType => + map.buildFormattedString(prefix, builder) + case _ => + } + } + + /** + * Compares two types, ignoring nullability of ArrayType, MapType, StructType. + */ + private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + (left, right) match { + case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => + equalsIgnoreNullability(leftElementType, rightElementType) + case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => + equalsIgnoreNullability(leftKeyType, rightKeyType) && + equalsIgnoreNullability(leftValueType, rightValueType) + case (StructType(leftFields), StructType(rightFields)) => + leftFields.length == rightFields.length && + leftFields.zip(rightFields).forall { case (l, r) => + l.name == r.name && equalsIgnoreNullability(l.dataType, r.dataType) + } + case (l, r) => l == r + } + } + + /** + * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. + * + * Compatible nullability is defined as follows: + * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` + * if and only if `to.containsNull` is true, or both of `from.containsNull` and + * `to.containsNull` are false. + * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` + * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and + * `to.valueContainsNull` are false. + * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` + * if and only if for all every pair of fields, `to.nullable` is true, or both + * of `fromField.nullable` and `toField.nullable` are false. + */ + private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { + (from, to) match { + case (ArrayType(fromElement, fn), ArrayType(toElement, tn)) => + (tn || !fn) && equalsIgnoreCompatibleNullability(fromElement, toElement) + + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + (tn || !fn) && + equalsIgnoreCompatibleNullability(fromKey, toKey) && + equalsIgnoreCompatibleNullability(fromValue, toValue) + + case (StructType(fromFields), StructType(toFields)) => + fromFields.length == toFields.length && + fromFields.zip(toFields).forall { case (fromField, toField) => + fromField.name == toField.name && + (toField.nullable || !fromField.nullable) && + equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) + } + + case (fromDataType, toDataType) => fromDataType == toDataType + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala deleted file mode 100644 index c243be07a91b6..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala +++ /dev/null @@ -1,77 +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.sql.types - -import java.text.SimpleDateFormat - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow - - -protected[sql] object DataTypeConversions { - - def productToRow(product: Product, schema: StructType): Row = { - val mutableRow = new GenericMutableRow(product.productArity) - val schemaFields = schema.fields.toArray - - var i = 0 - while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(product.productElement(i), schemaFields(i).dataType) - i += 1 - } - - mutableRow - } - - def stringToTime(s: String): java.util.Date = { - if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - java.sql.Timestamp.valueOf(s) - } else { - java.sql.Date.valueOf(s) - } - } else if (s.endsWith("Z")) { - // this is zero timezone of ISO8601 - stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") - } else if (s.indexOf("GMT") == -1) { - // timezone with ISO8601 - val inset = "+00.00".length - val s0 = s.substring(0, s.length - inset) - val s1 = s.substring(s.length - inset, s.length) - if (s0.substring(s0.lastIndexOf(':')).contains('.')) { - stringToTime(s0 + "GMT" + s1) - } else { - stringToTime(s0 + ".0GMT" + s1) - } - } else { - // ISO8601 with GMT insert - val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) - ISO8601GMT.parse(s) - } - } - - /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type - case (d: java.math.BigDecimal, _) => Decimal(d) - case (other, _) => other - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala new file mode 100644 index 0000000000000..04f3379afb38d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala @@ -0,0 +1,115 @@ +/* + * 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.sql.types + +import scala.language.implicitConversions +import scala.util.matching.Regex +import scala.util.parsing.combinator.syntactical.StandardTokenParsers + +import org.apache.spark.sql.catalyst.SqlLexical + +/** + * This is a data type parser that can be used to parse string representations of data types + * provided in SQL queries. This parser is mixed in with DDLParser and SqlParser. + */ +private[sql] trait DataTypeParser extends StandardTokenParsers { + + // This is used to create a parser from a regex. We are using regexes for data type strings + // since these strings can be also used as column names or field names. + import lexical.Identifier + implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", + { case Identifier(str) if regex.unapplySeq(str).isDefined => str } + ) + + protected lazy val primitiveType: Parser[DataType] = + "(?i)string".r ^^^ StringType | + "(?i)float".r ^^^ FloatType | + "(?i)(?:int|integer)".r ^^^ IntegerType | + "(?i)tinyint".r ^^^ ByteType | + "(?i)smallint".r ^^^ ShortType | + "(?i)double".r ^^^ DoubleType | + "(?i)bigint".r ^^^ LongType | + "(?i)binary".r ^^^ BinaryType | + "(?i)boolean".r ^^^ BooleanType | + fixedDecimalType | + "(?i)decimal".r ^^^ DecimalType.Unlimited | + "(?i)date".r ^^^ DateType | + "(?i)timestamp".r ^^^ TimestampType | + varchar + + protected lazy val fixedDecimalType: Parser[DataType] = + ("(?i)decimal".r ~> "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { + case precision ~ scale => + DecimalType(precision.toInt, scale.toInt) + } + + protected lazy val varchar: Parser[DataType] = + "(?i)varchar".r ~> "(" ~> (numericLit <~ ")") ^^^ StringType + + protected lazy val arrayType: Parser[DataType] = + "(?i)array".r ~> "<" ~> dataType <~ ">" ^^ { + case tpe => ArrayType(tpe) + } + + protected lazy val mapType: Parser[DataType] = + "(?i)map".r ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { + case t1 ~ _ ~ t2 => MapType(t1, t2) + } + + protected lazy val structField: Parser[StructField] = + ident ~ ":" ~ dataType ^^ { + case name ~ _ ~ tpe => StructField(name, tpe, nullable = true) + } + + protected lazy val structType: Parser[DataType] = + ("(?i)struct".r ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { + case fields => new StructType(fields.toArray) + }) | + ("(?i)struct".r ~ "<>" ^^^ StructType(Nil)) + + protected lazy val dataType: Parser[DataType] = + arrayType | + mapType | + structType | + primitiveType + + def toDataType(dataTypeString: String): DataType = synchronized { + phrase(dataType)(new lexical.Scanner(dataTypeString)) match { + case Success(result, _) => result + case failure: NoSuccess => throw new DataTypeException(failMessage(dataTypeString)) + } + } + + private def failMessage(dataTypeString: String): String = { + s"Unsupported dataType: $dataTypeString. If you have a struct and a field name of it has " + + "any special characters, please use backticks (`) to quote that field name, e.g. `x+y`. " + + "Please note that backtick itself is not supported in a field name." + } +} + +private[sql] object DataTypeParser { + lazy val dataTypeParser = new DataTypeParser { + override val lexical = new SqlLexical + } + + def parse(dataTypeString: String): DataType = dataTypeParser.toDataType(dataTypeString) +} + +/** The exception thrown from the [[DataTypeParser]]. */ +private[sql] class DataTypeException(message: String) extends Exception(message) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala new file mode 100644 index 0000000000000..03f0644bc784c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `java.sql.Date` values. + * Please use the singleton [[DataTypes.DateType]]. + * + * @group dataType + */ +@DeveloperApi +class DateType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DateType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Int + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the DateType is 4 bytes. + */ + override def defaultSize: Int = 4 + + private[spark] override def asNullable: DateType = this +} + + +case object DateType extends DateType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 8a1a3b81b3d2c..d36a49159b87f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.types import java.sql.Date +import java.text.SimpleDateFormat import java.util.{Calendar, TimeZone} import org.apache.spark.sql.catalyst.expressions.Cast @@ -39,6 +40,7 @@ object DateUtils { millisToDays(d.getTime) } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisLocal: Long): Int = { ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt } @@ -57,4 +59,32 @@ object DateUtils { } def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) + + def stringToTime(s: String): java.util.Date = { + if (!s.contains('T')) { + // JDBC escape string + if (s.contains(' ')) { + java.sql.Timestamp.valueOf(s) + } else { + java.sql.Date.valueOf(s) + } + } else if (s.endsWith("Z")) { + // this is zero timezone of ISO8601 + stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") + } else if (s.indexOf("GMT") == -1) { + // timezone with ISO8601 + val inset = "+00.00".length + val s0 = s.substring(0, s.length - inset) + val s1 = s.substring(s.length - inset, s.length) + if (s0.substring(s0.lastIndexOf(':')).contains('.')) { + stringToTime(s0 + "GMT" + s1) + } else { + stringToTime(s0 + ".0GMT" + s1) + } + } else { + // ISO8601 with GMT insert + val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) + ISO8601GMT.parse(s) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala new file mode 100644 index 0000000000000..0f8cecd28f7df --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -0,0 +1,110 @@ +/* + * 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.sql.types + +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.expressions.Expression + + +/** Precision parameters for a Decimal */ +case class PrecisionInfo(precision: Int, scale: Int) + + +/** + * :: DeveloperApi :: + * The data type representing `java.math.BigDecimal` values. + * A Decimal that might have fixed precision and scale, or unlimited values for these. + * + * Please use [[DataTypes.createDecimalType()]] to create a specific instance. + * + * @group dataType + */ +@DeveloperApi +case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { + + /** No-arg constructor for kryo. */ + protected def this() = this(null) + + private[sql] type InternalType = Decimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = Decimal.DecimalIsFractional + private[sql] val fractional = Decimal.DecimalIsFractional + private[sql] val ordering = Decimal.DecimalIsFractional + private[sql] val asIntegral = Decimal.DecimalAsIfIntegral + + def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) + + def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) + + override def typeName: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal" + } + + override def toString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" + case None => "DecimalType()" + } + + /** + * The default size of a value of the DecimalType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + override def simpleString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal(10,0)" + } + + private[spark] override def asNullable: DecimalType = this +} + + +/** Extra factory methods and pattern matchers for Decimals */ +object DecimalType { + val Unlimited: DecimalType = DecimalType(None) + + object Fixed { + def unapply(t: DecimalType): Option[(Int, Int)] = + t.precisionInfo.map(p => (p.precision, p.scale)) + } + + object Expression { + def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { + case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) + case _ => None + } + } + + def apply(): DecimalType = Unlimited + + def apply(precision: Int, scale: Int): DecimalType = + DecimalType(Some(PrecisionInfo(precision, scale))) + + def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] + + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] + + def isFixed(dataType: DataType): Boolean = dataType match { + case DecimalType.Fixed(_, _) => true + case _ => false + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala new file mode 100644 index 0000000000000..66766623213c9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.types + +import scala.math.{Ordering, Fractional, Numeric} +import scala.math.Numeric.DoubleAsIfIntegral +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. + * + * @group dataType + */ +@DeveloperApi +class DoubleType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Double]] + private[sql] val fractional = implicitly[Fractional[Double]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + private[sql] val asIntegral = DoubleAsIfIntegral + + /** + * The default size of a value of the DoubleType is 8 bytes. + */ + override def defaultSize: Int = 8 + + private[spark] override def asNullable: DoubleType = this +} + +case object DoubleType extends DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala new file mode 100644 index 0000000000000..1d5a2f4f6f86c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.types + +import scala.math.Numeric.FloatAsIfIntegral +import scala.math.{Ordering, Fractional, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. + * + * @group dataType + */ +@DeveloperApi +class FloatType private() extends FractionalType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Float]] + private[sql] val fractional = implicitly[Fractional[Float]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + private[sql] val asIntegral = FloatAsIfIntegral + + /** + * The default size of a value of the FloatType is 4 bytes. + */ + override def defaultSize: Int = 4 + + private[spark] override def asNullable: FloatType = this +} + +case object FloatType extends FloatType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala new file mode 100644 index 0000000000000..74e464c082873 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. + * + * @group dataType + */ +@DeveloperApi +class IntegerType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Int]] + private[sql] val integral = implicitly[Integral[Int]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the IntegerType is 4 bytes. + */ + override def defaultSize: Int = 4 + + override def simpleString: String = "int" + + private[spark] override def asNullable: IntegerType = this +} + +case object IntegerType extends IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala new file mode 100644 index 0000000000000..390675782e5fd --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. + * + * @group dataType + */ +@DeveloperApi +class LongType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "LongType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Long]] + private[sql] val integral = implicitly[Integral[Long]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the LongType is 8 bytes. + */ + override def defaultSize: Int = 8 + + override def simpleString: String = "bigint" + + private[spark] override def asNullable: LongType = this +} + + +case object LongType extends LongType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala new file mode 100644 index 0000000000000..cfdf493074415 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -0,0 +1,79 @@ +/* + * 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.sql.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + + +/** + * :: DeveloperApi :: + * The data type for Maps. Keys in a map are not allowed to have `null` values. + * + * Please use [[DataTypes.createMapType()]] to create a specific instance. + * + * @param keyType The data type of map keys. + * @param valueType The data type of map values. + * @param valueContainsNull Indicates if map values have `null` values. + * + * @group dataType + */ +case class MapType( + keyType: DataType, + valueType: DataType, + valueContainsNull: Boolean) extends DataType { + + /** No-arg constructor for kryo. */ + def this() = this(null, null, false) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- key: ${keyType.typeName}\n") + builder.append(s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") + DataType.buildFormattedString(keyType, s"$prefix |", builder) + DataType.buildFormattedString(valueType, s"$prefix |", builder) + } + + override private[sql] def jsonValue: JValue = + ("type" -> typeName) ~ + ("keyType" -> keyType.jsonValue) ~ + ("valueType" -> valueType.jsonValue) ~ + ("valueContainsNull" -> valueContainsNull) + + /** + * The default size of a value of the MapType is + * 100 * (the default size of the key type + the default size of the value type). + * (We assume that there are 100 elements). + */ + override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) + + override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" + + private[spark] override def asNullable: MapType = + MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) +} + + +object MapType { + /** + * Construct a [[MapType]] object with the given key type and value type. + * The `valueContainsNull` is true. + */ + def apply(keyType: DataType, valueType: DataType): MapType = + MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index e50e9761431f5..6ee24ee0c1913 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -41,6 +41,9 @@ import org.apache.spark.annotation.DeveloperApi sealed class Metadata private[types] (private[types] val map: Map[String, Any]) extends Serializable { + /** No-arg constructor for kryo. */ + protected def this() = this(null) + /** Tests whether this Metadata contains a binding for a key. */ def contains(key: String): Boolean = map.contains(key) diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala similarity index 53% rename from core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index faa4e2b12ddb6..b64b07431fa96 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -15,20 +15,25 @@ * limitations under the License. */ -package org.apache.spark.util.collection +package org.apache.spark.sql.types + +import org.apache.spark.annotation.DeveloperApi -import java.util.Comparator /** - * A common interface for our size-tracking collections of key-value pairs, which are used in - * external operations. These all support estimating the size and obtaining a memory-efficient - * sorted iterator. + * :: DeveloperApi :: + * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. + * + * @group dataType */ -// TODO: should extend Iterable[Product2[K, V]] instead of (K, V) -private[spark] trait SizeTrackingPairCollection[K, V] extends Iterable[(K, V)] { - /** Estimate the collection's current memory usage in bytes. */ - def estimateSize(): Long +@DeveloperApi +class NullType private() extends DataType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "NullType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + override def defaultSize: Int = 1 - /** Iterate through the data in a given key order. This may destroy the underlying collection. */ - def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] + private[spark] override def asNullable: NullType = this } + +case object NullType extends NullType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala new file mode 100644 index 0000000000000..73e9ec780b0af --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.types + +import scala.math.{Ordering, Integral, Numeric} +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. + * + * @group dataType + */ +@DeveloperApi +class ShortType private() extends IntegralType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val numeric = implicitly[Numeric[Short]] + private[sql] val integral = implicitly[Integral[Short]] + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the ShortType is 2 bytes. + */ + override def defaultSize: Int = 2 + + override def simpleString: String = "smallint" + + private[spark] override def asNullable: ShortType = this +} + +case object ShortType extends ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala new file mode 100644 index 0000000000000..134ab0af4e0de --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -0,0 +1,50 @@ +/* + * 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.sql.types + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + +/** + * :: DeveloperApi :: + * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. + * + * @group dataType + */ +@DeveloperApi +class StringType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "StringType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = UTF8String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + private[sql] val ordering = implicitly[Ordering[InternalType]] + + /** + * The default size of a value of the StringType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + private[spark] override def asNullable: StringType = this +} + +case object StringType extends StringType + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala new file mode 100644 index 0000000000000..83570a5eaee61 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -0,0 +1,54 @@ +/* + * 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.sql.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + +/** + * A field inside a StructType. + * @param name The name of this field. + * @param dataType The data type of this field. + * @param nullable Indicates if values of this field can be `null` values. + * @param metadata The metadata of this field. The metadata should be preserved during + * transformation if the content of the column is not modified, e.g, in selection. + */ +case class StructField( + name: String, + dataType: DataType, + nullable: Boolean = true, + metadata: Metadata = Metadata.empty) { + + /** No-arg constructor for kryo. */ + protected def this() = this(null, null) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") + DataType.buildFormattedString(dataType, s"$prefix |", builder) + } + + // override the default toString to be compatible with legacy parquet files. + override def toString: String = s"StructField($name,$dataType,$nullable)" + + private[sql] def jsonValue: JValue = { + ("name" -> name) ~ + ("type" -> dataType.jsonValue) ~ + ("nullable" -> nullable) ~ + ("metadata" -> metadata.jsonValue) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala new file mode 100644 index 0000000000000..d80ffca18ec9a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -0,0 +1,263 @@ +/* + * 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.sql.types + +import scala.collection.mutable.ArrayBuffer +import scala.math.max + +import org.json4s.JsonDSL._ + +import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} + + +/** + * :: DeveloperApi :: + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Any names without matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ +@DeveloperApi +case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { + + /** No-arg constructor for kryo. */ + protected def this() = this(null) + + /** Returns all field names in an array. */ + def fieldNames: Array[String] = fields.map(_.name) + + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet + private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + private lazy val nameToIndex: Map[String, Int] = fieldNames.zipWithIndex.toMap + + /** + * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not + * have a name matching the given name, `null` will be returned. + */ + def apply(name: String): StructField = { + nameToField.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) + } + + /** + * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the + * original order of fields. Those names which do not have matching fields will be ignored. + */ + def apply(names: Set[String]): StructType = { + val nonExistFields = names -- fieldNamesSet + if (nonExistFields.nonEmpty) { + throw new IllegalArgumentException( + s"Field ${nonExistFields.mkString(",")} does not exist.") + } + // Preserve the original order of fields. + StructType(fields.filter(f => names.contains(f.name))) + } + + /** + * Returns index of a given field + */ + def fieldIndex(name: String): Int = { + nameToIndex.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) + } + + protected[sql] def toAttributes: Seq[AttributeReference] = + map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + + def treeString: String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + fields.foreach(field => field.buildFormattedString(prefix, builder)) + + builder.toString() + } + + def printTreeString(): Unit = println(treeString) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, builder)) + } + + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("fields" -> map(_.jsonValue)) + + override def apply(fieldIndex: Int): StructField = fields(fieldIndex) + + override def length: Int = fields.length + + override def iterator: Iterator[StructField] = fields.iterator + + /** + * The default size of a value of the StructType is the total default sizes of all field types. + */ + override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum + + override def simpleString: String = { + val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") + s"struct<${fieldTypes.mkString(",")}>" + } + + /** + * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field + * B from `that`, + * + * 1. If A and B have the same name and data type, they are merged to a field C with the same name + * and data type. C is nullable if and only if either A or B is nullable. + * 2. If A doesn't exist in `that`, it's included in the result schema. + * 3. If B doesn't exist in `this`, it's also included in the result schema. + * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be + * thrown. + */ + private[sql] def merge(that: StructType): StructType = + StructType.merge(this, that).asInstanceOf[StructType] + + private[spark] override def asNullable: StructType = { + val newFields = fields.map { + case StructField(name, dataType, nullable, metadata) => + StructField(name, dataType.asNullable, nullable = true, metadata) + } + + StructType(newFields) + } +} + + +object StructType { + + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) + + def apply(fields: java.util.List[StructField]): StructType = { + StructType(fields.toArray.asInstanceOf[Array[StructField]]) + } + + protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + + private[sql] def merge(left: DataType, right: DataType): DataType = + (left, right) match { + case (ArrayType(leftElementType, leftContainsNull), + ArrayType(rightElementType, rightContainsNull)) => + ArrayType( + merge(leftElementType, rightElementType), + leftContainsNull || rightContainsNull) + + case (MapType(leftKeyType, leftValueType, leftContainsNull), + MapType(rightKeyType, rightValueType, rightContainsNull)) => + MapType( + merge(leftKeyType, rightKeyType), + merge(leftValueType, rightValueType), + leftContainsNull || rightContainsNull) + + case (StructType(leftFields), StructType(rightFields)) => + val newFields = ArrayBuffer.empty[StructField] + + leftFields.foreach { + case leftField @ StructField(leftName, leftType, leftNullable, _) => + rightFields + .find(_.name == leftName) + .map { case rightField @ StructField(_, rightType, rightNullable, _) => + leftField.copy( + dataType = merge(leftType, rightType), + nullable = leftNullable || rightNullable) + } + .orElse(Some(leftField)) + .foreach(newFields += _) + } + + rightFields + .filterNot(f => leftFields.map(_.name).contains(f.name)) + .foreach(newFields += _) + + StructType(newFields) + + case (DecimalType.Fixed(leftPrecision, leftScale), + DecimalType.Fixed(rightPrecision, rightScale)) => + DecimalType( + max(leftScale, rightScale) + max(leftPrecision - leftScale, rightPrecision - rightScale), + max(leftScale, rightScale)) + + case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) + if leftUdt.userClass == rightUdt.userClass => leftUdt + + case (leftType, rightType) if leftType == rightType => + leftType + + case _ => + throw new SparkException(s"Failed to merge incompatible data types $left and $right") + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala new file mode 100644 index 0000000000000..aebabfc475925 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala @@ -0,0 +1,57 @@ +/* + * 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.sql.types + +import java.sql.Timestamp + +import scala.math.Ordering +import scala.reflect.runtime.universe.typeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.ScalaReflectionLock + + +/** + * :: DeveloperApi :: + * The data type representing `java.sql.Timestamp` values. + * Please use the singleton [[DataTypes.TimestampType]]. + * + * @group dataType + */ +@DeveloperApi +class TimestampType private() extends AtomicType { + // The companion object and this class is separated so the companion object also subclasses + // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. + // Defined with a private constructor so the companion object is the only possible instantiation. + private[sql] type InternalType = Timestamp + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[InternalType] } + + private[sql] val ordering = new Ordering[InternalType] { + def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) + } + + /** + * The default size of a value of the TimestampType is 12 bytes. + */ + override def defaultSize: Int = 12 + + private[spark] override def asNullable: TimestampType = this +} + +case object TimestampType extends TimestampType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala new file mode 100644 index 0000000000000..fc02ba6c9c43e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala @@ -0,0 +1,214 @@ +/* +* 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.sql.types + +import java.util.Arrays + +/** + * A UTF-8 String, as internal representation of StringType in SparkSQL + * + * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + * + * Note: This is not designed for general use cases, should not be used outside SQL. + */ + +final class UTF8String extends Ordered[UTF8String] with Serializable { + + private[this] var bytes: Array[Byte] = _ + + /** + * Update the UTF8String with String. + */ + def set(str: String): UTF8String = { + bytes = str.getBytes("utf-8") + this + } + + /** + * Update the UTF8String with Array[Byte], which should be encoded in UTF-8 + */ + def set(bytes: Array[Byte]): UTF8String = { + this.bytes = bytes + this + } + + /** + * Return the number of bytes for a code point with the first byte as `b` + * @param b The first byte of a code point + */ + @inline + private[this] def numOfBytes(b: Byte): Int = { + val offset = (b & 0xFF) - 192 + if (offset >= 0) UTF8String.bytesOfCodePointInUTF8(offset) else 1 + } + + /** + * Return the number of code points in it. + * + * This is only used by Substring() when `start` is negative. + */ + def length(): Int = { + var len = 0 + var i: Int = 0 + while (i < bytes.length) { + i += numOfBytes(bytes(i)) + len += 1 + } + len + } + + def getBytes: Array[Byte] = { + bytes + } + + /** + * Return a substring of this, + * @param start the position of first code point + * @param until the position after last code point + */ + def slice(start: Int, until: Int): UTF8String = { + if (until <= start || start >= bytes.length || bytes == null) { + new UTF8String + } + + var c = 0 + var i: Int = 0 + while (c < start && i < bytes.length) { + i += numOfBytes(bytes(i)) + c += 1 + } + var j = i + while (c < until && j < bytes.length) { + j += numOfBytes(bytes(j)) + c += 1 + } + UTF8String(Arrays.copyOfRange(bytes, i, j)) + } + + def contains(sub: UTF8String): Boolean = { + val b = sub.getBytes + if (b.length == 0) { + return true + } + var i: Int = 0 + while (i <= bytes.length - b.length) { + // In worst case, it's O(N*K), but should works fine with SQL + if (bytes(i) == b(0) && Arrays.equals(Arrays.copyOfRange(bytes, i, i + b.length), b)) { + return true + } + i += 1 + } + false + } + + def startsWith(prefix: UTF8String): Boolean = { + val b = prefix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, 0, b.length), b) + } + + def endsWith(suffix: UTF8String): Boolean = { + val b = suffix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, bytes.length - b.length, bytes.length), b) + } + + def toUpperCase(): UTF8String = { + // upper case depends on locale, fallback to String. + UTF8String(toString().toUpperCase) + } + + def toLowerCase(): UTF8String = { + // lower case depends on locale, fallback to String. + UTF8String(toString().toLowerCase) + } + + override def toString(): String = { + new String(bytes, "utf-8") + } + + override def clone(): UTF8String = new UTF8String().set(this.bytes) + + override def compare(other: UTF8String): Int = { + var i: Int = 0 + val b = other.getBytes + while (i < bytes.length && i < b.length) { + val res = bytes(i).compareTo(b(i)) + if (res != 0) return res + i += 1 + } + bytes.length - b.length + } + + override def compareTo(other: UTF8String): Int = { + compare(other) + } + + override def equals(other: Any): Boolean = other match { + case s: UTF8String => + Arrays.equals(bytes, s.getBytes) + case s: String => + // This is only used for Catalyst unit tests + // fail fast + bytes.length >= s.length && length() == s.length && toString() == s + case _ => + false + } + + override def hashCode(): Int = { + Arrays.hashCode(bytes) + } +} + +object UTF8String { + // number of tailing bytes in a UTF8 sequence for a code point + // see http://en.wikipedia.org/wiki/UTF-8, 192-256 of Byte 1 + private[types] val bytesOfCodePointInUTF8: Array[Int] = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, + 6, 6, 6, 6) + + /** + * Create a UTF-8 String from String + */ + def apply(s: String): UTF8String = { + if (s != null) { + new UTF8String().set(s) + } else{ + null + } + } + + /** + * Create a UTF-8 String from Array[Byte], which should be encoded in UTF-8 + */ + def apply(bytes: Array[Byte]): UTF8String = { + if (bytes != null) { + new UTF8String().set(bytes) + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala new file mode 100644 index 0000000000000..6b20505c6009a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala @@ -0,0 +1,81 @@ +/* + * 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.sql.types + +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.DeveloperApi + +/** + * ::DeveloperApi:: + * The data type for User Defined Types (UDTs). + * + * This interface allows a user to make their own classes more interoperable with SparkSQL; + * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create + * a `DataFrame` which has class X in the schema. + * + * For SparkSQL to recognize UDTs, the UDT must be annotated with + * [[SQLUserDefinedType]]. + * + * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. + * The conversion via `deserialize` occurs when reading from a `DataFrame`. + */ +@DeveloperApi +abstract class UserDefinedType[UserType] extends DataType with Serializable { + + /** Underlying storage type for this UDT */ + def sqlType: DataType + + /** Paired Python UDT class, if exists. */ + def pyUDT: String = null + + /** + * Convert the user type to a SQL datum + * + * TODO: Can we make this take obj: UserType? The issue is in + * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. + */ + def serialize(obj: Any): Any + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType + + override private[sql] def jsonValue: JValue = { + ("type" -> "udt") ~ + ("class" -> this.getClass.getName) ~ + ("pyClass" -> pyUDT) ~ + ("sqlType" -> sqlType.jsonValue) + } + + /** + * Class object for the UserType + */ + def userClass: java.lang.Class[UserType] + + /** + * The default size of a value of the UserDefinedType is 4096 bytes. + */ + override def defaultSize: Int = 4096 + + /** + * For UDT, asNullable will not change the nullability of its internal sqlType and just returns + * itself. + */ + private[spark] override def asNullable: UserDefinedType[UserType] = this +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala deleted file mode 100644 index d973144de3468..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ /dev/null @@ -1,1208 +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.sql.types - -import java.sql.Timestamp - -import scala.collection.mutable.ArrayBuffer -import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} -import scala.util.parsing.combinator.RegexParsers - -import org.json4s._ -import org.json4s.JsonAST.JValue -import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods._ - -import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.ScalaReflectionLock -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} -import org.apache.spark.util.Utils - - -object DataType { - def fromJson(json: String): DataType = parseDataType(parse(json)) - - private object JSortedObject { - def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { - case JObject(seq) => Some(seq.toList.sortBy(_._1)) - case _ => None - } - } - - // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. - private def parseDataType(json: JValue): DataType = json match { - case JString(name) => - PrimitiveType.nameToType(name) - - case JSortedObject( - ("containsNull", JBool(n)), - ("elementType", t: JValue), - ("type", JString("array"))) => - ArrayType(parseDataType(t), n) - - case JSortedObject( - ("keyType", k: JValue), - ("type", JString("map")), - ("valueContainsNull", JBool(n)), - ("valueType", v: JValue)) => - MapType(parseDataType(k), parseDataType(v), n) - - case JSortedObject( - ("fields", JArray(fields)), - ("type", JString("struct"))) => - StructType(fields.map(parseStructField)) - - case JSortedObject( - ("class", JString(udtClass)), - ("pyClass", _), - ("sqlType", _), - ("type", JString("udt"))) => - Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] - } - - private def parseStructField(json: JValue): StructField = json match { - case JSortedObject( - ("metadata", metadata: JObject), - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => - StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) - // Support reading schema when 'metadata' is missing. - case JSortedObject( - ("name", JString(name)), - ("nullable", JBool(nullable)), - ("type", dataType: JValue)) => - StructField(name, parseDataType(dataType), nullable) - } - - @deprecated("Use DataType.fromJson instead", "1.2.0") - def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) - - private object CaseClassStringParser extends RegexParsers { - protected lazy val primitiveType: Parser[DataType] = - ( "StringType" ^^^ StringType - | "FloatType" ^^^ FloatType - | "IntegerType" ^^^ IntegerType - | "ByteType" ^^^ ByteType - | "ShortType" ^^^ ShortType - | "DoubleType" ^^^ DoubleType - | "LongType" ^^^ LongType - | "BinaryType" ^^^ BinaryType - | "BooleanType" ^^^ BooleanType - | "DateType" ^^^ DateType - | "DecimalType()" ^^^ DecimalType.Unlimited - | fixedDecimalType - | "TimestampType" ^^^ TimestampType - ) - - protected lazy val fixedDecimalType: Parser[DataType] = - ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) - } - - protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { - case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) - } - - protected lazy val mapType: Parser[DataType] = - "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { - case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) - } - - protected lazy val structField: Parser[StructField] = - ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { - case name ~ tpe ~ nullable => - StructField(name, tpe, nullable = nullable) - } - - protected lazy val boolVal: Parser[Boolean] = - ( "true" ^^^ true - | "false" ^^^ false - ) - - protected lazy val structType: Parser[DataType] = - "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { - case fields => StructType(fields) - } - - protected lazy val dataType: Parser[DataType] = - ( arrayType - | mapType - | structType - | primitiveType - ) - - /** - * Parses a string representation of a DataType. - * - * TODO: Generate parser as pickler... - */ - def apply(asString: String): DataType = parseAll(dataType, asString) match { - case Success(result, _) => result - case failure: NoSuccess => - throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") - } - } - - protected[types] def buildFormattedString( - dataType: DataType, - prefix: String, - builder: StringBuilder): Unit = { - dataType match { - case array: ArrayType => - array.buildFormattedString(prefix, builder) - case struct: StructType => - struct.buildFormattedString(prefix, builder) - case map: MapType => - map.buildFormattedString(prefix, builder) - case _ => - } - } - - /** - * Compares two types, ignoring nullability of ArrayType, MapType, StructType. - */ - private[types] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { - (left, right) match { - case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => - equalsIgnoreNullability(leftElementType, rightElementType) - case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) => - equalsIgnoreNullability(leftKeyType, rightKeyType) && - equalsIgnoreNullability(leftValueType, rightValueType) - case (StructType(leftFields), StructType(rightFields)) => - leftFields.size == rightFields.size && - leftFields.zip(rightFields) - .forall{ - case (left, right) => - left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType) - } - case (left, right) => left == right - } - } - - /** - * Compares two types, ignoring compatible nullability of ArrayType, MapType, StructType. - * - * Compatible nullability is defined as follows: - * - If `from` and `to` are ArrayTypes, `from` has a compatible nullability with `to` - * if and only if `to.containsNull` is true, or both of `from.containsNull` and - * `to.containsNull` are false. - * - If `from` and `to` are MapTypes, `from` has a compatible nullability with `to` - * if and only if `to.valueContainsNull` is true, or both of `from.valueContainsNull` and - * `to.valueContainsNull` are false. - * - If `from` and `to` are StructTypes, `from` has a compatible nullability with `to` - * if and only if for all every pair of fields, `to.nullable` is true, or both - * of `fromField.nullable` and `toField.nullable` are false. - */ - private[sql] def equalsIgnoreCompatibleNullability(from: DataType, to: DataType): Boolean = { - (from, to) match { - case (ArrayType(fromElement, fn), ArrayType(toElement, tn)) => - (tn || !fn) && equalsIgnoreCompatibleNullability(fromElement, toElement) - - case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => - (tn || !fn) && - equalsIgnoreCompatibleNullability(fromKey, toKey) && - equalsIgnoreCompatibleNullability(fromValue, toValue) - - case (StructType(fromFields), StructType(toFields)) => - fromFields.size == toFields.size && - fromFields.zip(toFields).forall { - case (fromField, toField) => - fromField.name == toField.name && - (toField.nullable || !fromField.nullable) && - equalsIgnoreCompatibleNullability(fromField.dataType, toField.dataType) - } - - case (fromDataType, toDataType) => fromDataType == toDataType - } - } -} - - -/** - * :: DeveloperApi :: - * The base type of all Spark SQL data types. - * - * @group dataType - */ -@DeveloperApi -abstract class DataType { - /** Matches any expression that evaluates to this DataType */ - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType == this => true - case _ => false - } - - /** The default size of a value of this data type. */ - def defaultSize: Int - - def isPrimitive: Boolean = false - - def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase - - private[sql] def jsonValue: JValue = typeName - - def json: String = compact(render(jsonValue)) - - def prettyJson: String = pretty(render(jsonValue)) - - def simpleString: String = typeName - - /** Check if `this` and `other` are the same data type when ignoring nullability - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). - */ - private[spark] def sameType(other: DataType): Boolean = - DataType.equalsIgnoreNullability(this, other) - - /** Returns the same data type but set all nullability fields are true - * (`StructField.nullable`, `ArrayType.containsNull`, and `MapType.valueContainsNull`). - */ - private[spark] def asNullable: DataType -} - -/** - * :: DeveloperApi :: - * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. - * - * @group dataType - */ -@DeveloperApi -class NullType private() extends DataType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "NullType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - override def defaultSize: Int = 1 - - private[spark] override def asNullable: NullType = this -} - -case object NullType extends NullType - - -protected[sql] object NativeType { - val all = Seq( - IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) - - def unapply(dt: DataType): Boolean = all.contains(dt) -} - - -protected[sql] trait PrimitiveType extends DataType { - override def isPrimitive: Boolean = true -} - - -protected[sql] object PrimitiveType { - private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all - private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap - - /** Given the string representation of a type, return its DataType */ - private[sql] def nameToType(name: String): DataType = { - val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r - name match { - case "decimal" => DecimalType.Unlimited - case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) - case other => nonDecimalNameToType(other) - } - } -} - -protected[sql] abstract class NativeType extends DataType { - private[sql] type JvmType - @transient private[sql] val tag: TypeTag[JvmType] - private[sql] val ordering: Ordering[JvmType] - - @transient private[sql] val classTag = ScalaReflectionLock.synchronized { - val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) - } -} - - -/** - * :: DeveloperApi :: - * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. - * - * @group dataType - */ -@DeveloperApi -class StringType private() extends NativeType with PrimitiveType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "StringType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = String - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the StringType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - private[spark] override def asNullable: StringType = this -} - -case object StringType extends StringType - - -/** - * :: DeveloperApi :: - * The data type representing `Array[Byte]` values. - * Please use the singleton [[DataTypes.BinaryType]]. - * - * @group dataType - */ -@DeveloperApi -class BinaryType private() extends NativeType with PrimitiveType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Array[Byte] - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = new Ordering[JvmType] { - def compare(x: Array[Byte], y: Array[Byte]): Int = { - for (i <- 0 until x.length; if i < y.length) { - val res = x(i).compareTo(y(i)) - if (res != 0) return res - } - x.length - y.length - } - } - - /** - * The default size of a value of the BinaryType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - private[spark] override def asNullable: BinaryType = this -} - -case object BinaryType extends BinaryType - - -/** - * :: DeveloperApi :: - * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. - * - *@group dataType - */ -@DeveloperApi -class BooleanType private() extends NativeType with PrimitiveType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Boolean - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the BooleanType is 1 byte. - */ - override def defaultSize: Int = 1 - - private[spark] override def asNullable: BooleanType = this -} - -case object BooleanType extends BooleanType - - -/** - * :: DeveloperApi :: - * The data type representing `java.sql.Timestamp` values. - * Please use the singleton [[DataTypes.TimestampType]]. - * - * @group dataType - */ -@DeveloperApi -class TimestampType private() extends NativeType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Timestamp - - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - - private[sql] val ordering = new Ordering[JvmType] { - def compare(x: Timestamp, y: Timestamp): Int = x.compareTo(y) - } - - /** - * The default size of a value of the TimestampType is 12 bytes. - */ - override def defaultSize: Int = 12 - - private[spark] override def asNullable: TimestampType = this -} - -case object TimestampType extends TimestampType - - -/** - * :: DeveloperApi :: - * The data type representing `java.sql.Date` values. - * Please use the singleton [[DataTypes.DateType]]. - * - * @group dataType - */ -@DeveloperApi -class DateType private() extends NativeType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "DateType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Int - - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the DateType is 4 bytes. - */ - override def defaultSize: Int = 4 - - private[spark] override def asNullable: DateType = this -} - -case object DateType extends DateType - - -/** - * :: DeveloperApi :: - * Numeric data types. - * - * @group dataType - */ -abstract class NumericType extends NativeType with PrimitiveType { - // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for - // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a - // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets - // desugared by the compiler into an argument to the objects constructor. This means there is no - // longer an no argument constructor and thus the JVM cannot serialize the object anymore. - private[sql] val numeric: Numeric[JvmType] -} - - -protected[sql] object NumericType { - def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] -} - - -/** Matcher for any expressions that evaluate to [[IntegralType]]s */ -protected[sql] object IntegralType { - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[IntegralType] => true - case _ => false - } -} - - -protected[sql] sealed abstract class IntegralType extends NumericType { - private[sql] val integral: Integral[JvmType] -} - - -/** - * :: DeveloperApi :: - * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. - * - * @group dataType - */ -@DeveloperApi -class LongType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "LongType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Long - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Long]] - private[sql] val integral = implicitly[Integral[Long]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the LongType is 8 bytes. - */ - override def defaultSize: Int = 8 - - override def simpleString: String = "bigint" - - private[spark] override def asNullable: LongType = this -} - -case object LongType extends LongType - - -/** - * :: DeveloperApi :: - * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. - * - * @group dataType - */ -@DeveloperApi -class IntegerType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Int - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Int]] - private[sql] val integral = implicitly[Integral[Int]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the IntegerType is 4 bytes. - */ - override def defaultSize: Int = 4 - - override def simpleString: String = "int" - - private[spark] override def asNullable: IntegerType = this -} - -case object IntegerType extends IntegerType - - -/** - * :: DeveloperApi :: - * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. - * - * @group dataType - */ -@DeveloperApi -class ShortType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Short - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Short]] - private[sql] val integral = implicitly[Integral[Short]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the ShortType is 2 bytes. - */ - override def defaultSize: Int = 2 - - override def simpleString: String = "smallint" - - private[spark] override def asNullable: ShortType = this -} - -case object ShortType extends ShortType - - -/** - * :: DeveloperApi :: - * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. - * - * @group dataType - */ -@DeveloperApi -class ByteType private() extends IntegralType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Byte - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Byte]] - private[sql] val integral = implicitly[Integral[Byte]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - - /** - * The default size of a value of the ByteType is 1 byte. - */ - override def defaultSize: Int = 1 - - override def simpleString: String = "tinyint" - - private[spark] override def asNullable: ByteType = this -} - -case object ByteType extends ByteType - - -/** Matcher for any expressions that evaluate to [[FractionalType]]s */ -protected[sql] object FractionalType { - def unapply(a: Expression): Boolean = a match { - case e: Expression if e.dataType.isInstanceOf[FractionalType] => true - case _ => false - } -} - - -protected[sql] sealed abstract class FractionalType extends NumericType { - private[sql] val fractional: Fractional[JvmType] - private[sql] val asIntegral: Integral[JvmType] -} - - -/** Precision parameters for a Decimal */ -case class PrecisionInfo(precision: Int, scale: Int) - - -/** - * :: DeveloperApi :: - * The data type representing `java.math.BigDecimal` values. - * A Decimal that might have fixed precision and scale, or unlimited values for these. - * - * Please use [[DataTypes.createDecimalType()]] to create a specific instance. - * - * @group dataType - */ -@DeveloperApi -case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { - private[sql] type JvmType = Decimal - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = Decimal.DecimalIsFractional - private[sql] val fractional = Decimal.DecimalIsFractional - private[sql] val ordering = Decimal.DecimalIsFractional - private[sql] val asIntegral = Decimal.DecimalAsIfIntegral - - def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) - - def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) - - override def typeName: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" - case None => "decimal" - } - - override def toString: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" - case None => "DecimalType()" - } - - /** - * The default size of a value of the DecimalType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - override def simpleString: String = precisionInfo match { - case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" - case None => "decimal(10,0)" - } - - private[spark] override def asNullable: DecimalType = this -} - - -/** Extra factory methods and pattern matchers for Decimals */ -object DecimalType { - val Unlimited: DecimalType = DecimalType(None) - - object Fixed { - def unapply(t: DecimalType): Option[(Int, Int)] = - t.precisionInfo.map(p => (p.precision, p.scale)) - } - - object Expression { - def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { - case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) - case _ => None - } - } - - def apply(): DecimalType = Unlimited - - def apply(precision: Int, scale: Int): DecimalType = - DecimalType(Some(PrecisionInfo(precision, scale))) - - def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] - - def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] - - def isFixed(dataType: DataType): Boolean = dataType match { - case DecimalType.Fixed(_, _) => true - case _ => false - } -} - - -/** - * :: DeveloperApi :: - * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. - * - * @group dataType - */ -@DeveloperApi -class DoubleType private() extends FractionalType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Double - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Double]] - private[sql] val fractional = implicitly[Fractional[Double]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - private[sql] val asIntegral = DoubleAsIfIntegral - - /** - * The default size of a value of the DoubleType is 8 bytes. - */ - override def defaultSize: Int = 8 - - private[spark] override def asNullable: DoubleType = this -} - -case object DoubleType extends DoubleType - - -/** - * :: DeveloperApi :: - * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. - * - * @group dataType - */ -@DeveloperApi -class FloatType private() extends FractionalType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = Float - @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[Float]] - private[sql] val fractional = implicitly[Fractional[Float]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - private[sql] val asIntegral = FloatAsIfIntegral - - /** - * The default size of a value of the FloatType is 4 bytes. - */ - override def defaultSize: Int = 4 - - private[spark] override def asNullable: FloatType = this -} - -case object FloatType extends FloatType - - -object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) -} - - -/** - * :: DeveloperApi :: - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. - * - * Please use [[DataTypes.createArrayType()]] to create a specific instance. - * - * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and - * `containsNull: Boolean`. The field of `elementType` is used to specify the type of - * array elements. The field of `containsNull` is used to specify if the array has `null` values. - * - * @param elementType The data type of values. - * @param containsNull Indicates if values have `null` values - * - * @group dataType - */ -@DeveloperApi -case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append( - s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") - DataType.buildFormattedString(elementType, s"$prefix |", builder) - } - - override private[sql] def jsonValue = - ("type" -> typeName) ~ - ("elementType" -> elementType.jsonValue) ~ - ("containsNull" -> containsNull) - - /** - * The default size of a value of the ArrayType is 100 * the default size of the element type. - * (We assume that there are 100 elements). - */ - override def defaultSize: Int = 100 * elementType.defaultSize - - override def simpleString: String = s"array<${elementType.simpleString}>" - - private[spark] override def asNullable: ArrayType = - ArrayType(elementType.asNullable, containsNull = true) -} - - -/** - * A field inside a StructType. - * @param name The name of this field. - * @param dataType The data type of this field. - * @param nullable Indicates if values of this field can be `null` values. - * @param metadata The metadata of this field. The metadata should be preserved during - * transformation if the content of the column is not modified, e.g, in selection. - */ -case class StructField( - name: String, - dataType: DataType, - nullable: Boolean = true, - metadata: Metadata = Metadata.empty) { - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") - DataType.buildFormattedString(dataType, s"$prefix |", builder) - } - - // override the default toString to be compatible with legacy parquet files. - override def toString: String = s"StructField($name,$dataType,$nullable)" - - private[sql] def jsonValue: JValue = { - ("name" -> name) ~ - ("type" -> dataType.jsonValue) ~ - ("nullable" -> nullable) ~ - ("metadata" -> metadata.jsonValue) - } -} - - -object StructType { - protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = - StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) - - def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) - - def apply(fields: java.util.List[StructField]): StructType = { - StructType(fields.toArray.asInstanceOf[Array[StructField]]) - } - - private[sql] def merge(left: DataType, right: DataType): DataType = - (left, right) match { - case (ArrayType(leftElementType, leftContainsNull), - ArrayType(rightElementType, rightContainsNull)) => - ArrayType( - merge(leftElementType, rightElementType), - leftContainsNull || rightContainsNull) - - case (MapType(leftKeyType, leftValueType, leftContainsNull), - MapType(rightKeyType, rightValueType, rightContainsNull)) => - MapType( - merge(leftKeyType, rightKeyType), - merge(leftValueType, rightValueType), - leftContainsNull || rightContainsNull) - - case (StructType(leftFields), StructType(rightFields)) => - val newFields = ArrayBuffer.empty[StructField] - - leftFields.foreach { - case leftField @ StructField(leftName, leftType, leftNullable, _) => - rightFields - .find(_.name == leftName) - .map { case rightField @ StructField(_, rightType, rightNullable, _) => - leftField.copy( - dataType = merge(leftType, rightType), - nullable = leftNullable || rightNullable) - } - .orElse(Some(leftField)) - .foreach(newFields += _) - } - - rightFields - .filterNot(f => leftFields.map(_.name).contains(f.name)) - .foreach(newFields += _) - - StructType(newFields) - - case (DecimalType.Fixed(leftPrecision, leftScale), - DecimalType.Fixed(rightPrecision, rightScale)) => - DecimalType(leftPrecision.max(rightPrecision), leftScale.max(rightScale)) - - case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) - if leftUdt.userClass == rightUdt.userClass => leftUdt - - case (leftType, rightType) if leftType == rightType => - leftType - - case _ => - throw new SparkException(s"Failed to merge incompatible data types $left and $right") - } -} - - -/** - * :: DeveloperApi :: - * A [[StructType]] object can be constructed by - * {{{ - * StructType(fields: Seq[StructField]) - * }}} - * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. - * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. - * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single StructField, a `null` will be returned. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val struct = - * StructType( - * StructField("a", IntegerType, true) :: - * StructField("b", LongType, false) :: - * StructField("c", BooleanType, false) :: Nil) - * - * // Extract a single StructField. - * val singleField = struct("b") - * // singleField: StructField = StructField(b,LongType,false) - * - * // This struct does not have a field called "d". null will be returned. - * val nonExisting = struct("d") - * // nonExisting: StructField = null - * - * // Extract multiple StructFields. Field names are provided in a set. - * // A StructType object will be returned. - * val twoFields = struct(Set("b", "c")) - * // twoFields: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * - * // Any names without matching fields will be ignored. - * // For the case shown below, "d" will be ignored and - * // it is treated as struct(Set("b", "c")). - * val ignoreNonExisting = struct(Set("b", "c", "d")) - * // ignoreNonExisting: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * }}} - * - * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val innerStruct = - * StructType( - * StructField("f1", IntegerType, true) :: - * StructField("f2", LongType, false) :: - * StructField("f3", BooleanType, false) :: Nil) - * - * val struct = StructType( - * StructField("a", innerStruct, true) :: Nil) - * - * // Create a Row with the schema defined by struct - * val row = Row(Row(1, 2, true)) - * // row: Row = [[1,2,true]] - * }}} - * - * @group dataType - */ -@DeveloperApi -case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { - - /** Returns all field names in an array. */ - def fieldNames: Array[String] = fields.map(_.name) - - private lazy val fieldNamesSet: Set[String] = fieldNames.toSet - private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap - - /** - * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not - * have a name matching the given name, `null` will be returned. - */ - def apply(name: String): StructField = { - nameToField.getOrElse(name, - throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) - } - - /** - * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the - * original order of fields. Those names which do not have matching fields will be ignored. - */ - def apply(names: Set[String]): StructType = { - val nonExistFields = names -- fieldNamesSet - if (nonExistFields.nonEmpty) { - throw new IllegalArgumentException( - s"Field ${nonExistFields.mkString(",")} does not exist.") - } - // Preserve the original order of fields. - StructType(fields.filter(f => names.contains(f.name))) - } - - protected[sql] def toAttributes: Seq[AttributeReference] = - map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - - def treeString: String = { - val builder = new StringBuilder - builder.append("root\n") - val prefix = " |" - fields.foreach(field => field.buildFormattedString(prefix, builder)) - - builder.toString() - } - - def printTreeString(): Unit = println(treeString) - - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - fields.foreach(field => field.buildFormattedString(prefix, builder)) - } - - override private[sql] def jsonValue = - ("type" -> typeName) ~ - ("fields" -> map(_.jsonValue)) - - override def apply(fieldIndex: Int): StructField = fields(fieldIndex) - - override def length: Int = fields.length - - override def iterator: Iterator[StructField] = fields.iterator - - /** - * The default size of a value of the StructType is the total default sizes of all field types. - */ - override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum - - override def simpleString: String = { - val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") - s"struct<${fieldTypes.mkString(",")}>" - } - - /** - * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field - * B from `that`, - * - * 1. If A and B have the same name and data type, they are merged to a field C with the same name - * and data type. C is nullable if and only if either A or B is nullable. - * 2. If A doesn't exist in `that`, it's included in the result schema. - * 3. If B doesn't exist in `this`, it's also included in the result schema. - * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be - * thrown. - */ - private[sql] def merge(that: StructType): StructType = - StructType.merge(this, that).asInstanceOf[StructType] - - private[spark] override def asNullable: StructType = { - val newFields = fields.map { - case StructField(name, dataType, nullable, metadata) => - StructField(name, dataType.asNullable, nullable = true, metadata) - } - - StructType(newFields) - } -} - - -object MapType { - /** - * Construct a [[MapType]] object with the given key type and value type. - * The `valueContainsNull` is true. - */ - def apply(keyType: DataType, valueType: DataType): MapType = - MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) -} - - -/** - * :: DeveloperApi :: - * The data type for Maps. Keys in a map are not allowed to have `null` values. - * - * Please use [[DataTypes.createMapType()]] to create a specific instance. - * - * @param keyType The data type of map keys. - * @param valueType The data type of map values. - * @param valueContainsNull Indicates if map values have `null` values. - * - * @group dataType - */ -case class MapType( - keyType: DataType, - valueType: DataType, - valueContainsNull: Boolean) extends DataType { - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- key: ${keyType.typeName}\n") - builder.append(s"$prefix-- value: ${valueType.typeName} " + - s"(valueContainsNull = $valueContainsNull)\n") - DataType.buildFormattedString(keyType, s"$prefix |", builder) - DataType.buildFormattedString(valueType, s"$prefix |", builder) - } - - override private[sql] def jsonValue: JValue = - ("type" -> typeName) ~ - ("keyType" -> keyType.jsonValue) ~ - ("valueType" -> valueType.jsonValue) ~ - ("valueContainsNull" -> valueContainsNull) - - /** - * The default size of a value of the MapType is - * 100 * (the default size of the key type + the default size of the value type). - * (We assume that there are 100 elements). - */ - override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) - - override def simpleString: String = s"map<${keyType.simpleString},${valueType.simpleString}>" - - private[spark] override def asNullable: MapType = - MapType(keyType.asNullable, valueType.asNullable, valueContainsNull = true) -} - - -/** - * ::DeveloperApi:: - * The data type for User Defined Types (UDTs). - * - * This interface allows a user to make their own classes more interoperable with SparkSQL; - * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create - * a `DataFrame` which has class X in the schema. - * - * For SparkSQL to recognize UDTs, the UDT must be annotated with - * [[SQLUserDefinedType]]. - * - * The conversion via `serialize` occurs when instantiating a `DataFrame` from another RDD. - * The conversion via `deserialize` occurs when reading from a `DataFrame`. - */ -@DeveloperApi -abstract class UserDefinedType[UserType] extends DataType with Serializable { - - /** Underlying storage type for this UDT */ - def sqlType: DataType - - /** Paired Python UDT class, if exists. */ - def pyUDT: String = null - - /** - * Convert the user type to a SQL datum - * - * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, - * where we need to convert Any to UserType. - */ - def serialize(obj: Any): Any - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType - - override private[sql] def jsonValue: JValue = { - ("type" -> "udt") ~ - ("class" -> this.getClass.getName) ~ - ("pyClass" -> pyUDT) ~ - ("sqlType" -> sqlType.jsonValue) - } - - /** - * Class object for the UserType - */ - def userClass: java.lang.Class[UserType] - - /** - * The default size of a value of the UserDefinedType is 4096 bytes. - */ - override def defaultSize: Int = 4096 - - /** - * For UDT, asNullable will not change the nullability of its internal sqlType and just returns - * itself. - */ - private[spark] override def asNullable: UserDefinedType[UserType] = this -} diff --git a/sql/catalyst/src/test/resources/log4j.properties b/sql/catalyst/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/sql/catalyst/src/test/resources/log4j.properties +++ b/sql/catalyst/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala new file mode 100644 index 0000000000000..bbb9739e9cc76 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowTest.scala @@ -0,0 +1,71 @@ +/* + * 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.sql + +import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} +import org.apache.spark.sql.types._ +import org.scalatest.{Matchers, FunSpec} + +class RowTest extends FunSpec with Matchers { + + val schema = StructType( + StructField("col1", StringType) :: + StructField("col2", StringType) :: + StructField("col3", IntegerType) :: Nil) + val values = Array("value1", "value2", 1) + + val sampleRow: Row = new GenericRowWithSchema(values, schema) + val noSchemaRow: Row = new GenericRow(values) + + describe("Row (without schema)") { + it("throws an exception when accessing by fieldName") { + intercept[UnsupportedOperationException] { + noSchemaRow.fieldIndex("col1") + } + intercept[UnsupportedOperationException] { + noSchemaRow.getAs("col1") + } + } + } + + describe("Row (with schema)") { + it("fieldIndex(name) returns field index") { + sampleRow.fieldIndex("col1") shouldBe 0 + sampleRow.fieldIndex("col3") shouldBe 2 + } + + it("getAs[T] retrieves a value by fieldname") { + sampleRow.getAs[String]("col1") shouldBe "value1" + sampleRow.getAs[Int]("col3") shouldBe 1 + } + + it("Accessing non existent field throws an exception") { + intercept[IllegalArgumentException] { + sampleRow.getAs[String]("non_existent") + } + } + + it("getValuesMap() retrieves values of multiple fields as a Map(field -> value)") { + val expected = Map( + "col1" -> "value1", + "col2" -> "value2" + ) + sampleRow.getValuesMap(List("col1", "col2")) shouldBe expected + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 46b2250aab231..ea82cd2622de9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -30,7 +30,7 @@ class DistributionSuite extends FunSuite { inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if (inputPartitioning.satisfies(requiredDistribution) != satisfied) + if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" |== Input Partitioning == @@ -40,6 +40,7 @@ class DistributionSuite extends FunSuite { |== Does input partitioning satisfy required distribution? == |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) + } } test("HashPartitioning is the output partitioning") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index eee00e3f7ea76..bbc0b661a0c0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -260,7 +260,7 @@ class ScalaReflectionSuite extends FunSuite { val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) val convertedData = Row(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) val dataType = schemaFor[PrimitiveData].dataType - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("convert Option[Product] to catalyst") { @@ -270,7 +270,7 @@ class ScalaReflectionSuite extends FunSuite { val dataType = schemaFor[OptionalData].dataType val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, Row(1, 1, 1, 1, 1, 1, true)) - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("infer schema from case class with multiple constructors") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala index 1a0a0e6154ad2..a652c70560990 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/SqlParserSuite.scala @@ -49,13 +49,14 @@ class SqlParserSuite extends FunSuite { test("test long keyword") { val parser = new SuperLongKeywordTestParser - assert(TestCommand("NotRealCommand") === parser("ThisIsASuperLongKeyWordTest NotRealCommand")) + assert(TestCommand("NotRealCommand") === + parser.parse("ThisIsASuperLongKeyWordTest NotRealCommand")) } test("test case insensitive") { val parser = new CaseInsensitiveTestParser - assert(TestCommand("NotRealCommand") === parser("EXECUTE NotRealCommand")) - assert(TestCommand("NotRealCommand") === parser("execute NotRealCommand")) - assert(TestCommand("NotRealCommand") === parser("exEcute NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("EXECUTE NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("execute NotRealCommand")) + assert(TestCommand("NotRealCommand") === parser.parse("exEcute NotRealCommand")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index c1dd5aa913ddc..971e1ff5ec2b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -32,18 +32,20 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseInsensitiveCatalog = new SimpleCatalog(false) val caseSensitiveAnalyzer = - new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } val caseInsensitiveAnalyzer = - new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) - - val checkAnalysis = new CheckAnalysis + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } - def caseSensitiveAnalyze(plan: LogicalPlan) = - checkAnalysis(caseSensitiveAnalyzer(plan)) + def caseSensitiveAnalyze(plan: LogicalPlan): Unit = + caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer.execute(plan)) - def caseInsensitiveAnalyze(plan: LogicalPlan) = - checkAnalysis(caseInsensitiveAnalyzer(plan)) + def caseInsensitiveAnalyze(plan: LogicalPlan): Unit = + caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer.execute(plan)) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) val testRelation2 = LocalRelation( @@ -53,6 +55,21 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) + val nestedRelation = LocalRelation( + AttributeReference("top", StructType( + StructField("duplicateField", StringType) :: + StructField("duplicateField", StringType) :: + StructField("differentCase", StringType) :: + StructField("differentcase", StringType) :: Nil + ))()) + + val nestedRelation2 = LocalRelation( + AttributeReference("top", StructType( + StructField("aField", StringType) :: + StructField("bField", StringType) :: + StructField("cField", StringType) :: Nil + ))()) + before { caseSensitiveCatalog.registerTable(Seq("TaBlE"), testRelation) caseInsensitiveCatalog.registerTable(Seq("TaBlE"), testRelation) @@ -65,7 +82,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { a.select(UnresolvedStar(None)).select('a).unionAll(b.select(UnresolvedStar(None))) } - assert(caseInsensitiveAnalyzer(plan).resolved) + assert(caseInsensitiveAnalyzer.execute(plan).resolved) } test("check project's resolved") { @@ -73,7 +90,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(!Project(Seq(UnresolvedAttribute("a")), testRelation).resolved) - val explode = Explode(Nil, AttributeReference("a", IntegerType, nullable = true)()) + val explode = Explode(AttributeReference("a", IntegerType, nullable = true)()) assert(!Project(Seq(Alias(explode, "explode")()), testRelation).resolved) assert(!Project(Seq(Alias(Count(Literal(1)), "count")()), testRelation).resolved) @@ -81,11 +98,11 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { test("analyze project") { assert( - caseSensitiveAnalyzer(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + caseSensitiveAnalyzer.execute(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) assert( - caseSensitiveAnalyzer( + caseSensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -98,13 +115,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage().toLowerCase.contains("cannot resolve")) assert( - caseInsensitiveAnalyzer( + caseInsensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("TbL.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) assert( - caseInsensitiveAnalyzer( + caseInsensitiveAnalyzer.execute( Project(Seq(UnresolvedAttribute("tBl.a")), UnresolvedRelation(Seq("TaBlE"), Some("TbL")))) === Project(testRelation.output, testRelation)) @@ -117,20 +134,20 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(e.getMessage == "Table Not Found: tAbLe") assert( - caseSensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) + caseSensitiveAnalyzer.execute(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) assert( - caseInsensitiveAnalyzer(UnresolvedRelation(Seq("tAbLe"), None)) === testRelation) + caseInsensitiveAnalyzer.execute(UnresolvedRelation(Seq("tAbLe"), None)) === testRelation) assert( - caseInsensitiveAnalyzer(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) + caseInsensitiveAnalyzer.execute(UnresolvedRelation(Seq("TaBlE"), None)) === testRelation) } def errorTest( name: String, plan: LogicalPlan, errorMessages: Seq[String], - caseSensitive: Boolean = true) = { + caseSensitive: Boolean = true): Unit = { test(name) { val error = intercept[AnalysisException] { if(caseSensitive) { @@ -165,9 +182,27 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { "'b'" :: "group by" :: Nil ) + errorTest( + "ambiguous field", + nestedRelation.select($"top.duplicateField"), + "Ambiguous reference to fields" :: "duplicateField" :: Nil, + caseSensitive = false) + + errorTest( + "ambiguous field due to case insensitivity", + nestedRelation.select($"top.differentCase"), + "Ambiguous reference to fields" :: "differentCase" :: "differentcase" :: Nil, + caseSensitive = false) + + errorTest( + "missing field", + nestedRelation2.select($"top.c"), + "No such struct field" :: "aField" :: "bField" :: "cField" :: Nil, + caseSensitive = false) + case class UnresolvedTestPlan() extends LeafNode { override lazy val resolved = false - override def output = Nil + override def output: Seq[Attribute] = Nil } errorTest( @@ -184,7 +219,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) - val plan = caseInsensitiveAnalyzer( + val plan = caseInsensitiveAnalyzer.execute( testRelation2.select( 'a / Literal(2) as 'div1, 'a / 'b as 'div2, @@ -199,4 +234,22 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(pl(3).dataType == DecimalType.Unlimited) assert(pl(4).dataType == DoubleType) } + + test("SPARK-6452 regression test") { + // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) + val plan = + Aggregate( + Nil, + Alias(Sum(AttributeReference("a", StringType)(exprId = ExprId(1))), "b")() :: Nil, + LocalRelation( + AttributeReference("a", StringType)(exprId = ExprId(2)))) + + assert(plan.resolved) + + val message = intercept[AnalysisException] { + caseSensitiveAnalyze(plan) + }.getMessage + + assert(message.contains("resolved attribute(s) a#1 missing from a#2")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index bc2ec754d5865..36b03d1c65e28 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation} +import org.apache.spark.sql.catalyst.plans.logical.{Union, Project, LocalRelation} import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -31,7 +31,8 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { AttributeReference("d1", DecimalType(2, 1))(), AttributeReference("d2", DecimalType(5, 2))(), AttributeReference("u", DecimalType.Unlimited)(), - AttributeReference("f", FloatType)() + AttributeReference("f", FloatType)(), + AttributeReference("b", DoubleType)() ) val i: Expression = UnresolvedAttribute("i") @@ -39,6 +40,7 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { val d2: Expression = UnresolvedAttribute("d2") val u: Expression = UnresolvedAttribute("u") val f: Expression = UnresolvedAttribute("f") + val b: Expression = UnresolvedAttribute("b") before { catalog.registerTable(Seq("table"), relation) @@ -46,18 +48,29 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { private def checkType(expression: Expression, expectedType: DataType): Unit = { val plan = Project(Seq(Alias(expression, "c")()), relation) - assert(analyzer(plan).schema.fields(0).dataType === expectedType) + assert(analyzer.execute(plan).schema.fields(0).dataType === expectedType) } private def checkComparison(expression: Expression, expectedType: DataType): Unit = { val plan = Project(Alias(expression, "c")() :: Nil, relation) - val comparison = analyzer(plan).collect { + val comparison = analyzer.execute(plan).collect { case Project(Alias(e: BinaryComparison, _) :: Nil, _) => e }.head assert(comparison.left.dataType === expectedType) assert(comparison.right.dataType === expectedType) } + private def checkUnion(left: Expression, right: Expression, expectedType: DataType): Unit = { + val plan = + Union(Project(Seq(Alias(left, "l")()), relation), + Project(Seq(Alias(right, "r")()), relation)) + val (l, r) = analyzer.execute(plan).collect { + case Union(left, right) => (left.output.head, right.output.head) + }.head + assert(l.dataType === expectedType) + assert(r.dataType === expectedType) + } + test("basic operations") { checkType(Add(d1, d2), DecimalType(6, 2)) checkType(Subtract(d1, d2), DecimalType(6, 2)) @@ -82,6 +95,19 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) } + test("decimal precision for union") { + checkUnion(d1, i, DecimalType(11, 1)) + checkUnion(i, d2, DecimalType(12, 2)) + checkUnion(d1, d2, DecimalType(5, 2)) + checkUnion(d2, d1, DecimalType(5, 2)) + checkUnion(d1, f, DecimalType(8, 7)) + checkUnion(f, d2, DecimalType(10, 7)) + checkUnion(d1, b, DecimalType(16, 15)) + checkUnion(b, d2, DecimalType(18, 15)) + checkUnion(d1, u, DecimalType.Unlimited) + checkUnion(u, d2, DecimalType.Unlimited) + } + test("bringing in primitive types") { checkType(Add(d1, i), DecimalType(12, 1)) checkType(Add(d1, f), DoubleType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index ecbb54218d457..fcd745f43cfbf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -96,7 +96,9 @@ class HiveTypeCoercionSuite extends PlanTest { widenTest(StringType, TimestampType, None) // ComplexType - widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, + MapType(IntegerType, StringType, false), + Some(MapType(IntegerType, StringType, false))) widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) widenTest(StringType, MapType(IntegerType, StringType, true), None) widenTest(ArrayType(IntegerType), StructType(Seq()), None) @@ -113,7 +115,9 @@ class HiveTypeCoercionSuite extends PlanTest { // Remove superflous boolean -> boolean casts. ruleTest(Cast(Literal(true), BooleanType), Literal(true)) // Stringify boolean when casting to string. - ruleTest(Cast(Literal(false), StringType), If(Literal(false), Literal("true"), Literal("false"))) + ruleTest( + Cast(Literal(false), StringType), + If(Literal(false), Literal("true"), Literal("false"))) } test("coalesce casts") { @@ -127,11 +131,11 @@ class HiveTypeCoercionSuite extends PlanTest { ruleTest( Coalesce(Literal(1.0) :: Literal(1) - :: Literal(1.0, FloatType) + :: Literal.create(1.0, FloatType) :: Nil), Coalesce(Cast(Literal(1.0), DoubleType) :: Cast(Literal(1), DoubleType) - :: Cast(Literal(1.0, FloatType), DoubleType) + :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) ruleTest( Coalesce(Literal(1L) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala new file mode 100644 index 0000000000000..f2f3a84d19380 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.sql.catalyst.expressions + +import org.scalatest.FunSuite + +import org.apache.spark.sql.types.IntegerType + +class AttributeSetSuite extends FunSuite { + + val aUpper = AttributeReference("A", IntegerType)(exprId = ExprId(1)) + val aLower = AttributeReference("a", IntegerType)(exprId = ExprId(1)) + val fakeA = AttributeReference("a", IntegerType)(exprId = ExprId(3)) + val aSet = AttributeSet(aLower :: Nil) + + val bUpper = AttributeReference("B", IntegerType)(exprId = ExprId(2)) + val bLower = AttributeReference("b", IntegerType)(exprId = ExprId(2)) + val bSet = AttributeSet(bUpper :: Nil) + + val aAndBSet = AttributeSet(aUpper :: bUpper :: Nil) + + test("sanity check") { + assert(aUpper != aLower) + assert(bUpper != bLower) + } + + test("checks by id not name") { + assert(aSet.contains(aUpper) === true) + assert(aSet.contains(aLower) === true) + assert(aSet.contains(fakeA) === false) + + assert(aSet.contains(bUpper) === false) + assert(aSet.contains(bLower) === false) + } + + test("++ preserves AttributeSet") { + assert((aSet ++ bSet).contains(aUpper) === true) + assert((aSet ++ bSet).contains(aLower) === true) + } + + test("extracts all references references") { + val addSet = AttributeSet(Add(aUpper, Alias(bUpper, "test")()):: Nil) + assert(addSet.contains(aUpper)) + assert(addSet.contains(aLower)) + assert(addSet.contains(bUpper)) + assert(addSet.contains(bLower)) + } + + test("dedups attributes") { + assert(AttributeSet(aUpper :: aLower :: Nil).size === 1) + } + + test("subset") { + assert(aSet.subsetOf(aAndBSet) === true) + assert(aAndBSet.subsetOf(aSet) === false) + } + + test("equality") { + assert(aSet != aAndBSet) + assert(aAndBSet != aSet) + assert(aSet != bSet) + assert(bSet != aSet) + + assert(aSet == aSet) + assert(aSet == AttributeSet(aUpper :: Nil)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index dcfd8b28cb02a..fa71001c9336e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -25,12 +25,45 @@ import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.mathfuncs._ import org.apache.spark.sql.types._ -class ExpressionEvaluationSuite extends FunSuite { +class ExpressionEvaluationBaseSuite extends FunSuite { + + def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { + expression.eval(inputRow) + } + + def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + def checkDoubleEvaluation( + expression: Expression, + expected: Spread[Double], + inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + actual.asInstanceOf[Double] shouldBe expected + } +} + +class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { + + def create_row(values: Any*): Row = { + new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) + } test("literals") { checkEvaluation(Literal(1), 1) @@ -55,10 +88,13 @@ class ExpressionEvaluationSuite extends FunSuite { assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte]) } + // scalastyle:off /** * Checks for three-valued-logic. Based on: * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 - * I.e. in flat cpo "False -> Unknown -> True", OR is lowest upper bound, AND is greatest lower bound. + * I.e. in flat cpo "False -> Unknown -> True", + * OR is lowest upper bound, + * AND is greatest lower bound. * p q p OR q p AND q p = q * True True True True True * True False True False False @@ -75,7 +111,7 @@ class ExpressionEvaluationSuite extends FunSuite { * False True * Unknown Unknown */ - + // scalastyle:on val notTrueTable = (true, false) :: (false, true) :: @@ -84,7 +120,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("3VL Not") { notTrueTable.foreach { case (v, answer) => - checkEvaluation(!Literal(v, BooleanType), answer) + checkEvaluation(!Literal.create(v, BooleanType), answer) } } @@ -128,38 +164,19 @@ class ExpressionEvaluationSuite extends FunSuite { test(s"3VL $name") { truthTable.foreach { case (l,r,answer) => - val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) + val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType)) checkEvaluation(expr, answer) } } } - def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.eval(inputRow) - } - - def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - if(actual != expected) { - val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" - fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") - } - } - - def checkDoubleEvaluation(expression: Expression, expected: Spread[Double], inputRow: Row = EmptyRow): Unit = { - val actual = try evaluate(expression, inputRow) catch { - case e: Exception => fail(s"Exception evaluating $expression", e) - } - actual.asInstanceOf[Double] shouldBe expected - } - test("IN") { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation( + In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), + true) } test("Divide") { @@ -169,12 +186,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Divide(Literal(1), Literal(0)), null) checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0), Literal(null, IntegerType)), null) - checkEvaluation(Divide(Literal(1), Literal(null, IntegerType)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(0)), null) - checkEvaluation(Divide(Literal(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal(null, IntegerType), Literal(null, IntegerType)), null) + checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("Remainder") { @@ -184,12 +202,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Remainder(Literal(1), Literal(0)), null) checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0), Literal(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(1), Literal(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(0)), null) - checkEvaluation(Remainder(Literal(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal(null, IntegerType), Literal(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) + checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("INSET") { @@ -216,14 +235,24 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(MaxOf(1L, 2L), 2L) checkEvaluation(MaxOf(2L, 1L), 2L) - checkEvaluation(MaxOf(Literal(null, IntegerType), 2), 2) - checkEvaluation(MaxOf(2, Literal(null, IntegerType)), 2) + checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) + } + + test("MinOf") { + checkEvaluation(MinOf(1, 2), 1) + checkEvaluation(MinOf(2, 1), 1) + checkEvaluation(MinOf(1L, 2L), 1L) + checkEvaluation(MinOf(2L, 1L), 1L) + + checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) + checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) } test("LIKE literal Regular Expression") { - checkEvaluation(Literal(null, StringType).like("a"), null) - checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) - checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like("a"), null) + checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) + checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null) checkEvaluation("abdef" like "abdef", true) checkEvaluation("a_%b" like "a\\__b", true) checkEvaluation("addb" like "a_%b", true) @@ -242,29 +271,29 @@ class ExpressionEvaluationSuite extends FunSuite { test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) - checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) - checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) - checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - - checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) + checkEvaluation("abcd" like regEx, null, create_row(null)) + checkEvaluation("abdef" like regEx, true, create_row("abdef")) + checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) + checkEvaluation("addb" like regEx, true, create_row("a_%b")) + checkEvaluation("addb" like regEx, false, create_row("a\\__b")) + checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) + checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) + checkEvaluation("addb" like regEx, true, create_row("a%")) + checkEvaluation("addb" like regEx, false, create_row("**")) + checkEvaluation("abc" like regEx, true, create_row("a%")) + checkEvaluation("abc" like regEx, false, create_row("b%")) + checkEvaluation("abc" like regEx, false, create_row("bc%")) + checkEvaluation("a\nb" like regEx, true, create_row("a_b")) + checkEvaluation("ab" like regEx, true, create_row("a%b")) + checkEvaluation("a\nb" like regEx, true, create_row("a%b")) + + checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) } test("RLIKE literal Regular Expression") { - checkEvaluation(Literal(null, StringType) rlike "abdef", null) - checkEvaluation("abdef" rlike Literal(null, StringType), null) - checkEvaluation(Literal(null, StringType) rlike Literal(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal.create(null, StringType), null) + checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null) checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) @@ -289,14 +318,14 @@ class ExpressionEvaluationSuite extends FunSuite { test("RLIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) - checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) - checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) - checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) + checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) + checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) + checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) + checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + evaluate("abbbbc" rlike regEx, create_row("**")) } } @@ -375,7 +404,7 @@ class ExpressionEvaluationSuite extends FunSuite { assert(("abcdef" cast DoubleType).nullable === true) assert(("abcdef" cast FloatType).nullable === true) - checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) + checkEvaluation(Cast(Literal.create(null, IntegerType), ShortType), null) } test("date") { @@ -501,8 +530,10 @@ class ExpressionEvaluationSuite extends FunSuite { } test("array casting") { - val array = Literal(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) - val array_notNull = Literal(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) + val array = Literal.create(Seq("123", "abc", "", null), + ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "abc", ""), + ArrayType(StringType, containsNull = false)) { val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) @@ -550,10 +581,10 @@ class ExpressionEvaluationSuite extends FunSuite { } test("map casting") { - val map = Literal( + val map = Literal.create( Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null), MapType(StringType, StringType, valueContainsNull = true)) - val map_notNull = Literal( + val map_notNull = Literal.create( Map("a" -> "123", "b" -> "abc", "c" -> ""), MapType(StringType, StringType, valueContainsNull = false)) @@ -611,14 +642,14 @@ class ExpressionEvaluationSuite extends FunSuite { } test("struct casting") { - val struct = Literal( + val struct = Literal.create( Row("123", "abc", "", null), StructType(Seq( StructField("a", StringType, nullable = true), StructField("b", StringType, nullable = true), StructField("c", StringType, nullable = true), StructField("d", StringType, nullable = true)))) - val struct_notNull = Literal( + val struct_notNull = Literal.create( Row("123", "abc", ""), StructType(Seq( StructField("a", StringType, nullable = false), @@ -706,7 +737,7 @@ class ExpressionEvaluationSuite extends FunSuite { } test("complex casting") { - val complex = Literal( + val complex = Literal.create( Row( Seq("123", "abc", ""), Map("a" -> "123", "b" -> "abc", "c" -> ""), @@ -737,7 +768,7 @@ class ExpressionEvaluationSuite extends FunSuite { } test("null checking") { - val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val row = create_row("^Ba*n", null, true, null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) val c3 = 'a.boolean.at(2) @@ -749,34 +780,35 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c2.isNull, true, row) checkEvaluation(c2.isNotNull, false, row) - checkEvaluation(Literal(1, ShortType).isNull, false) - checkEvaluation(Literal(1, ShortType).isNotNull, true) + checkEvaluation(Literal.create(1, ShortType).isNull, false) + checkEvaluation(Literal.create(1, ShortType).isNotNull, true) - checkEvaluation(Literal(null, ShortType).isNull, true) - checkEvaluation(Literal(null, ShortType).isNotNull, false) + checkEvaluation(Literal.create(null, ShortType).isNull, true) + checkEvaluation(Literal.create(null, ShortType).isNotNull, false) checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) - checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) + checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(If(c3, Literal("a", StringType), Literal("b", StringType)), "a", row) + checkEvaluation( + If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) checkEvaluation(If(c3, c1, c2), "^Ba*n", row) checkEvaluation(If(c4, c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), - Literal("a", StringType), Literal("b", StringType)), "b", row) + checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal.create(false, BooleanType), + Literal.create("a", StringType), Literal.create("b", StringType)), "b", row) checkEvaluation(c1 in (c1, c2), true, row) checkEvaluation( - Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType)), true, row) + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row) checkEvaluation( - Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType), c2), true, row) + Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row) } test("case when") { - val row = new GenericRow(Array[Any](null, false, true, "a", "b", "c")) + val row = create_row(null, false, true, "a", "b", "c") val c1 = 'a.boolean.at(0) val c2 = 'a.boolean.at(1) val c3 = 'a.boolean.at(2) @@ -787,9 +819,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(Literal(null, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal(false, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal(true, BooleanType), c4, c6)), "a", row) + checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) + checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) @@ -819,13 +851,13 @@ class ExpressionEvaluationSuite extends FunSuite { } test("complex type") { - val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 - null.asInstanceOf[String], // 1 - new GenericRow(Array[Any]("aa", "bb")), // 2 - Map("aa"->"bb"), // 3 - Seq("aa", "bb") // 4 - )) + val row = create_row( + "^Ba*n", // 0 + null.asInstanceOf[UTF8String], // 1 + create_row("aa", "bb"), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + ) val typeS = StructType( StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil @@ -835,19 +867,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) - checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) - checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) + checkEvaluation(GetItem(Literal.create(null, typeMap), Literal("aa")), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(3, typeMap, true), - Literal(null, StringType)), null, row) + Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) - checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) - checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) + checkEvaluation(GetItem(Literal.create(null, typeArray), Literal(1)), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), - Literal(null, IntegerType)), null, row) + Literal.create(null, IntegerType)), null, row) - def quickBuildGetField(expr: Expression, fieldName: String) = { + def quickBuildGetField(expr: Expression, fieldName: String): StructGetField = { expr.dataType match { case StructType(fields) => val field = fields.find(_.name == fieldName).get @@ -855,10 +889,12 @@ class ExpressionEvaluationSuite extends FunSuite { } } - def quickResolve(u: UnresolvedGetField) = quickBuildGetField(u.child, u.fieldName) + def quickResolve(u: UnresolvedGetField): StructGetField = { + quickBuildGetField(u.child, u.fieldName) + } checkEvaluation(quickBuildGetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) - checkEvaluation(quickBuildGetField(Literal(null, typeS), "a"), null, row) + checkEvaluation(quickBuildGetField(Literal.create(null, typeS), "a"), null, row) val typeS_notNullable = StructType( StructField("a", StringType, nullable = false) @@ -866,10 +902,11 @@ class ExpressionEvaluationSuite extends FunSuite { ) assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) + assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable + === false) - assert(quickBuildGetField(Literal(null, typeS), "a").nullable === true) - assert(quickBuildGetField(Literal(null, typeS_notNullable), "a").nullable === true) + assert(quickBuildGetField(Literal.create(null, typeS), "a").nullable === true) + assert(quickBuildGetField(Literal.create(null, typeS_notNullable), "a").nullable === true) checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) @@ -877,20 +914,21 @@ class ExpressionEvaluationSuite extends FunSuite { } test("arithmetic") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) val c4 = 'a.int.at(3) checkEvaluation(UnaryMinus(c1), -1, row) - checkEvaluation(UnaryMinus(Literal(100, IntegerType)), -100) + checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100) checkEvaluation(Add(c1, c4), null, row) checkEvaluation(Add(c1, c2), 3, row) - checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(Add(Literal(null, IntegerType), c2), null, row) - checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation( + Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(-c1, -1, row) checkEvaluation(c1 + c2, 3, row) @@ -901,19 +939,20 @@ class ExpressionEvaluationSuite extends FunSuite { } test("fractional arithmetic") { - val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val row = create_row(1.1, 2.0, 3.1, null) val c1 = 'a.double.at(0) val c2 = 'a.double.at(1) val c3 = 'a.double.at(2) val c4 = 'a.double.at(3) checkEvaluation(UnaryMinus(c1), -1.1, row) - checkEvaluation(UnaryMinus(Literal(100.0, DoubleType)), -100.0) + checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0) checkEvaluation(Add(c1, c4), null, row) checkEvaluation(Add(c1, c2), 3.1, row) - checkEvaluation(Add(c1, Literal(null, DoubleType)), null, row) - checkEvaluation(Add(Literal(null, DoubleType), c2), null, row) - checkEvaluation(Add(Literal(null, DoubleType), Literal(null, DoubleType)), null, row) + checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) + checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) + checkEvaluation( + Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) checkEvaluation(-c1, -1.1, row) checkEvaluation(c1 + c2, 3.1, row) @@ -924,7 +963,7 @@ class ExpressionEvaluationSuite extends FunSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) + val row = create_row(1, 2, 3, null, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -934,9 +973,10 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(LessThan(c1, c4), null, row) checkEvaluation(LessThan(c1, c2), true, row) - checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row) - checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) + checkEvaluation( + LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 < c2, true, row) checkEvaluation(c1 <= c2, true, row) @@ -948,85 +988,115 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 <=> c4, false, row) checkEvaluation(c4 <=> c6, true, row) checkEvaluation(c3 <=> c5, true, row) - checkEvaluation(Literal(true) <=> Literal(null, BooleanType), false, row) - checkEvaluation(Literal(null, BooleanType) <=> Literal(true), false, row) + checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row) + checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row) } test("StringComparison") { - val row = new GenericRow(Array[Any]("abc", null)) + val row = create_row("abc", null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) checkEvaluation(c1 contains "b", true, row) checkEvaluation(c1 contains "x", false, row) checkEvaluation(c2 contains "b", null, row) - checkEvaluation(c1 contains Literal(null, StringType), null, row) + checkEvaluation(c1 contains Literal.create(null, StringType), null, row) checkEvaluation(c1 startsWith "a", true, row) checkEvaluation(c1 startsWith "b", false, row) checkEvaluation(c2 startsWith "a", null, row) - checkEvaluation(c1 startsWith Literal(null, StringType), null, row) + checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row) checkEvaluation(c1 endsWith "c", true, row) checkEvaluation(c1 endsWith "b", false, row) checkEvaluation(c2 endsWith "b", null, row) - checkEvaluation(c1 endsWith Literal(null, StringType), null, row) + checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row) } test("Substring") { - val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + val row = create_row("example", "example".toArray.map(_.toByte)) val s = 'a.string.at(0) // substring from zero position with less-than-full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) // substring from zero position with full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(7, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) // substring from zero position with greater-than-full length - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(100, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), + "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), + "example", row) // substring from nonzero position with less-than-full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(2, IntegerType)), "xa", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), + "xa", row) // substring from nonzero position with full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(6, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), + "xample", row) // substring from nonzero position with greater-than-full length - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(100, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), + "xample", row) // zero-length substring (within string bounds) - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(0, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), + "", row) // zero-length substring (beyond string bounds) - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + "", row) // substring(null, _, _) -> null - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + null, create_row(null)) // substring(_, null, _) -> null - checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row) + checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), + null, row) // substring(_, _, null) -> null - checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation( + Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), + null, + row) // 2-arg substring from zero position - checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) // 2-arg substring from nonzero position - checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row) + checkEvaluation( + Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "xample", + row) val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) - assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === true) + assert( + Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === false) + assert(Substring(s_notNull, + Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, + Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -1037,20 +1107,20 @@ class ExpressionEvaluationSuite extends FunSuite { test("SQRT") { val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) - val rowSequence = inputSequence.map(l => new GenericRow(Array[Any](l.toDouble))) + val rowSequence = inputSequence.map(l => create_row(l.toDouble)) val d = 'a.double.at(0) for ((row, expected) <- rowSequence zip expectedResults) { checkEvaluation(Sqrt(d), expected, row) } - checkEvaluation(Sqrt(Literal(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) checkEvaluation(Sqrt(-1), null, EmptyRow) checkEvaluation(Sqrt(-1.5), null, EmptyRow) } test("Bitwise operations") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -1058,26 +1128,192 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(BitwiseAnd(c1, c4), null, row) checkEvaluation(BitwiseAnd(c1, c2), 0, row) - checkEvaluation(BitwiseAnd(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseAnd(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseOr(c1, c4), null, row) checkEvaluation(BitwiseOr(c1, c2), 3, row) - checkEvaluation(BitwiseOr(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseOr(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseXor(c1, c4), null, row) checkEvaluation(BitwiseXor(c1, c2), 3, row) - checkEvaluation(BitwiseXor(c1, Literal(null, IntegerType)), null, row) - checkEvaluation(BitwiseXor(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseNot(c4), null, row) checkEvaluation(BitwiseNot(c1), -2, row) - checkEvaluation(BitwiseNot(Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 & c2, 0, row) checkEvaluation(c1 | c2, 3, row) checkEvaluation(c1 ^ c2, 3, row) checkEvaluation(~c1, -2, row) } + + /** + * Used for testing math functions for DataFrames. + * @param c The DataFrame function + * @param f The functions in scala.math + * @param domain The set of values to run the function with + * @param expectNull Whether the given values should return null or not + * @tparam T Generic type for primitives + */ + def unaryMathFunctionEvaluation[@specialized(Int, Double, Float, Long) T]( + c: Expression => Expression, + f: T => T, + domain: Iterable[T] = (-20 to 20).map(_ * 0.1), + expectNull: Boolean = false): Unit = { + if (expectNull) { + domain.foreach { value => + checkEvaluation(c(Literal(value)), null, EmptyRow) + } + } else { + domain.foreach { value => + checkEvaluation(c(Literal(value)), f(value), EmptyRow) + } + } + checkEvaluation(c(Literal.create(null, DoubleType)), null, create_row(null)) + } + + test("sin") { + unaryMathFunctionEvaluation(Sin, math.sin) + } + + test("asin") { + unaryMathFunctionEvaluation(Asin, math.asin, (-10 to 10).map(_ * 0.1)) + unaryMathFunctionEvaluation(Asin, math.asin, (11 to 20).map(_ * 0.1), true) + } + + test("sinh") { + unaryMathFunctionEvaluation(Sinh, math.sinh) + } + + test("cos") { + unaryMathFunctionEvaluation(Cos, math.cos) + } + + test("acos") { + unaryMathFunctionEvaluation(Acos, math.acos, (-10 to 10).map(_ * 0.1)) + unaryMathFunctionEvaluation(Acos, math.acos, (11 to 20).map(_ * 0.1), true) + } + + test("cosh") { + unaryMathFunctionEvaluation(Cosh, math.cosh) + } + + test("tan") { + unaryMathFunctionEvaluation(Tan, math.tan) + } + + test("atan") { + unaryMathFunctionEvaluation(Atan, math.atan) + } + + test("tanh") { + unaryMathFunctionEvaluation(Tanh, math.tanh) + } + + test("toDeg") { + unaryMathFunctionEvaluation(ToDegrees, math.toDegrees) + } + + test("toRad") { + unaryMathFunctionEvaluation(ToRadians, math.toRadians) + } + + test("cbrt") { + unaryMathFunctionEvaluation(Cbrt, math.cbrt) + } + + test("ceil") { + unaryMathFunctionEvaluation(Ceil, math.ceil) + } + + test("floor") { + unaryMathFunctionEvaluation(Floor, math.floor) + } + + test("rint") { + unaryMathFunctionEvaluation(Rint, math.rint) + } + + test("exp") { + unaryMathFunctionEvaluation(Exp, math.exp) + } + + test("expm1") { + unaryMathFunctionEvaluation(Expm1, math.expm1) + } + + test("signum") { + unaryMathFunctionEvaluation[Double](Signum, math.signum) + } + + test("log") { + unaryMathFunctionEvaluation(Log, math.log, (0 to 20).map(_ * 0.1)) + unaryMathFunctionEvaluation(Log, math.log, (-5 to -1).map(_ * 0.1), true) + } + + test("log10") { + unaryMathFunctionEvaluation(Log10, math.log10, (0 to 20).map(_ * 0.1)) + unaryMathFunctionEvaluation(Log10, math.log10, (-5 to -1).map(_ * 0.1), true) + } + + test("log1p") { + unaryMathFunctionEvaluation(Log1p, math.log1p, (-1 to 20).map(_ * 0.1)) + unaryMathFunctionEvaluation(Log1p, math.log1p, (-10 to -2).map(_ * 1.0), true) + } + + /** + * Used for testing math functions for DataFrames. + * @param c The DataFrame function + * @param f The functions in scala.math + * @param domain The set of values to run the function with + */ + def binaryMathFunctionEvaluation( + c: (Expression, Expression) => Expression, + f: (Double, Double) => Double, + domain: Iterable[(Double, Double)] = (-20 to 20).map(v => (v * 0.1, v * -0.1)), + expectNull: Boolean = false): Unit = { + if (expectNull) { + domain.foreach { case (v1, v2) => + checkEvaluation(c(v1, v2), null, create_row(null)) + } + } else { + domain.foreach { case (v1, v2) => + checkEvaluation(c(v1, v2), f(v1 + 0.0, v2 + 0.0), EmptyRow) + checkEvaluation(c(v2, v1), f(v2 + 0.0, v1 + 0.0), EmptyRow) + } + } + checkEvaluation(c(Literal.create(null, DoubleType), 1.0), null, create_row(null)) + checkEvaluation(c(1.0, Literal.create(null, DoubleType)), null, create_row(null)) + } + + test("pow") { + binaryMathFunctionEvaluation(Pow, math.pow, (-5 to 5).map(v => (v * 1.0, v * 1.0))) + binaryMathFunctionEvaluation(Pow, math.pow, Seq((-1.0, 0.9), (-2.2, 1.7), (-2.2, -1.7)), true) + } + + test("hypot") { + binaryMathFunctionEvaluation(Hypot, math.hypot) + } + + test("atan2") { + binaryMathFunctionEvaluation(Atan2, math.atan2) + } +} + +// TODO: Make the tests work with codegen. +class ExpressionEvaluationWithoutCodeGenSuite extends ExpressionEvaluationBaseSuite { + + test("CreateStruct") { + val row = Row(1, 2, 3) + val c1 = 'a.int.at(0).as("a") + val c3 = 'c.int.at(2).as("c") + checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala index ef3114fd4dbab..b5ebe4b38e337 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -29,7 +29,7 @@ class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { expected: Any, inputRow: Row = EmptyRow): Unit = { val plan = try { - GenerateMutableProjection(Alias(expression, s"Optimized($expression)")() :: Nil)() + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)() } catch { case e: Throwable => val evaluated = GenerateProjection.expressionEvaluator(expression) @@ -56,10 +56,10 @@ class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { val futures = (1 to 20).map { _ => future { - GeneratePredicate(EqualTo(Literal(1), Literal(1))) - GenerateProjection(EqualTo(Literal(1), Literal(1)) :: Nil) - GenerateMutableProjection(EqualTo(Literal(1), Literal(1)) :: Nil) - GenerateOrdering(Add(Literal(1), Literal(1)).asc :: Nil) + GeneratePredicate.generate(EqualTo(Literal(1), Literal(1))) + GenerateProjection.generate(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateMutableProjection.generate(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateOrdering.generate(Add(Literal(1), Literal(1)).asc :: Nil) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 275ea2627ebcd..97af2e0fd0502 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.codegen._ /** @@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ */ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { override def checkEvaluation( - expression: Expression, - expected: Any, - inputRow: Row = EmptyRow): Unit = { + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { lazy val evaluated = GenerateProjection.expressionEvaluator(expression) val plan = try { - GenerateProjection(Alias(expression, s"Optimized($expression)")() :: Nil) + GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil) } catch { case e: Throwable => fail( @@ -43,7 +43,7 @@ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { } val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](expected)) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) if (actual.hashCode() != expectedRow.hashCode()) { fail( s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala new file mode 100644 index 0000000000000..7a19e511eb8b5 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala @@ -0,0 +1,119 @@ +/* + * 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.sql.catalyst.expressions + +import scala.collection.JavaConverters._ +import scala.util.Random + +import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, TaskMemoryManager, MemoryAllocator} +import org.scalatest.{BeforeAndAfterEach, FunSuite, Matchers} + +import org.apache.spark.sql.types._ + +class UnsafeFixedWidthAggregationMapSuite extends FunSuite with Matchers with BeforeAndAfterEach { + + import UnsafeFixedWidthAggregationMap._ + + private val groupKeySchema = StructType(StructField("product", StringType) :: Nil) + private val aggBufferSchema = StructType(StructField("salePrice", IntegerType) :: Nil) + private def emptyAggregationBuffer: Row = new GenericRow(Array[Any](0)) + + private var memoryManager: TaskMemoryManager = null + + override def beforeEach(): Unit = { + memoryManager = new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)) + } + + override def afterEach(): Unit = { + if (memoryManager != null) { + memoryManager.cleanUpAllAllocatedMemory() + memoryManager = null + } + } + + test("supported schemas") { + assert(!supportsAggregationBufferSchema(StructType(StructField("x", StringType) :: Nil))) + assert(supportsGroupKeySchema(StructType(StructField("x", StringType) :: Nil))) + + assert( + !supportsAggregationBufferSchema(StructType(StructField("x", ArrayType(IntegerType)) :: Nil))) + assert( + !supportsGroupKeySchema(StructType(StructField("x", ArrayType(IntegerType)) :: Nil))) + } + + test("empty map") { + val map = new UnsafeFixedWidthAggregationMap( + emptyAggregationBuffer, + aggBufferSchema, + groupKeySchema, + memoryManager, + 1024, // initial capacity + false // disable perf metrics + ) + assert(!map.iterator().hasNext) + map.free() + } + + test("updating values for a single key") { + val map = new UnsafeFixedWidthAggregationMap( + emptyAggregationBuffer, + aggBufferSchema, + groupKeySchema, + memoryManager, + 1024, // initial capacity + false // disable perf metrics + ) + val groupKey = new GenericRow(Array[Any](UTF8String("cats"))) + + // Looking up a key stores a zero-entry in the map (like Python Counters or DefaultDicts) + map.getAggregationBuffer(groupKey) + val iter = map.iterator() + val entry = iter.next() + assert(!iter.hasNext) + entry.key.getString(0) should be ("cats") + entry.value.getInt(0) should be (0) + + // Modifications to rows retrieved from the map should update the values in the map + entry.value.setInt(0, 42) + map.getAggregationBuffer(groupKey).getInt(0) should be (42) + + map.free() + } + + test("inserting large random keys") { + val map = new UnsafeFixedWidthAggregationMap( + emptyAggregationBuffer, + aggBufferSchema, + groupKeySchema, + memoryManager, + 128, // initial capacity + false // disable perf metrics + ) + val rand = new Random(42) + val groupKeys: Set[String] = Seq.fill(512)(rand.nextString(1024)).toSet + groupKeys.foreach { keyString => + map.getAggregationBuffer(new GenericRow(Array[Any](UTF8String(keyString)))) + } + val seenKeys: Set[String] = map.iterator().asScala.map { entry => + entry.key.getString(0) + }.toSet + seenKeys.size should be (groupKeys.size) + seenKeys should be (groupKeys) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala new file mode 100644 index 0000000000000..3a60c7fd32675 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -0,0 +1,153 @@ +/* + * 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.sql.catalyst.expressions + +import java.util.Arrays + +import org.scalatest.{FunSuite, Matchers} + +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.PlatformDependent +import org.apache.spark.unsafe.array.ByteArrayMethods + +class UnsafeRowConverterSuite extends FunSuite with Matchers { + + test("basic conversion with only primitive types") { + val fieldTypes: Array[DataType] = Array(LongType, LongType, IntegerType) + val converter = new UnsafeRowConverter(fieldTypes) + + val row = new SpecificMutableRow(fieldTypes) + row.setLong(0, 0) + row.setLong(1, 1) + row.setInt(2, 2) + + val sizeRequired: Int = converter.getSizeRequirement(row) + sizeRequired should be (8 + (3 * 8)) + val buffer: Array[Long] = new Array[Long](sizeRequired / 8) + val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET) + numBytesWritten should be (sizeRequired) + + val unsafeRow = new UnsafeRow() + unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + unsafeRow.getLong(0) should be (0) + unsafeRow.getLong(1) should be (1) + unsafeRow.getInt(2) should be (2) + } + + test("basic conversion with primitive and string types") { + val fieldTypes: Array[DataType] = Array(LongType, StringType, StringType) + val converter = new UnsafeRowConverter(fieldTypes) + + val row = new SpecificMutableRow(fieldTypes) + row.setLong(0, 0) + row.setString(1, "Hello") + row.setString(2, "World") + + val sizeRequired: Int = converter.getSizeRequirement(row) + sizeRequired should be (8 + (8 * 3) + + ByteArrayMethods.roundNumberOfBytesToNearestWord("Hello".getBytes.length + 8) + + ByteArrayMethods.roundNumberOfBytesToNearestWord("World".getBytes.length + 8)) + val buffer: Array[Long] = new Array[Long](sizeRequired / 8) + val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET) + numBytesWritten should be (sizeRequired) + + val unsafeRow = new UnsafeRow() + unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + unsafeRow.getLong(0) should be (0) + unsafeRow.getString(1) should be ("Hello") + unsafeRow.getString(2) should be ("World") + } + + test("null handling") { + val fieldTypes: Array[DataType] = Array( + NullType, + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType) + val converter = new UnsafeRowConverter(fieldTypes) + + val rowWithAllNullColumns: Row = { + val r = new SpecificMutableRow(fieldTypes) + for (i <- 0 to fieldTypes.length - 1) { + r.setNullAt(i) + } + r + } + + val sizeRequired: Int = converter.getSizeRequirement(rowWithAllNullColumns) + val createdFromNullBuffer: Array[Long] = new Array[Long](sizeRequired / 8) + val numBytesWritten = converter.writeRow( + rowWithAllNullColumns, createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET) + numBytesWritten should be (sizeRequired) + + val createdFromNull = new UnsafeRow() + createdFromNull.pointTo( + createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + for (i <- 0 to fieldTypes.length - 1) { + assert(createdFromNull.isNullAt(i)) + } + createdFromNull.getBoolean(1) should be (false) + createdFromNull.getByte(2) should be (0) + createdFromNull.getShort(3) should be (0) + createdFromNull.getInt(4) should be (0) + createdFromNull.getLong(5) should be (0) + assert(java.lang.Float.isNaN(createdFromNull.getFloat(6))) + assert(java.lang.Double.isNaN(createdFromNull.getFloat(7))) + + // If we have an UnsafeRow with columns that are initially non-null and we null out those + // columns, then the serialized row representation should be identical to what we would get by + // creating an entirely null row via the converter + val rowWithNoNullColumns: Row = { + val r = new SpecificMutableRow(fieldTypes) + r.setNullAt(0) + r.setBoolean(1, false) + r.setByte(2, 20) + r.setShort(3, 30) + r.setInt(4, 400) + r.setLong(5, 500) + r.setFloat(6, 600) + r.setDouble(7, 700) + r + } + val setToNullAfterCreationBuffer: Array[Long] = new Array[Long](sizeRequired / 8) + converter.writeRow( + rowWithNoNullColumns, setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET) + val setToNullAfterCreation = new UnsafeRow() + setToNullAfterCreation.pointTo( + setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + + setToNullAfterCreation.isNullAt(0) should be (rowWithNoNullColumns.isNullAt(0)) + setToNullAfterCreation.getBoolean(1) should be (rowWithNoNullColumns.getBoolean(1)) + setToNullAfterCreation.getByte(2) should be (rowWithNoNullColumns.getByte(2)) + setToNullAfterCreation.getShort(3) should be (rowWithNoNullColumns.getShort(3)) + setToNullAfterCreation.getInt(4) should be (rowWithNoNullColumns.getInt(4)) + setToNullAfterCreation.getLong(5) should be (rowWithNoNullColumns.getLong(5)) + setToNullAfterCreation.getFloat(6) should be (rowWithNoNullColumns.getFloat(6)) + setToNullAfterCreation.getDouble(7) should be (rowWithNoNullColumns.getDouble(7)) + + for (i <- 0 to fieldTypes.length - 1) { + setToNullAfterCreation.setNullAt(i) + } + assert(Arrays.equals(createdFromNullBuffer, setToNullAfterCreationBuffer)) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 72f06e26e05f1..6255578d7fa57 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -61,7 +61,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { def checkCondition(input: Expression, expected: Expression): Unit = { val plan = testRelation.where(input).analyze - val actual = Optimize(plan).expressions.head + val actual = Optimize.execute(plan).expressions.head compareConditions(actual, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index e2ae0d25db1a5..a30052b38fc11 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -27,6 +27,8 @@ class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = + Batch("Filter Pushdown", FixedPoint(100), + ColumnPruning) :: Batch("Combine Limit", FixedPoint(10), CombineLimits) :: Batch("Constant Folding", FixedPoint(10), @@ -44,7 +46,7 @@ class CombiningLimitsSuite extends PlanTest { .limit(10) .limit(5) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -61,7 +63,24 @@ class CombiningLimitsSuite extends PlanTest { .limit(7) .limit(5) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + testRelation + .select('a) + .limit(2).analyze + + comparePlans(optimized, correctAnswer) + } + + test("limits: combines two limits after ColumnPruning") { + val originalQuery = + testRelation + .select('a) + .limit(2) + .select('a) + .limit(5) + + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index ef10c0aece716..18f92150b0966 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -47,7 +47,7 @@ class ConstantFoldingSuite extends PlanTest { .subquery('y) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a.attr) @@ -74,7 +74,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(2) * Literal(3) - Literal(6) / (Literal(4) - Literal(2)) )(Literal(9) / Literal(3) as Symbol("9/3")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -99,7 +99,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(2) * 'a + Literal(4) as Symbol("c3"), 'a * (Literal(3) + Literal(4)) as Symbol("c4")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -127,7 +127,7 @@ class ConstantFoldingSuite extends PlanTest { (Literal(1) === Literal(1) || 'b > 1) && (Literal(1) === Literal(2) || 'b < 10))) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -144,7 +144,7 @@ class ConstantFoldingSuite extends PlanTest { Cast(Literal("2"), IntegerType) + Literal(3) + 'a as Symbol("c1"), Coalesce(Seq(Cast(Literal("abc"), IntegerType), Literal(3))) as Symbol("c2")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -160,15 +160,15 @@ class ConstantFoldingSuite extends PlanTest { val originalQuery = testRelation .select( - Rand + Literal(1) as Symbol("c1"), + Rand(5L) + Literal(1) as Symbol("c1"), Sum('a) as Symbol("c2")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select( - Rand + Literal(1.0) as Symbol("c1"), + Rand(5L) + Literal(1.0) as Symbol("c1"), Sum('a) as Symbol("c2")) .analyze @@ -176,42 +176,41 @@ class ConstantFoldingSuite extends PlanTest { } test("Constant folding test: expressions have null literals") { - val originalQuery = - testRelation - .select( - IsNull(Literal(null)) as 'c1, - IsNotNull(Literal(null)) as 'c2, + val originalQuery = testRelation.select( + IsNull(Literal(null)) as 'c1, + IsNotNull(Literal(null)) as 'c2, - GetItem(Literal(null, ArrayType(IntegerType)), 1) as 'c3, - GetItem(Literal(Seq(1), ArrayType(IntegerType)), Literal(null, IntegerType)) as 'c4, - UnresolvedGetField( - Literal(null, StructType(Seq(StructField("a", IntegerType, true)))), - "a") as 'c5, + GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + GetItem( + Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, + UnresolvedGetField( + Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), + "a") as 'c5, - UnaryMinus(Literal(null, IntegerType)) as 'c6, - Cast(Literal(null), IntegerType) as 'c7, - Not(Literal(null, BooleanType)) as 'c8, + UnaryMinus(Literal.create(null, IntegerType)) as 'c6, + Cast(Literal(null), IntegerType) as 'c7, + Not(Literal.create(null, BooleanType)) as 'c8, - Add(Literal(null, IntegerType), 1) as 'c9, - Add(1, Literal(null, IntegerType)) as 'c10, + Add(Literal.create(null, IntegerType), 1) as 'c9, + Add(1, Literal.create(null, IntegerType)) as 'c10, - EqualTo(Literal(null, IntegerType), 1) as 'c11, - EqualTo(1, Literal(null, IntegerType)) as 'c12, + EqualTo(Literal.create(null, IntegerType), 1) as 'c11, + EqualTo(1, Literal.create(null, IntegerType)) as 'c12, - Like(Literal(null, StringType), "abc") as 'c13, - Like("abc", Literal(null, StringType)) as 'c14, + Like(Literal.create(null, StringType), "abc") as 'c13, + Like("abc", Literal.create(null, StringType)) as 'c14, - Upper(Literal(null, StringType)) as 'c15, + Upper(Literal.create(null, StringType)) as 'c15, - Substring(Literal(null, StringType), 0, 1) as 'c16, - Substring("abc", Literal(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal(null, IntegerType)) as 'c18, + Substring(Literal.create(null, StringType), 0, 1) as 'c16, + Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, + Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, - Contains(Literal(null, StringType), "abc") as 'c19, - Contains("abc", Literal(null, StringType)) as 'c20 - ) + Contains(Literal.create(null, StringType), "abc") as 'c19, + Contains("abc", Literal.create(null, StringType)) as 'c20 + ) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation @@ -219,31 +218,31 @@ class ConstantFoldingSuite extends PlanTest { Literal(true) as 'c1, Literal(false) as 'c2, - Literal(null, IntegerType) as 'c3, - Literal(null, IntegerType) as 'c4, - Literal(null, IntegerType) as 'c5, + Literal.create(null, IntegerType) as 'c3, + Literal.create(null, IntegerType) as 'c4, + Literal.create(null, IntegerType) as 'c5, - Literal(null, IntegerType) as 'c6, - Literal(null, IntegerType) as 'c7, - Literal(null, BooleanType) as 'c8, + Literal.create(null, IntegerType) as 'c6, + Literal.create(null, IntegerType) as 'c7, + Literal.create(null, BooleanType) as 'c8, - Literal(null, IntegerType) as 'c9, - Literal(null, IntegerType) as 'c10, + Literal.create(null, IntegerType) as 'c9, + Literal.create(null, IntegerType) as 'c10, - Literal(null, BooleanType) as 'c11, - Literal(null, BooleanType) as 'c12, + Literal.create(null, BooleanType) as 'c11, + Literal.create(null, BooleanType) as 'c12, - Literal(null, BooleanType) as 'c13, - Literal(null, BooleanType) as 'c14, + Literal.create(null, BooleanType) as 'c13, + Literal.create(null, BooleanType) as 'c14, - Literal(null, StringType) as 'c15, + Literal.create(null, StringType) as 'c15, - Literal(null, StringType) as 'c16, - Literal(null, StringType) as 'c17, - Literal(null, StringType) as 'c18, + Literal.create(null, StringType) as 'c16, + Literal.create(null, StringType) as 'c17, + Literal.create(null, StringType) as 'c18, - Literal(null, BooleanType) as 'c19, - Literal(null, BooleanType) as 'c20 + Literal.create(null, BooleanType) as 'c19, + Literal.create(null, BooleanType) as 'c20 ).analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala index cf42d43823399..6841bd9890c97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala @@ -49,7 +49,7 @@ class ConvertToLocalRelationSuite extends PlanTest { UnresolvedAttribute("a").as("a1"), (UnresolvedAttribute("b") + 1).as("b1")) - val optimized = Optimize(projectOnLocal.analyze) + val optimized = Optimize.execute(projectOnLocal.analyze) comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index ae99a3f9ba287..a4a3a66b8b229 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -29,8 +29,8 @@ class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { - val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) - val optimizedPlan = DefaultOptimizer(plan) + val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation) + val optimizedPlan = DefaultOptimizer.execute(plan) super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 55c6766520a1e..58d415d9011e1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions.{Count, Explode} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.{LeftSemi, PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -43,6 +43,8 @@ class FilterPushdownSuite extends PlanTest { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation1 = LocalRelation('d.int) + // This test already passes. test("eliminate subqueries") { val originalQuery = @@ -50,7 +52,7 @@ class FilterPushdownSuite extends PlanTest { .subquery('y) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a.attr) @@ -65,7 +67,7 @@ class FilterPushdownSuite extends PlanTest { .groupBy('a)('a, Count('b)) .select('a) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -81,7 +83,7 @@ class FilterPushdownSuite extends PlanTest { .groupBy('a)('a as 'c, Count('b)) .select('c) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select('a) @@ -90,7 +92,23 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("column pruning for Project(ne, Limit)") { + val originalQuery = + testRelation + .select('a,'b) + .limit(2) + .select('a) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + testRelation + .select('a) + .limit(2).analyze + comparePlans(optimized, correctAnswer) + } + // After this line is unimplemented. test("simple push down") { val originalQuery = @@ -98,7 +116,7 @@ class FilterPushdownSuite extends PlanTest { .select('a) .where('a === 1) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a === 1) @@ -115,7 +133,7 @@ class FilterPushdownSuite extends PlanTest { .where('e === 1) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a + 'b === 1) @@ -131,7 +149,7 @@ class FilterPushdownSuite extends PlanTest { .where('a === 1) .where('a === 2) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where('a === 1 && 'a === 2) @@ -152,7 +170,7 @@ class FilterPushdownSuite extends PlanTest { .where("y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation.where('b === 2) val correctAnswer = @@ -170,7 +188,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation val correctAnswer = @@ -188,7 +206,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val right = testRelation.where('b === 2) val correctAnswer = @@ -197,6 +215,23 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("joins: push down left semi join") { + val x = testRelation.subquery('x) + val y = testRelation1.subquery('y) + + val originalQuery = { + x.join(y, LeftSemi, Option("x.a".attr === "y.d".attr && "x.b".attr >= 1 && "y.d".attr >= 2)) + } + + val optimized = Optimize.execute(originalQuery.analyze) + val left = testRelation.where('b >= 1) + val right = testRelation1.where('d >= 2) + val correctAnswer = + left.join(right, LeftSemi, Option("a".attr === "d".attr)).analyze + + comparePlans(optimized, correctAnswer) + } + test("joins: push down left outer join #1") { val x = testRelation.subquery('x) val y = testRelation.subquery('y) @@ -206,7 +241,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 1) val correctAnswer = left.join(y, LeftOuter).where("y.b".attr === 2).analyze @@ -223,7 +258,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 1 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('d) val correctAnswer = x.join(right, RightOuter).where("x.b".attr === 1).analyze @@ -240,7 +275,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('d) val correctAnswer = left.join(y, LeftOuter, Some("d.b".attr === 1)).where("y.b".attr === 2).analyze @@ -257,7 +292,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('d) val correctAnswer = x.join(right, RightOuter, Some("d.b".attr === 1)).where("x.b".attr === 2).analyze @@ -274,7 +309,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -292,7 +327,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = x.join(right, RightOuter, Some("r.b".attr === 1)).where("x.b".attr === 2).analyze @@ -309,7 +344,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -327,7 +362,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.subquery('l) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = @@ -346,7 +381,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('b === 2).subquery('l) val right = testRelation.where('b === 1).subquery('r) val correctAnswer = @@ -365,7 +400,7 @@ class FilterPushdownSuite extends PlanTest { .where("x.b".attr === 2 && "y.b".attr === 2 && "x.c".attr === "y.c".attr) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 3).subquery('l) val right = testRelation.where('b === 2).subquery('r) val correctAnswer = @@ -382,7 +417,7 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { x.join(y, condition = Some("x.b".attr === "y.b".attr)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) comparePlans(analysis.EliminateSubQueries(originalQuery.analyze), optimized) } @@ -396,7 +431,7 @@ class FilterPushdownSuite extends PlanTest { .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("y.a".attr === 1)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.where('a === 1).subquery('y) val correctAnswer = @@ -415,7 +450,7 @@ class FilterPushdownSuite extends PlanTest { .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) val correctAnswer = @@ -432,10 +467,11 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { z.join(x.join(y)) - .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && + ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val lleft = testRelation.where('a >= 3).subquery('z) val left = testRelation.where('a === 1).subquery('x) val right = testRelation.subquery('y) @@ -453,27 +489,27 @@ class FilterPushdownSuite extends PlanTest { test("generate: predicate referenced no generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .generate(Explode('c_arr), true, false, Some("arr")) .where(('b >= 5) && ('a > 6)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = { testRelationWithArrayType .where(('b >= 5) && ('a > 6)) - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")).analyze + .generate(Explode('c_arr), true, false, Some("arr")).analyze } comparePlans(optimized, correctAnswer) } test("generate: part of conjuncts referenced generated column") { - val generator = Explode(Seq("c"), 'c_arr) + val generator = Explode('c_arr) val originalQuery = { testRelationWithArrayType .generate(generator, true, false, Some("arr")) .where(('b >= 5) && ('c > 6)) } - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val referenceResult = { testRelationWithArrayType .where('b >= 5) @@ -498,10 +534,10 @@ class FilterPushdownSuite extends PlanTest { test("generate: all conjuncts referenced generated column") { val originalQuery = { testRelationWithArrayType - .generate(Explode(Seq("c"), 'c_arr), true, false, Some("arr")) + .generate(Explode('c_arr), true, false, Some("arr")) .where(('c > 6) || ('b > 5)).analyze } - val optimized = Optimize(originalQuery) + val optimized = Optimize.execute(originalQuery) comparePlans(optimized, originalQuery) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index b10577c8001e2..b3df487c84dc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -41,7 +41,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "abc%") || ('a like "abc\\%")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(StartsWith('a, "abc") || ('a like "abc\\%")) .analyze @@ -54,7 +54,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where('a like "%xyz") - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(EndsWith('a, "xyz")) .analyze @@ -67,7 +67,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "%mn%") || ('a like "%mn\\%")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(Contains('a, "mn") || ('a like "%mn\\%")) .analyze @@ -80,7 +80,7 @@ class LikeSimplificationSuite extends PlanTest { testRelation .where(('a like "") || ('a like "abc")) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(('a === "") || ('a === "abc")) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 233e329cb2038..3eb399e68e70c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -49,10 +49,10 @@ class OptimizeInSuite extends PlanTest { .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2)))) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation - .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2)) + .where(InSet(UnresolvedAttribute("a"), HashSet[Any]() + 1 + 2)) .analyze comparePlans(optimized, correctAnswer) @@ -64,7 +64,7 @@ class OptimizeInSuite extends PlanTest { .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) .analyze - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala index 22992fb6f50d4..6b1e53cd42b24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala @@ -41,7 +41,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Upper(Upper('a)) as 'u) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Upper('a) as 'u) @@ -55,7 +55,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Upper(Lower('a)) as 'u) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Upper('a) as 'u) @@ -69,7 +69,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Lower(Upper('a)) as 'l) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Lower('a) as 'l) .analyze @@ -82,7 +82,7 @@ class SimplifyCaseConversionExpressionsSuite extends PlanTest { testRelation .select(Lower(Lower('a)) as 'l) - val optimized = Optimize(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) val correctAnswer = testRelation .select(Lower('a) as 'l) .analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala index a54751dfa9a12..a3ad200800b02 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -41,7 +40,7 @@ class UnionPushdownSuite extends PlanTest { test("union: filter to each side") { val query = testUnion.where('a === 1) - val optimized = Optimize(query.analyze) + val optimized = Optimize.execute(query.analyze) val correctAnswer = Union(testRelation.where('a === 1), testRelation2.where('d === 1)).analyze @@ -52,7 +51,7 @@ class UnionPushdownSuite extends PlanTest { test("union: project to each side") { val query = testUnion.select('b) - val optimized = Optimize(query.analyze) + val optimized = Optimize.execute(query.analyze) val correctAnswer = Union(testRelation.select('b), testRelation2.select('e)).analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 48884040bfce7..e7cafcc96de87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{NoRelation, Filter, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Filter, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** @@ -45,16 +45,17 @@ class PlanTest extends FunSuite { protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizeExprIds(plan1) val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) + if (normalized1 != normalized2) { fail( s""" |== FAIL: Plans do not match === |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) + """.stripMargin) + } } /** Fails the test if the two expressions do not match */ protected def compareExpressions(e1: Expression, e2: Expression): Unit = { - comparePlans(Filter(e1, NoRelation), Filter(e2, NoRelation)) + comparePlans(Filter(e1, OneRowRelation), Filter(e2, OneRowRelation)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index 11e6831b24768..1273921f6394c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -32,7 +32,7 @@ class SameResultSuite extends FunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) - def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true) = { + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true): Unit = { val aAnalyzed = a.analyze val bAnalyzed = b.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index 4b2d45584045f..2a641c63f87bb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -34,7 +34,7 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("once", Once, DecrementLiterals) :: Nil } - assert(ApplyOnce(Literal(10)) === Literal(9)) + assert(ApplyOnce.execute(Literal(10)) === Literal(9)) } test("to fixed point") { @@ -42,7 +42,7 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("fixedPoint", FixedPoint(100), DecrementLiterals) :: Nil } - assert(ToFixedPoint(Literal(10)) === Literal(0)) + assert(ToFixedPoint.execute(Literal(10)) === Literal(0)) } test("to maxIterations") { @@ -50,6 +50,6 @@ class RuleExecutorSuite extends FunSuite { val batches = Batch("fixedPoint", FixedPoint(10), DecrementLiterals) :: Nil } - assert(ToFixedPoint(Literal(100)) === Literal(90)) + assert(ToFixedPoint.execute(Literal(100)) === Literal(90)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index e7ce92a2160b6..786ddba403f2c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -22,15 +22,15 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NullType} +import org.apache.spark.sql.types.{IntegerType, StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { - def children = optKey.toSeq - def nullable = true - def dataType = NullType + def children: Seq[Expression] = optKey.toSeq + def nullable: Boolean = true + def dataType: NullType = NullType override lazy val resolved = true type EvaluatedType = Any - def eval(input: Row) = null.asInstanceOf[Any] + def eval(input: Row): Any = null.asInstanceOf[Any] } class TreeNodeSuite extends FunSuite { @@ -90,7 +90,7 @@ class TreeNodeSuite extends FunSuite { } test("transform works on nodes with Option children") { - val dummy1 = Dummy(Some(Literal("1", StringType))) + val dummy1 = Dummy(Some(Literal.create("1", StringType))) val dummy2 = Dummy(None) val toZero: PartialFunction[Expression, Expression] = { case Literal(_, _) => Literal(0) } @@ -117,5 +117,59 @@ class TreeNodeSuite extends FunSuite { assert(transformed.origin.startPosition.isDefined) } + test("foreach up") { + val actual = new ArrayBuffer[String]() + val expected = Seq("1", "2", "3", "4", "-", "*", "+") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression foreachUp { + case b: BinaryExpression => actual.append(b.symbol); + case l: Literal => actual.append(l.toString); + } + + assert(expected === actual) + } + + test("find") { + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + // Find the top node. + var actual: Option[Expression] = expression.find { + case add: Add => true + case other => false + } + var expected: Option[Expression] = + Some(Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4))))) + assert(expected === actual) + // Find the first children. + actual = expression.find { + case Literal(1, IntegerType) => true + case other => false + } + expected = Some(Literal(1)) + assert(expected === actual) + + // Find an internal node (Subtract). + actual = expression.find { + case sub: Subtract => true + case other => false + } + expected = Some(Subtract(Literal(3), Literal(4))) + assert(expected === actual) + + // Find a leaf node. + actual = expression.find { + case Literal(3, IntegerType) => true + case other => false + } + expected = Some(Literal(3)) + assert(expected === actual) + + // Find nothing. + actual = expression.find { + case Literal(100, IntegerType) => true + case other => false + } + expected = None + assert(expected === actual) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala new file mode 100644 index 0000000000000..3e7cf7cbb5e63 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala @@ -0,0 +1,118 @@ +/* +* 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.sql.types + +import org.scalatest.FunSuite + +class DataTypeParserSuite extends FunSuite { + + def checkDataType(dataTypeString: String, expectedDataType: DataType): Unit = { + test(s"parse ${dataTypeString.replace("\n", "")}") { + assert(DataTypeParser.parse(dataTypeString) === expectedDataType) + } + } + + def unsupported(dataTypeString: String): Unit = { + test(s"$dataTypeString is not supported") { + intercept[DataTypeException](DataTypeParser.parse(dataTypeString)) + } + } + + checkDataType("int", IntegerType) + checkDataType("integer", IntegerType) + checkDataType("BooLean", BooleanType) + checkDataType("tinYint", ByteType) + checkDataType("smallINT", ShortType) + checkDataType("INT", IntegerType) + checkDataType("INTEGER", IntegerType) + checkDataType("bigint", LongType) + checkDataType("float", FloatType) + checkDataType("dOUBle", DoubleType) + checkDataType("decimal(10, 5)", DecimalType(10, 5)) + checkDataType("decimal", DecimalType.Unlimited) + checkDataType("DATE", DateType) + checkDataType("timestamp", TimestampType) + checkDataType("string", StringType) + checkDataType("varchAr(20)", StringType) + checkDataType("BINARY", BinaryType) + + checkDataType("array", ArrayType(DoubleType, true)) + checkDataType("Array>", ArrayType(MapType(IntegerType, ByteType, true), true)) + checkDataType( + "array>", + ArrayType(StructType(StructField("tinYint", ByteType, true) :: Nil), true) + ) + checkDataType("MAP", MapType(IntegerType, StringType, true)) + checkDataType("MAp>", MapType(IntegerType, ArrayType(DoubleType), true)) + checkDataType( + "MAP>", + MapType(IntegerType, StructType(StructField("varchar", StringType, true) :: Nil), true) + ) + + checkDataType( + "struct", + StructType( + StructField("intType", IntegerType, true) :: + StructField("ts", TimestampType, true) :: Nil) + ) + // It is fine to use the data type string as the column name. + checkDataType( + "Struct", + StructType( + StructField("int", IntegerType, true) :: + StructField("timestamp", TimestampType, true) :: Nil) + ) + checkDataType( + """ + |struct< + | struct:struct, + | MAP:Map, + | arrAy:Array> + """.stripMargin, + StructType( + StructField("struct", + StructType( + StructField("deciMal", DecimalType.Unlimited, true) :: + StructField("anotherDecimal", DecimalType(5, 2), true) :: Nil), true) :: + StructField("MAP", MapType(TimestampType, StringType), true) :: + StructField("arrAy", ArrayType(DoubleType, true), true) :: Nil) + ) + // A column name can be a reserved word in our DDL parser and SqlParser. + checkDataType( + "Struct", + StructType( + StructField("TABLE", StringType, true) :: + StructField("CASE", BooleanType, true) :: Nil) + ) + // Use backticks to quote column names having special characters. + checkDataType( + "struct<`x+y`:int, `!@#$%^&*()`:string, `1_2.345<>:\"`:varchar(20)>", + StructType( + StructField("x+y", IntegerType, true) :: + StructField("!@#$%^&*()", StringType, true) :: + StructField("1_2.345<>:\"", StringType, true) :: Nil) + ) + // Empty struct. + checkDataType("strUCt<>", StructType(Nil)) + + unsupported("it is not a data type") + unsupported("struct") + unsupported("struct") + unsupported("struct<`x``y` int>") +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index a1341ea13d810..d797510f36685 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -56,6 +56,19 @@ class DataTypeSuite extends FunSuite { } } + test("extract field index from a StructType") { + val struct = StructType( + StructField("a", LongType) :: + StructField("b", FloatType) :: Nil) + + assert(struct.fieldIndex("a") === 0) + assert(struct.fieldIndex("b") === 1) + + intercept[IllegalArgumentException] { + struct.fieldIndex("non_existent") + } + } + def checkDataTypeJsonRepr(dataType: DataType): Unit = { test(s"JSON - $dataType") { assert(DataType.fromJson(dataType.json) === dataType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala new file mode 100644 index 0000000000000..a22aa6f244c48 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala @@ -0,0 +1,70 @@ +/* +* 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.sql.types + +import org.scalatest.FunSuite + +// scalastyle:off +class UTF8StringSuite extends FunSuite { + test("basic") { + def check(str: String, len: Int) { + + assert(UTF8String(str).length == len) + assert(UTF8String(str.getBytes("utf8")).length() == len) + + assert(UTF8String(str) == str) + assert(UTF8String(str.getBytes("utf8")) == str) + assert(UTF8String(str).toString == str) + assert(UTF8String(str.getBytes("utf8")).toString == str) + assert(UTF8String(str.getBytes("utf8")) == UTF8String(str)) + + assert(UTF8String(str).hashCode() == UTF8String(str.getBytes("utf8")).hashCode()) + } + + check("hello", 5) + check("世 界", 3) + } + + test("contains") { + assert(UTF8String("hello").contains(UTF8String("ello"))) + assert(!UTF8String("hello").contains(UTF8String("vello"))) + assert(UTF8String("大千世界").contains(UTF8String("千世"))) + assert(!UTF8String("大千世界").contains(UTF8String("世千"))) + } + + test("prefix") { + assert(UTF8String("hello").startsWith(UTF8String("hell"))) + assert(!UTF8String("hello").startsWith(UTF8String("ell"))) + assert(UTF8String("大千世界").startsWith(UTF8String("大千"))) + assert(!UTF8String("大千世界").startsWith(UTF8String("千"))) + } + + test("suffix") { + assert(UTF8String("hello").endsWith(UTF8String("ello"))) + assert(!UTF8String("hello").endsWith(UTF8String("ellov"))) + assert(UTF8String("大千世界").endsWith(UTF8String("世界"))) + assert(!UTF8String("大千世界").endsWith(UTF8String("世"))) + } + + test("slice") { + assert(UTF8String("hello").slice(1, 3) == UTF8String("el")) + assert(UTF8String("大千世界").slice(0, 1) == UTF8String("大")) + assert(UTF8String("大千世界").slice(1, 3) == UTF8String("千世")) + assert(UTF8String("大千世界").slice(3, 5) == UTF8String("界")) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index ca4a127120b37..18584c2dcf797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -112,7 +112,7 @@ private[sql] class CacheManager(sqlContext: SQLContext) extends Logging { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") - cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.uncache(blocking) cachedData.remove(dataIndex) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index b7a13a1b26802..33f9d0b37d006 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import scala.language.implicitConversions import org.apache.spark.annotation.Experimental +import org.apache.spark.Logging import org.apache.spark.sql.functions.lit import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar, UnresolvedGetField} @@ -46,7 +47,7 @@ private[sql] object Column { * @groupname Ungrouped Support functions for DataFrames. */ @Experimental -class Column(protected[sql] val expr: Expression) { +class Column(protected[sql] val expr: Expression) extends Logging { def this(name: String) = this(name match { case "*" => UnresolvedStar(None) @@ -109,7 +110,15 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def === (other: Any): Column = EqualTo(expr, lit(other).expr) + def === (other: Any): Column = { + val right = lit(other).expr + if (this.expr == right) { + logWarning( + s"Constructing trivially true equals predicate, '${this.expr} = $right'. " + + "Perhaps you need to use aliases.") + } + EqualTo(expr, right) + } /** * Equality test. @@ -506,14 +515,15 @@ class Column(protected[sql] val expr: Expression) { def rlike(literal: String): Column = RLike(expr, lit(literal).expr) /** - * An expression that gets an item at position `ordinal` out of an array. + * An expression that gets an item at position `ordinal` out of an array, + * or gets a value by key `key` in a [[MapType]]. * * @group expr_ops */ - def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + def getItem(key: Any): Column = GetItem(expr, Literal(key)) /** - * An expression that gets a field by name in a [[StructField]]. + * An expression that gets a field by name in a [[StructType]]. * * @group expr_ops */ @@ -594,6 +604,19 @@ class Column(protected[sql] val expr: Expression) { */ def as(alias: Symbol): Column = Alias(expr, alias.name)() + /** + * Gives the column an alias with metadata. + * {{{ + * val metadata: Metadata = ... + * df.select($"colA".as("colB", metadata)) + * }}} + * + * @group expr_ops + */ + def as(alias: String, metadata: Metadata): Column = { + Alias(expr, alias)(explicitMetadata = Some(metadata)) + } + /** * Casts the column to a different data type. * {{{ @@ -624,20 +647,7 @@ class Column(protected[sql] val expr: Expression) { * * @group expr_ops */ - def cast(to: String): Column = cast(to.toLowerCase match { - case "string" | "str" => StringType - case "boolean" => BooleanType - case "byte" => ByteType - case "short" => ShortType - case "int" => IntegerType - case "long" => LongType - case "float" => FloatType - case "double" => DoubleType - case "decimal" => DecimalType.Unlimited - case "date" => DateType - case "timestamp" => TimestampType - case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") - }) + def cast(to: String): Column = cast(DataTypeParser.parse(to)) /** * Returns an ordering used in sorting. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 8b8f86c4127e0..cf344710ff8b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.CharArrayWriter import java.sql.DriverManager + import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -28,20 +29,21 @@ import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonFactory +import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{expressions, ScalaReflection, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD} import org.apache.spark.sql.jdbc.JDBCWriteDetails import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.sources.{ResolvedDataSource, CreateTableUsingAsSelect} import org.apache.spark.util.Utils @@ -89,7 +91,7 @@ private[sql] object DataFrame { * val people = sqlContext.parquetFile("...") * val department = sqlContext.parquetFile("...") * - * people.filter("age" > 30) + * people.filter("age > 30") * .join(department, people("deptId") === department("id")) * .groupBy(department("name"), "gender") * .agg(avg(people("salary")), max(people("age"))) @@ -146,7 +148,7 @@ class DataFrame private[sql]( _: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => - queryExecution.logical + queryExecution.analyzed } /** @@ -158,7 +160,7 @@ class DataFrame private[sql]( } protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + queryExecution.analyzed.resolve(colName.split("\\."), sqlContext.analyzer.resolver).getOrElse { throw new AnalysisException( s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") } @@ -166,7 +168,7 @@ class DataFrame private[sql]( protected[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + queryExecution.analyzed.resolve(n.name.split("\\."), sqlContext.analyzer.resolver).get } } @@ -175,6 +177,7 @@ class DataFrame private[sql]( * @param numRows Number of rows to show */ private[sql] def showString(numRows: Int): String = { + val sb = new StringBuilder val data = take(numRows) val numCols = schema.fieldNames.length @@ -194,12 +197,25 @@ class DataFrame private[sql]( } } - // Pad the cells - rows.map { row => - row.zipWithIndex.map { case (cell, i) => - String.format(s"%-${colWidths(i)}s", cell) - }.mkString(" ") - }.mkString("\n") + // Create SeparateLine + val sep: String = colWidths.map("-" * _).addString(sb, "+", "+", "+\n").toString() + + // column names + rows.head.zipWithIndex.map { case (cell, i) => + StringUtils.leftPad(cell.toString, colWidths(i)) + }.addString(sb, "|", "|", "|\n") + + sb.append(sep) + + // data + rows.tail.map { + _.zipWithIndex.map { case (cell, i) => + StringUtils.leftPad(cell.toString, colWidths(i)) + }.addString(sb, "|", "|", "|\n") + } + + sb.append(sep) + sb.toString() } override def toString: String = { @@ -237,11 +253,11 @@ class DataFrame private[sql]( def toDF(colNames: String*): DataFrame = { require(schema.size == colNames.size, "The number of columns doesn't match.\n" + - "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + colNames.mkString(", ")) + s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" + + s"New column names (${colNames.size}): " + colNames.mkString(", ")) - val newCols = schema.fieldNames.zip(colNames).map { case (oldName, newName) => - apply(oldName).as(newName) + val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => + Column(oldAttribute).as(newName) } select(newCols :_*) } @@ -273,7 +289,7 @@ class DataFrame private[sql]( def printSchema(): Unit = println(schema.treeString) /** - * Prints the plans (logical and physical) to the console for debugging purpose. + * Prints the plans (logical and physical) to the console for debugging purposes. * @group basic */ def explain(extended: Boolean): Unit = { @@ -285,7 +301,7 @@ class DataFrame private[sql]( } /** - * Only prints the physical plan to the console for debugging purpose. + * Only prints the physical plan to the console for debugging purposes. * @group basic */ def explain(): Unit = explain(extended = false) @@ -319,6 +335,28 @@ class DataFrame private[sql]( */ def show(): Unit = show(20) + /** + * Returns a [[DataFrameNaFunctions]] for working with missing data. + * {{{ + * // Dropping rows containing any null values. + * df.na.drop() + * }}} + * + * @group dfops + */ + def na: DataFrameNaFunctions = new DataFrameNaFunctions(this) + + /** + * Returns a [[DataFrameStatFunctions]] for working statistic functions support. + * {{{ + * // Finding frequent items in column with name 'a'. + * df.stat.freqItems(Seq("a")) + * }}} + * + * @group dfops + */ + def stat: DataFrameStatFunctions = new DataFrameStatFunctions(this) + /** * Cartesian join with another [[DataFrame]]. * @@ -331,6 +369,43 @@ class DataFrame private[sql]( Join(logicalPlan, right.logicalPlan, joinType = Inner, None) } + /** + * Inner equi-join with another [[DataFrame]] using the given column. + * + * Different from other join functions, the join column will only appear once in the output, + * i.e. similar to SQL's `JOIN USING` syntax. + * + * {{{ + * // Joining df1 and df2 using the column "user_id" + * df1.join(df2, "user_id") + * }}} + * + * Note that if you perform a self-join using this function without aliasing the input + * [[DataFrame]]s, you will NOT be able to reference any columns after the join, since + * there is no way to disambiguate which side of the join you would like to reference. + * + * @param right Right side of the join operation. + * @param usingColumn Name of the column to join on. This column must exist on both sides. + * @group dfops + */ + def join(right: DataFrame, usingColumn: String): DataFrame = { + // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right + // by creating a new instance for one of the branch. + val joined = sqlContext.executePlan( + Join(logicalPlan, right.logicalPlan, joinType = Inner, None)).analyzed.asInstanceOf[Join] + + // Project only one of the join column. + val joinedCol = joined.right.resolve(usingColumn) + Project( + joined.output.filterNot(_ == joinedCol), + Join( + joined.left, + joined.right, + joinType = Inner, + Some(EqualTo(joined.left.resolve(usingColumn), joined.right.resolve(usingColumn)))) + ) + } + /** * Inner join with another [[DataFrame]], using the given join expression. * @@ -575,17 +650,12 @@ class DataFrame private[sql]( } /** - * (Scala-specific) Compute aggregates by specifying a map from column name to - * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. - * - * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. - * {{{ - * // Selects the age of the oldest employee and the aggregate expense for each department - * df.groupBy("department").agg( - * "age" -> "max", - * "expense" -> "sum" - * ) - * }}} + * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. + * {{ + * // df.agg(...) is a shorthand for df.groupBy().agg(...) + * df.agg("age" -> "max", "salary" -> "avg") + * df.groupBy().agg("age" -> "max", "salary" -> "avg") + * }} * @group dfops */ def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { @@ -663,7 +733,7 @@ class DataFrame private[sql]( * @group dfops */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { - Sample(fraction, withReplacement, seed, logicalPlan) + Sample(0.0, fraction, withReplacement, seed, logicalPlan) } /** @@ -677,6 +747,42 @@ class DataFrame private[sql]( sample(withReplacement, fraction, Utils.random.nextLong) } + /** + * Randomly splits this [[DataFrame]] with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1. + * @param seed Seed for sampling. + * @group dfops + */ + def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame] = { + val sum = weights.sum + val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) + normalizedCumWeights.sliding(2).map { x => + new DataFrame(sqlContext, Sample(x(0), x(1), false, seed, logicalPlan)) + }.toArray + } + + /** + * Randomly splits this [[DataFrame]] with the provided weights. + * + * @param weights weights for splits, will be normalized if they don't sum to 1. + * @group dfops + */ + def randomSplit(weights: Array[Double]): Array[DataFrame] = { + randomSplit(weights, Utils.random.nextLong) + } + + /** + * Randomly splits this [[DataFrame]] with the provided weights. Provided for the Python Api. + * + * @param weights weights for splits, will be normalized if they don't sum to 1. + * @param seed Seed for sampling. + * @group dfops + */ + private[spark] def randomSplit(weights: List[Double], seed: Long): Array[DataFrame] = { + randomSplit(weights.toArray, seed) + } + /** * (Scala-specific) Returns a new [[DataFrame]] where each row has been expanded to zero or more * rows by the provided function. This is similar to a `LATERAL VIEW` in HiveQL. The columns of @@ -700,12 +806,16 @@ class DataFrame private[sql]( */ def explode[A <: Product : TypeTag](input: Column*)(f: Row => TraversableOnce[A]): DataFrame = { val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] - val attributes = schema.toAttributes + + val elementTypes = schema.toAttributes.map { attr => (attr.dataType, attr.nullable) } + val names = schema.toAttributes.map(_.name) + val rowFunction = - f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) - val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) + f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema).asInstanceOf[Row])) + val generator = UserDefinedGenerator(elementTypes, rowFunction, input.map(_.expr)) - Generate(generator, join = true, outer = false, None, logicalPlan) + Generate(generator, join = true, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), logicalPlan) } /** @@ -722,12 +832,17 @@ class DataFrame private[sql]( : DataFrame = { val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil + // TODO handle the metadata? + val elementTypes = attributes.map { attr => (attr.dataType, attr.nullable) } + val names = attributes.map(_.name) + def rowFunction(row: Row): TraversableOnce[Row] = { - f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) + f(row(0).asInstanceOf[A]).map(o => Row(CatalystTypeConverters.convertToCatalyst(o, dataType))) } - val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) + val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil) - Generate(generator, join = true, outer = false, None, logicalPlan) + Generate(generator, join = true, outer = false, + qualifier = None, names.map(UnresolvedAttribute(_)), logicalPlan) } ///////////////////////////////////////////////////////////////////////////// @@ -736,19 +851,117 @@ class DataFrame private[sql]( * Returns a new [[DataFrame]] by adding a column. * @group dfops */ - def withColumn(colName: String, col: Column): DataFrame = select(Column("*"), col.as(colName)) + def withColumn(colName: String, col: Column): DataFrame = { + val resolver = sqlContext.analyzer.resolver + val replaced = schema.exists(f => resolver(f.name, colName)) + if (replaced) { + val colNames = schema.map { field => + val name = field.name + if (resolver(name, colName)) col.as(colName) else Column(name) + } + select(colNames :_*) + } else { + select(Column("*"), col.as(colName)) + } + } /** * Returns a new [[DataFrame]] with a column renamed. + * This is a no-op if schema doesn't contain existingName. * @group dfops */ def withColumnRenamed(existingName: String, newName: String): DataFrame = { val resolver = sqlContext.analyzer.resolver - val colNames = schema.map { field => - val name = field.name - if (resolver(name, existingName)) Column(name).as(newName) else Column(name) + val shouldRename = schema.exists(f => resolver(f.name, existingName)) + if (shouldRename) { + val colNames = schema.map { field => + val name = field.name + if (resolver(name, existingName)) Column(name).as(newName) else Column(name) + } + select(colNames : _*) + } else { + this } - select(colNames :_*) + } + + /** + * Returns a new [[DataFrame]] with a column dropped. + * This is a no-op if schema doesn't contain column name. + * @group dfops + */ + def drop(colName: String): DataFrame = { + val resolver = sqlContext.analyzer.resolver + val shouldDrop = schema.exists(f => resolver(f.name, colName)) + if (shouldDrop) { + val colsAfterDrop = schema.filter { field => + val name = field.name + !resolver(name, colName) + }.map(f => Column(f.name)) + select(colsAfterDrop : _*) + } else { + this + } + } + + /** + * Computes statistics for numeric columns, including count, mean, stddev, min, and max. + * If no columns are given, this function computes statistics for all numerical columns. + * + * This function is meant for exploratory data analysis, as we make no guarantee about the + * backward compatibility of the schema of the resulting [[DataFrame]]. If you want to + * programmatically compute summary statistics, use the `agg` function instead. + * + * {{{ + * df.describe("age", "height").show() + * + * // output: + * // summary age height + * // count 10.0 10.0 + * // mean 53.3 178.05 + * // stddev 11.6 15.7 + * // min 18.0 163.0 + * // max 92.0 192.0 + * }}} + * + * @group action + */ + @scala.annotation.varargs + def describe(cols: String*): DataFrame = { + + // TODO: Add stddev as an expression, and remove it from here. + def stddevExpr(expr: Expression): Expression = + Sqrt(Subtract(Average(Multiply(expr, expr)), Multiply(Average(expr), Average(expr)))) + + // The list of summary statistics to compute, in the form of expressions. + val statistics = List[(String, Expression => Expression)]( + "count" -> Count, + "mean" -> Average, + "stddev" -> stddevExpr, + "min" -> Min, + "max" -> Max) + + val outputCols = (if (cols.isEmpty) numericColumns.map(_.prettyString) else cols).toList + + val ret: Seq[Row] = if (outputCols.nonEmpty) { + val aggExprs = statistics.flatMap { case (_, colToAgg) => + outputCols.map(c => Column(colToAgg(Column(c).expr)).as(c)) + } + + val row = agg(aggExprs.head, aggExprs.tail: _*).head().toSeq + + // Pivot the data so each summary is one row + row.grouped(outputCols.size).toSeq.zip(statistics).map { + case (aggregation, (statistic, _)) => Row(statistic :: aggregation.toList: _*) + } + } else { + // If there are no output columns, just output a single column that contains the stats. + statistics.map { case (name, _) => Row(name) } + } + + // The first column is string type, and the rest are double type. + val schema = StructType( + StructField("summary", StringType) :: outputCols.map(StructField(_, DoubleType))).toAttributes + LocalRelation(schema, ret) } /** @@ -831,8 +1044,18 @@ class DataFrame private[sql]( * @group rdd */ override def repartition(numPartitions: Int): DataFrame = { - sqlContext.createDataFrame( - queryExecution.toRdd.map(_.copy()).repartition(numPartitions), schema) + Repartition(numPartitions, shuffle = true, logicalPlan) + } + + /** + * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. + * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. + * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of + * the 100 new partitions will claim 10 of the current partitions. + * @group rdd + */ + override def coalesce(numPartitions: Int): DataFrame = { + Repartition(numPartitions, shuffle = false, logicalPlan) } /** @@ -880,13 +1103,18 @@ class DataFrame private[sql]( ///////////////////////////////////////////////////////////////////////////// /** - * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. + * Represents the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. Note that the RDD is + * memoized. Once called, it won't change even if you change any query planning related Spark SQL + * configurations (e.g. `spark.sql.shuffle.partitions`). * @group rdd */ - def rdd: RDD[Row] = { + lazy val rdd: RDD[Row] = { // use a local variable to make sure the map closure doesn't capture the whole DataFrame val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + queryExecution.executedPlan.execute().mapPartitions { rows => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]) + } } /** @@ -902,8 +1130,8 @@ class DataFrame private[sql]( def javaRDD: JavaRDD[Row] = toJavaRDD /** - * Registers this RDD as a temporary table using the given name. The lifetime of this temporary - * table is tied to the [[SQLContext]] that was used to create this DataFrame. + * Registers this [[DataFrame]] as a temporary table using the given name. The lifetime of this + * temporary table is tied to the [[SQLContext]] that was used to create this DataFrame. * * @group basic */ @@ -1131,7 +1359,7 @@ class DataFrame private[sql]( @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit = { sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd + Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd } /** @@ -1178,7 +1406,7 @@ class DataFrame private[sql]( //////////////////////////////////////////////////////////////////////////// /** - * Save this RDD to a JDBC database at `url` under the table name `table`. + * Save this [[DataFrame]] to a JDBC database at `url` under the table name `table`. * This will run a `CREATE TABLE` and a bunch of `INSERT INTO` statements. * If you pass `true` for `allowExisting`, it will drop any table with the * given name; if you pass `false`, it will throw if the table already @@ -1202,7 +1430,7 @@ class DataFrame private[sql]( } /** - * Save this RDD to a JDBC database at `url` under the table name `table`. + * Save this [[DataFrame]] to a JDBC database at `url` under the table name `table`. * Assumes the table already exists and has a compatible schema. If you * pass `true` for `overwrite`, it will `TRUNCATE` the table before * performing the `INSERT`s. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala new file mode 100644 index 0000000000000..481ed4924857e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -0,0 +1,375 @@ +/* +* 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.sql + +import java.{lang => jl} + +import scala.collection.JavaConversions._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + + +/** + * :: Experimental :: + * Functionality for working with missing data in [[DataFrame]]s. + */ +@Experimental +final class DataFrameNaFunctions private[sql](df: DataFrame) { + + /** + * Returns a new [[DataFrame]] that drops rows containing any null values. + */ + def drop(): DataFrame = drop("any", df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing null values. + * + * If `how` is "any", then drop rows containing any null values. + * If `how` is "all", then drop rows only if every column is null for that row. + */ + def drop(how: String): DataFrame = drop(how, df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing any null values + * in the specified columns. + */ + def drop(cols: Array[String]): DataFrame = drop(cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame ]] that drops rows containing any null values + * in the specified columns. + */ + def drop(cols: Seq[String]): DataFrame = drop(cols.size, cols) + + /** + * Returns a new [[DataFrame]] that drops rows containing null values + * in the specified columns. + * + * If `how` is "any", then drop rows containing any null values in the specified columns. + * If `how` is "all", then drop rows only if every specified column is null for that row. + */ + def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that drops rows containing null values + * in the specified columns. + * + * If `how` is "any", then drop rows containing any null values in the specified columns. + * If `how` is "all", then drop rows only if every specified column is null for that row. + */ + def drop(how: String, cols: Seq[String]): DataFrame = { + how.toLowerCase match { + case "any" => drop(cols.size, cols) + case "all" => drop(1, cols) + case _ => throw new IllegalArgumentException(s"how ($how) must be 'any' or 'all'") + } + } + + /** + * Returns a new [[DataFrame]] that drops rows containing less than `minNonNulls` non-null values. + */ + def drop(minNonNulls: Int): DataFrame = drop(minNonNulls, df.columns) + + /** + * Returns a new [[DataFrame]] that drops rows containing less than `minNonNulls` non-null + * values in the specified columns. + */ + def drop(minNonNulls: Int, cols: Array[String]): DataFrame = drop(minNonNulls, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that drops rows containing less than + * `minNonNulls` non-null values in the specified columns. + */ + def drop(minNonNulls: Int, cols: Seq[String]): DataFrame = { + // Filtering condition -- only keep the row if it has at least `minNonNulls` non-null values. + val predicate = AtLeastNNonNulls(minNonNulls, cols.map(name => df.resolve(name))) + df.filter(Column(predicate)) + } + + /** + * Returns a new [[DataFrame]] that replaces null values in numeric columns with `value`. + */ + def fill(value: Double): DataFrame = fill(value, df.columns) + + /** + * Returns a new [[DataFrame ]] that replaces null values in string columns with `value`. + */ + def fill(value: String): DataFrame = fill(value, df.columns) + + /** + * Returns a new [[DataFrame]] that replaces null values in specified numeric columns. + * If a specified column is not a numeric column, it is ignored. + */ + def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values in specified + * numeric columns. If a specified column is not a numeric column, it is ignored. + */ + def fill(value: Double, cols: Seq[String]): DataFrame = { + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + // Only fill if the column is part of the cols list. + if (f.dataType.isInstanceOf[NumericType] && cols.exists(col => columnEquals(f.name, col))) { + fillCol[Double](f, value) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + + /** + * Returns a new [[DataFrame]] that replaces null values in specified string columns. + * If a specified column is not a string column, it is ignored. + */ + def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values in + * specified string columns. If a specified column is not a string column, it is ignored. + */ + def fill(value: String, cols: Seq[String]): DataFrame = { + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + // Only fill if the column is part of the cols list. + if (f.dataType.isInstanceOf[StringType] && cols.exists(col => columnEquals(f.name, col))) { + fillCol[String](f, value) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + + /** + * Returns a new [[DataFrame]] that replaces null values. + * + * The key of the map is the column name, and the value of the map is the replacement value. + * The value must be of the following type: `Integer`, `Long`, `Float`, `Double`, `String`. + * + * For example, the following replaces null values in column "A" with string "unknown", and + * null values in column "B" with numeric value 1.0. + * {{{ + * import com.google.common.collect.ImmutableMap; + * df.na.fill(ImmutableMap.of("A", "unknown", "B", 1.0)); + * }}} + */ + def fill(valueMap: java.util.Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + + /** + * (Scala-specific) Returns a new [[DataFrame]] that replaces null values. + * + * The key of the map is the column name, and the value of the map is the replacement value. + * The value must be of the following type: `Int`, `Long`, `Float`, `Double`, `String`. + * + * For example, the following replaces null values in column "A" with string "unknown", and + * null values in column "B" with numeric value 1.0. + * {{{ + * df.na.fill(Map( + * "A" -> "unknown", + * "B" -> 1.0 + * )) + * }}} + */ + def fill(valueMap: Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", ImmutableMap.of("UNKNOWN", "unnamed")); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: java.util.Map[T, T]): DataFrame = { + replace[T](col, replacement.toMap : Map[T, T]) + } + + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace(new String[] {"height", "weight"}, ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace(new String[] {"firstname", "lastname"}, ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { + replace(cols.toSeq, replacement.toMap) + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", Map(1.0 -> 2.0)) + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", Map("UNKNOWN" -> "unnamed") + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", Map("UNKNOWN" -> "unnamed") + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: Map[T, T]): DataFrame = { + if (col == "*") { + replace0(df.columns, replacement) + } else { + replace0(Seq(col), replacement) + } + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace("height" :: "weight" :: Nil, Map(1.0 -> 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace("firstname" :: "lastname" :: Nil, Map("UNKNOWN" -> "unnamed"); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = replace0(cols, replacement) + + private def replace0[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = { + if (replacement.isEmpty || cols.isEmpty) { + return df + } + + // replacementMap is either Map[String, String] or Map[Double, Double] + val replacementMap: Map[_, _] = replacement.head._2 match { + case v: String => replacement + case _ => replacement.map { case (k, v) => (convertToDouble(k), convertToDouble(v)) } + } + + // targetColumnType is either DoubleType or StringType + val targetColumnType = replacement.head._1 match { + case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long => DoubleType + case _: String => StringType + } + + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + val shouldReplace = cols.exists(colName => columnEquals(colName, f.name)) + if (f.dataType.isInstanceOf[NumericType] && targetColumnType == DoubleType && shouldReplace) { + replaceCol(f, replacementMap) + } else if (f.dataType == targetColumnType && shouldReplace) { + replaceCol(f, replacementMap) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + + private def fill0(values: Seq[(String, Any)]): DataFrame = { + // Error handling + values.foreach { case (colName, replaceValue) => + // Check column name exists + df.resolve(colName) + + // Check data type + replaceValue match { + case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: String => + // This is good + case _ => throw new IllegalArgumentException( + s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).") + } + } + + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + values.find { case (k, _) => columnEquals(k, f.name) }.map { case (_, v) => + v match { + case v: jl.Float => fillCol[Double](f, v.toDouble) + case v: jl.Double => fillCol[Double](f, v) + case v: jl.Long => fillCol[Double](f, v.toDouble) + case v: jl.Integer => fillCol[Double](f, v.toDouble) + case v: String => fillCol[String](f, v) + } + }.getOrElse(df.col(f.name)) + } + df.select(projections : _*) + } + + /** + * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. + */ + private def fillCol[T](col: StructField, replacement: T): Column = { + coalesce(df.col(col.name), lit(replacement).cast(col.dataType)).as(col.name) + } + + /** + * Returns a [[Column]] expression that replaces value matching key in `replacementMap` with + * value in `replacementMap`, using [[CaseWhen]]. + * + * TODO: This can be optimized to use broadcast join when replacementMap is large. + */ + private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { + val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => + df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: + lit(target).cast(col.dataType).expr :: Nil + }.toSeq + new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + } + + private def convertToDouble(v: Any): Double = v match { + case v: Float => v.toDouble + case v: Double => v + case v: Long => v.toDouble + case v: Int => v.toDouble + case v => throw new IllegalArgumentException( + s"Unsupported value type ${v.getClass.getName} ($v).") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala new file mode 100644 index 0000000000000..fcf21ca741a7c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -0,0 +1,124 @@ +/* +* 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.sql + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.execution.stat._ + +/** + * :: Experimental :: + * Statistic functions for [[DataFrame]]s. + */ +@Experimental +final class DataFrameStatFunctions private[sql](df: DataFrame) { + + /** + * Calculate the sample covariance of two numerical columns of a DataFrame. + * @param col1 the name of the first column + * @param col2 the name of the second column + * @return the covariance of the two columns. + */ + def cov(col1: String, col2: String): Double = { + StatFunctions.calculateCov(df, Seq(col1, col2)) + } + + /* + * Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson + * Correlation Coefficient. For Spearman Correlation, consider using RDD methods found in + * MLlib's Statistics. + * + * @param col1 the name of the column + * @param col2 the name of the column to calculate the correlation against + * @return The Pearson Correlation Coefficient as a Double. + */ + def corr(col1: String, col2: String, method: String): Double = { + require(method == "pearson", "Currently only the calculation of the Pearson Correlation " + + "coefficient is supported.") + StatFunctions.pearsonCorrelation(df, Seq(col1, col2)) + } + + /** + * Calculates the Pearson Correlation Coefficient of two columns of a DataFrame. + * + * @param col1 the name of the column + * @param col2 the name of the column to calculate the correlation against + * @return The Pearson Correlation Coefficient as a Double. + */ + def corr(col1: String, col2: String): Double = { + corr(col1, col2, "pearson") + } + + /** + * Computes a pair-wise frequency table of the given columns. Also known as a contingency table. + * The number of distinct values for each column should be less than 1e4. The first + * column of each row will be the distinct values of `col1` and the column names will be the + * distinct values of `col2`. The name of the first column will be `$col1_$col2`. Counts will be + * returned as `Long`s. Pairs that have no occurrences will have `null` as their counts. + * + * @param col1 The name of the first column. Distinct items will make the first item of + * each row. + * @param col2 The name of the second column. Distinct items will make the column names + * of the DataFrame. + * @return A Local DataFrame containing the table + */ + def crosstab(col1: String, col2: String): DataFrame = { + StatFunctions.crossTabulate(df, col1, col2) + } + + /** + * Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * The `support` should be greater than 1e-4. + * + * @param cols the names of the columns to search frequent items in. + * @param support The minimum frequency for an item to be considered `frequent`. Should be greater + * than 1e-4. + * @return A Local DataFrame with the Array of frequent items for each column. + */ + def freqItems(cols: Array[String], support: Double): DataFrame = { + FrequentItems.singlePassFreqItems(df, cols, support) + } + + /** + * Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * Uses a `default` support of 1%. + * + * @param cols the names of the columns to search frequent items in. + * @return A Local DataFrame with the Array of frequent items for each column. + */ + def freqItems(cols: Array[String]): DataFrame = { + FrequentItems.singlePassFreqItems(df, cols, 0.01) + } + + /** + * Python friendly implementation for `freqItems` + */ + def freqItems(cols: Seq[String], support: Double): DataFrame = { + FrequentItems.singlePassFreqItems(df, cols, support) + } + + /** + * Python friendly implementation for `freqItems` with a default `support` of 1%. + */ + def freqItems(cols: Seq[String]): DataFrame = { + FrequentItems.singlePassFreqItems(df, cols, 0.01) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index 45a63ae26ed71..53ad67372e024 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.NumericType @Experimental class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { - private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { + private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() @@ -127,10 +127,7 @@ class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department * import com.google.common.collect.ImmutableMap; - * df.groupBy("department").agg(ImmutableMap.builder() - * .put("age", "max") - * .put("expense", "sum") - * .build()); + * df.groupBy("department").agg(ImmutableMap.of("age", "max", "expense", "sum")); * }}} */ def agg(exprs: java.util.Map[String, String]): DataFrame = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala b/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala new file mode 100644 index 0000000000000..db484c5f50074 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/JavaTypeInference.scala @@ -0,0 +1,110 @@ +/* + * 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.sql + +import java.beans.Introspector +import java.lang.{Iterable => JIterable} +import java.util.{Iterator => JIterator, Map => JMap} + +import com.google.common.reflect.TypeToken + +import org.apache.spark.sql.types._ + +import scala.language.existentials + +/** + * Type-inference utilities for POJOs and Java collections. + */ +private [sql] object JavaTypeInference { + + private val iterableType = TypeToken.of(classOf[JIterable[_]]) + private val mapType = TypeToken.of(classOf[JMap[_, _]]) + private val iteratorReturnType = classOf[JIterable[_]].getMethod("iterator").getGenericReturnType + private val nextReturnType = classOf[JIterator[_]].getMethod("next").getGenericReturnType + private val keySetReturnType = classOf[JMap[_, _]].getMethod("keySet").getGenericReturnType + private val valuesReturnType = classOf[JMap[_, _]].getMethod("values").getGenericReturnType + + /** + * Infers the corresponding SQL data type of a Java type. + * @param typeToken Java type + * @return (SQL data type, nullable) + */ + private [sql] def inferDataType(typeToken: TypeToken[_]): (DataType, Boolean) = { + // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. + typeToken.getRawType match { + case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) + + case _ if typeToken.isArray => + val (dataType, nullable) = inferDataType(typeToken.getComponentType) + (ArrayType(dataType, nullable), true) + + case _ if iterableType.isAssignableFrom(typeToken) => + val (dataType, nullable) = inferDataType(elementType(typeToken)) + (ArrayType(dataType, nullable), true) + + case _ if mapType.isAssignableFrom(typeToken) => + val typeToken2 = typeToken.asInstanceOf[TypeToken[_ <: JMap[_, _]]] + val mapSupertype = typeToken2.getSupertype(classOf[JMap[_, _]]) + val keyType = elementType(mapSupertype.resolveType(keySetReturnType)) + val valueType = elementType(mapSupertype.resolveType(valuesReturnType)) + val (keyDataType, _) = inferDataType(keyType) + val (valueDataType, nullable) = inferDataType(valueType) + (MapType(keyDataType, valueDataType, nullable), true) + + case _ => + val beanInfo = Introspector.getBeanInfo(typeToken.getRawType) + val properties = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + val fields = properties.map { property => + val returnType = typeToken.method(property.getReadMethod).getReturnType + val (dataType, nullable) = inferDataType(returnType) + new StructField(property.getName, dataType, nullable) + } + (new StructType(fields), true) + } + } + + private def elementType(typeToken: TypeToken[_]): TypeToken[_] = { + val typeToken2 = typeToken.asInstanceOf[TypeToken[_ <: JIterable[_]]] + val iterableSupertype = typeToken2.getSupertype(classOf[JIterable[_]]) + val iteratorType = iterableSupertype.resolveType(iteratorReturnType) + val itemType = iteratorType.resolveType(nextReturnType) + itemType + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index ba4373f0124b4..63dbab19947c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -61,5 +61,7 @@ private[sql] trait RDDApi[T] { def repartition(numPartitions: Int): DataFrame + def coalesce(numPartitions: Int): DataFrame + def distinct: DataFrame } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4815620c6fe57..99db959a8741c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,6 +30,7 @@ private[spark] object SQLConf { val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" val CODEGEN_ENABLED = "spark.sql.codegen" + val UNSAFE_ENABLED = "spark.sql.unsafe.enabled" val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" @@ -39,15 +40,20 @@ private[spark] object SQLConf { val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" val PARQUET_USE_DATA_SOURCE_API = "spark.sql.parquet.useDataSourceApi" + val HIVE_VERIFY_PARTITIONPATH = "spark.sql.hive.verifyPartitionPath" + val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" val BROADCAST_TIMEOUT = "spark.sql.broadcastTimeout" // Options that control which operators can be chosen by the query planner. These should be // considered hints and may be ignored by future versions of Spark SQL. val EXTERNAL_SORT = "spark.sql.planner.externalSort" + val SORTMERGE_JOIN = "spark.sql.planner.sortMergeJoin" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" + val THRIFTSERVER_UI_STATEMENT_LIMIT = "spark.sql.thriftserver.ui.retainedStatements" + val THRIFTSERVER_UI_SESSION_LIMIT = "spark.sql.thriftserver.ui.retainedSessions" // This is used to set the default data source val DEFAULT_DATA_SOURCE_NAME = "spark.sql.sources.default" @@ -61,6 +67,8 @@ private[spark] object SQLConf { // Set to false when debugging requires the ability to look at invalid query plans. val DATAFRAME_EAGER_ANALYSIS = "spark.sql.eagerAnalysis" + val USE_SQL_SERIALIZER2 = "spark.sql.useSerializer2" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -119,9 +127,20 @@ private[sql] class SQLConf extends Serializable { private[spark] def parquetUseDataSourceApi = getConf(PARQUET_USE_DATA_SOURCE_API, "true").toBoolean + /** When true uses verifyPartitionPath to prune the path which is not exists. */ + private[spark] def verifyPartitionPath = + getConf(HIVE_VERIFY_PARTITIONPATH, "true").toBoolean + /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean + /** + * Sort merge join would sort the two side of join first, and then iterate both sides together + * only once to get all matches. Using sort merge join can save a lot of memory usage compared + * to HashJoin. + */ + private[spark] def sortMergeJoinEnabled: Boolean = getConf(SORTMERGE_JOIN, "false").toBoolean + /** * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster @@ -133,6 +152,16 @@ private[sql] class SQLConf extends Serializable { */ private[spark] def codegenEnabled: Boolean = getConf(CODEGEN_ENABLED, "false").toBoolean + /** + * When set to true, Spark SQL will use managed memory for certain operations. This option only + * takes effect if codegen is enabled. + * + * Defaults to false as this feature is currently experimental. + */ + private[spark] def unsafeEnabled: Boolean = getConf(UNSAFE_ENABLED, "false").toBoolean + + private[spark] def useSqlSerializer2: Boolean = getConf(USE_SQL_SERIALIZER2, "true").toBoolean + /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to * a broadcast value during the physical executions of join operations. Setting this to -1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index dc9912b52dcab..7eabb93c1e3d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,16 +24,21 @@ import scala.collection.JavaConversions._ import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag +import scala.util.control.NonFatal + +import com.google.common.reflect.TypeToken import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.errors.DialectException import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, NoRelation} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.{ScalaReflection, expressions} +import org.apache.spark.sql.catalyst.Dialect +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, expressions} import org.apache.spark.sql.execution.{Filter, _} import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.json._ @@ -42,6 +47,42 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.{Partition, SparkContext} +/** + * Currently we support the default dialect named "sql", associated with the class + * [[DefaultDialect]] + * + * And we can also provide custom SQL Dialect, for example in Spark SQL CLI: + * {{{ + *-- switch to "hiveql" dialect + * spark-sql>SET spark.sql.dialect=hiveql; + * spark-sql>SELECT * FROM src LIMIT 1; + * + *-- switch to "sql" dialect + * spark-sql>SET spark.sql.dialect=sql; + * spark-sql>SELECT * FROM src LIMIT 1; + * + *-- register the new SQL dialect + * spark-sql> SET spark.sql.dialect=com.xxx.xxx.SQL99Dialect; + * spark-sql> SELECT * FROM src LIMIT 1; + * + *-- register the non-exist SQL dialect + * spark-sql> SET spark.sql.dialect=NotExistedClass; + * spark-sql> SELECT * FROM src LIMIT 1; + * + *-- Exception will be thrown and switch to dialect + *-- "sql" (for SQLContext) or + *-- "hiveql" (for HiveContext) + * }}} + */ +private[spark] class DefaultDialect extends Dialect { + @transient + protected val sqlParser = new catalyst.SqlParser + + override def parse(sqlText: String): LogicalPlan = { + sqlParser.parse(sqlText) + } +} + /** * The entry point for working with structured data (rows and columns) in Spark. Allows the * creation of [[DataFrame]] objects as well as the execution of SQL queries. @@ -120,23 +161,40 @@ class SQLContext(@transient val sparkContext: SparkContext) ExtractPythonUdfs :: sources.PreInsertCastAndRename :: Nil + + override val extendedCheckRules = Seq( + sources.PreWriteCheck(catalog) + ) } @transient protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer @transient - protected[sql] val ddlParser = new DDLParser(sqlParser.apply(_)) + protected[sql] val ddlParser = new DDLParser(sqlParser.parse(_)) @transient - protected[sql] val sqlParser = { - val fallback = new catalyst.SqlParser - new SparkSQLParser(fallback(_)) + protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect().parse(_)) + + protected[sql] def getSQLDialect(): Dialect = { + try { + val clazz = Utils.classForName(dialectClassName) + clazz.newInstance().asInstanceOf[Dialect] + } catch { + case NonFatal(e) => + // Since we didn't find the available SQL Dialect, it will fail even for SET command: + // SET spark.sql.dialect=sql; Let's reset as default dialect automatically. + val dialect = conf.dialect + // reset the sql dialect + conf.unsetConf(SQLConf.DIALECT) + // throw out the exception, and the default sql dialect will take effect for next query. + throw new DialectException( + s"""Instantiating dialect '$dialect' failed. + |Reverting to default dialect '${conf.dialect}'""".stripMargin, e) + } } - protected[sql] def parseSql(sql: String): LogicalPlan = { - ddlParser(sql, false).getOrElse(sqlParser(sql)) - } + protected[sql] def parseSql(sql: String): LogicalPlan = ddlParser.parse(sql, false) protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) @@ -150,6 +208,12 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val defaultSession = createSession() + protected[sql] def dialectClassName = if (conf.dialect == "sql") { + classOf[DefaultDialect].getCanonicalName + } else { + conf.dialect + } + sparkContext.getConf.getAll.foreach { case (key, value) if key.startsWith("spark.sql") => setConf(key, value) case _ => @@ -177,7 +241,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental @transient - lazy val emptyDataFrame = DataFrame(this, NoRelation) + lazy val emptyDataFrame: DataFrame = createDataFrame(sparkContext.emptyRDD[Row], StructType(Nil)) /** * A collection of methods for registering user-defined functions (UDF). @@ -388,9 +452,24 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @DeveloperApi def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema, needsConversion = true) + } + + /** + * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be + * converted to Catalyst rows. + */ + private[sql] + def createDataFrame(rowRDD: RDD[Row], schema: StructType, needsConversion: Boolean) = { // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. - val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) + val catalystRows = if (needsConversion) { + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + rowRDD.map(converter(_).asInstanceOf[Row]) + } else { + rowRDD + } + val logicalPlan = LogicalRDD(schema.toAttributes, catalystRows)(self) DataFrame(this, logicalPlan) } @@ -407,20 +486,6 @@ class SQLContext(@transient val sparkContext: SparkContext) createDataFrame(rowRDD.rdd, schema) } - /** - * Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s by applying - * a seq of names of columns to this RDD, the data type for each column will - * be inferred by the first row. - * - * @param rowRDD an JavaRDD of Row - * @param columns names for each column - * @return DataFrame - * @group dataframes - */ - def createDataFrame(rowRDD: JavaRDD[Row], columns: java.util.List[String]): DataFrame = { - createDataFrame(rowRDD.rdd, columns.toSeq) - } - /** * Applies a schema to an RDD of Java Beans. * @@ -441,7 +506,7 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { row => new GenericRow( extractors.zip(attributeSeq).map { case (e, attr) => - DataTypeConversions.convertJavaToCatalyst(e.invoke(row), attr.dataType) + CatalystTypeConverters.convertToCatalyst(e.invoke(row), attr.dataType) }.toArray[Any] ) : Row } @@ -600,7 +665,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) } /** @@ -629,7 +694,7 @@ class SQLContext(@transient val sparkContext: SparkContext) JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) - createDataFrame(rowRDD, appliedSchema) + createDataFrame(rowRDD, appliedSchema, needsConversion = false) } /** @@ -854,8 +919,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * passed to this function. * * @param columnName the name of a column of integral type that will be used for partitioning. - * @param lowerBound the minimum value of `columnName` to retrieve - * @param upperBound the maximum value of `columnName` to retrieve + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split * evenly into this many partitions * @@ -924,11 +989,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group basic */ def sql(sqlText: String): DataFrame = { - if (conf.dialect == "sql") { - DataFrame(this, parseSql(sqlText)) - } else { - sys.error(s"Unsupported SQL dialect: ${conf.dialect}") - } + DataFrame(this, parseSql(sqlText)) } /** @@ -990,6 +1051,8 @@ class SQLContext(@transient val sparkContext: SparkContext) def codegenEnabled: Boolean = self.conf.codegenEnabled + def unsafeEnabled: Boolean = self.conf.unsafeEnabled + def numPartitions: Int = self.conf.numShufflePartitions def strategies: Seq[Strategy] = @@ -1062,17 +1125,9 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: Nil - } - - @transient - protected[sql] lazy val checkAnalysis = new CheckAnalysis { - override val extendedCheckRules = Seq( - sources.PreWriteCheck(catalog) - ) + Batch("Add exchange", Once, EnsureRequirements(self)) :: Nil } - protected[sql] def openSession(): SQLSession = { detachSession() val session = createSession() @@ -1105,14 +1160,14 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @DeveloperApi protected[sql] class QueryExecution(val logical: LogicalPlan) { - def assertAnalyzed(): Unit = checkAnalysis(analyzed) + def assertAnalyzed(): Unit = analyzer.checkAnalysis(analyzed) - lazy val analyzed: LogicalPlan = analyzer(logical) + lazy val analyzed: LogicalPlan = analyzer.execute(logical) lazy val withCachedData: LogicalPlan = { assertAnalyzed() cacheManager.useCachedData(analyzed) } - lazy val optimizedPlan: LogicalPlan = optimizer(withCachedData) + lazy val optimizedPlan: LogicalPlan = optimizer.execute(withCachedData) // TODO: Don't just pick the first one... lazy val sparkPlan: SparkPlan = { @@ -1121,7 +1176,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan) + lazy val executedPlan: SparkPlan = prepareForExecution.execute(sparkPlan) /** Internal version of the RDD. Avoids copies and has no schema */ lazy val toRdd: RDD[Row] = executedPlan.execute() @@ -1184,6 +1239,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case FloatType => true case DateType => true case TimestampType => true + case StringType => true case ArrayType(_, _) => true case MapType(_, _, _) => true case StructType(_) => true @@ -1210,38 +1266,12 @@ class SQLContext(@transient val sparkContext: SparkContext) * Returns a Catalyst Schema for the given java bean class. */ protected def getSchema(beanClass: Class[_]): Seq[AttributeReference] = { - // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. - val beanInfo = Introspector.getBeanInfo(beanClass) - - // Note: The ordering of elements may differ from when the schema is inferred in Scala. - // This is because beanInfo.getPropertyDescriptors gives no guarantees about - // element ordering. - val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") - fields.map { property => - val (dataType, nullable) = property.getPropertyType match { - case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) - case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) - case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) - case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) - case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) - } - AttributeReference(property.getName, dataType, nullable)() + val (dataType, _) = JavaTypeInference.inferDataType(TypeToken.of(beanClass)) + dataType.asInstanceOf[StructType].fields.map { f => + AttributeReference(f.name, f.dataType, f.nullable)() } } + } + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala new file mode 100644 index 0000000000000..ae77f72998a22 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -0,0 +1,153 @@ +/* + * 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.sql.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.r.SerDe +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} + +private[r] object SQLUtils { + def createSQLContext(jsc: JavaSparkContext): SQLContext = { + new SQLContext(jsc) + } + + def getJavaSparkContext(sqlCtx: SQLContext): JavaSparkContext = { + new JavaSparkContext(sqlCtx.sparkContext) + } + + def toSeq[T](arr: Array[T]): Seq[T] = { + arr.toSeq + } + + def createStructType(fields : Seq[StructField]): StructType = { + StructType(fields) + } + + def getSQLDataType(dataType: String): DataType = { + dataType match { + case "byte" => org.apache.spark.sql.types.ByteType + case "integer" => org.apache.spark.sql.types.IntegerType + case "double" => org.apache.spark.sql.types.DoubleType + case "numeric" => org.apache.spark.sql.types.DoubleType + case "character" => org.apache.spark.sql.types.StringType + case "string" => org.apache.spark.sql.types.StringType + case "binary" => org.apache.spark.sql.types.BinaryType + case "raw" => org.apache.spark.sql.types.BinaryType + case "logical" => org.apache.spark.sql.types.BooleanType + case "boolean" => org.apache.spark.sql.types.BooleanType + case "timestamp" => org.apache.spark.sql.types.TimestampType + case "date" => org.apache.spark.sql.types.DateType + case _ => throw new IllegalArgumentException(s"Invaid type $dataType") + } + } + + def createStructField(name: String, dataType: String, nullable: Boolean): StructField = { + val dtObj = getSQLDataType(dataType) + StructField(name, dtObj, nullable) + } + + def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = { + val num = schema.fields.size + val rowRDD = rdd.map(bytesToRow) + sqlContext.createDataFrame(rowRDD, schema) + } + + // A helper to include grouping columns in Agg() + def aggWithGrouping(gd: GroupedData, exprs: Column*): DataFrame = { + val aggExprs = exprs.map { col => + col.expr match { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.simpleString)() + } + } + gd.toDF(aggExprs) + } + + def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = { + df.map(r => rowToRBytes(r)) + } + + private[this] def bytesToRow(bytes: Array[Byte]): Row = { + val bis = new ByteArrayInputStream(bytes) + val dis = new DataInputStream(bis) + val num = SerDe.readInt(dis) + Row.fromSeq((0 until num).map { i => + SerDe.readObject(dis) + }.toSeq) + } + + private[this] def rowToRBytes(row: Row): Array[Byte] = { + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, row.length) + (0 until row.length).map { idx => + val obj: Object = row(idx).asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def dfToCols(df: DataFrame): Array[Array[Byte]] = { + // localDF is Array[Row] + val localDF = df.collect() + val numCols = df.columns.length + // dfCols is Array[Array[Any]] + val dfCols = convertRowsToColumns(localDF, numCols) + + dfCols.map { col => + colToRBytes(col) + } + } + + def convertRowsToColumns(localDF: Array[Row], numCols: Int): Array[Array[Any]] = { + (0 until numCols).map { colIdx => + localDF.map { row => + row(colIdx) + } + }.toArray + } + + def colToRBytes(col: Array[Any]): Array[Byte] = { + val numRows = col.length + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, numRows) + + col.map { item => + val obj: Object = item.asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def saveMode(mode: String): SaveMode = { + mode match { + case "append" => SaveMode.Append + case "overwrite" => SaveMode.Overwrite + case "error" => SaveMode.ErrorIfExists + case "ignore" => SaveMode.Ignore + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index f615fb33a7c35..64449b2659b4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -61,7 +61,7 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( protected def underlyingBuffer = buffer } -private[sql] abstract class NativeColumnAccessor[T <: NativeType]( +private[sql] abstract class NativeColumnAccessor[T <: AtomicType]( override protected val buffer: ByteBuffer, override protected val columnType: NativeColumnType[T]) extends BasicColumnAccessor(buffer, columnType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c881747751520..aa10af400c815 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -84,10 +84,10 @@ private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( extends BasicColumnBuilder[T, JvmType](columnStats, columnType) with NullableColumnBuilder -private[sql] abstract class NativeColumnBuilder[T <: NativeType]( +private[sql] abstract class NativeColumnBuilder[T <: AtomicType]( override val columnStats: ColumnStats, override val columnType: NativeColumnType[T]) - extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) + extends BasicColumnBuilder[T, T#InternalType](columnStats, columnType) with NullableColumnBuilder with AllCompressionSchemes with CompressibleColumnBuilder[T] @@ -153,6 +153,7 @@ private[sql] object ColumnBuilder { val builder: ColumnBuilder = dataType match { case IntegerType => new IntColumnBuilder case LongType => new LongColumnBuilder + case FloatType => new FloatColumnBuilder case DoubleType => new DoubleColumnBuilder case BooleanType => new BooleanColumnBuilder case ByteType => new ByteColumnBuilder diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 87a6631da8300..b0f983c180673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -216,13 +216,13 @@ private[sql] class IntColumnStats extends ColumnStats { } private[sql] class StringColumnStats extends ColumnStats { - protected var upper: String = null - protected var lower: String = null + protected var upper: UTF8String = null + protected var lower: UTF8String = null override def gatherStats(row: Row, ordinal: Int): Unit = { super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { - val value = row.getString(ordinal) + val value = row(ordinal).asInstanceOf[UTF8String] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value sizeInBytes += STRING.actualSize(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index c47497e0662d9..20be5ca9d0046 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag @@ -101,16 +101,16 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( override def toString: String = getClass.getSimpleName.stripSuffix("$") } -private[sql] abstract class NativeColumnType[T <: NativeType]( +private[sql] abstract class NativeColumnType[T <: AtomicType]( val dataType: T, typeId: Int, defaultSize: Int) - extends ColumnType[T, T#JvmType](typeId, defaultSize) { + extends ColumnType[T, T#InternalType](typeId, defaultSize) { /** * Scala TypeTag. Can be used to create primitive arrays and hash tables. */ - def scalaTag: TypeTag[dataType.JvmType] = dataType.tag + def scalaTag: TypeTag[dataType.InternalType] = dataType.tag } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { @@ -312,26 +312,28 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { row.getString(ordinal).getBytes("utf-8").length + 4 } - override def append(v: String, buffer: ByteBuffer): Unit = { - val stringBytes = v.getBytes("utf-8") + override def append(v: UTF8String, buffer: ByteBuffer): Unit = { + val stringBytes = v.getBytes buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } - override def extract(buffer: ByteBuffer): String = { + override def extract(buffer: ByteBuffer): UTF8String = { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes, "utf-8") + UTF8String(stringBytes) } - override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { - row.setString(ordinal, value) + override def setField(row: MutableRow, ordinal: Int, value: UTF8String): Unit = { + row.update(ordinal, value) } - override def getField(row: Row, ordinal: Int): String = row.getString(ordinal) + override def getField(row: Row, ordinal: Int): UTF8String = { + row(ordinal).asInstanceOf[UTF8String] + } override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { - to.setString(toOrdinal, from.getString(fromOrdinal)) + to.update(toOrdinal, from(fromOrdinal)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 6eee0c86d6a1c..d9b6fb43ab83d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import org.apache.spark.Accumulator +import org.apache.spark.{Accumulable, Accumulator, Accumulators} import org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -53,11 +55,16 @@ private[sql] case class InMemoryRelation( child: SparkPlan, tableName: Option[String])( private var _cachedColumnBuffers: RDD[CachedBatch] = null, - private var _statistics: Statistics = null) + private var _statistics: Statistics = null, + private var _batchStats: Accumulable[ArrayBuffer[Row], Row] = null) extends LogicalPlan with MultiInstanceRelation { - private val batchStats = - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + private val batchStats: Accumulable[ArrayBuffer[Row], Row] = + if (_batchStats == null) { + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + } else { + _batchStats + } val partitionStatistics = new PartitionStatistics(output) @@ -161,7 +168,7 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated) + _cachedColumnBuffers, statisticsToBePropagated, batchStats) } override def children: Seq[LogicalPlan] = Seq.empty @@ -175,13 +182,20 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - statisticsToBePropagated).asInstanceOf[this.type] + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated) + Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) + + private[sql] def uncache(blocking: Boolean): Unit = { + Accumulators.remove(batchStats.id) + cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } } private[sql] case class InMemoryColumnarTableScan( @@ -244,15 +258,20 @@ private[sql] case class InMemoryColumnarTableScan( } } + lazy val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + // Accumulators used for testing purposes - val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) - val readBatches: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0) private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning override def execute(): RDD[Row] = { - readPartitions.setValue(0) - readBatches.setValue(0) + if (enableAccumulators) { + readPartitions.setValue(0) + readBatches.setValue(0) + } relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( @@ -302,7 +321,7 @@ private[sql] case class InMemoryColumnarTableScan( } } - if (rows.hasNext) { + if (rows.hasNext && enableAccumulators) { readPartitions += 1 } @@ -321,7 +340,9 @@ private[sql] case class InMemoryColumnarTableScan( logInfo(s"Skipping partition based on stats $statsString") false } else { - readBatches += 1 + if (enableAccumulators) { + readBatches += 1 + } true } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index d0b602a834dfe..cb205defbb1ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { +private[sql] trait CompressibleColumnAccessor[T <: AtomicType] extends ColumnAccessor { this: NativeColumnAccessor[T] => private var decoder: Decoder[T] = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index b9cfc5df550d1..8e2a1af6dae78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -22,7 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType /** * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.NativeType * header body * }}} */ -private[sql] trait CompressibleColumnBuilder[T <: NativeType] +private[sql] trait CompressibleColumnBuilder[T <: AtomicType] extends ColumnBuilder with Logging { this: NativeColumnBuilder[T] with WithCompressionSchemes => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index 879d29bcfa6f6..17c2d9b111188 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -22,9 +22,9 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -private[sql] trait Encoder[T <: NativeType] { +private[sql] trait Encoder[T <: AtomicType] { def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} def compressedSize: Int @@ -38,7 +38,7 @@ private[sql] trait Encoder[T <: NativeType] { def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer } -private[sql] trait Decoder[T <: NativeType] { +private[sql] trait Decoder[T <: AtomicType] { def next(row: MutableRow, ordinal: Int): Unit def hasNext: Boolean @@ -49,9 +49,9 @@ private[sql] trait CompressionScheme { def supports(columnType: ColumnType[_, _]): Boolean - def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] + def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] - def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] + def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } private[sql] trait WithCompressionSchemes { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 8727d71c48bb7..534ae90ddbc8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -35,16 +35,16 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]): Boolean = true - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType]( + override def decoder[T <: AtomicType]( buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { override def uncompressedSize: Int = 0 override def compressedSize: Int = 0 @@ -56,7 +56,7 @@ private[sql] case object PassThrough extends CompressionScheme { } } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { override def next(row: MutableRow, ordinal: Int): Unit = { @@ -70,11 +70,11 @@ private[sql] case object PassThrough extends CompressionScheme { private[sql] case object RunLengthEncoding extends CompressionScheme { override val typeId = 1 - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } - override def decoder[T <: NativeType]( + override def decoder[T <: AtomicType]( buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] = { new this.Decoder(buffer, columnType) } @@ -84,7 +84,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 @@ -152,12 +152,12 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { private var run = 0 private var valueCount = 0 - private var currentValue: T#JvmType = _ + private var currentValue: T#InternalType = _ override def next(row: MutableRow, ordinal: Int): Unit = { if (valueCount == run) { @@ -181,12 +181,12 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // 32K unique values allowed val MAX_DICT_SIZE = Short.MaxValue - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : Decoder[T] = { new this.Decoder(buffer, columnType) } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): Encoder[T] = { new this.Encoder[T](columnType) } @@ -195,7 +195,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { + class Encoder[T <: AtomicType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -208,7 +208,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private var count = 0 // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. - private var values = new mutable.ArrayBuffer[T#JvmType](1024) + private var values = new mutable.ArrayBuffer[T#InternalType](1024) // The dictionary that maps a value to the encoded short integer. private val dictionary = mutable.HashMap.empty[Any, Short] @@ -268,14 +268,14 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { override def compressedSize: Int = if (overflow) Int.MaxValue else dictionarySize + count * 2 } - class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { private val dictionary = { // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) + ClassTag[T#InternalType](mirror.runtimeClass(columnType.scalaTag.tpe)) } Array.fill(buffer.getInt()) { @@ -296,12 +296,12 @@ private[sql] case object BooleanBitSet extends CompressionScheme { val BITS_PER_LONG = 64 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new this.Encoder).asInstanceOf[compression.Encoder[T]] } @@ -384,12 +384,12 @@ private[sql] case object BooleanBitSet extends CompressionScheme { private[sql] case object IntDelta extends CompressionScheme { override def typeId: Int = 4 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new Decoder(buffer, INT).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } @@ -464,12 +464,12 @@ private[sql] case object IntDelta extends CompressionScheme { private[sql] case object LongDelta extends CompressionScheme { override def typeId: Int = 5 - override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + override def decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) : compression.Decoder[T] = { new Decoder(buffer, LONG).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType](columnType: NativeColumnType[T]): compression.Encoder[T] = { + override def encoder[T <: AtomicType](columnType: NativeColumnType[T]): compression.Encoder[T] = { (new Encoder).asInstanceOf[compression.Encoder[T]] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 437408d30bfd2..5b2e46962cd3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -19,24 +19,44 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} +import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner} import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.serializer.Serializer import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types.DataType import org.apache.spark.util.MutablePair +object Exchange { + /** + * Returns true when the ordering expressions are a subset of the key. + * if true, ShuffledRDD can use `setKeyOrdering(orderingKey)` to sort within [[Exchange]]. + */ + def canSortWithShuffle(partitioning: Partitioning, desiredOrdering: Seq[SortOrder]): Boolean = { + desiredOrdering.map(_.child).toSet.subsetOf(partitioning.keyExpressions.toSet) + } +} + /** * :: DeveloperApi :: + * Performs a shuffle that will result in the desired `newPartitioning`. Optionally sorts each + * resulting partition based on expressions from the partition key. It is invalid to construct an + * exchange operator with a `newOrdering` that cannot be calculated using the partitioning key. */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { +case class Exchange( + newPartitioning: Partitioning, + newOrdering: Seq[SortOrder], + child: SparkPlan) + extends UnaryNode { override def outputPartitioning: Partitioning = newPartitioning + override def outputOrdering: Seq[SortOrder] = newOrdering + override def output: Seq[Attribute] = child.output /** We must copy rows when sort based shuffle is on */ @@ -45,6 +65,61 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una private val bypassMergeThreshold = child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + private val keyOrdering = { + if (newOrdering.nonEmpty) { + val key = newPartitioning.keyExpressions + val boundOrdering = newOrdering.map { o => + val ordinal = key.indexOf(o.child) + if (ordinal == -1) sys.error(s"Invalid ordering on $o requested for $newPartitioning") + o.copy(child = BoundReference(ordinal, o.child.dataType, o.child.nullable)) + } + new RowOrdering(boundOrdering) + } else { + null // Ordering will not be used + } + } + + @transient private lazy val sparkConf = child.sqlContext.sparkContext.getConf + + def serializer( + keySchema: Array[DataType], + valueSchema: Array[DataType], + numPartitions: Int): Serializer = { + // In ExternalSorter's spillToMergeableFile function, key-value pairs are written out + // through write(key) and then write(value) instead of write((key, value)). Because + // SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use + // it when spillToMergeableFile in ExternalSorter will be used. + // So, we will not use SparkSqlSerializer2 when + // - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater + // then the bypassMergeThreshold; or + // - newOrdering is defined. + val cannotUseSqlSerializer2 = + (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) || newOrdering.nonEmpty + + // It is true when there is no field that needs to be write out. + // For now, we will not use SparkSqlSerializer2 when noField is true. + val noField = + (keySchema == null || keySchema.length == 0) && + (valueSchema == null || valueSchema.length == 0) + + val useSqlSerializer2 = + child.sqlContext.conf.useSqlSerializer2 && // SparkSqlSerializer2 is enabled. + !cannotUseSqlSerializer2 && // Safe to use Serializer2. + SparkSqlSerializer2.support(keySchema) && // The schema of key is supported. + SparkSqlSerializer2.support(valueSchema) && // The schema of value is supported. + !noField + + val serializer = if (useSqlSerializer2) { + logInfo("Using SparkSqlSerializer2.") + new SparkSqlSerializer2(keySchema, valueSchema) + } else { + logInfo("Using SparkSqlSerializer.") + new SparkSqlSerializer(sparkConf) + } + + serializer + } + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -56,7 +131,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una // we can avoid the defensive copies to improve performance. In the long run, we probably // want to include information in shuffle dependencies to indicate whether elements in the // source RDD should be copied. - val rdd = if (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) { + val willMergeSort = sortBasedShuffleOn && numPartitions > bypassMergeThreshold + + val rdd = if (willMergeSort || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -69,12 +146,20 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una } } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Row, Row](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Row, Row](rdd, part) + } + val keySchema = expressions.map(_.dataType).toArray + val valueSchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) + shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - val rdd = if (sortBasedShuffleOn) { + val rdd = if (sortBasedShuffleOn || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} } else { child.execute().mapPartitions { iter => @@ -87,8 +172,14 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Null, Null](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Null, Null](rdd, part) + } + val keySchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(keySchema, null, numPartitions)) shuffled.map(_._1) @@ -107,7 +198,8 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + val valueSchema = child.output.map(_.dataType).toArray + shuffled.setSerializer(serializer(null, valueSchema, 1)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") @@ -120,27 +212,34 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] * of input data meets the * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[Exchange]] Operators where required. + * each operator by inserting [[Exchange]] Operators where required. Also ensure that the + * required input partition ordering requirements are met. */ -private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { +private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. def numPartitions: Int = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => - // Check if every child's outputPartitioning satisfies the corresponding + // True iff every child's outputPartitioning satisfies the corresponding // required data distribution. def meetsRequirements: Boolean = - !operator.requiredChildDistribution.zip(operator.children).map { + operator.requiredChildDistribution.zip(operator.children).forall { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid - }.exists(!_) + } - // Check if outputPartitionings of children are compatible with each other. + // True iff any of the children are incorrectly sorted. + def needsAnySort: Boolean = + operator.requiredChildOrdering.zip(operator.children).exists { + case (required, child) => required.nonEmpty && required != child.outputOrdering + } + + // True iff outputPartitionings of children are compatible with each other. // It is possible that every child satisfies its required data distribution // but two children have incompatible outputPartitionings. For example, // A dataset is range partitioned by "a.asc" (RangePartitioning) and another @@ -157,28 +256,69 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl case Seq(a,b) => a compatibleWith b }.exists(!_) - // Check if the partitioning we want to ensure is the same as the child's output - // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan): SparkPlan = - if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child + // Adds Exchange or Sort operators as required + def addOperatorsIfNecessary( + partitioning: Partitioning, + rowOrdering: Seq[SortOrder], + child: SparkPlan): SparkPlan = { + val needSort = rowOrdering.nonEmpty && child.outputOrdering != rowOrdering + val needsShuffle = child.outputPartitioning != partitioning + val canSortWithShuffle = Exchange.canSortWithShuffle(partitioning, rowOrdering) + + if (needSort && needsShuffle && canSortWithShuffle) { + Exchange(partitioning, rowOrdering, child) + } else { + val withShuffle = if (needsShuffle) { + Exchange(partitioning, Nil, child) + } else { + child + } + + val withSort = if (needSort) { + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, withShuffle) + } else { + Sort(rowOrdering, global = false, withShuffle) + } + } else { + withShuffle + } + + withSort + } + } - if (meetsRequirements && compatible) { + if (meetsRequirements && compatible && !needsAnySort) { operator } else { // At least one child does not satisfies its required data distribution or // at least one child's outputPartitioning is not compatible with another child's // outputPartitioning. In this case, we need to add Exchange operators. - val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { - case (AllTuples, child) => - addExchangeIfNecessary(SinglePartition, child) - case (ClusteredDistribution(clustering), child) => - addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) - case (OrderedDistribution(ordering), child) => - addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) - case (UnspecifiedDistribution, child) => child - case (dist, _) => sys.error(s"Don't know how to ensure $dist") + val requirements = + (operator.requiredChildDistribution, operator.requiredChildOrdering, operator.children) + + val fixedChildren = requirements.zipped.map { + case (AllTuples, rowOrdering, child) => + addOperatorsIfNecessary(SinglePartition, rowOrdering, child) + case (ClusteredDistribution(clustering), rowOrdering, child) => + addOperatorsIfNecessary(HashPartitioning(clustering, numPartitions), rowOrdering, child) + case (OrderedDistribution(ordering), rowOrdering, child) => + addOperatorsIfNecessary(RangePartitioning(ordering, numPartitions), rowOrdering, child) + + case (UnspecifiedDistribution, Seq(), child) => + child + case (UnspecifiedDistribution, rowOrdering, child) => + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, child) + } else { + Sort(rowOrdering, global = false, child) + } + + case (dist, ordering, _) => + sys.error(s"Don't know how to ensure $dist with ordering $ordering") } - operator.withNewChildren(repartitionedChildren) + + operator.withNewChildren(fixedChildren) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index d8955725e59b1..57effbf7ec501 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType - -import scala.collection.immutable +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -39,13 +37,42 @@ object RDDConversions { Iterator.empty } else { val bufferedIterator = iterator.buffered - val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) + val mutableRow = new SpecificMutableRow(schema.fields.map(_.dataType)) val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = converters(i)(r.productElement(i)) + i += 1 + } + + mutableRow + } + } + } + } + + /** + * Convert the objects inside Row into the types Catalyst expected. + */ + def rowToRowRdd(data: RDD[Row], schema: StructType): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.toSeq.toArray) + val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(r.productElement(i), schemaFields(i).dataType) + mutableRow(i) = converters(i)(r(i)) i += 1 } @@ -57,7 +84,7 @@ object RDDConversions { } /** Logical plan node for scanning data from an RDD. */ -case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) +private[sql] case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) extends LogicalPlan with MultiInstanceRelation { override def children: Seq[LogicalPlan] = Nil @@ -78,11 +105,12 @@ case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLCont } /** Physical plan node for scanning data from an RDD. */ -case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { +private[sql] case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute(): RDD[Row] = rdd } /** Logical plan node for scanning data from a local collection. */ +private[sql] case class LogicalLocalTable(output: Seq[Attribute], rows: Seq[Row])(sqlContext: SQLContext) extends LogicalPlan with MultiInstanceRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 12271048bb39c..5201e20a10565 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -27,44 +27,34 @@ import org.apache.spark.sql.catalyst.expressions._ * output of each into a new stream of rows. This operation is similar to a `flatMap` in functional * programming with one important additional feature, which allows the input rows to be joined with * their output. + * @param generator the generator expression * @param join when true, each output row is implicitly joined with the input tuple that produced * it. * @param outer when true, each input row will be output at least once, even if the output of the * given `generator` is empty. `outer` has no effect when `join` is false. + * @param output the output attributes of this node, which constructed in analysis phase, + * and we can not change it, as the parent node bound with it already. */ @DeveloperApi case class Generate( generator: Generator, join: Boolean, outer: Boolean, + output: Seq[Attribute], child: SparkPlan) extends UnaryNode { - // This must be a val since the generator output expr ids are not preserved by serialization. - protected val generatorOutput: Seq[Attribute] = { - if (join && outer) { - generator.output.map(_.withNullability(true)) - } else { - generator.output - } - } - - // This must be a val since the generator output expr ids are not preserved by serialization. - override val output = - if (join) child.output ++ generatorOutput else generatorOutput - val boundGenerator = BindReferences.bindReference(generator, child.output) override def execute(): RDD[Row] = { if (join) { child.execute().mapPartitions { iter => - val nullValues = Seq.fill(generator.output.size)(Literal(null)) + val nullValues = Seq.fill(generator.elementTypes.size)(Literal(null)) // Used to produce rows with no matches when outer = true. val outerProjection = newProjection(child.output ++ nullValues, child.output) - val joinProjection = - newProjection(child.output ++ generatorOutput, child.output ++ generatorOutput) + val joinProjection = newProjection(output, output) val joinedRow = new JoinedRow iter.flatMap {row => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 89682d25ca7dc..5d9f202681045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.TaskContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.trees._ @@ -40,6 +41,7 @@ case class AggregateEvaluation( * ensure all values where `groupingExpressions` are equal are present. * @param groupingExpressions expressions that are evaluated to determine grouping. * @param aggregateExpressions expressions that are computed for each group. + * @param unsafeEnabled whether to allow Unsafe-based aggregation buffers to be used. * @param child the input data source. */ @DeveloperApi @@ -47,6 +49,7 @@ case class GeneratedAggregate( partial: Boolean, groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], + unsafeEnabled: Boolean, child: SparkPlan) extends UnaryNode { @@ -68,6 +71,8 @@ case class GeneratedAggregate( a.collect { case agg: AggregateExpression => agg} } + // If you add any new function support, please add tests in org.apache.spark.sql.SQLQuerySuite + // (in test "aggregation with codegen"). val computeFunctions = aggregatesToCompute.map { case c @ Count(expr) => // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its @@ -93,13 +98,16 @@ case class GeneratedAggregate( } val currentSum = AttributeReference("currentSum", calcType, nullable = true)() - val initialValue = Literal(null, calcType) + val initialValue = Literal.create(null, calcType) - // Coalasce avoids double calculation... + // Coalesce avoids double calculation... // but really, common sub expression elimination would be better.... val zero = Cast(Literal(0), calcType) val updateFunction = Coalesce( - Add(Coalesce(currentSum :: zero :: Nil), Cast(expr, calcType)) :: currentSum :: Nil) + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType) + ) :: currentSum :: zero :: Nil) val result = expr.dataType match { case DecimalType.Fixed(_, _) => @@ -109,8 +117,8 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) - case a @ Average(expr) => - val calcType = + case cs @ CombineSum(expr) => + val calcType = expr.dataType expr.dataType match { case DecimalType.Fixed(_, _) => DecimalType.Unlimited @@ -118,45 +126,39 @@ case class GeneratedAggregate( expr.dataType } - val currentCount = AttributeReference("currentCount", LongType, nullable = false)() - val currentSum = AttributeReference("currentSum", calcType, nullable = false)() - val initialCount = Literal(0L) - val initialSum = Cast(Literal(0L), calcType) + val currentSum = AttributeReference("currentSum", calcType, nullable = true)() + val initialValue = Literal.create(null, calcType) + // Coalasce avoids double calculation... + // but really, common sub expression elimination would be better.... + val zero = Cast(Literal(0), calcType) // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its // UnscaledValue will be null if and only if x is null; helps with Average on decimals - val toCount = expr match { + val actualExpr = expr match { case UnscaledValue(e) => e case _ => expr } - - val updateCount = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) - val updateSum = Coalesce(Add(Cast(expr, calcType), currentSum) :: currentSum :: Nil) - + // partial sum result can be null only when no input rows present + val updateFunction = If( + IsNotNull(actualExpr), + Coalesce( + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType)) :: currentSum :: zero :: Nil), + currentSum) + val result = expr.dataType match { case DecimalType.Fixed(_, _) => - If(EqualTo(currentCount, Literal(0L)), - Literal(null, a.dataType), - Cast(Divide( - Cast(currentSum, DecimalType.Unlimited), - Cast(currentCount, DecimalType.Unlimited)), a.dataType)) - case _ => - If(EqualTo(currentCount, Literal(0L)), - Literal(null, a.dataType), - Divide(Cast(currentSum, a.dataType), Cast(currentCount, a.dataType))) + Cast(currentSum, cs.dataType) + case _ => currentSum } - AggregateEvaluation( - currentCount :: currentSum :: Nil, - initialCount :: initialSum :: Nil, - updateCount :: updateSum :: Nil, - result - ) - + AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + case m @ Max(expr) => val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() - val initialValue = Literal(null, expr.dataType) + val initialValue = Literal.create(null, expr.dataType) val updateMax = MaxOf(currentMax, expr) AggregateEvaluation( @@ -165,8 +167,20 @@ case class GeneratedAggregate( updateMax :: Nil, currentMax) + case m @ Min(expr) => + val currentMin = AttributeReference("currentMin", expr.dataType, nullable = true)() + val initialValue = Literal.create(null, expr.dataType) + val updateMin = MinOf(currentMin, expr) + + AggregateEvaluation( + currentMin :: Nil, + initialValue :: Nil, + updateMin :: Nil, + currentMin) + case CollectHashSet(Seq(expr)) => - val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val set = + AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() val initialValue = NewSet(expr.dataType) val addToSet = AddItemToSet(expr, set) @@ -177,9 +191,10 @@ case class GeneratedAggregate( set) case CombineSetsAndCount(inputSet) => - val ArrayType(inputType, _) = inputSet.dataType - val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() - val initialValue = NewSet(inputType) + val elementType = inputSet.dataType.asInstanceOf[OpenHashSetUDT].elementType + val set = + AttributeReference("hashSet", new OpenHashSetUDT(elementType), nullable = false)() + val initialValue = NewSet(elementType) val collectSets = CombineSets(set, inputSet) AggregateEvaluation( @@ -187,6 +202,8 @@ case class GeneratedAggregate( initialValue :: Nil, collectSets :: Nil, CountSet(set)) + + case o => sys.error(s"$o can't be codegened.") } val computationSchema = computeFunctions.flatMap(_.schema) @@ -211,6 +228,21 @@ case class GeneratedAggregate( case e: Expression if groupMap.contains(e) => groupMap(e) }) + val aggregationBufferSchema: StructType = StructType.fromAttributes(computationSchema) + + val groupKeySchema: StructType = { + val fields = groupingExpressions.zipWithIndex.map { case (expr, idx) => + // This is a dummy field name + StructField(idx.toString, expr.dataType, expr.nullable) + } + StructType(fields) + } + + val schemaSupportsUnsafe: Boolean = { + UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema(aggregationBufferSchema) && + UnsafeFixedWidthAggregationMap.supportsGroupKeySchema(groupKeySchema) + } + child.execute().mapPartitions { iter => // Builds a new custom class for holding the results of aggregation for a group. val initialValues = computeFunctions.flatMap(_.initialValues) @@ -251,7 +283,49 @@ case class GeneratedAggregate( val resultProjection = resultProjectionBuilder() Iterator(resultProjection(buffer)) + } else if (unsafeEnabled && schemaSupportsUnsafe) { + log.info("Using Unsafe-based aggregator") + val aggregationMap = new UnsafeFixedWidthAggregationMap( + newAggregationBuffer(EmptyRow), + aggregationBufferSchema, + groupKeySchema, + TaskContext.get.taskMemoryManager(), + 1024 * 16, // initial capacity + false // disable tracking of performance metrics + ) + + while (iter.hasNext) { + val currentRow: Row = iter.next() + val groupKey: Row = groupProjection(currentRow) + val aggregationBuffer = aggregationMap.getAggregationBuffer(groupKey) + updateProjection.target(aggregationBuffer)(joinedRow(aggregationBuffer, currentRow)) + } + + new Iterator[Row] { + private[this] val mapIterator = aggregationMap.iterator() + private[this] val resultProjection = resultProjectionBuilder() + + def hasNext: Boolean = mapIterator.hasNext + + def next(): Row = { + val entry = mapIterator.next() + val result = resultProjection(joinedRow(entry.key, entry.value)) + if (hasNext) { + result + } else { + // This is the last element in the iterator, so let's free the buffer. Before we do, + // though, we need to make a defensive copy of the result so that we don't return an + // object that might contain dangling pointers to the freed memory + val resultCopy = result.copy() + aggregationMap.free() + resultCopy + } + } + } } else { + if (unsafeEnabled) { + log.info("Not using Unsafe-based aggregator because it is not supported for this schema") + } val buffers = new java.util.HashMap[Row, MutableRow]() var currentRow: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala index 5bd699a2fa949..ace9af5f384c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -19,22 +19,28 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.Attribute /** * Physical plan node for scanning data from a local collection. */ -case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNode { +private[sql] case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNode { private lazy val rdd = sqlContext.sparkContext.parallelize(rows) override def execute(): RDD[Row] = rdd - override def executeCollect(): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray - override def executeTake(limit: Int): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).toArray + } + + + override def executeTake(limit: Int): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).take(limit).toArray + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index d239637cd4b4e..59c89800da00f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.{ScalaReflection, trees} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan @@ -72,6 +72,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) + /** Specifies how data is ordered in each partition. */ + def outputOrdering: Seq[SortOrder] = Nil + + /** Specifies sort order for each partition requirements on the input data for this operator. */ + def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) + /** * Runs this query returning the result as an RDD. */ @@ -80,8 +86,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ + def executeCollect(): Array[Row] = { - execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() + execute().mapPartitions { iter => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + iter.map(converter(_).asInstanceOf[Row]) + }.collect() } /** @@ -125,7 +135,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ partsScanned += numPartsToTry } - buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) + val converter = CatalystTypeConverters.createToScalaConverter(schema) + buf.toArray.map(converter(_).asInstanceOf[Row]) } protected def newProjection( @@ -133,7 +144,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ log.debug( s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") if (codegenEnabled) { - GenerateProjection(expressions, inputSchema) + GenerateProjection.generate(expressions, inputSchema) } else { new InterpretedProjection(expressions, inputSchema) } @@ -145,7 +156,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ log.debug( s"Creating MutableProj: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") if(codegenEnabled) { - GenerateMutableProjection(expressions, inputSchema) + GenerateMutableProjection.generate(expressions, inputSchema) } else { () => new InterpretedMutableProjection(expressions, inputSchema) } @@ -155,15 +166,15 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def newPredicate( expression: Expression, inputSchema: Seq[Attribute]): (Row) => Boolean = { if (codegenEnabled) { - GeneratePredicate(expression, inputSchema) + GeneratePredicate.generate(expression, inputSchema) } else { - InterpretedPredicate(expression, inputSchema) + InterpretedPredicate.create(expression, inputSchema) } } protected def newOrdering(order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[Row] = { if (codegenEnabled) { - GenerateOrdering(order, inputSchema) + GenerateOrdering.generate(order, inputSchema) } else { new RowOrdering(order, inputSchema) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index c4534fd5f67e4..eea15aff5dbcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.types.Decimal @@ -26,20 +27,19 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} -import com.twitter.chill.{AllScalaRegistrar, ResourcePool} +import com.twitter.chill.ResourcePool import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair -import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { - val kryo = new Kryo() + val kryo = super.newKryo() kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) @@ -55,10 +55,9 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], new OpenHashSetSerializer) kryo.register(classOf[Decimal]) + kryo.register(classOf[JavaHashMap[_, _]]) kryo.setReferences(false) - kryo.setClassLoader(Utils.getSparkClassLoader) - new AllScalaRegistrar().apply(kryo) kryo } } @@ -66,12 +65,9 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co private[execution] class KryoResourcePool(size: Int) extends ResourcePool[SerializerInstance](size) { - val ser: KryoSerializer = { + val ser: SparkSqlSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - new KryoSerializer(sparkConf) + new SparkSqlSerializer(sparkConf) } def newInstance(): SerializerInstance = ser.newInstance() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala new file mode 100644 index 0000000000000..9552f41115866 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer2.scala @@ -0,0 +1,441 @@ +/* + * 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.sql.execution + +import java.io._ +import java.math.{BigDecimal, BigInteger} +import java.nio.ByteBuffer +import java.sql.Timestamp + +import scala.reflect.ClassTag + +import org.apache.spark.serializer._ +import org.apache.spark.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow +import org.apache.spark.sql.types._ + +/** + * The serialization stream for [[SparkSqlSerializer2]]. It assumes that the object passed in + * its `writeObject` are [[Product2]]. The serialization functions for the key and value of the + * [[Product2]] are constructed based on their schemata. + * The benefit of this serialization stream is that compared with general-purpose serializers like + * Kryo and Java serializer, it can significantly reduce the size of serialized and has a lower + * allocation cost, which can benefit the shuffle operation. Right now, its main limitations are: + * 1. It does not support complex types, i.e. Map, Array, and Struct. + * 2. It assumes that the objects passed in are [[Product2]]. So, it cannot be used when + * [[org.apache.spark.util.collection.ExternalSorter]]'s merge sort operation is used because + * the objects passed in the serializer are not in the type of [[Product2]]. Also also see + * the comment of the `serializer` method in [[Exchange]] for more information on it. + */ +private[sql] class Serializer2SerializationStream( + keySchema: Array[DataType], + valueSchema: Array[DataType], + out: OutputStream) + extends SerializationStream with Logging { + + val rowOut = new DataOutputStream(out) + val writeKeyFunc = SparkSqlSerializer2.createSerializationFunction(keySchema, rowOut) + val writeValueFunc = SparkSqlSerializer2.createSerializationFunction(valueSchema, rowOut) + + override def writeObject[T: ClassTag](t: T): SerializationStream = { + val kv = t.asInstanceOf[Product2[Row, Row]] + writeKey(kv._1) + writeValue(kv._2) + + this + } + + override def writeKey[T: ClassTag](t: T): SerializationStream = { + writeKeyFunc(t.asInstanceOf[Row]) + this + } + + override def writeValue[T: ClassTag](t: T): SerializationStream = { + writeValueFunc(t.asInstanceOf[Row]) + this + } + + def flush(): Unit = { + rowOut.flush() + } + + def close(): Unit = { + rowOut.close() + } +} + +/** + * The corresponding deserialization stream for [[Serializer2SerializationStream]]. + */ +private[sql] class Serializer2DeserializationStream( + keySchema: Array[DataType], + valueSchema: Array[DataType], + in: InputStream) + extends DeserializationStream with Logging { + + val rowIn = new DataInputStream(new BufferedInputStream(in)) + + val key = if (keySchema != null) new SpecificMutableRow(keySchema) else null + val value = if (valueSchema != null) new SpecificMutableRow(valueSchema) else null + val readKeyFunc = SparkSqlSerializer2.createDeserializationFunction(keySchema, rowIn, key) + val readValueFunc = SparkSqlSerializer2.createDeserializationFunction(valueSchema, rowIn, value) + + override def readObject[T: ClassTag](): T = { + readKeyFunc() + readValueFunc() + + (key, value).asInstanceOf[T] + } + + override def readKey[T: ClassTag](): T = { + readKeyFunc() + key.asInstanceOf[T] + } + + override def readValue[T: ClassTag](): T = { + readValueFunc() + value.asInstanceOf[T] + } + + override def close(): Unit = { + rowIn.close() + } +} + +private[sql] class ShuffleSerializerInstance( + keySchema: Array[DataType], + valueSchema: Array[DataType]) + extends SerializerInstance { + + def serialize[T: ClassTag](t: T): ByteBuffer = + throw new UnsupportedOperationException("Not supported.") + + def deserialize[T: ClassTag](bytes: ByteBuffer): T = + throw new UnsupportedOperationException("Not supported.") + + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = + throw new UnsupportedOperationException("Not supported.") + + def serializeStream(s: OutputStream): SerializationStream = { + new Serializer2SerializationStream(keySchema, valueSchema, s) + } + + def deserializeStream(s: InputStream): DeserializationStream = { + new Serializer2DeserializationStream(keySchema, valueSchema, s) + } +} + +/** + * SparkSqlSerializer2 is a special serializer that creates serialization function and + * deserialization function based on the schema of data. It assumes that values passed in + * are key/value pairs and values returned from it are also key/value pairs. + * The schema of keys is represented by `keySchema` and that of values is represented by + * `valueSchema`. + */ +private[sql] class SparkSqlSerializer2(keySchema: Array[DataType], valueSchema: Array[DataType]) + extends Serializer + with Logging + with Serializable{ + + def newInstance(): SerializerInstance = new ShuffleSerializerInstance(keySchema, valueSchema) +} + +private[sql] object SparkSqlSerializer2 { + + final val NULL = 0 + final val NOT_NULL = 1 + + /** + * Check if rows with the given schema can be serialized with ShuffleSerializer. + */ + def support(schema: Array[DataType]): Boolean = { + if (schema == null) return true + + var i = 0 + while (i < schema.length) { + schema(i) match { + case udt: UserDefinedType[_] => return false + case array: ArrayType => return false + case map: MapType => return false + case struct: StructType => return false + case _ => + } + i += 1 + } + + return true + } + + /** + * The util function to create the serialization function based on the given schema. + */ + def createSerializationFunction(schema: Array[DataType], out: DataOutputStream): Row => Unit = { + (row: Row) => + // If the schema is null, the returned function does nothing when it get called. + if (schema != null) { + var i = 0 + while (i < schema.length) { + schema(i) match { + // When we write values to the underlying stream, we also first write the null byte + // first. Then, if the value is not null, we write the contents out. + + case NullType => // Write nothing. + + case BooleanType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeBoolean(row.getBoolean(i)) + } + + case ByteType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeByte(row.getByte(i)) + } + + case ShortType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeShort(row.getShort(i)) + } + + case IntegerType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeInt(row.getInt(i)) + } + + case LongType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeLong(row.getLong(i)) + } + + case FloatType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeFloat(row.getFloat(i)) + } + + case DoubleType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeDouble(row.getDouble(i)) + } + + case decimal: DecimalType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val value = row.apply(i).asInstanceOf[Decimal] + val javaBigDecimal = value.toJavaBigDecimal + // First, write out the unscaled value. + val bytes: Array[Byte] = javaBigDecimal.unscaledValue().toByteArray + out.writeInt(bytes.length) + out.write(bytes) + // Then, write out the scale. + out.writeInt(javaBigDecimal.scale()) + } + + case DateType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + out.writeInt(row.getAs[Int](i)) + } + + case TimestampType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val timestamp = row.getAs[java.sql.Timestamp](i) + val time = timestamp.getTime + val nanos = timestamp.getNanos + out.writeLong(time - (nanos / 1000000)) // Write the milliseconds value. + out.writeInt(nanos) // Write the nanoseconds part. + } + + case StringType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val bytes = row.getAs[UTF8String](i).getBytes + out.writeInt(bytes.length) + out.write(bytes) + } + + case BinaryType => + if (row.isNullAt(i)) { + out.writeByte(NULL) + } else { + out.writeByte(NOT_NULL) + val bytes = row.getAs[Array[Byte]](i) + out.writeInt(bytes.length) + out.write(bytes) + } + } + i += 1 + } + } + } + + /** + * The util function to create the deserialization function based on the given schema. + */ + def createDeserializationFunction( + schema: Array[DataType], + in: DataInputStream, + mutableRow: SpecificMutableRow): () => Unit = { + () => { + // If the schema is null, the returned function does nothing when it get called. + if (schema != null) { + var i = 0 + while (i < schema.length) { + schema(i) match { + // When we read values from the underlying stream, we also first read the null byte + // first. Then, if the value is not null, we update the field of the mutable row. + + case NullType => mutableRow.setNullAt(i) // Read nothing. + + case BooleanType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setBoolean(i, in.readBoolean()) + } + + case ByteType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setByte(i, in.readByte()) + } + + case ShortType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setShort(i, in.readShort()) + } + + case IntegerType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setInt(i, in.readInt()) + } + + case LongType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setLong(i, in.readLong()) + } + + case FloatType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setFloat(i, in.readFloat()) + } + + case DoubleType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.setDouble(i, in.readDouble()) + } + + case decimal: DecimalType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + // First, read in the unscaled value. + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + val unscaledVal = new BigInteger(bytes) + // Then, read the scale. + val scale = in.readInt() + // Finally, create the Decimal object and set it in the row. + mutableRow.update(i, Decimal(new BigDecimal(unscaledVal, scale))) + } + + case DateType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + mutableRow.update(i, in.readInt()) + } + + case TimestampType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val time = in.readLong() // Read the milliseconds value. + val nanos = in.readInt() // Read the nanoseconds part. + val timestamp = new Timestamp(time) + timestamp.setNanos(nanos) + mutableRow.update(i, timestamp) + } + + case StringType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + mutableRow.update(i, UTF8String(bytes)) + } + + case BinaryType => + if (in.readByte() == NULL) { + mutableRow.setNullAt(i) + } else { + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + mutableRow.update(i, bytes) + } + } + i += 1 + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 2b581152e5f77..56a4689eb58f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -90,6 +90,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { left.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) + // If the sort merge join option is set, we want to use sort merge join prior to hashjoin + // for now let's support inner join first, then add outer join + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.conf.sortMergeJoinEnabled => + val mergeJoin = + joins.SortMergeJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, mergeJoin)).getOrElse(mergeJoin) :: Nil + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { @@ -128,10 +136,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { partial = false, namedGroupingAttributes, rewrittenAggregateExpressions, + unsafeEnabled, execution.GeneratedAggregate( partial = true, groupingExpressions, partialComputation, + unsafeEnabled, planLater(child))) :: Nil // Cases where some aggregate can not be codegened @@ -155,7 +165,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: Min | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false @@ -211,9 +221,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil - case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => + val partitionColNames = relation.partitioningAttributes.map(_.name).toSet + val filtersToPush = filters.filter { pred => + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } val prunePushedDownFilters = if (sqlContext.conf.parquetFilterPushDown) { (predicates: Seq[Expression]) => { @@ -225,6 +241,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // "A AND B" in the higher-level filter, not just "B". predicates.map(p => p -> ParquetFilters.createFilter(p)).collect { case (predicate, None) => predicate + // Filter needs to be applied above when it contains partitioning + // columns + case (predicate, _) if(!predicate.references.map(_.name).toSet + .intersect (partitionColNames).isEmpty) => predicate } } } else { @@ -237,7 +257,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetTableScan( _, relation, - if (sqlContext.conf.parquetFilterPushDown) filters else Nil)) :: Nil + if (sqlContext.conf.parquetFilterPushDown) filtersToPush else Nil)) :: Nil case _ => Nil } @@ -265,7 +285,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Distinct(child) => execution.Distinct(partial = false, execution.Distinct(partial = true, planLater(child))) :: Nil - + case logical.Repartition(numPartitions, shuffle, child) => + execution.Repartition(numPartitions, shuffle, planLater(child)) :: Nil case logical.SortPartitions(sortExprs, child) => // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. @@ -282,8 +303,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Expand(projections, output, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil - case logical.Sample(fraction, withReplacement, seed, child) => - execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil + case logical.Window(projectList, windowExpressions, spec, child) => + execution.Window(projectList, windowExpressions, spec, planLater(child)) :: Nil + case logical.Sample(lb, ub, withReplacement, seed, child) => + execution.Sample(lb, ub, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => LocalTableScan(output, data) :: Nil case logical.Limit(IntegerLiteral(limit), child) => @@ -294,12 +317,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Except(planLater(left), planLater(right)) :: Nil case logical.Intersect(left, right) => execution.Intersect(planLater(left), planLater(right)) :: Nil - case logical.Generate(generator, join, outer, _, child) => - execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil - case logical.NoRelation => + case g @ logical.Generate(generator, join, outer, _, _, child) => + execution.Generate( + generator, join = join, outer = outer, g.output, planLater(child)) :: Nil + case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil - case logical.Repartition(expressions, child) => - execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + case logical.RepartitionByExpression(expressions, child) => + execution.Exchange( + HashPartitioning(expressions, numPartitions), Nil, planLater(child)) :: Nil case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala new file mode 100644 index 0000000000000..217b559def512 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -0,0 +1,480 @@ +/* + * 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.sql.execution + +import java.util + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, ClusteredDistribution, Partitioning} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * For every row, evaluates `windowExpression` containing Window Functions and attaches + * the results with other regular expressions (presented by `projectList`). + * Evert operator handles a single Window Specification, `windowSpec`. + */ +case class Window( + projectList: Seq[Attribute], + windowExpression: Seq[NamedExpression], + windowSpec: WindowSpecDefinition, + child: SparkPlan) + extends UnaryNode { + + override def output: Seq[Attribute] = + (projectList ++ windowExpression).map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + if (windowSpec.partitionSpec.isEmpty) { + // This operator will be very expensive. + AllTuples :: Nil + } else { + ClusteredDistribution(windowSpec.partitionSpec) :: Nil + } + + // Since window functions are adding columns to the input rows, the child's outputPartitioning + // is preserved. + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + // The required child ordering has two parts. + // The first part is the expressions in the partition specification. + // We add these expressions to the required ordering to make sure input rows are grouped + // based on the partition specification. So, we only need to process a single partition + // at a time. + // The second part is the expressions specified in the ORDER BY cluase. + // Basically, we first use sort to group rows based on partition specifications and then sort + // Rows in a group based on the order specification. + (windowSpec.partitionSpec.map(SortOrder(_, Ascending)) ++ windowSpec.orderSpec) :: Nil + } + + // Since window functions basically add columns to input rows, this operator + // will not change the ordering of input rows. + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + case class ComputedWindow( + unbound: WindowExpression, + windowFunction: WindowFunction, + resultAttribute: AttributeReference) + + // A list of window functions that need to be computed for each group. + private[this] val computedWindowExpressions = windowExpression.flatMap { window => + window.collect { + case w: WindowExpression => + ComputedWindow( + w, + BindReferences.bindReference(w.windowFunction, child.output), + AttributeReference(s"windowResult:$w", w.dataType, w.nullable)()) + } + }.toArray + + private[this] val windowFrame = + windowSpec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + + // Create window functions. + private[this] def windowFunctions(): Array[WindowFunction] = { + val functions = new Array[WindowFunction](computedWindowExpressions.length) + var i = 0 + while (i < computedWindowExpressions.length) { + functions(i) = computedWindowExpressions(i).windowFunction.newInstance() + functions(i).init() + i += 1 + } + functions + } + + // The schema of the result of all window function evaluations + private[this] val computedSchema = computedWindowExpressions.map(_.resultAttribute) + + private[this] val computedResultMap = + computedWindowExpressions.map { w => w.unbound -> w.resultAttribute }.toMap + + private[this] val windowExpressionResult = windowExpression.map { window => + window.transform { + case w: WindowExpression if computedResultMap.contains(w) => computedResultMap(w) + } + } + + def execute(): RDD[Row] = { + child.execute().mapPartitions { iter => + new Iterator[Row] { + + // Although input rows are grouped based on windowSpec.partitionSpec, we need to + // know when we have a new partition. + // This is to manually construct an ordering that can be used to compare rows. + // TODO: We may want to have a newOrdering that takes BoundReferences. + // So, we can take advantave of code gen. + private val partitionOrdering: Ordering[Row] = + RowOrdering.forSchema(windowSpec.partitionSpec.map(_.dataType)) + + // This is used to project expressions for the partition specification. + protected val partitionGenerator = + newMutableProjection(windowSpec.partitionSpec, child.output)() + + // This is ued to project expressions for the order specification. + protected val rowOrderGenerator = + newMutableProjection(windowSpec.orderSpec.map(_.child), child.output)() + + // The position of next output row in the inputRowBuffer. + var rowPosition: Int = 0 + // The number of buffered rows in the inputRowBuffer (the size of the current partition). + var partitionSize: Int = 0 + // The buffer used to buffer rows in a partition. + var inputRowBuffer: CompactBuffer[Row] = _ + // The partition key of the current partition. + var currentPartitionKey: Row = _ + // The partition key of next partition. + var nextPartitionKey: Row = _ + // The first row of next partition. + var firstRowInNextPartition: Row = _ + // Indicates if this partition is the last one in the iter. + var lastPartition: Boolean = false + + def createBoundaryEvaluator(): () => Unit = { + def findPhysicalBoundary( + boundary: FrameBoundary): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case CurrentRow => () => rowPosition + case ValuePreceding(value) => + () => + val newPosition = rowPosition - value + if (newPosition > 0) newPosition else 0 + case ValueFollowing(value) => + () => + val newPosition = rowPosition + value + if (newPosition < partitionSize) newPosition else partitionSize - 1 + } + + def findLogicalBoundary( + boundary: FrameBoundary, + searchDirection: Int, + evaluator: Expression, + joinedRow: JoinedRow): () => Int = boundary match { + case UnboundedPreceding => () => 0 + case UnboundedFollowing => () => partitionSize - 1 + case other => + () => { + // CurrentRow, ValuePreceding, or ValueFollowing. + var newPosition = rowPosition + searchDirection + var stopSearch = false + // rowOrderGenerator is a mutable projection. + // We need to make a copy of the returned by rowOrderGenerator since we will + // compare searched row with this currentOrderByValue. + val currentOrderByValue = rowOrderGenerator(inputRowBuffer(rowPosition)).copy() + while (newPosition >= 0 && newPosition < partitionSize && !stopSearch) { + val r = rowOrderGenerator(inputRowBuffer(newPosition)) + stopSearch = + !(evaluator.eval(joinedRow(currentOrderByValue, r)).asInstanceOf[Boolean]) + if (!stopSearch) { + newPosition += searchDirection + } + } + newPosition -= searchDirection + + if (newPosition < 0) { + 0 + } else if (newPosition >= partitionSize) { + partitionSize - 1 + } else { + newPosition + } + } + } + + windowFrame.frameType match { + case RowFrame => + val findStart = findPhysicalBoundary(windowFrame.frameStart) + val findEnd = findPhysicalBoundary(windowFrame.frameEnd) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + case RangeFrame => + val joinedRowForBoundaryEvaluation: JoinedRow = new JoinedRow() + val orderByExpr = windowSpec.orderSpec.head + val currentRowExpr = + BoundReference(0, orderByExpr.dataType, orderByExpr.nullable) + val examedRowExpr = + BoundReference(1, orderByExpr.dataType, orderByExpr.nullable) + val differenceExpr = Abs(Subtract(currentRowExpr, examedRowExpr)) + + val frameStartEvaluator = windowFrame.frameStart match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val frameEndEvaluator = windowFrame.frameEnd match { + case CurrentRow => EqualTo(currentRowExpr, examedRowExpr) + case ValuePreceding(value) => + GreaterThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case ValueFollowing(value) => + LessThanOrEqual(differenceExpr, Cast(Literal(value), orderByExpr.dataType)) + case o => Literal(true) // This is just a dummy expression, we will not use it. + } + + val findStart = + findLogicalBoundary( + boundary = windowFrame.frameStart, + searchDirection = -1, + evaluator = frameStartEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + val findEnd = + findLogicalBoundary( + boundary = windowFrame.frameEnd, + searchDirection = 1, + evaluator = frameEndEvaluator, + joinedRow = joinedRowForBoundaryEvaluation) + () => { + frameStart = findStart() + frameEnd = findEnd() + } + } + } + + val boundaryEvaluator = createBoundaryEvaluator() + // Indicates if we the specified window frame requires us to maintain a sliding frame + // (e.g. RANGES BETWEEN 1 PRECEDING AND CURRENT ROW) or the window frame + // is the entire partition (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING). + val requireUpdateFrame: Boolean = { + def requireUpdateBoundary(boundary: FrameBoundary): Boolean = boundary match { + case UnboundedPreceding => false + case UnboundedFollowing => false + case _ => true + } + + requireUpdateBoundary(windowFrame.frameStart) || + requireUpdateBoundary(windowFrame.frameEnd) + } + // The start position of the current frame in the partition. + var frameStart: Int = 0 + // The end position of the current frame in the partition. + var frameEnd: Int = -1 + // Window functions. + val functions: Array[WindowFunction] = windowFunctions() + // Buffers used to store input parameters for window functions. Because we may need to + // maintain a sliding frame, we use this buffer to avoid evaluate the parameters from + // the same row multiple times. + val windowFunctionParameterBuffers: Array[util.LinkedList[AnyRef]] = + functions.map(_ => new util.LinkedList[AnyRef]()) + + // The projection used to generate the final result rows of this operator. + private[this] val resultProjection = + newMutableProjection( + projectList ++ windowExpressionResult, + projectList ++ computedSchema)() + + // The row used to hold results of window functions. + private[this] val windowExpressionResultRow = + new GenericMutableRow(computedSchema.length) + + private[this] val joinedRow = new JoinedRow6 + + // Initialize this iterator. + initialize() + + private def initialize(): Unit = { + if (iter.hasNext) { + val currentRow = iter.next().copy() + // partitionGenerator is a mutable projection. Since we need to track nextPartitionKey, + // we are making a copy of the returned partitionKey at here. + nextPartitionKey = partitionGenerator(currentRow).copy() + firstRowInNextPartition = currentRow + fetchNextPartition() + } else { + // The iter is an empty one. So, we set all of the following variables + // to make sure hasNext will return false. + lastPartition = true + rowPosition = 0 + partitionSize = 0 + } + } + + // Indicates if we will have new output row. + override final def hasNext: Boolean = { + !lastPartition || (rowPosition < partitionSize) + } + + override final def next(): Row = { + if (hasNext) { + if (rowPosition == partitionSize) { + // All rows of this buffer have been consumed. + // We will move to next partition. + fetchNextPartition() + } + // Get the input row for the current output row. + val inputRow = inputRowBuffer(rowPosition) + // Get all results of the window functions for this output row. + var i = 0 + while (i < functions.length) { + windowExpressionResultRow.update(i, functions(i).get(rowPosition)) + i += 1 + } + + // Construct the output row. + val outputRow = resultProjection(joinedRow(inputRow, windowExpressionResultRow)) + // We will move to the next one. + rowPosition += 1 + if (requireUpdateFrame && rowPosition < partitionSize) { + // If we need to maintain a sliding frame and + // we will still work on this partition when next is called next time, do the update. + updateFrame() + } + + // Return the output row. + outputRow + } else { + // no more result + throw new NoSuchElementException + } + } + + // Fetch the next partition. + private def fetchNextPartition(): Unit = { + // Create a new buffer for input rows. + inputRowBuffer = new CompactBuffer[Row]() + // We already have the first row for this partition + // (recorded in firstRowInNextPartition). Add it back. + inputRowBuffer += firstRowInNextPartition + // Set the current partition key. + currentPartitionKey = nextPartitionKey + // Now, we will start to find all rows belonging to this partition. + // Create a variable to track if we see the next partition. + var findNextPartition = false + // The search will stop when we see the next partition or there is no + // input row left in the iter. + while (iter.hasNext && !findNextPartition) { + // Make a copy of the input row since we will put it in the buffer. + val currentRow = iter.next().copy() + // Get the partition key based on the partition specification. + // For the below compare method, we do not need to make a copy of partitionKey. + val partitionKey = partitionGenerator(currentRow) + // Check if the current row belongs the current input row. + val comparing = partitionOrdering.compare(currentPartitionKey, partitionKey) + if (comparing == 0) { + // This row is still in the current partition. + inputRowBuffer += currentRow + } else { + // The current input row is in a different partition. + findNextPartition = true + // partitionGenerator is a mutable projection. + // Since we need to track nextPartitionKey and we determine that it should be set + // as partitionKey, we are making a copy of the partitionKey at here. + nextPartitionKey = partitionKey.copy() + firstRowInNextPartition = currentRow + } + } + + // We have not seen a new partition. It means that there is no new row in the + // iter. The current partition is the last partition of the iter. + if (!findNextPartition) { + lastPartition = true + } + + // We have got all rows for the current partition. + // Set rowPosition to 0 (the next output row will be based on the first + // input row of this partition). + rowPosition = 0 + // The size of this partition. + partitionSize = inputRowBuffer.size + // Reset all parameter buffers of window functions. + var i = 0 + while (i < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(i).clear() + i += 1 + } + frameStart = 0 + frameEnd = -1 + // Create the first window frame for this partition. + // If we do not need to maintain a sliding frame, this frame will + // have the entire partition. + updateFrame() + } + + /** The function used to maintain the sliding frame. */ + private def updateFrame(): Unit = { + // Based on the difference between the new frame and old frame, + // updates the buffers holding input parameters of window functions. + // We will start to prepare input parameters starting from the row + // indicated by offset in the input row buffer. + def updateWindowFunctionParameterBuffers( + numToRemove: Int, + numToAdd: Int, + offset: Int): Unit = { + // First, remove unneeded entries from the head of every buffer. + var i = 0 + while (i < numToRemove) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + windowFunctionParameterBuffers(j).remove() + j += 1 + } + i += 1 + } + // Then, add needed entries to the tail of every buffer. + i = 0 + while (i < numToAdd) { + var j = 0 + while (j < windowFunctionParameterBuffers.length) { + // Ask the function to prepare the input parameters. + val parameters = functions(j).prepareInputParameters(inputRowBuffer(i + offset)) + windowFunctionParameterBuffers(j).add(parameters) + j += 1 + } + i += 1 + } + } + + // Record the current frame start point and end point before + // we update them. + val previousFrameStart = frameStart + val previousFrameEnd = frameEnd + boundaryEvaluator() + updateWindowFunctionParameterBuffers( + frameStart - previousFrameStart, + frameEnd - previousFrameEnd, + previousFrameEnd + 1) + // Evaluate the current frame. + evaluateCurrentFrame() + } + + /** Evaluate the current window frame. */ + private def evaluateCurrentFrame(): Unit = { + var i = 0 + while (i < functions.length) { + // Reset the state of the window function. + functions(i).reset() + // Get all buffered input parameters based on rows of this window frame. + val inputParameters = windowFunctionParameterBuffers(i).toArray() + // Send these input parameters to the window function. + functions(i).batchUpdate(inputParameters) + // Ask the function to evaluate based on this window frame. + functions(i).evaluate() + i += 1 + } + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 20c9bc3e75542..5ca11e67a9434 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -21,11 +21,11 @@ import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.util.MutablePair +import org.apache.spark.util.{CompletionIterator, MutablePair} import org.apache.spark.util.collection.ExternalSorter /** @@ -41,6 +41,8 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends val resuableProjection = buildProjection() iter.map(resuableProjection) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -55,20 +57,38 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def execute(): RDD[Row] = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** * :: DeveloperApi :: + * Sample the dataset. + * @param lowerBound Lower-bound of the sampling probability (usually 0.0) + * @param upperBound Upper-bound of the sampling probability. The expected fraction sampled + * will be ub - lb. + * @param withReplacement Whether to sample with replacement. + * @param seed the random seed + * @param child the QueryPlan */ @DeveloperApi -case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: SparkPlan) +case class Sample( + lowerBound: Double, + upperBound: Double, + withReplacement: Boolean, + seed: Long, + child: SparkPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output // TODO: How to pick seed? override def execute(): RDD[Row] = { - child.execute().map(_.copy()).sample(withReplacement, fraction, seed) + if (withReplacement) { + child.execute().map(_.copy()).sample(withReplacement, upperBound - lowerBound, seed) + } else { + child.execute().map(_.copy()).randomSampleWithRange(lowerBound, upperBound, seed) + } } } @@ -117,7 +137,7 @@ case class Limit(limit: Int, child: SparkPlan) } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(child.sqlContext.sparkContext.getConf)) shuffled.mapPartitions(_.take(limit).map(_._2)) } } @@ -139,13 +159,16 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) private def collectData(): Array[Row] = child.execute().map(_.copy()).takeOrdered(limit)(ord) - // TODO: Is this copying for no reason? - override def executeCollect(): Array[Row] = - collectData().map(ScalaReflection.convertRowToScala(_, this.schema)) + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + collectData().map(converter(_).asInstanceOf[Row]) + } // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. override def execute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1) + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -171,6 +194,8 @@ case class Sort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -193,12 +218,16 @@ case class ExternalSort( child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) - sorter.insertAll(iterator.map(r => (r, null))) - sorter.iterator.map(_._1) + sorter.insertAll(iterator.map(r => (r.copy, null))) + val baseIterator = sorter.iterator.map(_._1) + // TODO(marmbrus): The complex type signature below thwarts inference for no reason. + CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop()) }, preservesPartitioning = true) } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -232,6 +261,20 @@ case class Distinct(partial: Boolean, child: SparkPlan) extends UnaryNode { } } +/** + * :: DeveloperApi :: + * Return a new RDD that has exactly `numPartitions` partitions. + */ +@DeveloperApi +case class Repartition(numPartitions: Int, shuffle: Boolean, child: SparkPlan) + extends UnaryNode { + override def output: Seq[Attribute] = child.output + + override def execute(): RDD[Row] = { + child.execute().map(_.copy()).coalesce(numPartitions, shuffle) + } +} + /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index fad7a281dc1e2..98df5bef34efa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType} -import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -41,7 +42,7 @@ trait RunnableCommand extends logical.Command { * A physical operator that executes the run method of a `RunnableCommand` and * saves the result to prevent multiple executions. */ -case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { +private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field @@ -61,7 +62,11 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray - override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) + override def execute(): RDD[Row] = { + val converted = sideEffectResult.map(r => + CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[Row]) + sqlContext.sparkContext.parallelize(converted, 1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e916e68e58b5d..710787096e6cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -164,7 +164,7 @@ package object debug { case (_: Long, LongType) => case (_: Int, IntegerType) => - case (_: String, StringType) => + case (_: UTF8String, StringType) => case (_: Float, FloatType) => case (_: Byte, ByteType) => case (_: Short, ShortType) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala new file mode 100644 index 0000000000000..9ac732b55b188 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.execution.expressions + +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.{Row, LeafExpression} +import org.apache.spark.sql.types.{LongType, DataType} + +/** + * Returns monotonically increasing 64-bit integers. + * + * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. + * The current implementation puts the partition ID in the upper 31 bits, and the lower 33 bits + * represent the record number within each partition. The assumption is that the data frame has + * less than 1 billion partitions, and each partition has less than 8 billion records. + * + * Since this expression is stateful, it cannot be a case object. + */ +private[sql] case class MonotonicallyIncreasingID() extends LeafExpression { + + /** + * Record ID within each partition. By being transient, count's value is reset to 0 every time + * we serialize and deserialize it. + */ + @transient private[this] var count: Long = 0L + + override type EvaluatedType = Long + + override def nullable: Boolean = false + + override def dataType: DataType = LongType + + override def eval(input: Row): Long = { + val currentCount = count + count += 1 + (TaskContext.get().partitionId().toLong << 33) + currentCount + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala new file mode 100644 index 0000000000000..c2c6cbd491598 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala @@ -0,0 +1,37 @@ +/* + * 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.sql.execution.expressions + +import org.apache.spark.TaskContext +import org.apache.spark.sql.catalyst.expressions.{LeafExpression, Row} +import org.apache.spark.sql.types.{IntegerType, DataType} + + +/** + * Expression that returns the current partition id of the Spark task. + */ +private[sql] case object SparkPartitionID extends LeafExpression { + + override type EvaluatedType = Int + + override def nullable: Boolean = false + + override def dataType: DataType = IntegerType + + override def eval(input: Row): Int = TaskContext.get().partitionId() +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/package.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/package.scala index b86abbda1d3e7..568b7ac2c5987 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/package.scala @@ -15,14 +15,9 @@ * limitations under the License. */ -package org.apache.spark.storage - -import tachyon.client.TachyonFile +package org.apache.spark.sql.execution /** - * References a particular segment of a file (potentially the entire file), based off an offset and - * a length. + * Package containing expressions that are specific to Spark runtime. */ -private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) { - override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) -} +package object expressions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala index 83b1a83765153..6aaf35fb429e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -59,10 +59,7 @@ case class BroadcastNestedLoopJoin( } @transient private lazy val boundCondition = - InterpretedPredicate( - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true))) + newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output) override def execute(): RDD[Row] = { val broadcastedRelation = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 2fa1cf5add3b5..ab84c123e0c0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution.joins +import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.util.collection.CompactBuffer @@ -29,16 +31,43 @@ import org.apache.spark.util.collection.CompactBuffer */ private[joins] sealed trait HashedRelation { def get(key: Row): CompactBuffer[Row] + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def writeBytes(out: ObjectOutput, serialized: Array[Byte]): Unit = { + out.writeInt(serialized.length) // Write the length of serialized bytes first + out.write(serialized) + } + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def readBytes(in: ObjectInput): Array[Byte] = { + val serializedSize = in.readInt() // Read the length of serialized bytes first + val bytes = new Array[Byte](serializedSize) + in.readFully(bytes) + bytes + } } /** * A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values. */ -private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) - extends HashedRelation with Serializable { +private[joins] final class GeneralHashedRelation( + private var hashTable: JavaHashMap[Row, CompactBuffer[Row]]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } @@ -46,8 +75,10 @@ private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, Com * A specialized [[HashedRelation]] that maps key into a single value. This implementation * assumes the key is unique. */ -private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) - extends HashedRelation with Serializable { +private[joins] final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[Row, Row]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = { val v = hashTable.get(key) @@ -55,6 +86,14 @@ private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, R } def getValue(key: Row): Row = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala index 1fa7e7bd0406c..b03af410dca08 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -45,10 +45,7 @@ case class LeftSemiJoinBNL( override def right: SparkPlan = broadcast @transient private lazy val boundCondition = - InterpretedPredicate( - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true))) + newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output) override def execute(): RDD[Row] = { val broadcastedRelation = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala new file mode 100644 index 0000000000000..b5123668ba11e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -0,0 +1,169 @@ +/* + * 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.sql.execution.joins + +import java.util.NoSuchElementException + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * Performs an sort merge join of two child relations. + */ +@DeveloperApi +case class SortMergeJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def output: Seq[Attribute] = left.output ++ right.output + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + // this is to manually construct an ordering that can be used to compare keys from both sides + private val keyOrdering: RowOrdering = RowOrdering.forSchema(leftKeys.map(_.dataType)) + + override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys) + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil + + @transient protected lazy val leftKeyGenerator = newProjection(leftKeys, left.output) + @transient protected lazy val rightKeyGenerator = newProjection(rightKeys, right.output) + + private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = + keys.map(SortOrder(_, Ascending)) + + override def execute(): RDD[Row] = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.zipPartitions(rightResults) { (leftIter, rightIter) => + new Iterator[Row] { + // Mutable per row objects. + private[this] val joinRow = new JoinedRow5 + private[this] var leftElement: Row = _ + private[this] var rightElement: Row = _ + private[this] var leftKey: Row = _ + private[this] var rightKey: Row = _ + private[this] var rightMatches: CompactBuffer[Row] = _ + private[this] var rightPosition: Int = -1 + private[this] var stop: Boolean = false + private[this] var matchKey: Row = _ + + // initialize iterator + initialize() + + override final def hasNext: Boolean = nextMatchingPair() + + override final def next(): Row = { + if (hasNext) { + // we are using the buffered right rows and run down left iterator + val joinedRow = joinRow(leftElement, rightMatches(rightPosition)) + rightPosition += 1 + if (rightPosition >= rightMatches.size) { + rightPosition = 0 + fetchLeft() + if (leftElement == null || keyOrdering.compare(leftKey, matchKey) != 0) { + stop = false + rightMatches = null + } + } + joinedRow + } else { + // no more result + throw new NoSuchElementException + } + } + + private def fetchLeft() = { + if (leftIter.hasNext) { + leftElement = leftIter.next() + leftKey = leftKeyGenerator(leftElement) + } else { + leftElement = null + } + } + + private def fetchRight() = { + if (rightIter.hasNext) { + rightElement = rightIter.next() + rightKey = rightKeyGenerator(rightElement) + } else { + rightElement = null + } + } + + private def initialize() = { + fetchLeft() + fetchRight() + } + + /** + * Searches the right iterator for the next rows that have matches in left side, and store + * them in a buffer. + * + * @return true if the search is successful, and false if the right iterator runs out of + * tuples. + */ + private def nextMatchingPair(): Boolean = { + if (!stop && rightElement != null) { + // run both side to get the first match pair + while (!stop && leftElement != null && rightElement != null) { + val comparing = keyOrdering.compare(leftKey, rightKey) + // for inner join, we need to filter those null keys + stop = comparing == 0 && !leftKey.anyNull + if (comparing > 0 || rightKey.anyNull) { + fetchRight() + } else if (comparing < 0 || leftKey.anyNull) { + fetchLeft() + } + } + rightMatches = new CompactBuffer[Row]() + if (stop) { + stop = false + // iterate the right side to buffer all rows that matches + // as the records should be ordered, exit when we meet the first that not match + while (!stop && rightElement != null) { + rightMatches += rightElement + fetchRight() + stop = keyOrdering.compare(leftKey, rightKey) != 0 + } + if (rightMatches.size > 0) { + rightPosition = 0 + matchKey = leftKey + } + } + } + rightMatches != null && rightMatches.size > 0 + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 5b308d88d4cdf..7a43bfd8bc8d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -140,6 +140,7 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) case (date: Int, DateType) => DateUtils.toJavaDate(date) + case (s: UTF8String, StringType) => s.toString // Pyrolite can handle Timestamp and Decimal case (other, _) => other @@ -192,7 +193,8 @@ object EvaluatePython { case (c: Long, IntegerType) => c.toInt case (c: Int, LongType) => c.toLong case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString + case (c: String, StringType) => UTF8String(c) + case (c, StringType) if !c.isInstanceOf[String] => UTF8String(c.toString) case (c, _) => c } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala new file mode 100644 index 0000000000000..5ae7e107544f8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala @@ -0,0 +1,121 @@ +/* +* 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.sql.execution.stat + +import scala.collection.mutable.{Map => MutableMap} + +import org.apache.spark.Logging +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.types.{ArrayType, StructField, StructType} + +private[sql] object FrequentItems extends Logging { + + /** A helper class wrapping `MutableMap[Any, Long]` for simplicity. */ + private class FreqItemCounter(size: Int) extends Serializable { + val baseMap: MutableMap[Any, Long] = MutableMap.empty[Any, Long] + + /** + * Add a new example to the counts if it exists, otherwise deduct the count + * from existing items. + */ + def add(key: Any, count: Long): this.type = { + if (baseMap.contains(key)) { + baseMap(key) += count + } else { + if (baseMap.size < size) { + baseMap += key -> count + } else { + // TODO: Make this more efficient... A flatMap? + baseMap.retain((k, v) => v > count) + baseMap.transform((k, v) => v - count) + } + } + this + } + + /** + * Merge two maps of counts. + * @param other The map containing the counts for that partition + */ + def merge(other: FreqItemCounter): this.type = { + other.baseMap.foreach { case (k, v) => + add(k, v) + } + this + } + } + + /** + * Finding frequent items for columns, possibly with false positives. Using the + * frequent element count algorithm described in + * [[http://dx.doi.org/10.1145/762471.762473, proposed by Karp, Schenker, and Papadimitriou]]. + * The `support` should be greater than 1e-4. + * For Internal use only. + * + * @param df The input DataFrame + * @param cols the names of the columns to search frequent items in + * @param support The minimum frequency for an item to be considered `frequent`. Should be greater + * than 1e-4. + * @return A Local DataFrame with the Array of frequent items for each column. + */ + private[sql] def singlePassFreqItems( + df: DataFrame, + cols: Seq[String], + support: Double): DataFrame = { + require(support >= 1e-4, s"support ($support) must be greater than 1e-4.") + val numCols = cols.length + // number of max items to keep counts for + val sizeOfMap = (1 / support).toInt + val countMaps = Seq.tabulate(numCols)(i => new FreqItemCounter(sizeOfMap)) + val originalSchema = df.schema + val colInfo = cols.map { name => + val index = originalSchema.fieldIndex(name) + (name, originalSchema.fields(index).dataType) + } + + val freqItems = df.select(cols.map(Column(_)):_*).rdd.aggregate(countMaps)( + seqOp = (counts, row) => { + var i = 0 + while (i < numCols) { + val thisMap = counts(i) + val key = row.get(i) + thisMap.add(key, 1L) + i += 1 + } + counts + }, + combOp = (baseCounts, counts) => { + var i = 0 + while (i < numCols) { + baseCounts(i).merge(counts(i)) + i += 1 + } + baseCounts + } + ) + val justItems = freqItems.map(m => m.baseMap.keys.toSeq) + val resultRow = Row(justItems:_*) + // append frequent Items to the column name for easy debugging + val outputCols = colInfo.map { v => + StructField(v._1 + "_freqItems", ArrayType(v._2, false)) + } + val schema = StructType(outputCols).toAttributes + new DataFrame(df.sqlContext, LocalRelation(schema, Seq(resultRow))) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala new file mode 100644 index 0000000000000..b50f606d9cbe3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -0,0 +1,129 @@ +/* + * 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.sql.execution.stat + +import org.apache.spark.Logging +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Cast} +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +private[sql] object StatFunctions extends Logging { + + /** Calculate the Pearson Correlation Coefficient for the given columns */ + private[sql] def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { + val counts = collectStatisticalData(df, cols) + counts.Ck / math.sqrt(counts.MkX * counts.MkY) + } + + /** Helper class to simplify tracking and merging counts. */ + private class CovarianceCounter extends Serializable { + var xAvg = 0.0 // the mean of all examples seen so far in col1 + var yAvg = 0.0 // the mean of all examples seen so far in col2 + var Ck = 0.0 // the co-moment after k examples + var MkX = 0.0 // sum of squares of differences from the (current) mean for col1 + var MkY = 0.0 // sum of squares of differences from the (current) mean for col1 + var count = 0L // count of observed examples + // add an example to the calculation + def add(x: Double, y: Double): this.type = { + val deltaX = x - xAvg + val deltaY = y - yAvg + count += 1 + xAvg += deltaX / count + yAvg += deltaY / count + Ck += deltaX * (y - yAvg) + MkX += deltaX * (x - xAvg) + MkY += deltaY * (y - yAvg) + this + } + // merge counters from other partitions. Formula can be found at: + // http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance + def merge(other: CovarianceCounter): this.type = { + val totalCount = count + other.count + val deltaX = xAvg - other.xAvg + val deltaY = yAvg - other.yAvg + Ck += other.Ck + deltaX * deltaY * count / totalCount * other.count + xAvg = (xAvg * count + other.xAvg * other.count) / totalCount + yAvg = (yAvg * count + other.yAvg * other.count) / totalCount + MkX += other.MkX + deltaX * deltaX * count / totalCount * other.count + MkY += other.MkY + deltaY * deltaY * count / totalCount * other.count + count = totalCount + this + } + // return the sample covariance for the observed examples + def cov: Double = Ck / (count - 1) + } + + private def collectStatisticalData(df: DataFrame, cols: Seq[String]): CovarianceCounter = { + require(cols.length == 2, "Currently cov supports calculating the covariance " + + "between two columns.") + cols.map(name => (name, df.schema.fields.find(_.name == name))).foreach { case (name, data) => + require(data.nonEmpty, s"Couldn't find column with name $name") + require(data.get.dataType.isInstanceOf[NumericType], "Covariance calculation for columns " + + s"with dataType ${data.get.dataType} not supported.") + } + val columns = cols.map(n => Column(Cast(Column(n).expr, DoubleType))) + df.select(columns: _*).rdd.aggregate(new CovarianceCounter)( + seqOp = (counter, row) => { + counter.add(row.getDouble(0), row.getDouble(1)) + }, + combOp = (baseCounter, other) => { + baseCounter.merge(other) + }) + } + + /** + * Calculate the covariance of two numerical columns of a DataFrame. + * @param df The DataFrame + * @param cols the column names + * @return the covariance of the two columns. + */ + private[sql] def calculateCov(df: DataFrame, cols: Seq[String]): Double = { + val counts = collectStatisticalData(df, cols) + counts.cov + } + + /** Generate a table of frequencies for the elements of two columns. */ + private[sql] def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { + val tableName = s"${col1}_$col2" + val counts = df.groupBy(col1, col2).agg(col(col1), col(col2), count("*")).take(1e8.toInt) + if (counts.length == 1e8.toInt) { + logWarning("The maximum limit of 1e8 pairs have been collected, which may not be all of " + + "the pairs. Please try reducing the amount of distinct items in your columns.") + } + // get the distinct values of column 2, so that we can make them the column names + val distinctCol2 = counts.map(_.get(1)).distinct.zipWithIndex.toMap + val columnSize = distinctCol2.size + require(columnSize < 1e4, s"The number of distinct values for $col2, can't " + + s"exceed 1e4. Currently $columnSize") + val table = counts.groupBy(_.get(0)).map { case (col1Item, rows) => + val countsRow = new GenericMutableRow(columnSize + 1) + rows.foreach { row => + countsRow.setLong(distinctCol2.get(row.get(1)).get + 1, row.getLong(2)) + } + // the value of col1 is the first value, the rest are the counts + countsRow.setString(0, col1Item.toString) + countsRow + }.toSeq + val headerNames = distinctCol2.map(r => StructField(r._1.toString, LongType)).toSeq + val schema = StructType(StructField(tableName, StringType) +: headerNames) + + new DataFrame(df.sqlContext, LocalRelation(schema.toAttributes, table)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 111e751588a8b..7e283393d0563 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,10 +22,10 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ - +import org.apache.spark.util.Utils /** * :: Experimental :: @@ -276,6 +276,30 @@ object functions { // Non-aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Computes the absolute value. + * + * @group normal_funcs + */ + def abs(e: Column): Column = Abs(e.expr) + + /** + * Creates a new array column. The input columns must all have the same data type. + * + * @group normal_funcs + */ + @scala.annotation.varargs + def array(cols: Column*): Column = CreateArray(cols.map(_.expr)) + + /** + * Creates a new array column. The input columns must all have the same data type. + * + * @group normal_funcs + */ + def array(colName: String, colNames: String*): Column = { + array((colName +: colNames).map(col) : _*) + } + /** * Returns the first column that is not null. * {{{ @@ -287,6 +311,29 @@ object functions { @scala.annotation.varargs def coalesce(e: Column*): Column = Coalesce(e.map(_.expr)) + /** + * Converts a string exprsesion to lower case. + * + * @group normal_funcs + */ + def lower(e: Column): Column = Lower(e.expr) + + /** + * A column expression that generates monotonically increasing 64-bit integers. + * + * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. + * The current implementation puts the partition ID in the upper 31 bits, and the record number + * within each partition in the lower 33 bits. The assumption is that the data frame has + * less than 1 billion partitions, and each partition has less than 8 billion records. + * + * As an example, consider a [[DataFrame]] with two partitions, each with 3 records. + * This expression would return the following IDs: + * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. + * + * @group normal_funcs + */ + def monotonicallyIncreasingId(): Column = execution.expressions.MonotonicallyIncreasingID() + /** * Unary minus, i.e. negate the expression. * {{{ @@ -317,18 +364,41 @@ object functions { def not(e: Column): Column = !e /** - * Converts a string expression to upper case. + * Generate a random column with i.i.d. samples from U[0.0, 1.0]. * * @group normal_funcs */ - def upper(e: Column): Column = Upper(e.expr) + def rand(seed: Long): Column = Rand(seed) /** - * Converts a string exprsesion to lower case. + * Generate a random column with i.i.d. samples from U[0.0, 1.0]. * * @group normal_funcs */ - def lower(e: Column): Column = Lower(e.expr) + def rand(): Column = rand(Utils.random.nextLong) + + /** + * Generate a column with i.i.d. samples from the standard normal distribution. + * + * @group normal_funcs + */ + def randn(seed: Long): Column = Randn(seed) + + /** + * Generate a column with i.i.d. samples from the standard normal distribution. + * + * @group normal_funcs + */ + def randn(): Column = randn(Utils.random.nextLong) + + /** + * Partition ID of the Spark task. + * + * Note that this is indeterministic because it depends on data partitioning and task scheduling. + * + * @group normal_funcs + */ + def sparkPartitionId(): Column = execution.expressions.SparkPartitionID /** * Computes the square root of the specified float value. @@ -338,11 +408,33 @@ object functions { def sqrt(e: Column): Column = Sqrt(e.expr) /** - * Computes the absolutle value. + * Creates a new struct column. The input column must be a column in a [[DataFrame]], or + * a derived column expression that is named (i.e. aliased). * * @group normal_funcs */ - def abs(e: Column): Column = Abs(e.expr) + @scala.annotation.varargs + def struct(cols: Column*): Column = { + require(cols.forall(_.expr.isInstanceOf[NamedExpression]), + s"struct input columns must all be named or aliased ($cols)") + CreateStruct(cols.map(_.expr.asInstanceOf[NamedExpression])) + } + + /** + * Creates a new struct column that composes multiple input columns. + * + * @group normal_funcs + */ + def struct(colName: String, colNames: String*): Column = { + struct((colName +: colNames).map(col) : _*) + } + + /** + * Converts a string expression to upper case. + * + * @group normal_funcs + */ + def upper(e: Column): Column = Upper(e.expr) ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// @@ -605,4 +697,23 @@ object functions { } // scalastyle:on + + /** + * Call an user-defined function. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + * val sqlContext = df.sqlContext + * sqlContext.udf.register("simpleUdf", (v: Int) => v * v) + * df.select($"id", callUdf("simpleUdf", $"value")) + * }}} + * + * @group udf_funcs + */ + def callUdf(udfName: String, cols: Column*): Column = { + UnresolvedFunction(udfName, cols.map(_.expr)) + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala index 1704be7fcbd30..0feabc4282f4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala @@ -49,9 +49,9 @@ private[sql] object DriverQuirks { * Fetch the DriverQuirks class corresponding to a given database url. */ def get(url: String): DriverQuirks = { - if (url.substring(0, 10).equals("jdbc:mysql")) { + if (url.startsWith("jdbc:mysql")) { new MySQLQuirks() - } else if (url.substring(0, 15).equals("jdbc:postgresql")) { + } else if (url.startsWith("jdbc:postgresql")) { new PostgresQuirks() } else { new NoQuirks() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 3266b972128ea..2f6ba48dbc3d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -18,12 +18,16 @@ package org.apache.spark.sql.jdbc import java.sql.{Connection, DriverManager, ResultSet, ResultSetMetaData, SQLException} +import java.util.Properties + +import org.apache.commons.lang3.StringUtils import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow} import org.apache.spark.sql.types._ import org.apache.spark.sql.sources._ +import org.apache.spark.util.Utils private[sql] object JDBCRDD extends Logging { /** @@ -33,7 +37,7 @@ private[sql] object JDBCRDD extends Logging { * @param sqlType - A field of java.sql.Types * @return The Catalyst type corresponding to sqlType. */ - private def getCatalystType(sqlType: Int): DataType = { + private def getCatalystType(sqlType: Int, precision: Int, scale: Int): DataType = { val answer = sqlType match { case java.sql.Types.ARRAY => null case java.sql.Types.BIGINT => LongType @@ -45,6 +49,8 @@ private[sql] object JDBCRDD extends Logging { case java.sql.Types.CLOB => StringType case java.sql.Types.DATALINK => null case java.sql.Types.DATE => DateType + case java.sql.Types.DECIMAL + if precision != 0 || scale != 0 => DecimalType(precision, scale) case java.sql.Types.DECIMAL => DecimalType.Unlimited case java.sql.Types.DISTINCT => null case java.sql.Types.DOUBLE => DoubleType @@ -57,7 +63,10 @@ private[sql] object JDBCRDD extends Logging { case java.sql.Types.NCHAR => StringType case java.sql.Types.NCLOB => StringType case java.sql.Types.NULL => null + case java.sql.Types.NUMERIC + if precision != 0 || scale != 0 => DecimalType(precision, scale) case java.sql.Types.NUMERIC => DecimalType.Unlimited + case java.sql.Types.NVARCHAR => StringType case java.sql.Types.OTHER => null case java.sql.Types.REAL => DoubleType case java.sql.Types.REF => StringType @@ -89,9 +98,9 @@ private[sql] object JDBCRDD extends Logging { * @throws SQLException if the table specification is garbage. * @throws SQLException if the table contains an unsupported type. */ - def resolveTable(url: String, table: String): StructType = { + def resolveTable(url: String, table: String, properties: Properties): StructType = { val quirks = DriverQuirks.get(url) - val conn: Connection = DriverManager.getConnection(url) + val conn: Connection = DriverManager.getConnection(url, properties) try { val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0").executeQuery() try { @@ -104,10 +113,11 @@ private[sql] object JDBCRDD extends Logging { val dataType = rsmd.getColumnType(i + 1) val typeName = rsmd.getColumnTypeName(i + 1) val fieldSize = rsmd.getPrecision(i + 1) + val fieldScale = rsmd.getScale(i + 1) val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls val metadata = new MetadataBuilder().putString("name", columnName) var columnType = quirks.getCatalystType(dataType, typeName, fieldSize, metadata) - if (columnType == null) columnType = getCatalystType(dataType) + if (columnType == null) columnType = getCatalystType(dataType, fieldSize, fieldScale) fields(i) = StructField(columnName, columnType, nullable, metadata.build()) i = i + 1 } @@ -146,16 +156,16 @@ private[sql] object JDBCRDD extends Logging { * * @return A function that loads the driver and connects to the url. */ - def getConnector(driver: String, url: String): () => Connection = { + def getConnector(driver: String, url: String, properties: Properties): () => Connection = { () => { try { - if (driver != null) Class.forName(driver) + if (driver != null) DriverRegistry.register(driver) } catch { case e: ClassNotFoundException => { logWarning(s"Couldn't find class $driver", e); } } - DriverManager.getConnection(url) + DriverManager.getConnection(url, properties) } } /** @@ -178,6 +188,7 @@ private[sql] object JDBCRDD extends Logging { schema: StructType, driver: String, url: String, + properties: Properties, fqTable: String, requiredColumns: Array[String], filters: Array[Filter], @@ -188,7 +199,7 @@ private[sql] object JDBCRDD extends Logging { return new JDBCRDD( sc, - getConnector(driver, url), + getConnector(driver, url, properties), prunedSchema, fqTable, requiredColumns, @@ -226,16 +237,27 @@ private[sql] class JDBCRDD( if (sb.length == 0) "1" else sb.substring(1) } + /** + * Converts value to SQL expression. + */ + private def compileValue(value: Any): Any = value match { + case stringValue: UTF8String => s"'${escapeSql(stringValue.toString)}'" + case _ => value + } + + private def escapeSql(value: String): String = + if (value == null) null else StringUtils.replace(value, "'", "''") + /** * Turns a single Filter into a String representing a SQL expression. * Returns null for an unhandled filter. */ private def compileFilter(f: Filter): String = f match { - case EqualTo(attr, value) => s"$attr = $value" - case LessThan(attr, value) => s"$attr < $value" - case GreaterThan(attr, value) => s"$attr > $value" - case LessThanOrEqual(attr, value) => s"$attr <= $value" - case GreaterThanOrEqual(attr, value) => s"$attr >= $value" + case EqualTo(attr, value) => s"$attr = ${compileValue(value)}" + case LessThan(attr, value) => s"$attr < ${compileValue(value)}" + case GreaterThan(attr, value) => s"$attr > ${compileValue(value)}" + case LessThanOrEqual(attr, value) => s"$attr <= ${compileValue(value)}" + case GreaterThanOrEqual(attr, value) => s"$attr >= ${compileValue(value)}" case _ => null } @@ -290,6 +312,7 @@ private[sql] class JDBCRDD( case BooleanType => BooleanConversion case DateType => DateConversion case DecimalType.Unlimited => DecimalConversion + case DecimalType.Fixed(d) => DecimalConversion case DoubleType => DoubleConversion case FloatType => FloatConversion case IntegerType => IntegerConversion @@ -338,12 +361,14 @@ private[sql] class JDBCRDD( val pos = i + 1 conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) - case DateConversion => mutableRow.update(i, rs.getDate(pos)) + case DateConversion => + mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) case LongConversion => mutableRow.setLong(i, rs.getLong(pos)) + // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 case StringConversion => mutableRow.setString(i, rs.getString(pos)) case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) @@ -352,7 +377,7 @@ private[sql] class JDBCRDD( var ans = 0L var j = 0 while (j < bytes.size) { - ans = 256*ans + (255 & bytes(j)) + ans = 256 * ans + (255 & bytes(j)) j = j + 1; } mutableRow.setLong(i, ans) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index df687e6da9bea..d6b3fb3291a2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -17,16 +17,18 @@ package org.apache.spark.sql.jdbc -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.types.StructType +import java.sql.DriverManager +import java.util.Properties import scala.collection.mutable.ArrayBuffer -import java.sql.DriverManager import org.apache.spark.Partition +import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils /** * Data corresponding to one partition of a JDBCRDD. @@ -98,7 +100,7 @@ private[sql] class DefaultSource extends RelationProvider { val upperBound = parameters.getOrElse("upperBound", null) val numPartitions = parameters.getOrElse("numPartitions", null) - if (driver != null) Class.forName(driver) + if (driver != null) DriverRegistry.register(driver) if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { @@ -115,26 +117,32 @@ private[sql] class DefaultSource extends RelationProvider { numPartitions.toInt) } val parts = JDBCRelation.columnPartition(partitionInfo) - JDBCRelation(url, table, parts)(sqlContext) + val properties = new Properties() // Additional properties that we will pass to getConnection + parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) + JDBCRelation(url, table, parts, properties)(sqlContext) } } private[sql] case class JDBCRelation( url: String, table: String, - parts: Array[Partition])(@transient val sqlContext: SQLContext) + parts: Array[Partition], + properties: Properties = new Properties())(@transient val sqlContext: SQLContext) extends BaseRelation with PrunedFilteredScan { - override val schema: StructType = JDBCRDD.resolveTable(url, table) + override val needConversion: Boolean = false + + override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { - val driver: String = DriverManager.getDriver(url).getClass.getCanonicalName + val driver: String = DriverRegistry.getDriverClassName(url) JDBCRDD.scanTable( sqlContext.sparkContext, schema, driver, url, + properties, table, requiredColumns, filters, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index 34f864f5fda7a..3a6c2c1e9101f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql -import java.sql.{Connection, DriverManager, PreparedStatement} -import org.apache.spark.{Logging, Partition} -import org.apache.spark.sql._ -import org.apache.spark.sql.sources.LogicalRelation +import java.sql.{Connection, Driver, DriverManager, DriverPropertyInfo, PreparedStatement, SQLFeatureNotSupportedException} +import java.util.Properties -import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartition} +import scala.collection.mutable + +import org.apache.spark.Logging import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils package object jdbc { private[sql] object JDBCWriteDetails extends Logging { @@ -182,4 +183,60 @@ package object jdbc { } } + + private [sql] class DriverWrapper(val wrapped: Driver) extends Driver { + override def acceptsURL(url: String): Boolean = wrapped.acceptsURL(url) + + override def jdbcCompliant(): Boolean = wrapped.jdbcCompliant() + + override def getPropertyInfo(url: String, info: Properties): Array[DriverPropertyInfo] = { + wrapped.getPropertyInfo(url, info) + } + + override def getMinorVersion: Int = wrapped.getMinorVersion + + def getParentLogger: java.util.logging.Logger = + throw new SQLFeatureNotSupportedException( + s"${this.getClass().getName}.getParentLogger is not yet implemented.") + + override def connect(url: String, info: Properties): Connection = wrapped.connect(url, info) + + override def getMajorVersion: Int = wrapped.getMajorVersion + } + + /** + * java.sql.DriverManager is always loaded by bootstrap classloader, + * so it can't load JDBC drivers accessible by Spark ClassLoader. + * + * To solve the problem, drivers from user-supplied jars are wrapped + * into thin wrapper. + */ + private [sql] object DriverRegistry extends Logging { + + private val wrapperMap: mutable.Map[String, DriverWrapper] = mutable.Map.empty + + def register(className: String): Unit = { + val cls = Utils.getContextOrSparkClassLoader.loadClass(className) + if (cls.getClassLoader == null) { + logTrace(s"$className has been loaded with bootstrap ClassLoader, wrapper is not required") + } else if (wrapperMap.get(className).isDefined) { + logTrace(s"Wrapper for $className already exists") + } else { + synchronized { + if (wrapperMap.get(className).isEmpty) { + val wrapper = new DriverWrapper(cls.newInstance().asInstanceOf[Driver]) + DriverManager.registerDriver(wrapper) + wrapperMap(className) = wrapper + logTrace(s"Wrapper for $className registered") + } + } + } + } + + def getDriverClassName(url: String): String = DriverManager.getDriver(url) match { + case wrapper: DriverWrapper => wrapper.wrapped.getClass.getCanonicalName + case driver => driver.getClass.getCanonicalName + } + } + } // package object jdbc diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f4c99b4b56606..e3352d02787fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row - -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] class DefaultSource @@ -113,6 +113,8 @@ private[sql] case class JSONRelation( // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) + override val needConversion: Boolean = false + override val schema = userSpecifiedSchema.getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 2b0358c4e2a1e..6e94e7056eb0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -49,7 +49,7 @@ private[sql] object JsonRDD extends Logging { val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = if (schemaData.isEmpty()) { - Set.empty[(String,DataType)] + Set.empty[(String, DataType)] } else { parseJson(schemaData, columnNameOfCorruptRecords).map(allKeysWithValueTypes).reduce(_ ++ _) } @@ -183,7 +183,7 @@ private[sql] object JsonRDD extends Logging { private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = { // For Integer values, use LongType by default. val useLongType: PartialFunction[Any, DataType] = { - case value: IntegerType.JvmType => LongType + case value: IntegerType.InternalType => LongType } useLongType orElse ScalaReflection.typeOfObject orElse { @@ -391,7 +391,7 @@ private[sql] object JsonRDD extends Logging { value match { // only support string as date case value: java.lang.String => - DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) + DateUtils.millisToDays(DateUtils.stringToTime(value).getTime) case value: java.sql.Date => DateUtils.fromJavaDate(value) } } @@ -400,7 +400,7 @@ private[sql] object JsonRDD extends Logging { value match { case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => toTimestamp(DataTypeConversions.stringToTime(value).getTime) + case value: java.lang.String => toTimestamp(DateUtils.stringToTime(value).getTime) } } @@ -409,13 +409,13 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case StringType => toString(value) + case StringType => UTF8String(toString(value)) case _ if value == null || value == "" => null // guard the non string type - case IntegerType => value.asInstanceOf[IntegerType.JvmType] + case IntegerType => value.asInstanceOf[IntegerType.InternalType] case LongType => toLong(value) case DoubleType => toDouble(value) case DecimalType() => toDecimal(value) - case BooleanType => value.asInstanceOf[BooleanType.JvmType] + case BooleanType => value.asInstanceOf[BooleanType.InternalType] case NullType => null case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala new file mode 100644 index 0000000000000..db47480c3864d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/mathfunctions.scala @@ -0,0 +1,383 @@ +/* + * 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.sql + +import scala.language.implicitConversions + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.mathfuncs._ +import org.apache.spark.sql.functions.lit + +/** + * :: Experimental :: + * Mathematical Functions available for [[DataFrame]]. + */ +@Experimental +// scalastyle:off +object mathfunctions { +// scalastyle:on + + private[this] implicit def toColumn(expr: Expression): Column = Column(expr) + + /** + * Computes the cosine inverse of the given value; the returned angle is in the range + * 0.0 through pi. + */ + def acos(e: Column): Column = Acos(e.expr) + + /** + * Computes the cosine inverse of the given column; the returned angle is in the range + * 0.0 through pi. + */ + def acos(columnName: String): Column = acos(Column(columnName)) + + /** + * Computes the sine inverse of the given value; the returned angle is in the range + * -pi/2 through pi/2. + */ + def asin(e: Column): Column = Asin(e.expr) + + /** + * Computes the sine inverse of the given column; the returned angle is in the range + * -pi/2 through pi/2. + */ + def asin(columnName: String): Column = asin(Column(columnName)) + + /** + * Computes the tangent inverse of the given value. + */ + def atan(e: Column): Column = Atan(e.expr) + + /** + * Computes the tangent inverse of the given column. + */ + def atan(columnName: String): Column = atan(Column(columnName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(l: Column, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(leftName: String, r: Column): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(leftName: String, rightName: String): Column = + atan2(Column(leftName), Column(rightName)) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta).= + */ + def atan2(leftName: String, r: Double): Column = atan2(Column(leftName), r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) + + /** + * Returns the angle theta from the conversion of rectangular coordinates (x, y) to + * polar coordinates (r, theta). + */ + def atan2(l: Double, rightName: String): Column = atan2(l, Column(rightName)) + + /** + * Computes the cube-root of the given value. + */ + def cbrt(e: Column): Column = Cbrt(e.expr) + + /** + * Computes the cube-root of the given column. + */ + def cbrt(columnName: String): Column = cbrt(Column(columnName)) + + /** + * Computes the ceiling of the given value. + */ + def ceil(e: Column): Column = Ceil(e.expr) + + /** + * Computes the ceiling of the given column. + */ + def ceil(columnName: String): Column = ceil(Column(columnName)) + + /** + * Computes the cosine of the given value. + */ + def cos(e: Column): Column = Cos(e.expr) + + /** + * Computes the cosine of the given column. + */ + def cos(columnName: String): Column = cos(Column(columnName)) + + /** + * Computes the hyperbolic cosine of the given value. + */ + def cosh(e: Column): Column = Cosh(e.expr) + + /** + * Computes the hyperbolic cosine of the given column. + */ + def cosh(columnName: String): Column = cosh(Column(columnName)) + + /** + * Computes the exponential of the given value. + */ + def exp(e: Column): Column = Exp(e.expr) + + /** + * Computes the exponential of the given column. + */ + def exp(columnName: String): Column = exp(Column(columnName)) + + /** + * Computes the exponential of the given value minus one. + */ + def expm1(e: Column): Column = Expm1(e.expr) + + /** + * Computes the exponential of the given column. + */ + def expm1(columnName: String): Column = expm1(Column(columnName)) + + /** + * Computes the floor of the given value. + */ + def floor(e: Column): Column = Floor(e.expr) + + /** + * Computes the floor of the given column. + */ + def floor(columnName: String): Column = floor(Column(columnName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(l: Column, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(leftName: String, r: Column): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(leftName: String, rightName: String): Column = + hypot(Column(leftName), Column(rightName)) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(leftName: String, r: Double): Column = hypot(Column(leftName), r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) + + /** + * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. + */ + def hypot(l: Double, rightName: String): Column = hypot(l, Column(rightName)) + + /** + * Computes the natural logarithm of the given value. + */ + def log(e: Column): Column = Log(e.expr) + + /** + * Computes the natural logarithm of the given column. + */ + def log(columnName: String): Column = log(Column(columnName)) + + /** + * Computes the logarithm of the given value in Base 10. + */ + def log10(e: Column): Column = Log10(e.expr) + + /** + * Computes the logarithm of the given value in Base 10. + */ + def log10(columnName: String): Column = log10(Column(columnName)) + + /** + * Computes the natural logarithm of the given value plus one. + */ + def log1p(e: Column): Column = Log1p(e.expr) + + /** + * Computes the natural logarithm of the given column plus one. + */ + def log1p(columnName: String): Column = log1p(Column(columnName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(l: Column, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(leftName: String, r: Column): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(leftName: String, rightName: String): Column = pow(Column(leftName), Column(rightName)) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(leftName: String, r: Double): Column = pow(Column(leftName), r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) + + /** + * Returns the value of the first argument raised to the power of the second argument. + */ + def pow(l: Double, rightName: String): Column = pow(l, Column(rightName)) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + */ + def rint(e: Column): Column = Rint(e.expr) + + /** + * Returns the double value that is closest in value to the argument and + * is equal to a mathematical integer. + */ + def rint(columnName: String): Column = rint(Column(columnName)) + + /** + * Computes the signum of the given value. + */ + def signum(e: Column): Column = Signum(e.expr) + + /** + * Computes the signum of the given column. + */ + def signum(columnName: String): Column = signum(Column(columnName)) + + /** + * Computes the sine of the given value. + */ + def sin(e: Column): Column = Sin(e.expr) + + /** + * Computes the sine of the given column. + */ + def sin(columnName: String): Column = sin(Column(columnName)) + + /** + * Computes the hyperbolic sine of the given value. + */ + def sinh(e: Column): Column = Sinh(e.expr) + + /** + * Computes the hyperbolic sine of the given column. + */ + def sinh(columnName: String): Column = sinh(Column(columnName)) + + /** + * Computes the tangent of the given value. + */ + def tan(e: Column): Column = Tan(e.expr) + + /** + * Computes the tangent of the given column. + */ + def tan(columnName: String): Column = tan(Column(columnName)) + + /** + * Computes the hyperbolic tangent of the given value. + */ + def tanh(e: Column): Column = Tanh(e.expr) + + /** + * Computes the hyperbolic tangent of the given column. + */ + def tanh(columnName: String): Column = tanh(Column(columnName)) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + */ + def toDeg(e: Column): Column = ToDegrees(e.expr) + + /** + * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. + */ + def toDeg(columnName: String): Column = toDeg(Column(columnName)) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + */ + def toRad(e: Column): Column = ToRadians(e.expr) + + /** + * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. + */ + def toRad(columnName: String): Column = toRad(Column(columnName)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala new file mode 100644 index 0000000000000..f5ce2718bec4a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala @@ -0,0 +1,73 @@ +/* + * 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.sql.parquet + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + +import parquet.Log +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter, ParquetOutputFormat} + +private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) + extends ParquetOutputCommitter(outputPath, context) { + val LOG = Log.getLog(classOf[ParquetOutputCommitter]) + + override def getWorkPath(): Path = outputPath + override def abortTask(taskContext: TaskAttemptContext): Unit = {} + override def commitTask(taskContext: TaskAttemptContext): Unit = {} + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + override def setupJob(jobContext: JobContext): Unit = {} + override def setupTask(taskContext: TaskAttemptContext): Unit = {} + + override def commitJob(jobContext: JobContext) { + val configuration = ContextUtil.getConfiguration(jobContext) + val fileSystem = outputPath.getFileSystem(configuration) + + if (configuration.getBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, true)) { + try { + val outputStatus = fileSystem.getFileStatus(outputPath) + val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) + try { + ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) + } catch { + case e: Exception => { + LOG.warn("could not write summary file for " + outputPath, e) + val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) + if (fileSystem.exists(metadataPath)) { + fileSystem.delete(metadataPath, true) + } + } + } + } catch { + case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) + } + } + + if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { + try { + val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) + fileSystem.create(successPath).close() + } catch { + case e: Exception => LOG.warn("could not write success file for " + outputPath, e) + } + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index f898e4b37a56b..36cb5e03bbca7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -90,7 +90,7 @@ private[sql] object CatalystConverter { createConverter(field.copy(dataType = udt.sqlType), fieldIndex, parent) } // For native JVM types we use a converter with native arrays - case ArrayType(elementType: NativeType, false) => { + case ArrayType(elementType: AtomicType, false) => { new CatalystNativeArrayConverter(elementType, fieldIndex, parent) } // This is for other types of arrays, including those with nested fields @@ -118,13 +118,19 @@ private[sql] object CatalystConverter { case ShortType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addInt(value: Int): Unit = - parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.JvmType]) + parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.InternalType]) } } case ByteType => { new CatalystPrimitiveConverter(parent, fieldIndex) { override def addInt(value: Int): Unit = - parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.JvmType]) + parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.InternalType]) + } + } + case DateType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addInt(value: Int): Unit = + parent.updateDate(fieldIndex, value.asInstanceOf[DateType.InternalType]) } } case d: DecimalType => { @@ -140,7 +146,8 @@ private[sql] object CatalystConverter { } } // All other primitive types use the default converter - case ctype: PrimitiveType => { // note: need the type tag here! + case ctype: DataType if ParquetTypesConverter.isPrimitiveType(ctype) => { + // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) } case _ => throw new RuntimeException( @@ -192,6 +199,9 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = updateField(fieldIndex, value) + protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = + updateField(fieldIndex, value) + protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = updateField(fieldIndex, value) @@ -210,8 +220,8 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, value.getBytes) - protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - updateField(fieldIndex, value) + protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + updateField(fieldIndex, UTF8String(value)) protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, readTimestamp(value)) @@ -315,9 +325,9 @@ private[parquet] class CatalystGroupConverter( override def start(): Unit = { current = ArrayBuffer.fill(size)(null) - converters.foreach { - converter => if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer + converters.foreach { converter => + if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer() } } } @@ -388,6 +398,9 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = current.setInt(fieldIndex, value) + override protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = + current.update(fieldIndex, value) + override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = current.setLong(fieldIndex, value) @@ -406,8 +419,8 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, value.getBytes) - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - current.setString(fieldIndex, value) + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + current.update(fieldIndex, UTF8String(value)) override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, readTimestamp(value)) @@ -463,19 +476,18 @@ private[parquet] class CatalystPrimitiveConverter( private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { - private[this] var dict: Array[String] = null + private[this] var dict: Array[Array[Byte]] = null override def hasDictionarySupport: Boolean = true override def setDictionary(dictionary: Dictionary):Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) {dictionary.decodeToBinary(_).toStringUsingUTF8} - + dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } override def addValueFromDictionary(dictionaryId: Int): Unit = parent.updateString(fieldIndex, dict(dictionaryId)) override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.toStringUsingUTF8) + parent.updateString(fieldIndex, value.getBytes) } private[parquet] object CatalystArrayConverter { @@ -601,7 +613,7 @@ private[parquet] class CatalystArrayConverter( override def start(): Unit = { if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer + converter.asInstanceOf[CatalystConverter].clearBuffer() } } @@ -625,13 +637,13 @@ private[parquet] class CatalystArrayConverter( * @param capacity The (initial) capacity of the buffer */ private[parquet] class CatalystNativeArrayConverter( - val elementType: NativeType, + val elementType: AtomicType, val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) extends CatalystConverter { - type NativeType = elementType.JvmType + type NativeType = elementType.InternalType private var buffer: Array[NativeType] = elementType.classTag.newArray(capacity) @@ -702,9 +714,9 @@ private[parquet] class CatalystNativeArrayConverter( elements += 1 } - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = { + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] + buffer(elements) = UTF8String(value).asInstanceOf[NativeType] elements += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 0357dcc4688be..5eb1c6abc2432 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -55,7 +55,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), @@ -76,7 +76,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), @@ -94,7 +94,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -111,7 +111,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -128,7 +128,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -145,7 +145,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5130d8ad5e003..aded126ea0615 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.parquet import java.io.IOException import java.lang.{Long => JLong} -import java.text.SimpleDateFormat -import java.text.NumberFormat +import java.text.{NumberFormat, SimpleDateFormat} import java.util.concurrent.{Callable, TimeUnit} -import java.util.{ArrayList, Collections, Date, List => JList} +import java.util.{Date, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable @@ -43,12 +42,13 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.{Logging, SerializableWritable, TaskContext} /** @@ -268,7 +268,7 @@ private[sql] case class InsertIntoParquetTable( val job = new Job(sqlContext.sparkContext.hadoopConfiguration) val writeSupport = - if (child.output.map(_.dataType).forall(_.isPrimitive)) { + if (child.output.map(_.dataType).forall(ParquetTypesConverter.isPrimitiveType)) { log.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { @@ -356,7 +356,7 @@ private[sql] case class InsertIntoParquetTable( } finally { writer.close(hadoopContext) } - committer.commitTask(hadoopContext) + SparkHadoopMapRedUtil.commitTask(committer, hadoopContext, context) 1 } val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) @@ -381,6 +381,7 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} + var committer: OutputCommitter = null // override to choose output filename so not overwrite existing ones override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { @@ -403,6 +404,26 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } + + // override to create output committer from configuration + override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { + if (committer == null) { + val output = getOutputPath(context) + val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", + classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) + val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] + } + committer + } + + // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 + private def getOutputPath(context: TaskAttemptContext): Path = { + context.getConfiguration().get("mapred.output.dir") match { + case null => null + case name => new Path(name) + } + } } /** @@ -512,6 +533,7 @@ private[parquet] class FilteringParquetRowInputFormat import parquet.filter2.compat.FilterCompat.Filter import parquet.filter2.compat.RowGroupFilter + import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.blockLocationCache val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 19bfba34b8f4a..c45c431438efc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -189,7 +189,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case t @ StructType(_) => writeStruct( t, value.asInstanceOf[CatalystConverter.StructScalaType[_]]) - case _ => writePrimitive(schema.asInstanceOf[NativeType], value) + case _ => writePrimitive(schema.asInstanceOf[AtomicType], value) } } } @@ -198,10 +198,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { if (value != null) { schema match { case StringType => writer.addBinary( - Binary.fromByteArray( - value.asInstanceOf[String].getBytes("utf-8") - ) - ) + Binary.fromByteArray(value.asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) @@ -212,6 +209,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) + case DateType => writer.addInteger(value.asInstanceOf[Int]) case d: DecimalType => if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { sys.error(s"Unsupported datatype $d, cannot write to consumer") @@ -348,7 +346,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { index: Int): Unit = { ctype match { case StringType => writer.addBinary( - Binary.fromByteArray(record(index).asInstanceOf[String].getBytes("utf-8"))) + Binary.fromByteArray(record(index).asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) @@ -358,6 +356,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { case DoubleType => writer.addDouble(record.getDouble(index)) case FloatType => writer.addFloat(record.getFloat(index)) case BooleanType => writer.addBoolean(record.getBoolean(index)) + case DateType => writer.addInteger(record.getInt(index)) case TimestampType => writeTimestamp(record(index).asInstanceOf[java.sql.Timestamp]) case d: DecimalType => if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 5209581fa8357..1dc819b5d7b9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -48,8 +48,10 @@ private[parquet] case class ParquetTypeInfo( length: Option[Int] = None) private[parquet] object ParquetTypesConverter extends Logging { - def isPrimitiveType(ctype: DataType): Boolean = - classOf[PrimitiveType] isAssignableFrom ctype.getClass + def isPrimitiveType(ctype: DataType): Boolean = ctype match { + case _: NumericType | BooleanType | StringType | BinaryType => true + case _: DataType => false + } def toPrimitiveDataType( parquetType: ParquetPrimitiveType, @@ -64,6 +66,8 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetPrimitiveTypeName.BOOLEAN => BooleanType case ParquetPrimitiveTypeName.DOUBLE => DoubleType case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 + if originalType == ParquetOriginalType.DATE => DateType case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType case ParquetPrimitiveTypeName.INT96 if int96AsTimestamp => TimestampType @@ -222,6 +226,8 @@ private[parquet] object ParquetTypesConverter extends Logging { // There is no type for Byte or Short so we promote them to INT32. case ShortType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) case ByteType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) + case DateType => Some(ParquetTypeInfo( + ParquetPrimitiveTypeName.INT32, Some(ParquetOriginalType.DATE))) case LongType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT64)) case TimestampType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT96)) case DecimalType.Fixed(precision, scale) if precision <= 18 => @@ -386,6 +392,7 @@ private[parquet] object ParquetTypesConverter extends Logging { def convertFromAttributes(attributes: Seq[Attribute], toThriftSchemaNames: Boolean = false): MessageType = { + checkSpecialCharacters(attributes) val fields = attributes.map( attribute => fromDataType(attribute.dataType, attribute.name, attribute.nullable, @@ -400,7 +407,20 @@ private[parquet] object ParquetTypesConverter extends Logging { } } + private def checkSpecialCharacters(schema: Seq[Attribute]) = { + // ,;{}()\n\t= and space character are special characters in Parquet schema + schema.map(_.name).foreach { name => + if (name.matches(".*[ ,;{}()\n\t=].*")) { + sys.error( + s"""Attribute name "$name" contains invalid character(s) among " ,;{}()\n\t=". + |Please use alias to rename it. + """.stripMargin.split("\n").mkString(" ")) + } + } + } + def convertToString(schema: Seq[Attribute]): String = { + checkSpecialCharacters(schema) StructType.fromAttributes(schema).json } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 410600b0529d3..85e60733bc57a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} - import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.metadata.CompressionCodecName @@ -42,15 +41,16 @@ import parquet.hadoop.{ParquetInputFormat, _} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, expressions} import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode} -import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWritable, SparkException, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, SparkException, TaskContext, Partition => SparkPartition} /** * Allows creation of Parquet based tables using the syntax: @@ -121,7 +121,8 @@ private[sql] class DefaultSource val df = sqlContext.createDataFrame( data.queryExecution.toRdd, - data.schema.asNullable) + data.schema.asNullable, + needsConversion = false) val createdRelation = createRelation(sqlContext, parameters, df.schema).asInstanceOf[ParquetRelation2] createdRelation.insert(df, overwrite = mode == SaveMode.Overwrite) @@ -266,7 +267,8 @@ private[sql] case class ParquetRelation2( // containing Parquet files (e.g. partitioned Parquet table). val baseStatuses = paths.distinct.map { p => val fs = FileSystem.get(URI.create(p), sparkContext.hadoopConfiguration) - val qualified = fs.makeQualified(new Path(p)) + val path = new Path(p) + val qualified = path.makeQualified(fs.getUri, fs.getWorkingDirectory) if (!fs.exists(qualified) && maybeSchema.isDefined) { fs.mkdirs(qualified) @@ -406,6 +408,9 @@ private[sql] case class ParquetRelation2( file.getName == ParquetFileWriter.PARQUET_METADATA_FILE } + // Skip type conversion + override val needConversion: Boolean = false + // TODO Should calculate per scan size // It's common that a query only scans a fraction of a large Parquet file. Returning size of the // whole Parquet file disables some optimizations in this case (e.g. broadcast join). @@ -429,17 +434,28 @@ private[sql] case class ParquetRelation2( // FileInputFormat cannot handle empty lists. if (selectedFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, selectedFiles.map(_.getPath): _*) + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val selectedPaths = selectedFiles.map(status => new Path(status.getPath.toUri.toString)) + FileInputFormat.setInputPaths(job, selectedPaths: _*) } - // Push down filters when possible. Notice that not all filters can be converted to Parquet - // filter predicate. Here we try to convert each individual predicate and only collect those - // convertible ones. - predicates - .flatMap(ParquetFilters.createFilter) - .reduceOption(FilterApi.and) - .filter(_ => sqlContext.conf.parquetFilterPushDown) - .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) + // Try to push down filters when filter push-down is enabled. + if (sqlContext.conf.parquetFilterPushDown) { + val partitionColNames = partitionColumns.map(_.name).toSet + predicates + // Don't push down predicates which reference partition columns + .filter { pred => + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(ParquetFilters.createFilter) + .reduceOption(FilterApi.and) + .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) + } if (isPartitioned) { logInfo { @@ -473,10 +489,31 @@ private[sql] case class ParquetRelation2( val cacheMetadata = useCache @transient - val cachedStatus = selectedFiles + val cachedStatus = selectedFiles.map { st => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val newPath = new Path(st.getPath.toUri.toString) + + new FileStatus( + st.getLen, + st.isDir, + st.getReplication, + st.getBlockSize, + st.getModificationTime, + st.getAccessTime, + st.getPermission, + st.getOwner, + st.getGroup, + newPath) + } @transient - val cachedFooters = selectedFooters + val cachedFooters = selectedFooters.map { f => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + new Footer(new Path(f.getFile.toUri.toString), f.getParquetMetadata) + } + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { @@ -515,7 +552,8 @@ private[sql] case class ParquetRelation2( baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { - case p if split.getPath.getParent.toString == p.path => p.values + case p if split.getPath.getParent.toString == p.path => + CatalystTypeConverters.convertToCatalyst(p.values).asInstanceOf[Row] }.get val requiredPartOrdinal = partitionKeyLocations.keys.toSeq @@ -573,7 +611,7 @@ private[sql] case class ParquetRelation2( val rawPredicate = partitionPruningPredicates.reduceOption(expressions.And).getOrElse(Literal(true)) - val boundPredicate = InterpretedPredicate(rawPredicate transform { + val boundPredicate = InterpretedPredicate.create(rawPredicate transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) @@ -596,12 +634,13 @@ private[sql] case class ParquetRelation2( // before calling execute(). val job = new Job(sqlContext.sparkContext.hadoopConfiguration) - val writeSupport = if (parquetSchema.map(_.dataType).forall(_.isPrimitive)) { - log.debug("Initializing MutableRowWriteSupport") - classOf[MutableRowWriteSupport] - } else { - classOf[RowWriteSupport] - } + val writeSupport = + if (parquetSchema.map(_.dataType).forall(ParquetTypesConverter.isPrimitiveType)) { + log.debug("Initializing MutableRowWriteSupport") + classOf[MutableRowWriteSupport] + } else { + classOf[RowWriteSupport] + } ParquetOutputFormat.setWriteSupportClass(job, writeSupport) @@ -662,7 +701,8 @@ private[sql] case class ParquetRelation2( } finally { writer.close(hadoopContext) } - committer.commitTask(hadoopContext) + + SparkHadoopMapRedUtil.commitTask(committer, hadoopContext, context) } val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) /* apparently we need a TaskAttemptID to construct an OutputCommitter; @@ -758,12 +798,15 @@ private[sql] object ParquetRelation2 extends Logging { |${parquetSchema.prettyJson} """.stripMargin - assert(metastoreSchema.size == parquetSchema.size, schemaConflictMessage) + val mergedParquetSchema = mergeMissingNullableFields(metastoreSchema, parquetSchema) + + assert(metastoreSchema.size <= mergedParquetSchema.size, schemaConflictMessage) val ordinalMap = metastoreSchema.zipWithIndex.map { case (field, index) => field.name.toLowerCase -> index }.toMap - val reorderedParquetSchema = parquetSchema.sortBy(f => ordinalMap(f.name.toLowerCase)) + val reorderedParquetSchema = mergedParquetSchema.sortBy(f => + ordinalMap.getOrElse(f.name.toLowerCase, metastoreSchema.size + 1)) StructType(metastoreSchema.zip(reorderedParquetSchema).map { // Uses Parquet field names but retains Metastore data types. @@ -774,6 +817,32 @@ private[sql] object ParquetRelation2 extends Logging { }) } + /** + * Returns the original schema from the Parquet file with any missing nullable fields from the + * Hive Metastore schema merged in. + * + * When constructing a DataFrame from a collection of structured data, the resulting object has + * a schema corresponding to the union of the fields present in each element of the collection. + * Spark SQL simply assigns a null value to any field that isn't present for a particular row. + * In some cases, it is possible that a given table partition stored as a Parquet file doesn't + * contain a particular nullable field in its schema despite that field being present in the + * table schema obtained from the Hive Metastore. This method returns a schema representing the + * Parquet file schema along with any additional nullable fields from the Metastore schema + * merged in. + */ + private[parquet] def mergeMissingNullableFields( + metastoreSchema: StructType, + parquetSchema: StructType): StructType = { + val fieldMap = metastoreSchema.map(f => f.name.toLowerCase -> f).toMap + val missingFields = metastoreSchema + .map(_.name.toLowerCase) + .diff(parquetSchema.map(_.name.toLowerCase)) + .map(fieldMap(_)) + .filter(_.nullable) + StructType(parquetSchema ++ missingFields) + } + + // TODO Data source implementations shouldn't touch Catalyst types (`Literal`). // However, we are already using Catalyst expressions for partition pruning and predicate // push-down here... @@ -834,9 +903,9 @@ private[sql] object ParquetRelation2 extends Logging { * PartitionValues( * Seq("a", "b", "c"), * Seq( - * Literal(42, IntegerType), - * Literal("hello", StringType), - * Literal(3.14, FloatType))) + * Literal.create(42, IntegerType), + * Literal.create("hello", StringType), + * Literal.create(3.14, FloatType))) * }}} */ private[parquet] def parsePartition( @@ -915,15 +984,16 @@ private[sql] object ParquetRelation2 extends Logging { raw: String, defaultPartitionName: String): Literal = { // First tries integral types - Try(Literal(Integer.parseInt(raw), IntegerType)) - .orElse(Try(Literal(JLong.parseLong(raw), LongType))) + Try(Literal.create(Integer.parseInt(raw), IntegerType)) + .orElse(Try(Literal.create(JLong.parseLong(raw), LongType))) // Then falls back to fractional types - .orElse(Try(Literal(JFloat.parseFloat(raw), FloatType))) - .orElse(Try(Literal(JDouble.parseDouble(raw), DoubleType))) - .orElse(Try(Literal(new JBigDecimal(raw), DecimalType.Unlimited))) + .orElse(Try(Literal.create(JFloat.parseFloat(raw), FloatType))) + .orElse(Try(Literal.create(JDouble.parseDouble(raw), DoubleType))) + .orElse(Try(Literal.create(new JBigDecimal(raw), DecimalType.Unlimited))) // Then falls back to string .getOrElse { - if (raw == defaultPartitionName) Literal(null, NullType) else Literal(raw, StringType) + if (raw == defaultPartitionName) Literal.create(null, NullType) + else Literal.create(raw, StringType) } } @@ -942,7 +1012,7 @@ private[sql] object ParquetRelation2 extends Logging { } literals.map { case l @ Literal(_, dataType) => - Literal(Cast(l, desiredType).eval(), desiredType) + Literal.create(Cast(l, desiredType).eval(), desiredType) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 67f3507c61ab6..b3d71f687a60a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -23,6 +23,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.{UTF8String, StringType} import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -52,10 +54,10 @@ private[sql] object DataSourceStrategy extends Strategy { (a, _) => t.buildScan(a)) :: Nil case l @ LogicalRelation(t: TableScan) => - execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), part, query, overwrite) if part.isEmpty => + l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil @@ -101,20 +103,30 @@ private[sql] object DataSourceStrategy extends Strategy { projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. .map(relation.attributeMap) // Match original case of attributes. - val scan = - execution.PhysicalRDD( - projectList.map(_.toAttribute), + val scan = createPhysicalRDD(relation.relation, projectList.map(_.toAttribute), scanBuilder(requestedColumns, pushedFilters)) filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) } else { val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq - val scan = - execution.PhysicalRDD(requestedColumns, scanBuilder(requestedColumns, pushedFilters)) + val scan = createPhysicalRDD(relation.relation, requestedColumns, + scanBuilder(requestedColumns, pushedFilters)) execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) } } + private[this] def createPhysicalRDD( + relation: BaseRelation, + output: Seq[Attribute], + rdd: RDD[Row]): SparkPlan = { + val converted = if (relation.needConversion) { + execution.RDDConversions.rowToRowRdd(rdd, relation.schema) + } else { + rdd + } + execution.PhysicalRDD(output, converted) + } + /** * Selects Catalyst predicate [[Expression]]s which are convertible into data source [[Filter]]s, * and convert them. @@ -166,6 +178,15 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.Not(child) => translate(child).map(sources.Not) + case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringStartsWith(a.name, v.toString)) + + case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringEndsWith(a.name, v.toString)) + + case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringContains(a.name, v.toString)) + case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 9bbe06e59ba30..dbdb0d39c26a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -31,7 +31,8 @@ private[sql] case class InsertIntoDataSource( val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] val data = DataFrame(sqlContext, query) // Apply the schema of the existing table to the new data. - val df = sqlContext.createDataFrame(data.queryExecution.toRdd, logicalRelation.schema) + val df = sqlContext.createDataFrame( + data.queryExecution.toRdd, logicalRelation.schema, needsConversion = false) relation.insert(df, overwrite) // Invalidate the cache. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index d57406645eefa..1abf3aa51cb25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.sources import scala.language.existentials +import scala.util.matching.Regex import scala.language.implicitConversions import org.apache.spark.Logging -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} +import org.apache.spark.sql.{AnalysisException, SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.AbstractSparkSQLParser import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -34,26 +35,19 @@ import org.apache.spark.util.Utils * A parser for foreign DDL commands. */ private[sql] class DDLParser( - parseQuery: String => LogicalPlan) extends AbstractSparkSQLParser with Logging { + parseQuery: String => LogicalPlan) + extends AbstractSparkSQLParser with DataTypeParser with Logging { - def apply(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = { + def parse(input: String, exceptionOnError: Boolean): LogicalPlan = { try { - Some(apply(input)) + parse(input) } catch { case ddlException: DDLException => throw ddlException - case _ if !exceptionOnError => None + case _ if !exceptionOnError => parseQuery(input) case x: Throwable => throw x } } - def parseType(input: String): DataType = { - lexical.initialize(reservedWords) - phrase(dataType)(new lexical.Scanner(input)) match { - case Success(r, x) => r - case x => throw new DDLException(s"Unsupported dataType: $x") - } - } - // Keyword is a convention with AbstractSparkSQLParser, which will scan all of the `Keyword` // properties via reflection the class in runtime for constructing the SqlLexical object protected val CREATE = Keyword("CREATE") @@ -70,24 +64,6 @@ private[sql] class DDLParser( protected val COMMENT = Keyword("COMMENT") protected val REFRESH = Keyword("REFRESH") - // Data types. - protected val STRING = Keyword("STRING") - protected val BINARY = Keyword("BINARY") - protected val BOOLEAN = Keyword("BOOLEAN") - protected val TINYINT = Keyword("TINYINT") - protected val SMALLINT = Keyword("SMALLINT") - protected val INT = Keyword("INT") - protected val BIGINT = Keyword("BIGINT") - protected val FLOAT = Keyword("FLOAT") - protected val DOUBLE = Keyword("DOUBLE") - protected val DECIMAL = Keyword("DECIMAL") - protected val DATE = Keyword("DATE") - protected val TIMESTAMP = Keyword("TIMESTAMP") - protected val VARCHAR = Keyword("VARCHAR") - protected val ARRAY = Keyword("ARRAY") - protected val MAP = Keyword("MAP") - protected val STRUCT = Keyword("STRUCT") - protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable | refreshTable protected def start: Parser[LogicalPlan] = ddl @@ -180,7 +156,19 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} - protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) } + override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", { + case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str + case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str + } + ) + + protected lazy val optionName: Parser[String] = "[_a-zA-Z][a-zA-Z0-9]*".r ^^ { + case name => name + } + + protected lazy val pair: Parser[(String, String)] = + optionName ~ stringLit ^^ { case k ~ v => (k,v) } protected lazy val column: Parser[StructField] = ident ~ dataType ~ (COMMENT ~> stringLit).? ^^ { case columnName ~ typ ~ cm => @@ -189,58 +177,9 @@ private[sql] class DDLParser( new MetadataBuilder().putString(COMMENT.str.toLowerCase, comment).build() case None => Metadata.empty } - StructField(columnName, typ, nullable = true, meta) - } - protected lazy val primitiveType: Parser[DataType] = - STRING ^^^ StringType | - BINARY ^^^ BinaryType | - BOOLEAN ^^^ BooleanType | - TINYINT ^^^ ByteType | - SMALLINT ^^^ ShortType | - INT ^^^ IntegerType | - BIGINT ^^^ LongType | - FLOAT ^^^ FloatType | - DOUBLE ^^^ DoubleType | - fixedDecimalType | // decimal with precision/scale - DECIMAL ^^^ DecimalType.Unlimited | // decimal with no precision/scale - DATE ^^^ DateType | - TIMESTAMP ^^^ TimestampType | - VARCHAR ~ "(" ~ numericLit ~ ")" ^^^ StringType - - protected lazy val fixedDecimalType: Parser[DataType] = - (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { - case precision ~ scale => DecimalType(precision.toInt, scale.toInt) - } - - protected lazy val arrayType: Parser[DataType] = - ARRAY ~> "<" ~> dataType <~ ">" ^^ { - case tpe => ArrayType(tpe) - } - - protected lazy val mapType: Parser[DataType] = - MAP ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { - case t1 ~ _ ~ t2 => MapType(t1, t2) - } - - protected lazy val structField: Parser[StructField] = - ident ~ ":" ~ dataType ^^ { - case fieldName ~ _ ~ tpe => StructField(fieldName, tpe, nullable = true) + StructField(columnName, typ, nullable = true, meta) } - - protected lazy val structType: Parser[DataType] = - (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { - case fields => StructType(fields) - }) | - (STRUCT ~> "<>" ^^ { - case fields => StructType(Nil) - }) - - private[sql] lazy val dataType: Parser[DataType] = - arrayType | - mapType | - structType | - primitiveType } private[sql] object ResolvedDataSource { @@ -278,19 +217,25 @@ private[sql] object ResolvedDataSource { provider: String, options: Map[String, String]): ResolvedDataSource = { val clazz: Class[_] = lookupDataSource(provider) + def className: String = clazz.getCanonicalName val relation = userSpecifiedSchema match { case Some(schema: StructType) => clazz.newInstance() match { case dataSource: SchemaRelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema) case dataSource: org.apache.spark.sql.sources.RelationProvider => - sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") + throw new AnalysisException(s"$className does not allow user-specified schemas.") + case _ => + throw new AnalysisException(s"$className is not a RelationProvider.") } case None => clazz.newInstance() match { case dataSource: RelationProvider => dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options)) case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => - sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") + throw new AnalysisException( + s"A schema needs to be specified when using $className.") + case _ => + throw new AnalysisException(s"$className is not a RelationProvider.") } } new ResolvedDataSource(clazz, relation) @@ -402,7 +347,24 @@ private[sql] case class RefreshTable(databaseName: String, tableName: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { + // Refresh the given table's metadata first. sqlContext.catalog.refreshTable(databaseName, tableName) + + // If this table is cached as a InMemoryColumnarRelation, drop the original + // cached version and make the new version cached lazily. + val logicalPlan = sqlContext.catalog.lookupRelation(Seq(databaseName, tableName)) + // Use lookupCachedData directly since RefreshTable also takes databaseName. + val isCached = sqlContext.cacheManager.lookupCachedData(logicalPlan).nonEmpty + if (isCached) { + // Create a data frame to represent the table. + // TODO: Use uncacheTable once it supports database name. + val df = DataFrame(sqlContext, logicalPlan) + // Uncache the logicalPlan. + sqlContext.cacheManager.tryUncacheQuery(df, blocking = true) + // Cache it again. + sqlContext.cacheManager.cacheQuery(df, Some(tableName)) + } + Seq.empty[Row] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index 1e4505e36d2f0..791046e0079d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,16 +17,85 @@ package org.apache.spark.sql.sources +/** + * A filter predicate for data sources. + */ abstract class Filter +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * equal to `value`. + */ case class EqualTo(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * greater than `value`. + */ case class GreaterThan(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * greater than or equal to `value`. + */ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * less than `value`. + */ case class LessThan(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a value + * less than or equal to `value`. + */ case class LessThanOrEqual(attribute: String, value: Any) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to one of the values in the array. + */ case class In(attribute: String, values: Array[Any]) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to null. + */ case class IsNull(attribute: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to a non-null value. + */ case class IsNotNull(attribute: String) extends Filter + +/** + * A filter that evaluates to `true` iff both `left` or `right` evaluate to `true`. + */ case class And(left: Filter, right: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff at least one of `left` or `right` evaluates to `true`. + */ case class Or(left: Filter, right: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff `child` is evaluated to `false`. + */ case class Not(child: Filter) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that starts with `value`. + */ +case class StringStartsWith(attribute: String, value: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that starts with `value`. + */ +case class StringEndsWith(attribute: String, value: String) extends Filter + +/** + * A filter that evaluates to `true` iff the attribute evaluates to + * a string that contains the string `value`. + */ +case class StringContains(attribute: String, value: String) extends Filter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index a046a48c1733d..ca53dcdb92c52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -126,6 +126,16 @@ abstract class BaseRelation { * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). */ def sizeInBytes: Long = sqlContext.conf.defaultSizeInBytes + + /** + * Whether does it need to convert the objects in Row to internal representation, for example: + * java.lang.String -> UTF8String + * java.lang.Decimal -> Decimal + * + * Note: The internal representation is not stable across releases and thus data sources outside + * of Spark SQL should leave this as true. + */ + def needConversion: Boolean = true } /** @@ -152,6 +162,9 @@ trait PrunedScan { * A BaseRelation that can eliminate unneeded columns and filter using selected * predicates before producing an RDD containing all matching tuples as Row objects. * + * The actual filter should be the conjunction of all `filters`, + * i.e. they should be "and" together. + * * The pushed down filters are currently purely an optimization as they will all be evaluated * again. This means it is safe to use them with methods that produce false positives such * as filtering partitions based on a bloom filter. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 5a78001117d1b..6ed68d179edc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -37,7 +37,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation. case i @ InsertIntoTable( - l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite) => { + l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite, ifNotExists) => { // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -84,7 +84,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => + l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") @@ -102,7 +102,8 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite) if !l.isInstanceOf[InsertableRelation] => + l: LogicalRelation, partition, query, overwrite, ifNotExists) + if !l.isInstanceOf[InsertableRelation] => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index c11d0ae5bf1cc..2fdd798b44bb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ * @param y y coordinate */ @SQLUserDefinedType(udt = classOf[ExamplePointUDT]) -private[sql] class ExamplePoint(val x: Double, val y: Double) +private[sql] class ExamplePoint(val x: Double, val y: Double) extends Serializable /** * User-defined type for [[ExamplePoint]]. diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 2d586f784ac5a..58cc8e5be6075 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -17,29 +17,45 @@ package test.org.apache.spark.sql; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import com.google.common.collect.ImmutableMap; +import com.google.common.primitives.Ints; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; +import org.apache.spark.sql.test.TestSQLContext; import org.apache.spark.sql.test.TestSQLContext$; -import static org.apache.spark.sql.functions.*; +import org.apache.spark.sql.types.*; +import org.junit.*; + +import scala.collection.JavaConversions; +import scala.collection.Seq; +import scala.collection.mutable.Buffer; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import static org.apache.spark.sql.functions.*; +import static org.apache.spark.sql.mathfunctions.*; public class JavaDataFrameSuite { + private transient JavaSparkContext jsc; private transient SQLContext context; @Before public void setUp() { // Trigger static initializer of TestData TestData$.MODULE$.testData(); + jsc = new JavaSparkContext(TestSQLContext.sparkContext()); context = TestSQLContext$.MODULE$; } @After public void tearDown() { + jsc = null; context = null; } @@ -81,6 +97,17 @@ public void testVarargMethods() { df.groupBy().agg(countDistinct("key", "value")); df.groupBy().agg(countDistinct(col("key"), col("value"))); df.select(coalesce(col("key"))); + + // Varargs with mathfunctions + DataFrame df2 = context.table("testData2"); + df2.select(exp("a"), exp("b")); + df2.select(exp(log("a"))); + df2.select(pow("a", "a"), pow("b", 2.0)); + df2.select(pow(col("a"), col("b")), exp("b")); + df2.select(sin("a"), acos("b")); + + df2.select(rand(), acos("b")); + df2.select(col("*"), randn(5L)); } @Ignore @@ -90,4 +117,115 @@ public void testShow() { df.show(); df.show(1000); } + + public static class Bean implements Serializable { + private double a = 0.0; + private Integer[] b = new Integer[]{0, 1}; + private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); + private List d = Arrays.asList("floppy", "disk"); + + public double getA() { + return a; + } + + public Integer[] getB() { + return b; + } + + public Map getC() { + return c; + } + + public List getD() { + return d; + } + } + + @Test + public void testCreateDataFrameFromJavaBeans() { + Bean bean = new Bean(); + JavaRDD rdd = jsc.parallelize(Arrays.asList(bean)); + DataFrame df = context.createDataFrame(rdd, Bean.class); + StructType schema = df.schema(); + Assert.assertEquals(new StructField("a", DoubleType$.MODULE$, false, Metadata.empty()), + schema.apply("a")); + Assert.assertEquals( + new StructField("b", new ArrayType(IntegerType$.MODULE$, true), true, Metadata.empty()), + schema.apply("b")); + ArrayType valueType = new ArrayType(DataTypes.IntegerType, false); + MapType mapType = new MapType(DataTypes.StringType, valueType, true); + Assert.assertEquals( + new StructField("c", mapType, true, Metadata.empty()), + schema.apply("c")); + Assert.assertEquals( + new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), + schema.apply("d")); + Row first = df.select("a", "b", "c", "d").first(); + Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); + // Now Java lists and maps are converetd to Scala Seq's and Map's. Once we get a Seq below, + // verify that it has the expected length, and contains expected elements. + Seq result = first.getAs(1); + Assert.assertEquals(bean.getB().length, result.length()); + for (int i = 0; i < result.length(); i++) { + Assert.assertEquals(bean.getB()[i], result.apply(i)); + } + Buffer outputBuffer = (Buffer) first.getJavaMap(2).get("hello"); + Assert.assertArrayEquals( + bean.getC().get("hello"), + Ints.toArray(JavaConversions.bufferAsJavaList(outputBuffer))); + Seq d = first.getAs(3); + Assert.assertEquals(bean.getD().size(), d.length()); + for (int i = 0; i < d.length(); i++) { + Assert.assertEquals(bean.getD().get(i), d.apply(i)); + } + } + + private static Comparator CrosstabRowComparator = new Comparator() { + public int compare(Row row1, Row row2) { + String item1 = row1.getString(0); + String item2 = row2.getString(0); + return item1.compareTo(item2); + } + }; + + @Test + public void testCrosstab() { + DataFrame df = context.table("testData2"); + DataFrame crosstab = df.stat().crosstab("a", "b"); + String[] columnNames = crosstab.schema().fieldNames(); + Assert.assertEquals(columnNames[0], "a_b"); + Assert.assertEquals(columnNames[1], "1"); + Assert.assertEquals(columnNames[2], "2"); + Row[] rows = crosstab.collect(); + Arrays.sort(rows, CrosstabRowComparator); + Integer count = 1; + for (Row row : rows) { + Assert.assertEquals(row.get(0).toString(), count.toString()); + Assert.assertEquals(row.getLong(1), 1L); + Assert.assertEquals(row.getLong(2), 1L); + count++; + } + } + + @Test + public void testFrequentItems() { + DataFrame df = context.table("testData2"); + String[] cols = new String[]{"a"}; + DataFrame results = df.stat().freqItems(cols, 0.2); + Assert.assertTrue(results.collect()[0].getSeq(0).contains(1)); + } + + @Test + public void testCorrelation() { + DataFrame df = context.table("testData2"); + Double pearsonCorr = df.stat().corr("a", "b", "pearson"); + Assert.assertTrue(Math.abs(pearsonCorr) < 1e-6); + } + + @Test + public void testCovariance() { + DataFrame df = context.table("testData2"); + Double result = df.stat().cov("a", "b"); + Assert.assertTrue(Math.abs(result) < 1e-6); + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c240f2be955ca..0772e5e187425 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,6 +22,7 @@ import scala.language.{implicitConversions, postfixOps} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.Accumulators import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ @@ -92,7 +93,8 @@ class CachedTableSuite extends QueryTest { test("too big for memory") { val data = "*" * 10000 - sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF().registerTempTable("bigData") + sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() + .registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) table("bigData").unpersist(blocking = true) @@ -296,4 +298,28 @@ class CachedTableSuite extends QueryTest { sql("Clear CACHE") assert(cacheManager.isEmpty) } + + test("Clear accumulators when uncacheTable to prevent memory leaking") { + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + + Accumulators.synchronized { + val accsSize = Accumulators.originals.size + cacheTable("t1") + cacheTable("t2") + assert((accsSize + 2) == Accumulators.originals.size) + } + + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + + Accumulators.synchronized { + val accsSize = Accumulators.originals.size + uncacheTable("t1") + uncacheTable("t2") + assert((accsSize - 2) == Accumulators.originals.size) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a53ae97d6243a..6322faf4d9907 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.NamedExpression -import org.apache.spark.sql.catalyst.plans.logical.{Project, NoRelation} +import org.scalatest.Matchers._ + import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{BooleanType, IntegerType, StructField, StructType} - +import org.apache.spark.sql.types._ class ColumnExpressionSuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -312,6 +311,25 @@ class ColumnExpressionSuite extends QueryTest { ) } + test("monotonicallyIncreasingId") { + // Make sure we have 2 partitions, each with 2 records. + val df = TestSQLContext.sparkContext.parallelize(1 to 2, 2).mapPartitions { iter => + Iterator(Tuple1(1), Tuple1(2)) + }.toDF("a") + checkAnswer( + df.select(monotonicallyIncreasingId()), + Row(0L) :: Row(1L) :: Row((1L << 33) + 0L) :: Row((1L << 33) + 1L) :: Nil + ) + } + + test("sparkPartitionId") { + val df = TestSQLContext.sparkContext.parallelize(1 to 1, 1).map(i => (i, i)).toDF("a", "b") + checkAnswer( + df.select(sparkPartitionId()), + Row(0) + ) + } + test("lift alias out of cast") { compareExpressions( col("1234").as("name").cast("int").expr, @@ -322,4 +340,35 @@ class ColumnExpressionSuite extends QueryTest { assert('key.desc == 'key.desc) assert('key.desc != 'key.asc) } + + test("alias with metadata") { + val metadata = new MetadataBuilder() + .putString("originName", "value") + .build() + val schema = testData + .select($"*", col("value").as("abc", metadata)) + .schema + assert(schema("value").metadata === Metadata.empty) + assert(schema("abc").metadata === metadata) + } + + test("rand") { + val randCol = testData.select('key, rand(5L).as("rand")) + randCol.columns.length should be (2) + val rows = randCol.collect() + rows.foreach { row => + assert(row.getDouble(1) <= 1.0) + assert(row.getDouble(1) >= 0.0) + } + } + + test("randn") { + val randCol = testData.select('key, randn(5L).as("rand")) + randCol.columns.length should be (2) + val rows = randCol.collect() + rows.foreach { row => + assert(row.getDouble(1) <= 4.0) + assert(row.getDouble(1) >= -4.0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala new file mode 100644 index 0000000000000..ca03713ef4658 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -0,0 +1,84 @@ +/* + * 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.sql + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.types._ + +/** + * Test suite for functions in [[org.apache.spark.sql.functions]]. + */ +class DataFrameFunctionsSuite extends QueryTest { + + test("array with column name") { + val df = Seq((0, 1)).toDF("a", "b") + val row = df.select(array("a", "b")).first() + + val expectedType = ArrayType(IntegerType, containsNull = false) + assert(row.schema(0).dataType === expectedType) + assert(row.getAs[Seq[Int]](0) === Seq(0, 1)) + } + + test("array with column expression") { + val df = Seq((0, 1)).toDF("a", "b") + val row = df.select(array(col("a"), col("b") + col("b"))).first() + + val expectedType = ArrayType(IntegerType, containsNull = false) + assert(row.schema(0).dataType === expectedType) + assert(row.getAs[Seq[Int]](0) === Seq(0, 2)) + } + + // Turn this on once we add a rule to the analyzer to throw a friendly exception + ignore("array: throw exception if putting columns of different types into an array") { + val df = Seq((0, "str")).toDF("a", "b") + intercept[AnalysisException] { + df.select(array("a", "b")) + } + } + + test("struct with column name") { + val df = Seq((1, "str")).toDF("a", "b") + val row = df.select(struct("a", "b")).first() + + val expectedType = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType) + )) + assert(row.schema(0).dataType === expectedType) + assert(row.getAs[Row](0) === Row(1, "str")) + } + + test("struct with column expression") { + val df = Seq((1, "str")).toDF("a", "b") + val row = df.select(struct((col("a") * 2).as("c"), col("b"))).first() + + val expectedType = StructType(Seq( + StructField("c", IntegerType, nullable = false), + StructField("b", StringType) + )) + assert(row.schema(0).dataType === expectedType) + assert(row.getAs[Row](0) === Row(2, "str")) + } + + test("struct: must use named column expression") { + intercept[IllegalArgumentException] { + struct(col("a") * 2) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala new file mode 100644 index 0000000000000..41b4f02e6a294 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -0,0 +1,191 @@ +/* + * 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.sql + +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.test.TestSQLContext.implicits._ + + +class DataFrameNaFunctionsSuite extends QueryTest { + + def createDF(): DataFrame = { + Seq[(String, java.lang.Integer, java.lang.Double)]( + ("Bob", 16, 176.5), + ("Alice", null, 164.3), + ("David", 60, null), + ("Amy", null, null), + (null, null, null)).toDF("name", "age", "height") + } + + test("drop") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop("name" :: Nil), + rows(0) :: rows(1) :: rows(2) :: rows(3) :: Nil) + + checkAnswer( + input.na.drop("age" :: Nil), + rows(0) :: rows(2) :: Nil) + + checkAnswer( + input.na.drop("age" :: "height" :: Nil), + rows(0) :: Nil) + + checkAnswer( + input.na.drop(), + rows(0)) + + // dropna on an a dataframe with no column should return an empty data frame. + val empty = input.sqlContext.emptyDataFrame.select() + assert(empty.na.drop().count() === 0L) + + // Make sure the columns are properly named. + assert(input.na.drop().columns.toSeq === input.columns.toSeq) + } + + test("drop with how") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop("all"), + rows(0) :: rows(1) :: rows(2) :: rows(3) :: Nil) + + checkAnswer( + input.na.drop("any"), + rows(0) :: Nil) + + checkAnswer( + input.na.drop("any", Seq("age", "height")), + rows(0) :: Nil) + + checkAnswer( + input.na.drop("all", Seq("age", "height")), + rows(0) :: rows(1) :: rows(2) :: Nil) + } + + test("drop with threshold") { + val input = createDF() + val rows = input.collect() + + checkAnswer( + input.na.drop(2, Seq("age", "height")), + rows(0) :: Nil) + + checkAnswer( + input.na.drop(3, Seq("name", "age", "height")), + rows(0)) + + // Make sure the columns are properly named. + assert(input.na.drop(2, Seq("age", "height")).columns.toSeq === input.columns.toSeq) + } + + test("fill") { + val input = createDF() + + val fillNumeric = input.na.fill(50.6) + checkAnswer( + fillNumeric, + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, 50.6) :: + Row("Amy", 50, 50.6) :: + Row(null, 50, 50.6) :: Nil) + + // Make sure the columns are properly named. + assert(fillNumeric.columns.toSeq === input.columns.toSeq) + + // string + checkAnswer( + input.na.fill("unknown").select("name"), + Row("Bob") :: Row("Alice") :: Row("David") :: Row("Amy") :: Row("unknown") :: Nil) + assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) + + // fill double with subset columns + checkAnswer( + input.na.fill(50.6, "age" :: Nil), + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, null) :: + Row("Amy", 50, null) :: + Row(null, 50, null) :: Nil) + + // fill string with subset columns + checkAnswer( + Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), + Row("test", null)) + } + + test("fill with map") { + val df = Seq[(String, String, java.lang.Long, java.lang.Double)]( + (null, null, null, null)).toDF("a", "b", "c", "d") + checkAnswer( + df.na.fill(Map( + "a" -> "test", + "c" -> 1, + "d" -> 2.2 + )), + Row("test", null, 1, 2.2)) + + // Test Java version + checkAnswer( + df.na.fill(mapAsJavaMap(Map( + "a" -> "test", + "c" -> 1, + "d" -> 2.2 + ))), + Row("test", null, 1, 2.2)) + } + + test("replace") { + val input = createDF() + + // Replace two numeric columns: age and height + val out = input.na.replace(Seq("age", "height"), Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 461.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + + // Replace only the age column + val out1 = input.na.replace("age", Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out1, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 164.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala new file mode 100644 index 0000000000000..46b1845a9180c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -0,0 +1,101 @@ +/* + * 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.sql + +import org.scalatest.FunSuite +import org.scalatest.Matchers._ + +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ + +class DataFrameStatSuite extends FunSuite { + + val sqlCtx = TestSQLContext + def toLetter(i: Int): String = (i + 97).toChar.toString + + test("pearson correlation") { + val df = Seq.tabulate(10)(i => (i, 2 * i, i * -1.0)).toDF("a", "b", "c") + val corr1 = df.stat.corr("a", "b", "pearson") + assert(math.abs(corr1 - 1.0) < 1e-12) + val corr2 = df.stat.corr("a", "c", "pearson") + assert(math.abs(corr2 + 1.0) < 1e-12) + // non-trivial example. To reproduce in python, use: + // >>> from scipy.stats import pearsonr + // >>> import numpy as np + // >>> a = np.array(range(20)) + // >>> b = np.array([x * x - 2 * x + 3.5 for x in range(20)]) + // >>> pearsonr(a, b) + // (0.95723391394758572, 3.8902121417802199e-11) + // In R, use: + // > a <- 0:19 + // > b <- mapply(function(x) x * x - 2 * x + 3.5, a) + // > cor(a, b) + // [1] 0.957233913947585835 + val df2 = Seq.tabulate(20)(x => (x, x * x - 2 * x + 3.5)).toDF("a", "b") + val corr3 = df2.stat.corr("a", "b", "pearson") + assert(math.abs(corr3 - 0.95723391394758572) < 1e-12) + } + + test("covariance") { + val df = Seq.tabulate(10)(i => (i, 2.0 * i, toLetter(i))).toDF("singles", "doubles", "letters") + + val results = df.stat.cov("singles", "doubles") + assert(math.abs(results - 55.0 / 3) < 1e-12) + intercept[IllegalArgumentException] { + df.stat.cov("singles", "letters") // doesn't accept non-numerical dataTypes + } + val decimalData = Seq.tabulate(6)(i => (BigDecimal(i % 3), BigDecimal(i % 2))).toDF("a", "b") + val decimalRes = decimalData.stat.cov("a", "b") + assert(math.abs(decimalRes) < 1e-12) + } + + test("crosstab") { + val df = Seq((0, 0), (2, 1), (1, 0), (2, 0), (0, 0), (2, 0)).toDF("a", "b") + val crosstab = df.stat.crosstab("a", "b") + val columnNames = crosstab.schema.fieldNames + assert(columnNames(0) === "a_b") + assert(columnNames(1) === "0") + assert(columnNames(2) === "1") + val rows: Array[Row] = crosstab.collect().sortBy(_.getString(0)) + assert(rows(0).get(0).toString === "0") + assert(rows(0).getLong(1) === 2L) + assert(rows(0).get(2) === null) + assert(rows(1).get(0).toString === "1") + assert(rows(1).getLong(1) === 1L) + assert(rows(1).get(2) === null) + assert(rows(2).get(0).toString === "2") + assert(rows(2).getLong(1) === 2L) + assert(rows(2).getLong(2) === 1L) + } + + test("Frequent Items") { + val rows = Seq.tabulate(1000) { i => + if (i % 3 == 0) (1, toLetter(1), -1.0) else (i, toLetter(i), i * -1.0) + } + val df = rows.toDF("numbers", "letters", "negDoubles") + + val results = df.stat.freqItems(Array("numbers", "letters"), 0.1) + val items = results.collect().head + items.getSeq[Int](0) should contain (1) + items.getSeq[String](1) should contain (toLetter(1)) + + val singleColResults = df.stat.freqItems(Array("negDoubles"), 0.1) + val items2 = singleColResults.collect().head + items2.getSeq[Double](0) should contain (-1.0) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index ff441ef26f9c0..ff31e15e2d472 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -21,7 +21,7 @@ import scala.language.postfixOps import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.{ExamplePointUDT, ExamplePoint, TestSQLContext} import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.test.TestSQLContext.sql @@ -60,6 +60,14 @@ class DataFrameSuite extends QueryTest { assert($"test".toString === "test") } + test("rename nested groupby") { + val df = Seq((1,(1,1))).toDF() + + checkAnswer( + df.groupBy("_1").agg(col("_1"), sum("_2._1")).toDF("key", "total"), + Row(1, 1) :: Nil) + } + test("invalid plan toString, debug mode") { val oldSetting = TestSQLContext.conf.dataFrameEagerAnalysis TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "true") @@ -78,19 +86,57 @@ class DataFrameSuite extends QueryTest { TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } + test("access complex data") { + assert(complexData.filter(complexData("a").getItem(0) === 2).count() == 1) + assert(complexData.filter(complexData("m").getItem("1") === 1).count() == 1) + assert(complexData.filter(complexData("s").getField("key") === 1).count() == 1) + } + test("table scan") { checkAnswer( testData, testData.collect().toSeq) } + test("empty data frame") { + assert(TestSQLContext.emptyDataFrame.columns.toSeq === Seq.empty[String]) + assert(TestSQLContext.emptyDataFrame.count() === 0) + } + test("head and take") { assert(testData.take(2) === testData.collect().take(2)) assert(testData.head(2) === testData.collect().take(2)) assert(testData.head(2).head.schema === testData.schema) } - test("self join") { + test("simple explode") { + val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words") + + checkAnswer( + df.explode("words", "word") { word: String => word.split(" ").toSeq }.select('word), + Row("a") :: Row("b") :: Row("c") :: Row("d") ::Row("e") :: Nil + ) + } + + test("join - join using") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = Seq(1, 2, 3).map(i => (i, (i + 1).toString)).toDF("int", "str") + + checkAnswer( + df.join(df2, "int"), + Row(1, "1", "2") :: Row(2, "2", "3") :: Row(3, "3", "4") :: Nil) + } + + test("join - join using self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + + // self join + checkAnswer( + df.join(df, "int"), + Row(1, "1", "1") :: Row(2, "2", "2") :: Row(3, "3", "3") :: Nil) + } + + test("join - self join") { val df1 = testData.select(testData("key")).as('df1) val df2 = testData.select(testData("key")).as('df2) @@ -99,13 +145,15 @@ class DataFrameSuite extends QueryTest { sql("SELECT a.key, b.key FROM testData a JOIN testData b ON a.key = b.key").collect().toSeq) } - test("simple explode") { - val df = Seq(Tuple1("a b c"), Tuple1("d e")).toDF("words") + test("join - using aliases after self join") { + val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") + checkAnswer( + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) checkAnswer( - df.explode("words", "word") { word: String => word.split(" ").toSeq }.select('word), - Row("a") :: Row("b") :: Row("c") :: Row("d") ::Row("e") :: Nil - ) + df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) } test("explode") { @@ -148,6 +196,14 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } + test("coalesce") { + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) + + checkAnswer( + testData.select('key).coalesce(1).select('key), + testData.select('key).collect().toSeq) + } + test("groupBy") { checkAnswer( testData2.groupBy("a").agg($"a", sum($"b")), @@ -305,8 +361,9 @@ class DataFrameSuite extends QueryTest { checkAnswer( decimalData.agg(avg('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0))) + // non-partial checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), // non-partial + decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) } @@ -415,6 +472,15 @@ class DataFrameSuite extends QueryTest { ) } + test("call udf in SQLContext") { + val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + val sqlctx = df.sqlContext + sqlctx.udf.register("simpleUdf", (v: Int) => v * v) + checkAnswer( + df.select($"id", callUdf("simpleUdf", $"value")), + Row("id1", 1) :: Row("id2", 16) :: Row("id3", 25) :: Nil) + } + test("withColumn") { val df = testData.toDF().withColumn("newCol", col("key") + 1) checkAnswer( @@ -425,6 +491,30 @@ class DataFrameSuite extends QueryTest { assert(df.schema.map(_.name).toSeq === Seq("key", "value", "newCol")) } + test("replace column using withColumn") { + val df2 = TestSQLContext.sparkContext.parallelize(Array(1, 2, 3)).toDF("x") + val df3 = df2.withColumn("x", df2("x") + 1) + checkAnswer( + df3.select("x"), + Row(2) :: Row(3) :: Row(4) :: Nil) + } + + test("drop column using drop") { + val df = testData.drop("key") + checkAnswer( + df, + testData.collect().map(x => Row(x.getString(1))).toSeq) + assert(df.schema.map(_.name) === Seq("value")) + } + + test("drop unknown column (no-op)") { + val df = testData.drop("random") + checkAnswer( + df, + testData.collect().toSeq) + assert(df.schema.map(_.name) === Seq("key","value")) + } + test("withColumnRenamed") { val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") @@ -436,6 +526,67 @@ class DataFrameSuite extends QueryTest { assert(df.schema.map(_.name).toSeq === Seq("key", "valueRenamed", "newCol")) } + test("randomSplit") { + val n = 600 + val data = TestSQLContext.sparkContext.parallelize(1 to n, 2).toDF("id") + for (seed <- 1 to 5) { + val splits = data.randomSplit(Array[Double](1, 2, 3), seed) + assert(splits.length == 3, "wrong number of splits") + + assert(splits.reduce((a, b) => a.unionAll(b)).sort("id").collect().toList == + data.collect().toList, "incomplete or wrong split") + + val s = splits.map(_.count()) + assert(math.abs(s(0) - 100) < 50) // std = 9.13 + assert(math.abs(s(1) - 200) < 50) // std = 11.55 + assert(math.abs(s(2) - 300) < 50) // std = 12.25 + } + } + + test("describe") { + val describeTestData = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + + val describeResult = Seq( + Row("count", 4, 4), + Row("mean", 33.0, 178.0), + Row("stddev", 16.583123951777, 10.0), + Row("min", 16, 164), + Row("max", 60, 192)) + + val emptyDescribeResult = Seq( + Row("count", 0, 0), + Row("mean", null, null), + Row("stddev", null, null), + Row("min", null, null), + Row("max", null, null)) + + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) + + val describeTwoCols = describeTestData.describe("age", "height") + assert(getSchemaAsSeq(describeTwoCols) === Seq("summary", "age", "height")) + checkAnswer(describeTwoCols, describeResult) + + val describeAllCols = describeTestData.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "age", "height")) + checkAnswer(describeAllCols, describeResult) + + val describeOneCol = describeTestData.describe("age") + assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) + checkAnswer(describeOneCol, describeResult.map { case Row(s, d, _) => Row(s, d)} ) + + val describeNoCol = describeTestData.select("name").describe() + assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _) => Row(s)} ) + + val emptyDescription = describeTestData.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "age", "height")) + checkAnswer(emptyDescription, emptyDescribeResult) + } + test("apply on query results (SPARK-5462)") { val df = testData.sqlContext.sql("select key from testData") checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) @@ -446,4 +597,39 @@ class DataFrameSuite extends QueryTest { testData.select($"*").show() testData.select($"*").show(1000) } + + test("SPARK-7319 showString") { + val expectedAnswer = """+---+-----+ + ||key|value| + |+---+-----+ + || 1| 1| + |+---+-----+ + |""".stripMargin + assert(testData.select($"*").showString(1) === expectedAnswer) + } + + test("SPARK-7327 show with empty dataFrame") { + val expectedAnswer = """+---+-----+ + ||key|value| + |+---+-----+ + |+---+-----+ + |""".stripMargin + assert(testData.select($"*").filter($"key" < 0).showString(1) === expectedAnswer) + } + + test("createDataFrame(RDD[Row], StructType) should convert UDTs (SPARK-6672)") { + val rowRDD = TestSQLContext.sparkContext.parallelize(Seq(Row(new ExamplePoint(1.0, 2.0)))) + val schema = StructType(Array(StructField("point", new ExamplePointUDT(), false))) + val df = TestSQLContext.createDataFrame(rowRDD, schema) + df.rdd.collect() + } + + test("SPARK-6899") { + val originalValue = TestSQLContext.conf.codegenEnabled + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, "true") + checkAnswer( + decimalData.agg(avg('a)), + Row(new java.math.BigDecimal(2.0))) + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index dd0948ad824be..037d392c1f929 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -34,7 +34,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") - val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.analyzed + val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.optimizedPlan val planned = planner.HashJoin(join) assert(planned.size === 1) } @@ -51,6 +51,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j case j: BroadcastLeftSemiJoinHash => j + case j: SortMergeJoin => j } assert(operators.size === 1) @@ -62,6 +63,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("join operator selection") { cacheManager.clearCache() + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), @@ -91,17 +93,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", classOf[BroadcastNestedLoopJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } } test("broadcasted hash join operator selection") { cacheManager.clearCache() sql("CACHE TABLE testData") + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } sql("UNCACHE TABLE testData") } @@ -109,7 +135,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("multiple-key equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") - val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.analyzed + val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.optimizedPlan val planned = planner.HashJoin(join) assert(planned.size === 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala new file mode 100644 index 0000000000000..9e19bb7482e9b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -0,0 +1,221 @@ +/* + * 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.sql + +import java.lang.{Double => JavaDouble} + +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.mathfunctions._ +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ + +private[this] object MathExpressionsTestData { + + case class DoubleData(a: JavaDouble, b: JavaDouble) + val doubleData = TestSQLContext.sparkContext.parallelize( + (1 to 10).map(i => DoubleData(i * 0.2 - 1, i * -0.2 + 1))).toDF() + + val nnDoubleData = TestSQLContext.sparkContext.parallelize( + (1 to 10).map(i => DoubleData(i * 0.1, i * -0.1))).toDF() + + case class NullDoubles(a: JavaDouble) + val nullDoubles = + TestSQLContext.sparkContext.parallelize( + NullDoubles(1.0) :: + NullDoubles(2.0) :: + NullDoubles(3.0) :: + NullDoubles(null) :: Nil + ).toDF() +} + +class MathExpressionsSuite extends QueryTest { + + import MathExpressionsTestData._ + + def testOneToOneMathFunction[@specialized(Int, Long, Float, Double) T]( + c: Column => Column, + f: T => T): Unit = { + checkAnswer( + doubleData.select(c('a)), + (1 to 10).map(n => Row(f((n * 0.2 - 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c('b)), + (1 to 10).map(n => Row(f((-n * 0.2 + 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c(lit(null))), + (1 to 10).map(_ => Row(null)) + ) + } + + def testOneToOneNonNegativeMathFunction(c: Column => Column, f: Double => Double): Unit = { + checkAnswer( + nnDoubleData.select(c('a)), + (1 to 10).map(n => Row(f(n * 0.1))) + ) + + if (f(-1) === math.log1p(-1)) { + checkAnswer( + nnDoubleData.select(c('b)), + (1 to 9).map(n => Row(f(n * -0.1))) :+ Row(Double.NegativeInfinity) + ) + } else { + checkAnswer( + nnDoubleData.select(c('b)), + (1 to 10).map(n => Row(null)) + ) + } + + checkAnswer( + nnDoubleData.select(c(lit(null))), + (1 to 10).map(_ => Row(null)) + ) + } + + def testTwoToOneMathFunction( + c: (Column, Column) => Column, + d: (Column, Double) => Column, + f: (Double, Double) => Double): Unit = { + checkAnswer( + nnDoubleData.select(c('a, 'a)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) + ) + + checkAnswer( + nnDoubleData.select(c('a, 'b)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(1)))) + ) + + checkAnswer( + nnDoubleData.select(d('a, 2.0)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), 2.0))) + ) + + checkAnswer( + nnDoubleData.select(d('a, -0.5)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), -0.5))) + ) + + val nonNull = nullDoubles.collect().toSeq.filter(r => r.get(0) != null) + + checkAnswer( + nullDoubles.select(c('a, 'a)).orderBy('a.asc), + Row(null) +: nonNull.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) + ) + } + + test("sin") { + testOneToOneMathFunction(sin, math.sin) + } + + test("asin") { + testOneToOneMathFunction(asin, math.asin) + } + + test("sinh") { + testOneToOneMathFunction(sinh, math.sinh) + } + + test("cos") { + testOneToOneMathFunction(cos, math.cos) + } + + test("acos") { + testOneToOneMathFunction(acos, math.acos) + } + + test("cosh") { + testOneToOneMathFunction(cosh, math.cosh) + } + + test("tan") { + testOneToOneMathFunction(tan, math.tan) + } + + test("atan") { + testOneToOneMathFunction(atan, math.atan) + } + + test("tanh") { + testOneToOneMathFunction(tanh, math.tanh) + } + + test("toDeg") { + testOneToOneMathFunction(toDeg, math.toDegrees) + } + + test("toRad") { + testOneToOneMathFunction(toRad, math.toRadians) + } + + test("cbrt") { + testOneToOneMathFunction(cbrt, math.cbrt) + } + + test("ceil") { + testOneToOneMathFunction(ceil, math.ceil) + } + + test("floor") { + testOneToOneMathFunction(floor, math.floor) + } + + test("rint") { + testOneToOneMathFunction(rint, math.rint) + } + + test("exp") { + testOneToOneMathFunction(exp, math.exp) + } + + test("expm1") { + testOneToOneMathFunction(expm1, math.expm1) + } + + test("signum") { + testOneToOneMathFunction[Double](signum, math.signum) + } + + test("pow") { + testTwoToOneMathFunction(pow, pow, math.pow) + } + + test("hypot") { + testTwoToOneMathFunction(hypot, hypot, math.hypot) + } + + test("atan2") { + testTwoToOneMathFunction(atan2, atan2, math.atan2) + } + + test("log") { + testOneToOneNonNegativeMathFunction(log, math.log) + } + + test("log10") { + testOneToOneNonNegativeMathFunction(log10, math.log10) + } + + test("log1p") { + testOneToOneNonNegativeMathFunction(log1p, math.log1p) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9b4dd6c620fec..bbf9ab113ca43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -67,7 +67,7 @@ class QueryTest extends PlanTest { checkAnswer(df, Seq(expectedAnswer)) } - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) { test(sqlString) { checkAnswer(sqlContext.sql(sqlString), expectedAnswer) } @@ -104,9 +104,12 @@ object QueryTest { // Converts data to types that we can do equality comparison using Scala collections. // For BigDecimal type, the Scala type has a better definition of equality test (similar to // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. val converted: Seq[Row] = answer.map { s => Row.fromSeq(s.toSeq.map { case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq case o => o }) } @@ -129,11 +132,7 @@ object QueryTest { val errorMessage = s""" |Results do not match for query: - |${df.logicalPlan} - |== Analyzed Plan == - |${df.queryExecution.analyzed} - |== Physical Plan == - |${df.queryExecution.executedPlan} + |${df.queryExecution} |== Results == |${sideBySide( s"== Correct Answer - ${expectedAnswer.size} ==" +: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index f5b945f468dad..fb3ba4bc1b908 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.SparkSqlSerializer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ class RowSuite extends FunSuite { @@ -27,7 +30,7 @@ class RowSuite extends FunSuite { test("create row") { val expected = new GenericMutableRow(4) expected.update(0, 2147483647) - expected.update(1, "this is a string") + expected.setString(1, "this is a string") expected.update(2, false) expected.update(3, null) val actual1 = Row(2147483647, "this is a string", false, null) @@ -50,4 +53,23 @@ class RowSuite extends FunSuite { row(0) = null assert(row.isNullAt(0)) } + + test("serialize w/ kryo") { + val row = Seq((1, Seq(1), Map(1 -> 1), BigDecimal(1))).toDF().first() + val serializer = new SparkSqlSerializer(TestSQLContext.sparkContext.getConf) + val instance = serializer.newInstance() + val ser = instance.serialize(row) + val de = instance.deserialize(ser).asInstanceOf[Row] + assert(de === row) + } + + test("get values by field name on Row created via .toDF") { + val row = Seq((1, Seq(1))).toDF("a", "b").first() + assert(row.getAs[Int]("a") === 1) + assert(row.getAs[Seq[Int]]("b") === Seq(1)) + + intercept[IllegalArgumentException]{ + row.getAs[Int]("c") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a3c0076e16d6c..0ab8558c1db13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.sql -import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.catalyst.errors.DialectException +import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ -import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types._ - import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} +import org.apache.spark.sql.types._ + +/** A SQL Dialect for testing purpose, and it can not be nested type */ +class MyDialect extends DefaultDialect class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. @@ -49,6 +51,16 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) } + test("support table.star") { + checkAnswer( + sql( + """ + |SELECT r.* + |FROM testData l join testData2 r on (l.key = r.a) + """.stripMargin), + Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) + } + test("self join with alias in agg") { Seq(1,2,3) .map(i => (i, i.toString)) @@ -67,6 +79,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) } + test("SQL Dialect Switching to a new SQL parser") { + val newContext = new SQLContext(TestSQLContext.sparkContext) + newContext.setConf("spark.sql.dialect", classOf[MyDialect].getCanonicalName()) + assert(newContext.getSQLDialect().getClass === classOf[MyDialect]) + assert(newContext.sql("SELECT 1").collect() === Array(Row(1))) + } + + test("SQL Dialect Switch to an invalid parser with alias") { + val newContext = new SQLContext(TestSQLContext.sparkContext) + newContext.sql("SET spark.sql.dialect=MyTestClass") + intercept[DialectException] { + newContext.sql("SELECT 1") + } + // test if the dialect set back to DefaultSQLDialect + assert(newContext.getSQLDialect().getClass === classOf[DefaultDialect]) + } + test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( sql("SELECT a FROM testData2 SORT BY a"), @@ -102,11 +131,105 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT ABS(2.5)"), Row(2.5)) } - + test("aggregation with codegen") { val originalValue = conf.codegenEnabled setConf(SQLConf.CODEGEN_ENABLED, "true") - sql("SELECT key FROM testData GROUP BY key").collect() + // Prepare a table that we can group some rows. + table("testData") + .unionAll(table("testData")) + .unionAll(table("testData")) + .registerTempTable("testData3x") + + def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = { + val df = sql(sqlText) + // First, check if we have GeneratedAggregate. + var hasGeneratedAgg = false + df.queryExecution.executedPlan.foreach { + case generatedAgg: GeneratedAggregate => hasGeneratedAgg = true + case _ => + } + if (!hasGeneratedAgg) { + fail( + s""" + |Codegen is enabled, but query $sqlText does not have GeneratedAggregate in the plan. + |${df.queryExecution.simpleString} + """.stripMargin) + } + // Then, check results. + checkAnswer(df, expectedResults) + } + + // Just to group rows. + testCodeGen( + "SELECT key FROM testData3x GROUP BY key", + (1 to 100).map(Row(_))) + // COUNT + testCodeGen( + "SELECT key, count(value) FROM testData3x GROUP BY key", + (1 to 100).map(i => Row(i, 3))) + testCodeGen( + "SELECT count(key) FROM testData3x", + Row(300) :: Nil) + // COUNT DISTINCT ON int + testCodeGen( + "SELECT value, count(distinct key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 1))) + testCodeGen( + "SELECT count(distinct key) FROM testData3x", + Row(100) :: Nil) + // SUM + testCodeGen( + "SELECT value, sum(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 3 * i))) + testCodeGen( + "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x", + Row(5050 * 3, 5050 * 3.0) :: Nil) + // AVERAGE + testCodeGen( + "SELECT value, avg(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT avg(key) FROM testData3x", + Row(50.5) :: Nil) + // MAX + testCodeGen( + "SELECT value, max(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT max(key) FROM testData3x", + Row(100) :: Nil) + // MIN + testCodeGen( + "SELECT value, min(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT min(key) FROM testData3x", + Row(1) :: Nil) + // Some combinations. + testCodeGen( + """ + |SELECT + | value, + | sum(key), + | max(key), + | min(key), + | avg(key), + | count(key), + | count(distinct key) + |FROM testData3x + |GROUP BY value + """.stripMargin, + (1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1))) + testCodeGen( + "SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 1, 50.5, 300, 100) :: Nil) + // Aggregate with Code generation handling all null values + testCodeGen( + "SELECT sum('a'), avg('a'), count(null) FROM testData", + Row(0, null, 0) :: Nil) + + dropTempTable("testData3x") setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } @@ -182,7 +305,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')"), + """ + |SELECT time FROM timestamps + |WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002') + """.stripMargin), Seq(Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")), Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002")))) @@ -248,7 +374,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row("1")) } - def sortTest() = { + def sortTest(): Unit = { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"), Seq(Row(1,1), Row(1,2), Row(2,1), Row(2,2), Row(3,1), Row(3,2))) @@ -304,6 +430,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { setConf(SQLConf.EXTERNAL_SORT, before.toString) } + test("SPARK-6927 sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.EXTERNAL_SORT, "false") + setConf(SQLConf.CODEGEN_ENABLED, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + + test("SPARK-6927 external sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.CODEGEN_ENABLED, "true") + setConf(SQLConf.EXTERNAL_SORT, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + test("limit") { checkAnswer( sql("SELECT * FROM testData LIMIT 10"), @@ -318,6 +464,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("CTE feature") { + checkAnswer( + sql("with q1 as (select * from testData limit 10) select * from q1"), + testData.take(10).toSeq) + + checkAnswer( + sql(""" + |with q1 as (select * from testData where key= '5'), + |q2 as (select * from testData where key = '4') + |select * from q1 union all select * from q2""".stripMargin), + Row(5, "5") :: Row(4, "4") :: Nil) + + } + + test("Allow only a single WITH clause per query") { + intercept[RuntimeException] { + sql("with q1 as (select * from testData) with q2 as (select * from q1) select * from q2") + } + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), @@ -327,7 +493,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("from follow multiple brackets") { checkAnswer(sql( - "select key from ((select * from testData limit 1) union all (select * from testData limit 1)) x limit 1"), + """ + |select key from ((select * from testData limit 1) + | union all (select * from testData limit 1)) x limit 1 + """.stripMargin), Row(1) ) @@ -337,7 +506,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ) checkAnswer(sql( - "select key from (select * from testData limit 1 union all select * from testData limit 1) x limit 1"), + """ + |select key from + | (select * from testData limit 1 union all select * from testData limit 1) x + | limit 1 + """.stripMargin), Row(1) ) } @@ -384,7 +557,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Seq(Row(1, 0), Row(2, 1))) checkAnswer( - sql("SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"), + sql( + """ + |SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3 + """.stripMargin), Row(2, 1, 2, 2, 1)) } @@ -997,9 +1173,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3483 Special chars in column names") { - val data = sparkContext.parallelize(Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) + val data = sparkContext.parallelize( + Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") - sql("SELECT `key?number1` FROM records") + sql("SELECT `key?number1`, `key.number2` FROM records") } test("SPARK-3814 Support Bitwise & operator") { @@ -1082,12 +1259,29 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-6145: ORDER BY test for nested fields") { + jsonRDD(sparkContext.makeRDD("""{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) + .registerTempTable("nestedOrder") + + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) + checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a"), Row(1)) + checkAnswer(sql("SELECT a.a.a FROM nestedOrder ORDER BY a.a.a"), Row(1)) + checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d"), Row(1)) + checkAnswer(sql("SELECT c[0].d FROM nestedOrder ORDER BY c[0].d"), Row(1)) + } + + test("SPARK-6145: special cases") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"b": [1]}, "b": [{"a": 1}], "c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY c0.a"), Row(1)) + checkAnswer(sql("SELECT b[0].a FROM t ORDER BY c0.a"), Row(1)) + } + + test("SPARK-6898: complete support for special chars in column names") { jsonRDD(sparkContext.makeRDD( - """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder") - // These should be successfully analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY a.b").queryExecution.analyzed - sql("SELECT a.b FROM nestedOrder ORDER BY a.b").queryExecution.analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a").queryExecution.analyzed - sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d").queryExecution.analyzed + """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) + .registerTempTable("t") + + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 17e923ca48502..3fa00fd9d0ccb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -80,7 +80,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) + new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectData") @@ -103,7 +103,8 @@ class ScalaReflectionRelationSuite extends FunSuite { val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectOptionalData") - assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) + assert(sql("SELECT * FROM reflectOptionalData").collect().head === + Row.fromSeq(Seq.fill(7)(null))) } // Equality is broken for Arrays, so we test that separately. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 637f59b2e68ca..225b51bd73d6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.test._ case class TestData(key: Int, value: String) @@ -199,11 +198,11 @@ object TestData { Salary(1, 1000.0) :: Nil).toDF() salary.registerTempTable("salary") - case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) + case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) val complexData = TestSQLContext.sparkContext.parallelize( - ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) - :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) + ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1), true) + :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2), false) :: Nil).toDF() complexData.registerTempTable("complexData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index be105c6e83594..d615542ab50a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -50,4 +50,10 @@ class UDFSuite extends QueryTest { .select($"ret.f1").head().getString(0) assert(result === "test") } + + test("udf that is transformed") { + udf.register("makeStruct", (x: Int, y: Int) => (x, y)) + // 1 + 1 is constant folded causing a transformation. + assert(sql("SELECT makeStruct(1 + 1, 2)").first().getAs[Row](0) === Row(2, 2)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index fe618e0e8e767..2672e20deadc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -23,13 +23,16 @@ import org.apache.spark.util.Utils import scala.beans.{BeanInfo, BeanProperty} +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{OpenHashSetUDT, HyperLogLogUDT} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ - +import org.apache.spark.util.collection.OpenHashSet @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { @@ -63,7 +66,7 @@ private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { } } - override def userClass = classOf[MyDenseVector] + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] private[spark] override def asNullable: MyDenseVectorUDT = this } @@ -119,4 +122,23 @@ class UserDefinedTypeSuite extends QueryTest { df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) } + + test("HyperLogLogUDT") { + val hyperLogLogUDT = HyperLogLogUDT + val hyperLogLog = new HyperLogLog(0.4) + (1 to 10).foreach(i => hyperLogLog.offer(Row(i))) + + val actual = hyperLogLogUDT.deserialize(hyperLogLogUDT.serialize(hyperLogLog)) + assert(actual.cardinality() === hyperLogLog.cardinality()) + assert(java.util.Arrays.equals(actual.getBytes, hyperLogLog.getBytes)) + } + + test("OpenHashSetUDT") { + val openHashSetUDT = new OpenHashSetUDT(IntegerType) + val set = new OpenHashSet[Int] + (1 to 10).foreach(i => set.add(i)) + + val actual = openHashSetUDT.deserialize(openHashSetUDT.serialize(set)) + assert(actual.iterator.toSet === set.iterator.toSet) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index fec487f1d2c82..7cefcf44061ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -34,7 +34,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) - def testColumnStats[T <: NativeType, U <: ColumnStats]( + def testColumnStats[T <: AtomicType, U <: ColumnStats]( columnStatsClass: Class[U], columnType: NativeColumnType[T], initialStatistics: Row): Unit = { @@ -55,8 +55,8 @@ class ColumnStatsSuite extends FunSuite { val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) - val values = rows.take(10).map(_(0).asInstanceOf[T#JvmType]) - val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + val values = rows.take(10).map(_(0).asInstanceOf[T#InternalType]) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] val stats = columnStats.collectedStatistics assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 5f08834f73c6b..1e105e259dce7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import java.sql.Timestamp +import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.spark.serializer.KryoRegistrator import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -65,7 +68,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(STRING, UTF8String("hello"), 4 + "hello".getBytes("utf-8").length) checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) @@ -73,7 +76,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(BINARY, binary, 4 + 4) val generic = Map(1 -> "a") - checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 8) } testNativeColumnType[BooleanType.type]( @@ -108,8 +111,8 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[StringType.type]( STRING, - (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") + (buffer: ByteBuffer, string: UTF8String) => { + val bytes = string.getBytes buffer.putInt(bytes.length) buffer.put(bytes) }, @@ -117,7 +120,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes) - new String(bytes, "utf-8") + UTF8String(bytes) }) testColumnType[BinaryType.type, Array[Byte]]( @@ -158,12 +161,47 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - def testNativeColumnType[T <: NativeType]( + test("CUSTOM") { + val conf = new SparkConf() + conf.set("spark.kryo.registrator", "org.apache.spark.sql.columnar.Registrator") + val serializer = new SparkSqlSerializer(conf).newInstance() + + val buffer = ByteBuffer.allocate(512) + val obj = CustomClass(Int.MaxValue,Long.MaxValue) + val serializedObj = serializer.serialize(obj).array() + + GENERIC.append(serializer.serialize(obj).array(), buffer) + buffer.rewind() + + val length = buffer.getInt + assert(length === serializedObj.length) + assert(13 == length) // id (1) + int (4) + long (8) + + val genericSerializedObj = SparkSqlSerializer.serialize(obj) + assert(length != genericSerializedObj.length) + assert(length < genericSerializedObj.length) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + serializer.deserialize(ByteBuffer.wrap(bytes)) + } + + buffer.rewind() + buffer.putInt(serializedObj.length).put(serializedObj) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + buffer.rewind() + serializer.deserialize(ByteBuffer.wrap(GENERIC.extract(buffer))) + } + } + + def testNativeColumnType[T <: AtomicType]( columnType: NativeColumnType[T], - putter: (ByteBuffer, T#JvmType) => Unit, - getter: (ByteBuffer) => T#JvmType): Unit = { + putter: (ByteBuffer, T#InternalType) => Unit, + getter: (ByteBuffer) => T#InternalType): Unit = { - testColumnType[T, T#JvmType](columnType, putter, getter) + testColumnType[T, T#InternalType](columnType, putter, getter) } def testColumnType[T <: DataType, JvmType]( @@ -229,3 +267,23 @@ class ColumnTypeSuite extends FunSuite with Logging { } } } + +private[columnar] final case class CustomClass(a: Int, b: Long) + +private[columnar] object CustomerSerializer extends Serializer[CustomClass] { + override def write(kryo: Kryo, output: Output, t: CustomClass) { + output.writeInt(t.a) + output.writeLong(t.b) + } + override def read(kryo: Kryo, input: Input, aClass: Class[CustomClass]): CustomClass = { + val a = input.readInt() + val b = input.readLong() + CustomClass(a,b) + } +} + +private[columnar] final class Registrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[CustomClass], CustomerSerializer) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index c7a40845db16c..75d993e563e06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,10 +24,10 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{Decimal, DataType, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, AtomicType} object ColumnarTestUtils { - def makeNullRow(length: Int) = { + def makeNullRow(length: Int): GenericMutableRow = { val row = new GenericMutableRow(length) (0 until length).foreach(row.setNullAt) row @@ -48,7 +48,7 @@ object ColumnarTestUtils { case FLOAT => Random.nextFloat() case DOUBLE => Random.nextDouble() case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) - case STRING => Random.nextString(Random.nextInt(32)) + case STRING => UTF8String(Random.nextString(Random.nextInt(32))) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) case DATE => Random.nextInt() @@ -91,9 +91,9 @@ object ColumnarTestUtils { row } - def makeUniqueValuesAndSingleValueRows[T <: NativeType]( + def makeUniqueValuesAndSingleValueRows[T <: AtomicType]( columnType: NativeColumnType[T], - count: Int) = { + count: Int): (Seq[T#InternalType], Seq[GenericMutableRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 27dfabca90217..56591d9dba29e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.columnar +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{DecimalType, Decimal} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -42,7 +44,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { .toDF().registerTempTable("sizeTst") cacheTable("sizeTst") assert( - table("sizeTst").queryExecution.logical.statistics.sizeInBytes > + table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > conf.autoBroadcastJoinThreshold) } @@ -132,4 +134,59 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) } + + test("test different data types") { + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val dataTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), MapType(StringType, LongType), struct) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD for the schema + val rdd = + sparkContext.parallelize((1 to 100), 10).map { i => + Row( + s"str${i}: test cache.", + s"binary${i}: test cache.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i), + (1 to i).toSeq, + (0 to i).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, + Row((i - 0.25).toFloat, (1 to i).toSeq)) + } + createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + // Cache the table. + sql("cache table InMemoryCache_different_data_types") + // Make sure the table is indeed cached. + val tableScan = table("InMemoryCache_different_data_types").queryExecution.executedPlan + assert( + isCached("InMemoryCache_different_data_types"), + "InMemoryCache_different_data_types should be cached.") + // Issue a query and check the results. + checkAnswer( + sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), + table("InMemoryCache_different_data_types").collect()) + dropTempTable("InMemoryCache_different_data_types") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index bb305355276bf..a0702144f942c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -31,7 +31,8 @@ class TestNullableColumnAccessor[T <: DataType, JvmType]( with NullableColumnAccessor object TestNullableColumnAccessor { - def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = { + def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) + : TestNullableColumnAccessor[T, JvmType] = { // Skips the column type ID buffer.getInt() new TestNullableColumnAccessor(buffer, columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 75a47498683f4..3a5605d2335d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -27,7 +27,8 @@ class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T with NullableColumnBuilder object TestNullableColumnBuilder { - def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = { + def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) + : TestNullableColumnBuilder[T, JvmType] = { val builder = new TestNullableColumnBuilder(columnType) builder.initialize(initialSize) builder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index e57bb06e7263b..2a0b701cad7fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -39,6 +39,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + // Enable in-memory table scan accumulators + setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") } override protected def afterAll(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index c82d9799359c7..64b70552eb047 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -24,14 +24,14 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) testDictionaryEncoding(new LongColumnStats, LONG) testDictionaryEncoding(new StringColumnStats, STRING) - def testDictionaryEncoding[T <: NativeType]( + def testDictionaryEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 88011631ee4e3..bfd99f143bedc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -33,7 +33,7 @@ class IntegralDeltaSuite extends FunSuite { columnType: NativeColumnType[I], scheme: CompressionScheme) { - def skeleton(input: Seq[I#JvmType]) { + def skeleton(input: Seq[I#InternalType]) { // ------------- // Tests encoder // ------------- @@ -120,13 +120,13 @@ class IntegralDeltaSuite extends FunSuite { case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) } - skeleton(input.map(_.asInstanceOf[I#JvmType])) + skeleton(input.map(_.asInstanceOf[I#InternalType])) } test(s"$scheme: long random series") { // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeleton(input.map(_.asInstanceOf[I#JvmType])) + skeleton(input.map(_.asInstanceOf[I#InternalType])) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 08df1db375097..fde7a4595be0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new NoopColumnStats, BOOLEAN) @@ -32,7 +32,7 @@ class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new LongColumnStats, LONG) testRunLengthEncoding(new StringColumnStats, STRING) - def testRunLengthEncoding[T <: NativeType]( + def testRunLengthEncoding[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 0b18b4119268f..5268dfe0aa03e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.types.NativeType +import org.apache.spark.sql.types.AtomicType -class TestCompressibleColumnBuilder[T <: NativeType]( +class TestCompressibleColumnBuilder[T <: AtomicType]( override val columnStats: ColumnStats, override val columnType: NativeColumnType[T], override val schemes: Seq[CompressionScheme]) @@ -32,10 +32,10 @@ class TestCompressibleColumnBuilder[T <: NativeType]( } object TestCompressibleColumnBuilder { - def apply[T <: NativeType]( + def apply[T <: AtomicType]( columnStats: ColumnStats, columnType: NativeColumnType[T], - scheme: CompressionScheme) = { + scheme: CompressionScheme): TestCompressibleColumnBuilder[T] = { val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) builder.initialize(0, "", useCompression = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala new file mode 100644 index 0000000000000..27f063d73a9a9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala @@ -0,0 +1,195 @@ +/* + * 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.sql.execution + +import java.sql.{Timestamp, Date} + +import org.scalatest.{FunSuite, BeforeAndAfterAll} + +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.serializer.Serializer +import org.apache.spark.ShuffleDependency +import org.apache.spark.sql.types._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.{MyDenseVectorUDT, QueryTest} + +class SparkSqlSerializer2DataTypeSuite extends FunSuite { + // Make sure that we will not use serializer2 for unsupported data types. + def checkSupported(dataType: DataType, isSupported: Boolean): Unit = { + val testName = + s"${if (dataType == null) null else dataType.toString} is " + + s"${if (isSupported) "supported" else "unsupported"}" + + test(testName) { + assert(SparkSqlSerializer2.support(Array(dataType)) === isSupported) + } + } + + checkSupported(null, isSupported = true) + checkSupported(NullType, isSupported = true) + checkSupported(BooleanType, isSupported = true) + checkSupported(ByteType, isSupported = true) + checkSupported(ShortType, isSupported = true) + checkSupported(IntegerType, isSupported = true) + checkSupported(LongType, isSupported = true) + checkSupported(FloatType, isSupported = true) + checkSupported(DoubleType, isSupported = true) + checkSupported(DateType, isSupported = true) + checkSupported(TimestampType, isSupported = true) + checkSupported(StringType, isSupported = true) + checkSupported(BinaryType, isSupported = true) + checkSupported(DecimalType(10, 5), isSupported = true) + checkSupported(DecimalType.Unlimited, isSupported = true) + + // For now, ArrayType, MapType, and StructType are not supported. + checkSupported(ArrayType(DoubleType, true), isSupported = false) + checkSupported(ArrayType(StringType, false), isSupported = false) + checkSupported(MapType(IntegerType, StringType, true), isSupported = false) + checkSupported(MapType(IntegerType, ArrayType(DoubleType), false), isSupported = false) + checkSupported(StructType(StructField("a", IntegerType, true) :: Nil), isSupported = false) + // UDTs are not supported right now. + checkSupported(new MyDenseVectorUDT, isSupported = false) +} + +abstract class SparkSqlSerializer2Suite extends QueryTest with BeforeAndAfterAll { + var allColumns: String = _ + val serializerClass: Class[Serializer] = + classOf[SparkSqlSerializer2].asInstanceOf[Class[Serializer]] + var numShufflePartitions: Int = _ + var useSerializer2: Boolean = _ + + override def beforeAll(): Unit = { + numShufflePartitions = conf.numShufflePartitions + useSerializer2 = conf.useSqlSerializer2 + + sql("set spark.sql.useSerializer2=true") + + val supportedTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType) + + val fields = supportedTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD with all data types supported by SparkSqlSerializer2. + val rdd = + sparkContext.parallelize((1 to 1000), 10).map { i => + Row( + s"str${i}: test serializer2.", + s"binary${i}: test serializer2.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i)) + } + + createDataFrame(rdd, schema).registerTempTable("shuffle") + + super.beforeAll() + } + + override def afterAll(): Unit = { + dropTempTable("shuffle") + sql(s"set spark.sql.shuffle.partitions=$numShufflePartitions") + sql(s"set spark.sql.useSerializer2=$useSerializer2") + super.afterAll() + } + + def checkSerializer[T <: Serializer]( + executedPlan: SparkPlan, + expectedSerializerClass: Class[T]): Unit = { + executedPlan.foreach { + case exchange: Exchange => + val shuffledRDD = exchange.execute().firstParent.asInstanceOf[ShuffledRDD[_, _, _]] + val dependency = shuffledRDD.getDependencies.head.asInstanceOf[ShuffleDependency[_, _, _]] + val serializerNotSetMessage = + s"Expected $expectedSerializerClass as the serializer of Exchange. " + + s"However, the serializer was not set." + val serializer = dependency.serializer.getOrElse(fail(serializerNotSetMessage)) + assert(serializer.getClass === expectedSerializerClass) + case _ => // Ignore other nodes. + } + } + + test("key schema and value schema are not nulls") { + val df = sql(s"SELECT DISTINCT ${allColumns} FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + table("shuffle").collect()) + } + + test("value schema is null") { + val df = sql(s"SELECT col0 FROM shuffle ORDER BY col0") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + assert( + df.map(r => r.getString(0)).collect().toSeq === + table("shuffle").select("col0").map(r => r.getString(0)).collect().sorted.toSeq) + } + + test("no map output field") { + val df = sql(s"SELECT 1 + 1 FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, classOf[SparkSqlSerializer]) + } +} + +/** Tests SparkSqlSerializer2 with sort based shuffle without sort merge. */ +class SparkSqlSerializer2SortShuffleSuite extends SparkSqlSerializer2Suite { + override def beforeAll(): Unit = { + super.beforeAll() + // Sort merge will not be triggered. + sql("set spark.sql.shuffle.partitions = 200") + } + + test("key schema is null") { + val aggregations = allColumns.split(",").map(c => s"COUNT($c)").mkString(",") + val df = sql(s"SELECT $aggregations FROM shuffle") + checkSerializer(df.queryExecution.executedPlan, serializerClass) + checkAnswer( + df, + Row(1000, 1000, 0, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000, 1000)) + } +} + +/** For now, we will use SparkSqlSerializer for sort based shuffle with sort merge. */ +class SparkSqlSerializer2SortMergeShuffleSuite extends SparkSqlSerializer2Suite { + + // We are expecting SparkSqlSerializer. + override val serializerClass: Class[Serializer] = + classOf[SparkSqlSerializer].asInstanceOf[Class[Serializer]] + + override def beforeAll(): Unit = { + super.beforeAll() + // To trigger the sort merge. + sql("set spark.sql.shuffle.partitions = 201") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 4e9472c60249e..358d8cf06e463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -30,4 +30,4 @@ class DebuggingSuite extends FunSuite { test("DataFrame.typeCheck()") { testData.typeCheck() } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index cd737c0b62767..b165ab2b1deb3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -19,41 +19,54 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal import java.sql.DriverManager -import java.util.{Calendar, GregorianCalendar} +import java.util.{Calendar, GregorianCalendar, Properties} import org.apache.spark.sql.test._ +import org.apache.spark.sql.types._ +import org.h2.jdbc.JdbcSQLException import org.scalatest.{FunSuite, BeforeAndAfter} import TestSQLContext._ +import TestSQLContext.implicits._ class JDBCSuite extends FunSuite with BeforeAndAfter { val url = "jdbc:h2:mem:testdb0" + val urlWithUserAndPass = "jdbc:h2:mem:testdb0;user=testUser;password=testPass" var conn: java.sql.Connection = null val testBytes = Array[Byte](99.toByte, 134.toByte, 135.toByte, 200.toByte, 205.toByte) before { Class.forName("org.h2.Driver") - conn = DriverManager.getConnection(url) + // Extra properties that will be specified for our database. We need these to test + // usage of parameters from OPTIONS clause in queries. + val properties = new Properties() + properties.setProperty("user", "testUser") + properties.setProperty("password", "testPass") + properties.setProperty("rowId", "false") + + conn = DriverManager.getConnection(url, properties) conn.prepareStatement("create schema test").executeUpdate() - conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() + conn.prepareStatement( + "create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() - conn.prepareStatement("insert into test.people values ('joe', 3)").executeUpdate() + conn.prepareStatement( + "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() conn.commit() sql( s""" |CREATE TEMPORARY TABLE foobar |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.PEOPLE') + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) sql( s""" |CREATE TEMPORARY TABLE parts |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', - |partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', + | partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.inttypes (a INT, b BOOLEAN, c TINYINT, " @@ -67,12 +80,12 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE inttypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.INTTYPES') + |OPTIONS (url '$url', dbtable 'TEST.INTTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.strtypes (a BINARY(20), b VARCHAR(20), " + "c VARCHAR_IGNORECASE(20), d CHAR(20), e BLOB, f CLOB)").executeUpdate() - var stmt = conn.prepareStatement("insert into test.strtypes values (?, ?, ?, ?, ?, ?)") + val stmt = conn.prepareStatement("insert into test.strtypes values (?, ?, ?, ?, ?, ?)") stmt.setBytes(1, testBytes) stmt.setString(2, "Sensitive") stmt.setString(3, "Insensitive") @@ -84,7 +97,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE strtypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.STRTYPES') + |OPTIONS (url '$url', dbtable 'TEST.STRTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.timetypes (a TIME, b DATE, c TIMESTAMP)" @@ -96,7 +109,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE timetypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES') + |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) @@ -111,7 +124,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { s""" |CREATE TEMPORARY TABLE flttypes |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES') + |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. @@ -122,29 +135,36 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT *") { - assert(sql("SELECT * FROM foobar").collect().size == 3) + assert(sql("SELECT * FROM foobar").collect().size === 3) } test("SELECT * WHERE (simple predicates)") { - assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size == 1) + assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size === 2) + } + + test("SELECT * WHERE (quoted strings)") { + assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size === 1) } test("SELECT first field") { val names = sql("SELECT NAME FROM foobar").collect().map(x => x.getString(0)).sortWith(_ < _) - assert(names.size == 3) + assert(names.size === 3) assert(names(0).equals("fred")) - assert(names(1).equals("joe")) + assert(names(1).equals("joe 'foo' \"bar\"")) assert(names(2).equals("mary")) } test("SELECT second field") { val ids = sql("SELECT THEID FROM foobar").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("SELECT * partitioned") { @@ -152,45 +172,46 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT WHERE (simple predicates) partitioned") { - assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size == 1) + assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size === 1) } test("SELECT second field partitioned") { val ids = sql("SELECT THEID FROM parts").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("Basic API") { - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE").collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect().size === 3) } test("Partitioning via JDBCPartitioningInfo API") { - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", "THEID", 0, 4, 3).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3) + .collect.size === 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(TestSQLContext.jdbc(url, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect().size === 3) } test("H2 integral types") { val rows = sql("SELECT * FROM inttypes WHERE A IS NOT NULL").collect() - assert(rows.size == 1) - assert(rows(0).getInt(0) == 1) - assert(rows(0).getBoolean(1) == false) - assert(rows(0).getInt(2) == 3) - assert(rows(0).getInt(3) == 4) - assert(rows(0).getLong(4) == 1234567890123L) + assert(rows.size === 1) + assert(rows(0).getInt(0) === 1) + assert(rows(0).getBoolean(1) === false) + assert(rows(0).getInt(2) === 3) + assert(rows(0).getInt(3) === 4) + assert(rows(0).getLong(4) === 1234567890123L) } test("H2 null entries") { val rows = sql("SELECT * FROM inttypes WHERE A IS NULL").collect() - assert(rows.size == 1) + assert(rows.size === 1) assert(rows(0).isNullAt(0)) assert(rows(0).isNullAt(1)) assert(rows(0).isNullAt(2)) @@ -208,47 +229,77 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { assert(rows(0).getString(5).equals("I am a clob!")) } - test("H2 time types") { val rows = sql("SELECT * FROM timetypes").collect() val cal = new GregorianCalendar(java.util.Locale.ROOT) cal.setTime(rows(0).getAs[java.sql.Timestamp](0)) - assert(cal.get(Calendar.HOUR_OF_DAY) == 12) - assert(cal.get(Calendar.MINUTE) == 34) - assert(cal.get(Calendar.SECOND) == 56) + assert(cal.get(Calendar.HOUR_OF_DAY) === 12) + assert(cal.get(Calendar.MINUTE) === 34) + assert(cal.get(Calendar.SECOND) === 56) cal.setTime(rows(0).getAs[java.sql.Timestamp](1)) - assert(cal.get(Calendar.YEAR) == 1996) - assert(cal.get(Calendar.MONTH) == 0) - assert(cal.get(Calendar.DAY_OF_MONTH) == 1) + assert(cal.get(Calendar.YEAR) === 1996) + assert(cal.get(Calendar.MONTH) === 0) + assert(cal.get(Calendar.DAY_OF_MONTH) === 1) cal.setTime(rows(0).getAs[java.sql.Timestamp](2)) - assert(cal.get(Calendar.YEAR) == 2002) - assert(cal.get(Calendar.MONTH) == 1) - assert(cal.get(Calendar.DAY_OF_MONTH) == 20) - assert(cal.get(Calendar.HOUR) == 11) - assert(cal.get(Calendar.MINUTE) == 22) - assert(cal.get(Calendar.SECOND) == 33) - assert(rows(0).getAs[java.sql.Timestamp](2).getNanos == 543543543) + assert(cal.get(Calendar.YEAR) === 2002) + assert(cal.get(Calendar.MONTH) === 1) + assert(cal.get(Calendar.DAY_OF_MONTH) === 20) + assert(cal.get(Calendar.HOUR) === 11) + assert(cal.get(Calendar.MINUTE) === 22) + assert(cal.get(Calendar.SECOND) === 33) + assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543543) + } + + test("test DATE types") { + val rows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").collect() + val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").cache().collect() + assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + } + + test("test DATE types in cache") { + val rows = TestSQLContext.jdbc(urlWithUserAndPass, "TEST.TIMETYPES").collect() + TestSQLContext + .jdbc(urlWithUserAndPass, "TEST.TIMETYPES").cache().registerTempTable("mycached_date") + val cachedRows = sql("select * from mycached_date").collect() + assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) } test("H2 floating-point types") { val rows = sql("SELECT * FROM flttypes").collect() - assert(rows(0).getDouble(0) == 1.00000000000000022) // Yes, I meant ==. - assert(rows(0).getDouble(1) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. + assert(rows(0).getDouble(1) === 1.00000011920928955) // Yes, I meant ==. assert(rows(0).getAs[BigDecimal](2) .equals(new BigDecimal("123456789012345.54321543215432100000"))) + assert(rows(0).schema.fields(2).dataType === DecimalType(40, 20)) } - test("SQL query as table name") { sql( s""" |CREATE TEMPORARY TABLE hack |USING org.apache.spark.sql.jdbc - |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)') + |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)', + | user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) val rows = sql("SELECT * FROM hack").collect() - assert(rows(0).getDouble(0) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000011920928955) // Yes, I meant ==. // For some reason, H2 computes this square incorrectly... assert(math.abs(rows(0).getDouble(1) - 1.00000023841859331) < 1e-12) } + + test("Pass extra properties via OPTIONS") { + // We set rowId to false during setup, which means that _ROWID_ column should be absent from + // all tables. If rowId is true (default), the query below doesn't throw an exception. + intercept[JdbcSQLException] { + sql( + s""" + |CREATE TEMPORARY TABLE abc + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable '(SELECT _ROWID_ FROM test.people)', + | user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 706c966ee05f5..fd0e2746dc045 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -380,8 +380,10 @@ class JsonSuite extends QueryTest { sql("select * from jsonTable"), Row("true", 11L, null, 1.1, "13.1", "str1") :: Row("12", null, new java.math.BigDecimal("21474836470.9"), null, null, "true") :: - Row("false", 21474836470L, new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: - Row(null, 21474836570L, new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil + Row("false", 21474836470L, + new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: + Row(null, 21474836570L, + new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil ) // Number and Boolean conflict: resolve the type as number in this query. @@ -404,7 +406,8 @@ class JsonSuite extends QueryTest { // Widening to DecimalType checkAnswer( sql("select num_num_2 + 1.2 from jsonTable where num_num_2 > 1.1"), - Row(new java.math.BigDecimal("21474836472.1")) :: Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil + Row(new java.math.BigDecimal("21474836472.1")) :: + Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil ) // Widening to DoubleType @@ -892,8 +895,7 @@ class JsonSuite extends QueryTest { ) } - test("SPARK-4228 DataFrame to JSON") - { + test("SPARK-4228 DataFrame to JSON") { val schema1 = StructType( StructField("f1", IntegerType, false) :: StructField("f2", StringType, false) :: @@ -913,8 +915,10 @@ class JsonSuite extends QueryTest { df1.registerTempTable("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() + // scalastyle:off assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") + // scalastyle:on val schema2 = StructType( StructField("f1", StructType( @@ -968,7 +972,8 @@ class JsonSuite extends QueryTest { // Access elements of a BigInteger array (we use DecimalType internally). checkAnswer( - sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from complexTable"), + sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] " + + " from complexTable"), Row(new java.math.BigDecimal("922337203685477580700"), new java.math.BigDecimal("-922337203685477580800"), null) ) @@ -1008,7 +1013,8 @@ class JsonSuite extends QueryTest { // Access elements of an array field of a struct. checkAnswer( - sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from complexTable"), + sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] " + + "from complexTable"), Row(5, null) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index 4d32e84fc1115..10d0ede4dc0dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -22,7 +22,7 @@ import parquet.filter2.predicate.Operators._ import parquet.filter2.predicate.{FilterPredicate, Operators} import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, Predicate, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext @@ -321,6 +321,23 @@ class ParquetDataSourceOnFilterSuite extends ParquetFilterSuiteBase with BeforeA override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6554: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + checkAnswer( + sqlContext.parquetFile(path).filter("part = 1"), + (1 to 3).map(i => Row(i, i.toString, 1))) + } + } + } } class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with BeforeAndAfterAll { @@ -333,4 +350,26 @@ class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with Before override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6742: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + val df = DataFrame(sqlContext, org.apache.spark.sql.parquet.ParquetRelation( + path, + Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext, + Seq(AttributeReference("part", IntegerType, false)()) )) + + checkAnswer( + df.filter("a = 1 or part = 1"), + (1 to 3).map(i => Row(1, i, i.toString))) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 5438095addeaf..b504842053690 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -135,6 +135,21 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } + test("date type") { + def makeDateRDD(): DataFrame = + sparkContext + .parallelize(0 to 1000) + .map(i => Tuple1(DateUtils.toJavaDate(i))) + .toDF() + .select($"_1") + + withTempPath { dir => + val data = makeDateRDD() + data.saveAsParquetFile(dir.getCanonicalPath) + checkAnswer(parquetFile(dir.getCanonicalPath), data.collect().toSeq) + } + } + test("map") { val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) checkParquetFile(data) @@ -203,7 +218,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } test("compression codec") { - def compressionCodecFor(path: String) = { + def compressionCodecFor(path: String): String = { val codecs = ParquetTypesConverter .readMetaData(new Path(path), Some(configuration)) .getBlocks @@ -366,6 +381,28 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } + + test("SPARK-6352 DirectParquetOutputCommitter") { + // Write to a parquet file and let it fail. + // _temporary should be missing if direct output committer works. + try { + configuration.set("spark.sql.parquet.output.committer.class", + "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") + sqlContext.udf.register("div0", (x: Int) => x / 0) + withTempPath { dir => + intercept[org.apache.spark.SparkException] { + sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) + } + val path = new Path(dir.getCanonicalPath, "_temporary") + val fs = path.getFileSystem(configuration) + assert(!fs.exists(path)) + } + } + finally { + configuration.set("spark.sql.parquet.output.committer.class", + "parquet.hadoop.ParquetOutputCommitter") + } + } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala index adb3c9391f6c2..b7561ce7298cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala @@ -45,11 +45,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { assert(inferPartitionColumnValue(raw, defaultPartitionName) === literal) } - check("10", Literal(10, IntegerType)) - check("1000000000000000", Literal(1000000000000000L, LongType)) - check("1.5", Literal(1.5, FloatType)) - check("hello", Literal("hello", StringType)) - check(defaultPartitionName, Literal(null, NullType)) + check("10", Literal.create(10, IntegerType)) + check("1000000000000000", Literal.create(1000000000000000L, LongType)) + check("1.5", Literal.create(1.5, FloatType)) + check("hello", Literal.create("hello", StringType)) + check(defaultPartitionName, Literal.create(null, NullType)) } test("parse partition") { @@ -75,22 +75,22 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest { "file://path/a=10", PartitionValues( ArrayBuffer("a"), - ArrayBuffer(Literal(10, IntegerType)))) + ArrayBuffer(Literal.create(10, IntegerType)))) check( "file://path/a=10/b=hello/c=1.5", PartitionValues( ArrayBuffer("a", "b", "c"), ArrayBuffer( - Literal(10, IntegerType), - Literal("hello", StringType), - Literal(1.5, FloatType)))) + Literal.create(10, IntegerType), + Literal.create("hello", StringType), + Literal.create(1.5, FloatType)))) check( "file://path/a=10/b_hello/c=1.5", PartitionValues( ArrayBuffer("c"), - ArrayBuffer(Literal(1.5, FloatType)))) + ArrayBuffer(Literal.create(1.5, FloatType)))) checkThrows[AssertionError]("file://path/=10", "Empty partition column name") checkThrows[AssertionError]("file://path/a=", "Empty partition column value") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index ad880e2bc3679..c964b6d984557 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -57,7 +57,7 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { |} """.stripMargin) - testSchema[(Byte, Short, Int, Long)]( + testSchema[(Byte, Short, Int, Long, java.sql.Date)]( "logical integral types", """ |message root { @@ -65,6 +65,7 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { | required int32 _2 (INT_16); | required int32 _3 (INT_32); | required int64 _4 (INT_64); + | optional int32 _5 (DATE); |} """.stripMargin) @@ -179,10 +180,12 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { val caseClassString = "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))" + // scalastyle:off val jsonString = """ |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]} """.stripMargin + // scalastyle:on val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString) val fromJson = ParquetTypesConverter.convertFromString(jsonString) @@ -211,8 +214,11 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { StructField("UPPERCase", IntegerType, nullable = true)))) } - // Conflicting field count - assert(intercept[Throwable] { + // MetaStore schema is subset of parquet schema + assertResult( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = false)))) { + ParquetRelation2.mergeMetastoreParquetSchema( StructType(Seq( StructField("uppercase", DoubleType, nullable = false))), @@ -220,13 +226,56 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { StructType(Seq( StructField("lowerCase", BinaryType), StructField("UPPERCase", IntegerType, nullable = true)))) + } + + // Metastore schema contains additional non-nullable fields. + assert(intercept[Throwable] { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("uppercase", DoubleType, nullable = false), + StructField("lowerCase", BinaryType, nullable = false))), + + StructType(Seq( + StructField("UPPERCase", IntegerType, nullable = true)))) }.getMessage.contains("detected conflicting schemas")) - // Conflicting field names + // Conflicting non-nullable field names intercept[Throwable] { ParquetRelation2.mergeMetastoreParquetSchema( - StructType(Seq(StructField("lower", StringType))), + StructType(Seq(StructField("lower", StringType, nullable = false))), StructType(Seq(StructField("lowerCase", BinaryType)))) } } + + test("merge missing nullable fields from Metastore schema") { + // Standard case: Metastore schema contains additional nullable fields not present + // in the Parquet file schema. + assertResult( + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true)))) { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + } + + // Merge should fail if the Metastore contains any additional fields that are not + // nullable. + assert(intercept[Throwable] { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = false))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + }.getMessage.contains("detected conflicting schemas")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 54af50c6e10ad..ca25751b9583d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -24,17 +25,17 @@ class DDLScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt, parameters("Table"))(sqlContext) } } -case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, - new MetadataBuilder().putString("comment", "test comment").build()), + new MetadataBuilder().putString("comment", s"test comment $table").build()), StructField("stringType", StringType, nullable = false), StructField("dateType", DateType, nullable = false), StructField("timestampType", TimestampType, nullable = false), @@ -57,8 +58,9 @@ case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLConte )) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to). - map(e => Row(s"people$e", e * 2)) + override def buildScan(): RDD[Row] = { + sqlContext.sparkContext.parallelize(from to to).map(e => Row(s"people$e", e * 2)) + } } class DDLTestSuite extends DataSourceTest { @@ -71,7 +73,8 @@ class DDLTestSuite extends DataSourceTest { |USING org.apache.spark.sql.sources.DDLScanSource |OPTIONS ( | From '1', - | To '10' + | To '10', + | Table 'test1' |) """.stripMargin) } @@ -79,7 +82,7 @@ class DDLTestSuite extends DataSourceTest { sqlTest( "describe ddlPeople", Seq( - Row("intType", "int", "test comment"), + Row("intType", "int", "test comment test1"), Row("stringType", "string", ""), Row("dateType", "date", ""), Row("timestampType", "timestamp", ""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 91c6367371f15..33c67355967dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -32,6 +32,10 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { override val extendedResolutionRules = PreInsertCastAndRename :: Nil + + override val extendedCheckRules = Seq( + sources.PreWriteCheck(catalog) + ) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index ffeccf0b69394..cb5e5147ff189 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -35,20 +36,25 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL extends BaseRelation with PrunedFilteredScan { - override def schema = + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: - StructField("b", IntegerType, nullable = false) :: Nil) + StructField("b", IntegerType, nullable = false) :: + StructField("c", StringType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) + case "c" => (i: Int) => + val c = (i - 1 + 'a').toChar.toString + Seq(c * 5 + c.toUpperCase() * 5) } FiltersPushed.list = filters - def translateFilter(filter: Filter): Int => Boolean = filter match { + // Predicate test on integer column + def translateFilterOnA(filter: Filter): Int => Boolean = filter match { case EqualTo("a", v) => (a: Int) => a == v case LessThan("a", v: Int) => (a: Int) => a < v case LessThanOrEqual("a", v: Int) => (a: Int) => a <= v @@ -57,13 +63,27 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL case In("a", values) => (a: Int) => values.map(_.asInstanceOf[Int]).toSet.contains(a) case IsNull("a") => (a: Int) => false // Int can't be null case IsNotNull("a") => (a: Int) => true - case Not(pred) => (a: Int) => !translateFilter(pred)(a) - case And(left, right) => (a: Int) => translateFilter(left)(a) && translateFilter(right)(a) - case Or(left, right) => (a: Int) => translateFilter(left)(a) || translateFilter(right)(a) + case Not(pred) => (a: Int) => !translateFilterOnA(pred)(a) + case And(left, right) => (a: Int) => + translateFilterOnA(left)(a) && translateFilterOnA(right)(a) + case Or(left, right) => (a: Int) => + translateFilterOnA(left)(a) || translateFilterOnA(right)(a) case _ => (a: Int) => true } - def eval(a: Int) = !filters.map(translateFilter(_)(a)).contains(false) + // Predicate test on string column + def translateFilterOnC(filter: Filter): String => Boolean = filter match { + case StringStartsWith("c", v) => _.startsWith(v) + case StringEndsWith("c", v) => _.endsWith(v) + case StringContains("c", v) => _.contains(v) + case _ => (c: String) => true + } + + def eval(a: Int) = { + val c = (a - 1 + 'a').toChar.toString * 5 + (a - 1 + 'a').toChar.toString.toUpperCase() * 5 + !filters.map(translateFilterOnA(_)(a)).contains(false) && + !filters.map(translateFilterOnC(_)(c)).contains(false) + } sqlContext.sparkContext.parallelize(from to to).filter(eval).map(i => Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) @@ -93,7 +113,8 @@ class FilteredScanSuite extends DataSourceTest { sqlTest( "SELECT * FROM oneToTenFiltered", - (1 to 10).map(i => Row(i, i * 2)).toSeq) + (1 to 10).map(i => Row(i, i * 2, (i - 1 + 'a').toChar.toString * 5 + + (i - 1 + 'a').toChar.toString.toUpperCase() * 5)).toSeq) sqlTest( "SELECT a, b FROM oneToTenFiltered", @@ -128,41 +149,53 @@ class FilteredScanSuite extends DataSourceTest { (2 to 10 by 2).map(i => Row(i, i)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a = 1", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a = 1", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IN (1,3,5)", - Seq(1,3,5).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a IN (1,3,5)", + Seq(1,3,5).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE A = 1", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE A = 1", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE b = 2", - Seq(1).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE b = 2", + Seq(1).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IS NULL", + "SELECT a, b FROM oneToTenFiltered WHERE a IS NULL", Seq.empty[Row]) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a IS NOT NULL", + "SELECT a, b FROM oneToTenFiltered WHERE a IS NOT NULL", (1 to 10).map(i => Row(i, i * 2)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a < 5 AND a > 1", + "SELECT a, b FROM oneToTenFiltered WHERE a < 5 AND a > 1", (2 to 4).map(i => Row(i, i * 2)).toSeq) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", - Seq(1, 2, 9, 10).map(i => Row(i, i * 2)).toSeq) + "SELECT a, b FROM oneToTenFiltered WHERE a < 3 OR a > 8", + Seq(1, 2, 9, 10).map(i => Row(i, i * 2))) sqlTest( - "SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", + "SELECT a, b FROM oneToTenFiltered WHERE NOT (a < 6)", (6 to 10).map(i => Row(i, i * 2)).toSeq) + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like 'c%'", + Seq(Row(3, 3 * 2, "c" * 5 + "C" * 5))) + + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%D'", + Seq(Row(4, 4 * 2, "d" * 5 + "D" * 5))) + + sqlTest( + "SELECT a, b, c FROM oneToTenFiltered WHERE c like '%eE%'", + Seq(Row(5, 5 * 2, "e" * 5 + "E" * 5))) + testPushDown("SELECT * FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT a FROM oneToTenFiltered WHERE A = 1", 1) testPushDown("SELECT b FROM oneToTenFiltered WHERE A = 1", 1) @@ -193,6 +226,15 @@ class FilteredScanSuite extends DataSourceTest { testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", 4) testPushDown("SELECT * FROM oneToTenFiltered WHERE NOT (a < 6)", 5) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like 'c%'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like 'C%'", 0) + + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%D'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%d'", 0) + + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%eE%'", 1) + testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%Ee%'", 0) + def testPushDown(sqlString: String, expectedCount: Int): Unit = { test(s"PushDown Returns $expectedCount: $sqlString") { val queryExecution = sql(sqlString).queryExecution diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 08fb5380dc026..6a1ddf2f8e98b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -34,12 +35,12 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo extends BaseRelation with PrunedScan { - override def schema = + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: StructField("b", IntegerType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String]) = { + override def buildScan(requiredColumns: Array[String]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 43bc8eb2d11a7..cb287ba85c1f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -114,4 +114,4 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { message.contains("Append mode is not supported"), "We should complain that 'Append mode is not supported' for JSON source.") } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 7928600ac2fb5..3b47b8adf313b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import java.sql.{Timestamp, Date} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -35,10 +36,10 @@ class SimpleScanSource extends RelationProvider { case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) + override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) } class AllDataTypesScanSource extends SchemaRelationProvider { @@ -57,9 +58,9 @@ case class AllDataTypesScan( extends BaseRelation with TableScan { - override def schema = userSpecifiedSchema + override def schema: StructType = userSpecifiedSchema - override def buildScan() = { + override def buildScan(): RDD[Row] = { sqlContext.sparkContext.parallelize(from to to).map { i => Row( s"str_$i", @@ -73,7 +74,7 @@ case class AllDataTypesScan( i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -81,7 +82,7 @@ case class AllDataTypesScan( Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) } } } @@ -102,7 +103,7 @@ class TableScanSuite extends DataSourceTest { i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -110,7 +111,7 @@ class TableScanSuite extends DataSourceTest { Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) }.toSeq before { @@ -265,7 +266,7 @@ class TableScanSuite extends DataSourceTest { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(new Date((i + 2) * 8640000)))).toSeq) + (1 to 10).map(i => Row(Seq(new Date(1970, 1, i + 1)))).toSeq) test("Caching") { // Cached Query Execution diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a96b1ffc26966..437f697d25bf3 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -44,7 +44,6 @@ com.google.guava guava - runtime ${hive.group} @@ -58,6 +57,18 @@ ${hive.group} hive-beeline + + + org.seleniumhq.selenium + selenium-java + test + + + io.netty + netty + + + target/scala-${scala.binary.version}/classes diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index 59f3a75768082..48ac9062af96a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import scala.collection.JavaConversions._ -import org.apache.commons.lang.exception.ExceptionUtils +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse @@ -61,7 +61,7 @@ private[hive] abstract class AbstractSparkSQLDriver( } catch { case cause: Throwable => logError(s"Failed in [$command]", cause) - new CommandProcessorResponse(1, ExceptionUtils.getFullStackTrace(cause), null) + new CommandProcessorResponse(1, ExceptionUtils.getStackTrace(cause), null) } } 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 c3a3f8c0f41df..0be5a92c2546c 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 @@ -22,12 +22,18 @@ 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.server.{HiveServer2, ServerOptionsProcessor} +import org.apache.spark.sql.SQLConf -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListener} +import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListenerApplicationEnd, SparkListener} +import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab +import org.apache.spark.util.Utils + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a @@ -35,6 +41,8 @@ import org.apache.spark.scheduler.{SparkListenerApplicationEnd, SparkListener} */ object HiveThriftServer2 extends Logging { var LOG = LogFactory.getLog(classOf[HiveServer2]) + var uiTab: Option[ThriftServerTab] = _ + var listener: HiveThriftServer2Listener = _ /** * :: DeveloperApi :: @@ -45,7 +53,13 @@ object HiveThriftServer2 extends Logging { val server = new HiveThriftServer2(sqlContext) server.init(sqlContext.hiveconf) server.start() - sqlContext.sparkContext.addSparkListener(new HiveThriftServer2Listener(server)) + listener = new HiveThriftServer2Listener(server, sqlContext.conf) + sqlContext.sparkContext.addSparkListener(listener) + uiTab = if (sqlContext.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + Some(new ThriftServerTab(sqlContext.sparkContext)) + } else { + None + } } def main(args: Array[String]) { @@ -57,20 +71,23 @@ object HiveThriftServer2 extends Logging { logInfo("Starting SparkContext") SparkSQLEnv.init() - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) + Utils.addShutdownHook { () => + SparkSQLEnv.stop() + uiTab.foreach(_.detach()) + } try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) server.init(SparkSQLEnv.hiveContext.hiveconf) server.start() logInfo("HiveThriftServer2 started") - SparkSQLEnv.sparkContext.addSparkListener(new HiveThriftServer2Listener(server)) + listener = new HiveThriftServer2Listener(server, SparkSQLEnv.hiveContext.conf) + SparkSQLEnv.sparkContext.addSparkListener(listener) + uiTab = if (SparkSQLEnv.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) + } else { + None + } } catch { case e: Exception => logError("Error starting HiveThriftServer2", e) @@ -78,15 +95,140 @@ object HiveThriftServer2 extends Logging { } } + private[thriftserver] class SessionInfo( + val sessionId: String, + val startTimestamp: Long, + val ip: String, + val userName: String) { + var finishTimestamp: Long = 0L + var totalExecution: Int = 0 + def totalTime: Long = { + if (finishTimestamp == 0L) { + System.currentTimeMillis - startTimestamp + } else { + finishTimestamp - startTimestamp + } + } + } + + private[thriftserver] object ExecutionState extends Enumeration { + val STARTED, COMPILED, FAILED, FINISHED = Value + type ExecutionState = Value + } + + private[thriftserver] class ExecutionInfo( + val statement: String, + val sessionId: String, + val startTimestamp: Long, + val userName: String) { + var finishTimestamp: Long = 0L + var executePlan: String = "" + var detail: String = "" + var state: ExecutionState.Value = ExecutionState.STARTED + val jobId: ArrayBuffer[String] = ArrayBuffer[String]() + var groupId: String = "" + def totalTime: Long = { + if (finishTimestamp == 0L) { + System.currentTimeMillis - startTimestamp + } else { + finishTimestamp - startTimestamp + } + } + } + + /** * A inner sparkListener called in sc.stop to clean up the HiveThriftServer2 */ - class HiveThriftServer2Listener(val server: HiveServer2) extends SparkListener { + private[thriftserver] class HiveThriftServer2Listener( + val server: HiveServer2, + val conf: SQLConf) extends SparkListener { + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { server.stop() } - } + val sessionList = new mutable.LinkedHashMap[String, SessionInfo] + val executionList = new mutable.LinkedHashMap[String, ExecutionInfo] + val retainedStatements = + conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT, "200").toInt + val retainedSessions = + conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT, "200").toInt + var totalRunning = 0 + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + for { + props <- Option(jobStart.properties) + groupId <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + (_, info) <- executionList if info.groupId == groupId + } { + info.jobId += jobStart.jobId.toString + info.groupId = groupId + } + } + + def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { + val info = new SessionInfo(sessionId, System.currentTimeMillis, ip, userName) + sessionList.put(sessionId, info) + trimSessionIfNecessary() + } + + def onSessionClosed(sessionId: String): Unit = { + sessionList(sessionId).finishTimestamp = System.currentTimeMillis + } + + def onStatementStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + userName: String = "UNKNOWN"): Unit = { + val info = new ExecutionInfo(statement, sessionId, System.currentTimeMillis, userName) + info.state = ExecutionState.STARTED + executionList.put(id, info) + trimExecutionIfNecessary() + sessionList(sessionId).totalExecution += 1 + executionList(id).groupId = groupId + totalRunning += 1 + } + + def onStatementParsed(id: String, executionPlan: String): Unit = { + executionList(id).executePlan = executionPlan + executionList(id).state = ExecutionState.COMPILED + } + + def onStatementError(id: String, errorMessage: String, errorTrace: String): Unit = { + executionList(id).finishTimestamp = System.currentTimeMillis + executionList(id).detail = errorMessage + executionList(id).state = ExecutionState.FAILED + totalRunning -= 1 + } + + def onStatementFinish(id: String): Unit = { + executionList(id).finishTimestamp = System.currentTimeMillis + executionList(id).state = ExecutionState.FINISHED + totalRunning -= 1 + } + + private def trimExecutionIfNecessary() = synchronized { + if (executionList.size > retainedStatements) { + val toRemove = math.max(retainedStatements / 10, 1) + executionList.take(toRemove).foreach { s => + executionList.remove(s._1) + } + } + } + + private def trimSessionIfNecessary() = synchronized { + if (sessionList.size > retainedSessions) { + val toRemove = math.max(retainedSessions / 10, 1) + sessionList.take(toRemove).foreach { s => + sessionList.remove(s._1) + } + } + + } + } } private[hive] class HiveThriftServer2(hiveContext: HiveContext) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 895688ab2ec2e..b7b6925aa87f7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -24,22 +24,21 @@ import java.util.{ArrayList => JArrayList} import jline.{ConsoleReader, History} -import org.apache.commons.lang.StringUtils +import org.apache.commons.lang3.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException -import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor} import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.util.Utils private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" @@ -101,13 +100,7 @@ private[hive] object SparkSQLCLIDriver { SessionState.start(sessionState) // Clean up after we exit - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) + Utils.addShutdownHook { () => SparkSQLEnv.stop() } // "-h" option has been passed, so connect to Hive thrift server. if (sessionState.getHost != null) { @@ -145,6 +138,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } + // use the specified database if specified + cli.processSelectDatabase(sessionState); + // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) @@ -194,8 +190,8 @@ private[hive] object SparkSQLCLIDriver { val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) - def promptWithCurrentDB = s"$prompt$currentDB" - def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + def promptWithCurrentDB: String = s"$prompt$currentDB" + def continuedPromptWithDBSpaces: String = continuedPrompt + ReflectionUtils.invokeStatic( classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) var currentPrompt = promptWithCurrentDB @@ -264,7 +260,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { - if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || + proc.isInstanceOf[AddResourceProcessor]) { val driver = new SparkSQLDriver driver.init() 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 158c225159720..97b46a01ba5b4 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 @@ -22,6 +22,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.{HiveShim, HiveContext} import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.util.Utils /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -37,7 +38,7 @@ private[hive] object SparkSQLEnv extends Logging { val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") sparkConf - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") + .setAppName(s"SparkSQL::${Utils.localHostName()}") .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala new file mode 100644 index 0000000000000..71b16b6bebffb --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -0,0 +1,190 @@ +/* + * 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.sql.hive.thriftserver.ui + +import java.util.Calendar +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.Logging +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{SessionInfo, ExecutionState, ExecutionInfo} +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui._ + + +/** Page for Spark Web UI that shows statistics of a streaming job */ +private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { + + private val listener = parent.listener + private val startTime = Calendar.getInstance().getTime() + private val emptyCell = "-" + + /** Render the page */ + def render(request: HttpServletRequest): Seq[Node] = { + val content = + generateBasicStats() ++ +
    ++ +

    + {listener.sessionList.size} session(s) are online, + running {listener.totalRunning} SQL statement(s) +

    ++ + generateSessionStatsTable() ++ + generateSQLStatsTable() + UIUtils.headerSparkPage("ThriftServer", content, parent, Some(5000)) + } + + /** Generate basic stats of the streaming program */ + private def generateBasicStats(): Seq[Node] = { + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
      +
    • + Started at: {startTime.toString} +
    • +
    • + Time since start: {formatDurationVerbose(timeSinceStart)} +
    • +
    + } + + /** Generate stats of batch statements of the thrift server program */ + private def generateSQLStatsTable(): Seq[Node] = { + val numStatement = listener.executionList.size + val table = if (numStatement > 0) { + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", + "Statement", "State", "Detail") + val dataRows = listener.executionList.values + + def generateDataRow(info: ExecutionInfo): Seq[Node] = { + val jobLink = info.jobId.map { id: String => + + [{id}] + + } + val detail = if(info.state == ExecutionState.FAILED) info.detail else info.executePlan +
    + + + + + + + + + {errorMessageCell(detail)} + + } + + Some(UIUtils.listingTable(headerRow, generateDataRow, + dataRows, false, None, Seq(null), false)) + } else { + None + } + + val content = +
    SQL Statistics
    ++ +
    +
      + {table.getOrElse("No statistics have been generated yet.")} +
    +
    + + content + } + + private def errorMessageCell(errorMessage: String): Seq[Node] = { + val isMultiline = errorMessage.indexOf('\n') >= 0 + val errorSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + errorMessage.substring(0, errorMessage.indexOf('\n')) + } else { + errorMessage + }) + val details = if (isMultiline) { + // scalastyle:off + + + details + ++ + + // scalastyle:on + } else { + "" + } + + } + + /** Generate stats of batch sessions of the thrift server program */ + private def generateSessionStatsTable(): Seq[Node] = { + val numBatches = listener.sessionList.size + val table = if (numBatches > 0) { + val dataRows = + listener.sessionList.values + val headerRow = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", + "Total Execute") + def generateDataRow(session: SessionInfo): Seq[Node] = { + val sessionLink = "%s/ThriftServer/session?id=%s" + .format(UIUtils.prependBaseUri(parent.basePath), session.sessionId) + + + + , + + + + + + } + Some(UIUtils.listingTable(headerRow, generateDataRow, dataRows, true, None, Seq(null), false)) + } else { + None + } + + val content = +
    Session Statistics
    ++ +
    +
      + {table.getOrElse("No statistics have been generated yet.")} +
    +
    + + content + } + + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCell) + } + + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { + def generateDataRow(data: Seq[String]): Seq[Node] = { + {data.map(d => )} + } + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala new file mode 100644 index 0000000000000..33ba038ecce73 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -0,0 +1,197 @@ +/* + * 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.sql.hive.thriftserver.ui + +import java.util.Calendar +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.Logging +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState} +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui._ + +/** Page for Spark Web UI that shows statistics of a streaming job */ +private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) + extends WebUIPage("session") with Logging { + + private val listener = parent.listener + private val startTime = Calendar.getInstance().getTime() + private val emptyCell = "-" + + /** Render the page */ + def render(request: HttpServletRequest): Seq[Node] = { + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + val sessionStat = listener.sessionList.find(stat => { + stat._1 == parameterId + }).getOrElse(null) + require(sessionStat != null, "Invalid sessionID[" + parameterId + "]") + + val content = + generateBasicStats() ++ +
    ++ +

    + User {sessionStat._2.userName}, + IP {sessionStat._2.ip}, + Session created at {formatDate(sessionStat._2.startTimestamp)}, + Total run {sessionStat._2.totalExecution} SQL +

    ++ + generateSQLStatsTable(sessionStat._2.sessionId) + UIUtils.headerSparkPage("ThriftServer", content, parent, Some(5000)) + } + + /** Generate basic stats of the streaming program */ + private def generateBasicStats(): Seq[Node] = { + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
      +
    • + Started at: {startTime.toString} +
    • +
    • + Time since start: {formatDurationVerbose(timeSinceStart)} +
    • +
    + } + + /** Generate stats of batch statements of the thrift server program */ + private def generateSQLStatsTable(sessionID: String): Seq[Node] = { + val executionList = listener.executionList + .filter(_._2.sessionId == sessionID) + val numStatement = executionList.size + val table = if (numStatement > 0) { + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", + "Statement", "State", "Detail") + val dataRows = executionList.values.toSeq.sortBy(_.startTimestamp).reverse + + def generateDataRow(info: ExecutionInfo): Seq[Node] = { + val jobLink = info.jobId.map { id: String => + + [{id}] + + } + val detail = if(info.state == ExecutionState.FAILED) info.detail else info.executePlan + + + + + + + + + + {errorMessageCell(detail)} + + } + + Some(UIUtils.listingTable(headerRow, generateDataRow, + dataRows, false, None, Seq(null), false)) + } else { + None + } + + val content = +
    SQL Statistics
    ++ +
    +
      + {table.getOrElse("No statistics have been generated yet.")} +
    +
    + + content + } + + private def errorMessageCell(errorMessage: String): Seq[Node] = { + val isMultiline = errorMessage.indexOf('\n') >= 0 + val errorSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + errorMessage.substring(0, errorMessage.indexOf('\n')) + } else { + errorMessage + }) + val details = if (isMultiline) { + // scalastyle:off + + + details + ++ + + // scalastyle:on + } else { + "" + } + + } + + /** Generate stats of batch sessions of the thrift server program */ + private def generateSessionStatsTable(): Seq[Node] = { + val numBatches = listener.sessionList.size + val table = if (numBatches > 0) { + val dataRows = + listener.sessionList.values.toSeq.sortBy(_.startTimestamp).reverse.map ( session => + Seq( + session.userName, + session.ip, + session.sessionId, + formatDate(session.startTimestamp), + formatDate(session.finishTimestamp), + formatDurationOption(Some(session.totalTime)), + session.totalExecution.toString + ) + ).toSeq + val headerRow = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", + "Total Execute") + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
    Session Statistics
    ++ +
    +
      + {table.getOrElse("No statistics have been generated yet.")} +
    +
    + + content + } + + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCell) + } + + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { + def generateDataRow(data: Seq[String]): Seq[Node] = { + {data.map(d => )} + } + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala new file mode 100644 index 0000000000000..343031f10c75c --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -0,0 +1,50 @@ +/* + * 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.sql.hive.thriftserver.ui + +import org.apache.spark.sql.hive.thriftserver.{HiveThriftServer2, SparkSQLEnv} +import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ +import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.{SparkContext, Logging, SparkException} + +/** + * Spark Web UI tab that shows statistics of a streaming job. + * This assumes the given SparkContext has enabled its SparkUI. + */ +private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) + extends SparkUITab(getSparkUI(sparkContext), "ThriftServer") with Logging { + + val parent = getSparkUI(sparkContext) + val listener = HiveThriftServer2.listener + + attachPage(new ThriftServerPage(this)) + attachPage(new ThriftServerSessionPage(this)) + parent.attachTab(this) + + def detach() { + getSparkUI(sparkContext).detachTab(this) + } +} + +private[thriftserver] object ThriftServerTab { + def getSparkUI(sparkContext: SparkContext): SparkUI = { + sparkContext.ui.getOrElse { + throw new SparkException("Parent SparkUI to attach this tab to not found!") + } + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 75738fa22b572..b070fa8eaa469 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, @@ -26,22 +25,31 @@ import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging import org.apache.spark.util.Utils -class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { +class CliSuite extends FunSuite with BeforeAndAfter with Logging { + val warehousePath = Utils.createTempDir() + val metastorePath = Utils.createTempDir() + + before { + warehousePath.delete() + metastorePath.delete() + } + + after { + warehousePath.delete() + metastorePath.delete() + } + def runCliWithin( timeout: FiniteDuration, extraArgs: Seq[String] = Seq.empty)( - queriesAndExpectedAnswers: (String, String)*) { + queriesAndExpectedAnswers: (String, String)*): Unit = { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip - val warehousePath = Utils.createTempDir() - warehousePath.delete() - val metastorePath = Utils.createTempDir() - metastorePath.delete() val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val command = { @@ -96,8 +104,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin, cause) throw cause } finally { - warehousePath.delete() - metastorePath.delete() process.destroy() } } @@ -125,4 +131,24 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { test("Single command with -e") { runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") } + + test("Single command with --database") { + runCliWithin(1.minute)( + "CREATE DATABASE hive_test_db;" + -> "OK", + "USE hive_test_db;" + -> "OK", + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "Time taken: " + ) + + runCliWithin(1.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( + "" + -> "OK", + "" + -> "hive_test" + ) + } } 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 bf20acecb1f32..1fadea97fd07f 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File +import java.net.URL import java.sql.{Date, DriverManager, Statement} import scala.collection.mutable.ArrayBuffer @@ -41,7 +42,7 @@ import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils object TestData { - def getTestDataFilePath(name: String) = { + def getTestDataFilePath(name: String): URL = { Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") } @@ -50,7 +51,7 @@ object TestData { } class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.binary private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logics below mimics HiveConnection.createBinaryTransport @@ -337,7 +338,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.http + override def mode: ServerMode.Value = ServerMode.http test("JDBC query execution") { withJdbcStatement { statement => @@ -408,24 +409,24 @@ abstract class HiveThriftServer2Test extends FunSuite with BeforeAndAfterAll wit private val CLASS_NAME = HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$") private val LOG_FILE_MARK = s"starting $CLASS_NAME, logging to " - private val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) - private val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) + protected val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + protected val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) private var listeningPort: Int = _ protected def serverPort: Int = listeningPort protected def user = System.getProperty("user.name") - private var warehousePath: File = _ - private var metastorePath: File = _ - private def metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + protected var warehousePath: File = _ + protected var metastorePath: File = _ + protected def metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" private val pidDir: File = Utils.createTempDir("thriftserver-pid") private var logPath: File = _ private var logTailingProcess: Process = _ private var diagnosisBuffer: ArrayBuffer[String] = ArrayBuffer.empty[String] - private def serverStartCommand(port: Int) = { + protected def serverStartCommand(port: Int) = { val portConf = if (mode == ServerMode.binary) { ConfVars.HIVE_SERVER2_THRIFT_PORT } else { 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 new file mode 100644 index 0000000000000..47541015a3611 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.sql.hive.thriftserver + + + +import scala.util.Random + +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.scalatest.{Matchers, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.selenium.WebBrowser +import org.scalatest.time.SpanSugar._ + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.spark.sql.hive.HiveContext + + +class UISeleniumSuite + extends HiveThriftJdbcTest + with WebBrowser with Matchers with BeforeAndAfterAll { + + implicit var webDriver: WebDriver = _ + var server: HiveThriftServer2 = _ + var hc: HiveContext = _ + val uiPort = 20000 + Random.nextInt(10000) + override def mode: ServerMode.Value = ServerMode.binary + + override def beforeAll(): Unit = { + webDriver = new HtmlUnitDriver + super.beforeAll() + } + + override def afterAll(): Unit = { + if (webDriver != null) { + webDriver.quit() + } + super.afterAll() + } + + override protected def serverStartCommand(port: Int) = { + val portConf = if (mode == ServerMode.binary) { + ConfVars.HIVE_SERVER2_THRIFT_PORT + } else { + ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT + } + + s"""$startScript + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost + | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=$mode + | --hiveconf $portConf=$port + | --driver-class-path ${sys.props("java.class.path")} + | --conf spark.ui.enabled=true + | --conf spark.ui.port=$uiPort + """.stripMargin.split("\\s+").toSeq + } + + test("thrift server ui test") { + withJdbcStatement(statement =>{ + val baseURL = s"http://localhost:${uiPort}" + + val queries = Seq( + "CREATE TABLE test_map(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") + + queries.foreach(statement.execute) + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (baseURL) + find(cssSelector("""ul li a[href*="ThriftServer"]""")) should not be(None) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (baseURL + "/ThriftServer") + find(id("sessionstat")) should not be(None) + find(id("sqlstat")) should not be(None) + + // check whether statements exists + queries.foreach { line => + findAll(cssSelector("""ul table tbody tr td""")).map(_.text).toList should contain (line) + } + } + }) + } +} diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 95a6e86d0546d..b3a79ba1c7d6b 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} import java.util.concurrent.Executors -import java.util.{ArrayList => JArrayList, Map => JMap} +import java.util.{ArrayList => JArrayList, Map => JMap, UUID} import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf @@ -190,9 +190,12 @@ private[hive] class SparkExecuteStatementOperation( } def run(): Unit = { + val statementId = UUID.randomUUID().toString logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) - hiveContext.sparkContext.setJobDescription(statement) + HiveThriftServer2.listener.onStatementStart( + statementId, parentSession.getSessionHandle.getSessionId.toString, statement, statementId) + hiveContext.sparkContext.setJobGroup(statementId, statement) sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } @@ -205,6 +208,7 @@ private[hive] class SparkExecuteStatementOperation( logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } + HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) iter = { val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean @@ -221,10 +225,13 @@ private[hive] class SparkExecuteStatementOperation( // HiveServer will silently swallow them. case e: Throwable => setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, e.getStackTraceString) logError("Error executing query:",e) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) + HiveThriftServer2.listener.onStatementFinish(statementId) } } @@ -255,11 +262,14 @@ private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) withImpersonation: Boolean, delegationToken: String): SessionHandle = { hiveContext.openSession() - - super.openSession(username, passwd, sessionConf, withImpersonation, delegationToken) + val sessionHandle = super.openSession( + username, passwd, sessionConf, withImpersonation, delegationToken) + HiveThriftServer2.listener.onSessionCreated("UNKNOWN", sessionHandle.getSessionId.toString) + sessionHandle } override def closeSession(sessionHandle: SessionHandle) { + HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool -= sessionHandle diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 178eb1af7cdcd..b9d4f1c58c982 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.sql.{Date, Timestamp} import java.util.concurrent.Executors -import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, UUID} import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf @@ -36,7 +36,7 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.{SessionManager, HiveSession} -import org.apache.spark.Logging +import org.apache.spark.{SparkContext, Logging} import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf} import org.apache.spark.sql.execution.SetCommand import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -161,9 +161,16 @@ private[hive] class SparkExecuteStatementOperation( } def run(): Unit = { + val statementId = UUID.randomUUID().toString logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) - hiveContext.sparkContext.setJobDescription(statement) + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + statement, + statementId, + parentSession.getUsername) + hiveContext.sparkContext.setJobGroup(statementId, statement) sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } @@ -176,6 +183,7 @@ private[hive] class SparkExecuteStatementOperation( logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } + HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) iter = { val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean @@ -192,10 +200,13 @@ private[hive] class SparkExecuteStatementOperation( // HiveServer will silently swallow them. case e: Throwable => setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, e.getStackTraceString) logError("Error executing query:", e) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) + HiveThriftServer2.listener.onStatementFinish(statementId) } } @@ -227,11 +238,16 @@ private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) withImpersonation: Boolean, delegationToken: String): SessionHandle = { hiveContext.openSession() - - super.openSession(protocol, username, passwd, sessionConf, withImpersonation, delegationToken) + val sessionHandle = super.openSession( + protocol, username, passwd, sessionConf, withImpersonation, delegationToken) + val session = super.getSession(sessionHandle) + HiveThriftServer2.listener.onSessionCreated( + session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) + sessionHandle } override def closeSession(sessionHandle: SessionHandle) { + HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool -= sessionHandle 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 5180a7f09d80f..5e411c2fdba9d 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 @@ -185,7 +185,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive does not support buckets. ".*bucket.*", - // No window support yet + // We have our own tests based on these query files. ".*window.*", // Fails in hive with authorization errors. @@ -532,6 +532,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl7", "inputddl8", "insert1", + "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", "join0", @@ -800,6 +801,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udaf_covar_pop", "udaf_covar_samp", "udaf_histogram_numeric", + "udaf_number_format", "udf2", "udf5", "udf6", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala new file mode 100644 index 0000000000000..65d070bd3cbde --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala @@ -0,0 +1,162 @@ +/* + * 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.sql.hive.execution + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.test.TestHive + +/** + * Runs the test cases that are included in the hive distribution with sort merge join is true. + */ +class SortMergeCompatibilitySuite extends HiveCompatibilitySuite { + override def beforeAll() { + super.beforeAll() + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "true") + } + + override def afterAll() { + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "false") + super.afterAll() + } + + override def whiteList = Seq( + "auto_join0", + "auto_join1", + "auto_join10", + "auto_join11", + "auto_join12", + "auto_join13", + "auto_join14", + "auto_join14_hadoop20", + "auto_join15", + "auto_join17", + "auto_join18", + "auto_join19", + "auto_join2", + "auto_join20", + "auto_join21", + "auto_join22", + "auto_join23", + "auto_join24", + "auto_join25", + "auto_join26", + "auto_join27", + "auto_join28", + "auto_join3", + "auto_join30", + "auto_join31", + "auto_join32", + "auto_join4", + "auto_join5", + "auto_join6", + "auto_join7", + "auto_join8", + "auto_join9", + "auto_join_filters", + "auto_join_nulls", + "auto_join_reordering_values", + "auto_smb_mapjoin_14", + "auto_sortmerge_join_1", + "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", + "auto_sortmerge_join_15", + "auto_sortmerge_join_16", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", + "auto_sortmerge_join_9", + "correlationoptimizer1", + "correlationoptimizer10", + "correlationoptimizer11", + "correlationoptimizer13", + "correlationoptimizer14", + "correlationoptimizer15", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", + "correlationoptimizer8", + "correlationoptimizer9", + "join0", + "join1", + "join10", + "join11", + "join12", + "join13", + "join14", + "join14_hadoop20", + "join15", + "join16", + "join17", + "join18", + "join19", + "join2", + "join20", + "join21", + "join22", + "join23", + "join24", + "join25", + "join26", + "join27", + "join28", + "join29", + "join3", + "join30", + "join31", + "join32", + "join32_lessSize", + "join33", + "join34", + "join35", + "join36", + "join37", + "join38", + "join39", + "join4", + "join40", + "join41", + "join5", + "join6", + "join7", + "join8", + "join9", + "join_1to1", + "join_array", + "join_casesensitive", + "join_empty", + "join_filters", + "join_hive_626", + "join_map_ppr", + "join_nulls", + "join_nullsafe", + "join_rc", + "join_reorder2", + "join_reorder3", + "join_reorder4", + "join_star" + ) +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a9816f6c38cd2..e322340094e6f 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -59,6 +59,11 @@ ${hive.group}hive-exec + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + org.codehaus.jackson jackson-mapper-asl @@ -89,6 +94,20 @@ junit test + + org.apache.spark + spark-sql_${scala.binary.version} + test-jar + ${project.version} + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + ${project.version} + test + @@ -164,7 +183,6 @@ org.apache.maven.plugins maven-dependency-plugin - 2.4 copy-dependencies diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index a5c435fdfa778..f25723e53f07c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -20,6 +20,9 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader, PrintStream} import java.sql.Timestamp +import org.apache.hadoop.hive.ql.parse.VariableSubstitution +import org.apache.spark.sql.catalyst.Dialect + import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -42,6 +45,15 @@ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNative import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ +/** + * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext + */ +private[hive] class HiveQLDialect extends Dialect { + override def parse(sqlText: String): LogicalPlan = { + HiveQl.parseSql(sqlText) + } +} + /** * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. @@ -57,6 +69,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertMetastoreParquet: Boolean = getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true" + /** + * When true, also tries to merge possibly different but compatible Parquet schemas in different + * Parquet data files. + * + * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. + */ + protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean = + getConf("spark.sql.hive.convertMetastoreParquet.mergeSchema", "false") == "true" + /** * When true, a table created by a Hive CTAS statement (no USING clause) will be * converted to a data source table, using the data source set by spark.sql.sources.default. @@ -72,25 +93,16 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] def convertCTAS: Boolean = getConf("spark.sql.hive.convertCTAS", "false").toBoolean - override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) - @transient - protected[sql] val ddlParserWithHiveQL = new DDLParser(HiveQl.parseSql(_)) - - override def sql(sqlText: String): DataFrame = { - val substituted = new VariableSubstitution().substitute(hiveconf, sqlText) - // TODO: Create a framework for registering parsers instead of just hardcoding if statements. - if (conf.dialect == "sql") { - super.sql(substituted) - } else if (conf.dialect == "hiveql") { - val ddlPlan = ddlParserWithHiveQL(sqlText, exceptionOnError = false) - DataFrame(this, ddlPlan.getOrElse(HiveQl.parseSql(substituted))) - } else { - sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") - } + protected[sql] lazy val substitutor = new VariableSubstitution() + + protected[sql] override def parseSql(sql: String): LogicalPlan = { + super.parseSql(substitutor.substitute(hiveconf, sql)) } + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = + new this.QueryExecution(plan) + /** * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, * Spark SQL or the external data source library it uses might cache certain metadata about a @@ -172,18 +184,19 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val tableFullName = relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName - catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + catalog.synchronized { + catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + } } case otherRelation => - throw new NotImplementedError( - s"Analyze has only implemented for Hive tables, " + - s"but $tableName is a ${otherRelation.nodeName}") + throw new UnsupportedOperationException( + s"Analyze only works for Hive tables, but $tableName is a ${otherRelation.nodeName}") } } // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient - protected lazy val outputBuffer = new java.io.OutputStream { + protected lazy val outputBuffer = new java.io.OutputStream { var pos: Int = 0 var buffer = new Array[Int](10240) def write(i: Int): Unit = { @@ -191,7 +204,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { pos = (pos + 1) % buffer.size } - override def toString = { + override def toString: String = { val (end, start) = buffer.splitAt(pos) val input = new java.io.InputStream { val iterator = (start ++ end).iterator @@ -227,7 +240,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry { - def caseSensitive = false + def caseSensitive: Boolean = false } /* An analyzer that uses the Hive metastore. */ @@ -239,7 +252,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: - ResolveUdtfsAlias :: + ResolveHiveWindowFunction :: sources.PreInsertCastAndRename :: Nil } @@ -347,6 +360,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } + override protected[sql] def dialectClassName = if (conf.dialect == "hiveql") { + classOf[HiveQLDialect].getCanonicalName + } else { + super.dialectClassName + } + @transient private val hivePlanner = new SparkPlanner with HiveStrategies { val hiveContext = self diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 4afa2e71d77cc..74ae984f34866 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -34,7 +34,7 @@ import scala.collection.JavaConversions._ * 1. The Underlying data type in catalyst and in Hive * In catalyst: * Primitive => - * java.lang.String + * UTF8String * int / scala.Int * boolean / scala.Boolean * float / scala.Float @@ -239,9 +239,10 @@ private[hive] trait HiveInspectors { */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null - case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString + case poi: WritableConstantStringObjectInspector => + UTF8String(poi.getWritableConstantValue.toString) case poi: WritableConstantHiveVarcharObjectInspector => - poi.getWritableConstantValue.getHiveVarchar.getValue + UTF8String(poi.getWritableConstantValue.getHiveVarchar.getValue) case poi: WritableConstantHiveDecimalObjectInspector => HiveShim.toCatalystDecimal( PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, @@ -284,10 +285,13 @@ private[hive] trait HiveInspectors { case pi: PrimitiveObjectInspector => pi match { // We think HiveVarchar is also a String case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => - hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue - case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue + UTF8String(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) + case hvoi: HiveVarcharObjectInspector => + UTF8String(hvoi.getPrimitiveJavaObject(data).getValue) case x: StringObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).toString + UTF8String(x.getPrimitiveWritableObject(data).toString) + case x: StringObjectInspector => + UTF8String(x.getPrimitiveJavaObject(data)) case x: IntObjectInspector if x.preferWritable() => x.get(data) case x: BooleanObjectInspector if x.preferWritable() => x.get(data) case x: FloatObjectInspector if x.preferWritable() => x.get(data) @@ -340,7 +344,9 @@ private[hive] trait HiveInspectors { */ protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { case _: JavaHiveVarcharObjectInspector => - (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + (o: Any) => + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.size) case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) @@ -409,7 +415,7 @@ private[hive] trait HiveInspectors { case x: PrimitiveObjectInspector => x match { // TODO we don't support the HiveVarcharObjectInspector yet. case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a) - case _: StringObjectInspector => a.asInstanceOf[java.lang.String] + case _: StringObjectInspector => a.asInstanceOf[UTF8String].toString() case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a) case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a) @@ -593,7 +599,7 @@ private[hive] trait HiveInspectors { case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") // ideally, we don't test the foldable here(but in optimizer), however, some of the // Hive UDF / UDAF requires its argument to be constant objectinspector, we do it eagerly. - case _ if expr.foldable => toInspector(Literal(expr.eval(), expr.dataType)) + case _ if expr.foldable => toInspector(Literal.create(expr.eval(), expr.dataType)) // For those non constant expression, map to object inspector according to its data type case _ => toInspector(expr.dataType) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fe86bd206a71c..4d222cf88e5e8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} +import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.metastore.{TableType, Warehouse} @@ -32,7 +33,7 @@ import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, Catalog, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical @@ -58,7 +59,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // TODO: Use this everywhere instead of tuples or databaseName, tableName,. /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) { - def toLowerCase = QualifiedTableName(database.toLowerCase, name.toLowerCase) + def toLowerCase: QualifiedTableName = QualifiedTableName(database.toLowerCase, name.toLowerCase) } /** A cache of Spark SQL data source tables that have been accessed. */ @@ -66,10 +67,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { override def load(in: QualifiedTableName): LogicalPlan = { logDebug(s"Creating new cached data source for $in") - val table = synchronized { + val table = HiveMetastoreCatalog.this.synchronized { client.getTable(in.database, in.name) } - val userSpecifiedSchema = + + def schemaStringFromParts: Option[String] = { Option(table.getProperty("spark.sql.sources.schema.numParts")).map { numParts => val parts = (0 until numParts.toInt).map { index => val part = table.getProperty(s"spark.sql.sources.schema.part.${index}") @@ -81,10 +83,19 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with part } - // Stick all parts back to a single schema string in the JSON representation - // and convert it back to a StructType. - DataType.fromJson(parts.mkString).asInstanceOf[StructType] + // Stick all parts back to a single schema string. + parts.mkString } + } + + // Originally, we used spark.sql.sources.schema to store the schema of a data source table. + // After SPARK-6024, we removed this flag. + // Although we are not using spark.sql.sources.schema any more, we need to still support. + val schemaString = + Option(table.getProperty("spark.sql.sources.schema")).orElse(schemaStringFromParts) + + val userSpecifiedSchema = + schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) // It does not appear that the ql client for the metastore has a way to enumerate all the // SerDe properties directly... @@ -105,7 +116,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } override def refreshTable(databaseName: String, tableName: String): Unit = { - cachedDataSourceTables.refresh(QualifiedTableName(databaseName, tableName).toLowerCase) + // refreshTable does not eagerly reload the cache. It just invalidate the cache. + // Next time when we use the table, it will be populated in the cache. + // Since we also cache ParquetRealtions converted from Hive Parquet tables and + // adding converted ParquetRealtions into the cache is not defined in the load function + // of the cache (instead, we add the cache entry in convertToParquetRelation), + // it is better at here to invalidate the cache to avoid confusing waring logs from the + // cache loader (e.g. cannot find data source provider, which is only defined for + // data source table.). + invalidateTable(databaseName, tableName) } def invalidateTable(databaseName: String, tableName: String): Unit = { @@ -172,12 +191,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with def lookupRelation( tableIdentifier: Seq[String], - alias: Option[String]): LogicalPlan = synchronized { + alias: Option[String]): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - val table = try client.getTable(databaseName, tblName) catch { + val table = try { + synchronized { + client.getTable(databaseName, tblName) + } + } catch { case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException => throw new NoSuchTableException } @@ -199,7 +222,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } else { val partitions: Seq[Partition] = if (table.isPartitioned) { - HiveShim.getAllPartitionsOf(client, table).toSeq + synchronized { + HiveShim.getAllPartitionsOf(client, table).toSeq + } } else { Nil } @@ -211,11 +236,50 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with private def convertToParquetRelation(metastoreRelation: MetastoreRelation): LogicalRelation = { val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) + val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging // NOTE: Instead of passing Metastore schema directly to `ParquetRelation2`, we have to // serialize the Metastore schema to JSON and pass it as a data source option because of the // evil case insensitivity issue, which is reconciled within `ParquetRelation2`. - if (metastoreRelation.hiveQlTable.isPartitioned) { + val parquetOptions = Map( + ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json, + ParquetRelation2.MERGE_SCHEMA -> mergeSchema.toString) + val tableIdentifier = + QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) + + def getCached( + tableIdentifier: QualifiedTableName, + pathsInMetastore: Seq[String], + schemaInMetastore: StructType, + partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { + cachedDataSourceTables.getIfPresent(tableIdentifier) match { + case null => None // Cache miss + case logical@LogicalRelation(parquetRelation: ParquetRelation2) => + // If we have the same paths, same schema, and same partition spec, + // we will use the cached Parquet Relation. + val useCached = + parquetRelation.paths.toSet == pathsInMetastore.toSet && + logical.schema.sameType(metastoreSchema) && + parquetRelation.maybePartitionSpec == partitionSpecInMetastore + + if (useCached) { + Some(logical) + } else { + // If the cached relation is not updated, we invalidate it right away. + cachedDataSourceTables.invalidate(tableIdentifier) + None + } + case other => + logWarning( + s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be stored " + + s"as Parquet. However, we are getting a ${other} from the metastore cache. " + + s"This cached entry will be invalidated.") + cachedDataSourceTables.invalidate(tableIdentifier) + None + } + } + + val result = if (metastoreRelation.hiveQlTable.isPartitioned) { val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) val partitionColumnDataTypes = partitionSchema.map(_.dataType) val partitions = metastoreRelation.hiveQlPartitions.map { p => @@ -227,19 +291,31 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } val partitionSpec = PartitionSpec(partitionSchema, partitions) val paths = partitions.map(_.path) - LogicalRelation( - ParquetRelation2( - paths, - Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json), - None, - Some(partitionSpec))(hive)) + + val cached = getCached(tableIdentifier, paths, metastoreSchema, Some(partitionSpec)) + val parquetRelation = cached.getOrElse { + val created = + LogicalRelation(ParquetRelation2(paths, parquetOptions, None, Some(partitionSpec))(hive)) + cachedDataSourceTables.put(tableIdentifier, created) + created + } + + parquetRelation } else { val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) - LogicalRelation( - ParquetRelation2( - paths, - Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json))(hive)) + + val cached = getCached(tableIdentifier, paths, metastoreSchema, None) + val parquetRelation = cached.getOrElse { + val created = + LogicalRelation(ParquetRelation2(paths, parquetOptions)(hive)) + cachedDataSourceTables.put(tableIdentifier, created) + created + } + + parquetRelation } + + result.newInstance() } override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { @@ -451,7 +527,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Collects all `MetastoreRelation`s which should be replaced val toBeReplaced = plan.collect { // Write path - case InsertIntoTable(relation: MetastoreRelation, _, _, _) + case InsertIntoTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -462,7 +538,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with (relation, parquetRelation, attributedRewrites) // Write path - case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) + case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -488,24 +564,21 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Replaces all `MetastoreRelation`s with corresponding `ParquetRelation2`s, and fixes // attribute IDs referenced in other nodes. plan.transformUp { - case r: MetastoreRelation if relationMap.contains(r) => { + case r: MetastoreRelation if relationMap.contains(r) => val parquetRelation = relationMap(r) - val withAlias = - r.alias.map(a => Subquery(a, parquetRelation)).getOrElse( - Subquery(r.tableName, parquetRelation)) + val alias = r.alias.getOrElse(r.tableName) + Subquery(alias, parquetRelation) - withAlias - } - case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r) => { + case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) + if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) - } - case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) - if relationMap.contains(r) => { + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) + + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) + if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) - } + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) + case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) } @@ -625,11 +698,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => castChildOutput(p, table, child) } - def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { + def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) + : LogicalPlan = { val childOutputDataTypes = child.output.map(_.dataType) val tableOutputDataTypes = (table.attributes ++ table.partitionKeys).take(child.output.length).map(_.dataType) @@ -641,7 +715,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with .forall { case (left, right) => left.sameType(right) }) { // If both types ignoring nullability of ArrayType, MapType, StructType are the same, // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite) + InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite, p.ifNotExists) } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { @@ -667,7 +741,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ override def unregisterTable(tableIdentifier: Seq[String]): Unit = ??? - override def unregisterAllTables() = {} + override def unregisterAllTables(): Unit = {} } /** @@ -679,13 +753,14 @@ private[hive] case class InsertIntoHiveTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { - override def children = child :: Nil - override def output = child.output + override def children: Seq[LogicalPlan] = child :: Nil + override def output: Seq[Attribute] = child.output - override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => childAttr.dataType.sameType(tableAttr.dataType) } } @@ -694,23 +769,36 @@ private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) (@transient sqlContext: SQLContext) - extends LeafNode { + extends LeafNode with MultiInstanceRelation { self: Product => + override def equals(other: scala.Any): Boolean = other match { + case relation: MetastoreRelation => + databaseName == relation.databaseName && + tableName == relation.tableName && + alias == relation.alias && + output == relation.output + case _ => false + } + + override def hashCode(): Int = { + Objects.hashCode(databaseName, tableName, alias, output) + } + // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. // Right now, using org.apache.hadoop.hive.ql.metadata.Table and // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - @transient val hiveQlTable = new Table(table) + @transient val hiveQlTable: Table = new Table(table) - @transient val hiveQlPartitions = partitions.map { p => + @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p => new Partition(hiveQlTable, p) } - @transient override lazy val statistics = Statistics( + @transient override lazy val statistics: Statistics = Statistics( sizeInBytes = { val totalSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize) val rawDataSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize) @@ -754,9 +842,9 @@ private[hive] case class MetastoreRelation ) implicit class SchemaAttribute(f: FieldSchema) { - def toAttribute = AttributeReference( + def toAttribute: AttributeReference = AttributeReference( f.getName, - sqlContext.ddlParser.parseType(f.getType), + HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true )(qualifiers = Seq(alias.getOrElse(tableName))) @@ -775,15 +863,15 @@ private[hive] case class MetastoreRelation /** An attribute map for determining the ordinal for non-partition columns. */ val columnOrdinals = AttributeMap(attributes.zipWithIndex) + + override def newInstance(): MetastoreRelation = { + MetastoreRelation(databaseName, tableName, alias)(table, partitions)(sqlContext) + } } private[hive] object HiveMetastoreTypes { - protected val ddlParser = new DDLParser(HiveQl.parseSql(_)) - - def toDataType(metastoreType: String): DataType = synchronized { - ddlParser.parseType(metastoreType) - } + def toDataType(metastoreType: String): DataType = DataTypeParser.parse(metastoreType) def toMetastoreType(dt: DataType): String = dt match { case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index ced99cd082614..8a0686a2d81c9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive import java.sql.Date - import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.parse._ @@ -55,37 +55,8 @@ private[hive] case object NativePlaceholder extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( - "TOK_DESCFUNCTION", - "TOK_DESCDATABASE", - "TOK_SHOW_CREATETABLE", - "TOK_SHOWCOLUMNS", - "TOK_SHOW_TABLESTATUS", - "TOK_SHOWDATABASES", - "TOK_SHOWFUNCTIONS", - "TOK_SHOWINDEXES", - "TOK_SHOWINDEXES", - "TOK_SHOWPARTITIONS", - "TOK_SHOW_TBLPROPERTIES", - - "TOK_LOCKTABLE", - "TOK_SHOWLOCKS", - "TOK_UNLOCKTABLE", - - "TOK_SHOW_ROLES", - "TOK_CREATEROLE", - "TOK_DROPROLE", - "TOK_GRANT", - "TOK_GRANT_ROLE", - "TOK_REVOKE", - "TOK_SHOW_GRANT", - "TOK_SHOW_ROLE_GRANT", - "TOK_SHOW_SET_ROLE", - - "TOK_CREATEFUNCTION", - "TOK_DROPFUNCTION", - - "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERDATABASE_OWNER", + "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", "TOK_ALTERTABLE_ADDCOLS", @@ -102,28 +73,64 @@ private[hive] object HiveQl { "TOK_ALTERTABLE_SKEWED", "TOK_ALTERTABLE_TOUCH", "TOK_ALTERTABLE_UNARCHIVE", - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", - "TOK_CREATEINDEX", - "TOK_DROPDATABASE", - "TOK_DROPINDEX", - "TOK_DROPTABLE_PROPERTIES", - "TOK_MSCK", - "TOK_ALTERVIEW_ADDPARTS", "TOK_ALTERVIEW_AS", "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", + + "TOK_CREATEDATABASE", + "TOK_CREATEFUNCTION", + "TOK_CREATEINDEX", + "TOK_CREATEROLE", "TOK_CREATEVIEW", - "TOK_DROPVIEW_PROPERTIES", + + "TOK_DESCDATABASE", + "TOK_DESCFUNCTION", + + "TOK_DROPDATABASE", + "TOK_DROPFUNCTION", + "TOK_DROPINDEX", + "TOK_DROPROLE", + "TOK_DROPTABLE_PROPERTIES", "TOK_DROPVIEW", - + "TOK_DROPVIEW_PROPERTIES", + "TOK_EXPORT", + + "TOK_GRANT", + "TOK_GRANT_ROLE", + "TOK_IMPORT", + "TOK_LOAD", - - "TOK_SWITCHDATABASE" + + "TOK_LOCKTABLE", + + "TOK_MSCK", + + "TOK_REVOKE", + + "TOK_SHOW_COMPACTIONS", + "TOK_SHOW_CREATETABLE", + "TOK_SHOW_GRANT", + "TOK_SHOW_ROLE_GRANT", + "TOK_SHOW_ROLE_PRINCIPALS", + "TOK_SHOW_ROLES", + "TOK_SHOW_SET_ROLE", + "TOK_SHOW_TABLESTATUS", + "TOK_SHOW_TBLPROPERTIES", + "TOK_SHOW_TRANSACTIONS", + "TOK_SHOWCOLUMNS", + "TOK_SHOWDATABASES", + "TOK_SHOWFUNCTIONS", + "TOK_SHOWINDEXES", + "TOK_SHOWLOCKS", + "TOK_SHOWPARTITIONS", + + "TOK_SWITCHDATABASE", + + "TOK_UNLOCKTABLE" ) // Commands that we do not need to explain. @@ -135,7 +142,7 @@ private[hive] object HiveQl { protected val hqlParser = { val fallback = new ExtendedHiveQlParser - new SparkSQLParser(fallback(_)) + new SparkSQLParser(fallback.parse(_)) } /** @@ -196,8 +203,8 @@ private[hive] object HiveQl { * Right now this function only checks the name, type, text and children of the node * for equality. */ - def checkEquals(other: ASTNode) { - def check(field: String, f: ASTNode => Any) = if (f(n) != f(other)) { + def checkEquals(other: ASTNode): Unit = { + def check(field: String, f: ASTNode => Any): Unit = if (f(n) != f(other)) { sys.error(s"$field does not match for trees. " + s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") } @@ -209,7 +216,7 @@ private[hive] object HiveQl { val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]] val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]] leftChildren zip rightChildren foreach { - case (l,r) => l checkEquals r + case (l, r) => l checkEquals r } } } @@ -231,7 +238,7 @@ private[hive] object HiveQl { /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = hqlParser(sql) + def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql) val errorRegEx = "line (\\d+):(\\d+) (.*)".r @@ -269,7 +276,7 @@ private[hive] object HiveQl { } /** Creates LogicalPlan for a given VIEW */ - def createPlanForView(view: Table, alias: Option[String]) = alias match { + def createPlanForView(view: Table, alias: Option[String]): Subquery = alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText)) @@ -323,7 +330,7 @@ private[hive] object HiveQl { clauses } - def getClause(clauseName: String, nodeList: Seq[Node]) = + def getClause(clauseName: String, nodeList: Seq[Node]): Node = getClauseOption(clauseName, nodeList).getOrElse(sys.error( s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) @@ -414,16 +421,16 @@ private[hive] object HiveQl { } /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to + * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) + * is equivalent to * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 * Check the following link for details. - * + * https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup * * The bitmask denotes the grouping expressions validity for a grouping set, * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of + * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. */ protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { @@ -437,7 +444,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val bitmasks: Seq[Int] = setASTs.map(set => set match { case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => + case Token("TOK_GROUPING_SETS_EXPRESSION", children) => children.foldLeft(0)((bitmap, col) => { val colString = col.asInstanceOf[ASTNode].toStringTree() require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") @@ -475,7 +482,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => - ExplainCommand(NoRelation) + ExplainCommand(OneRowRelation) case Token("TOK_EXPLAIN", explainArgs) if "TOK_CREATETABLE" == explainArgs.head.getText => val Some(crtTbl) :: _ :: extended :: Nil = @@ -570,11 +577,23 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_QUERY", queryArgs) if Seq("TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) => - val (fromClause: Option[ASTNode], insertClauses) = queryArgs match { - case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => - (Some(args.head), insertClauses) - case Token("TOK_INSERT", _) :: Nil => (None, queryArgs) - } + val (fromClause: Option[ASTNode], insertClauses, cteRelations) = + queryArgs match { + case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => + // check if has CTE + insertClauses.last match { + case Token("TOK_CTE", cteClauses) => + val cteRelations = cteClauses.map(node => { + val relation = nodeToRelation(node).asInstanceOf[Subquery] + (relation.alias, relation) + }).toMap + (Some(args.head), insertClauses.init, Some(cteRelations)) + + case _ => (Some(args.head), insertClauses, None) + } + + case Token("TOK_INSERT", _) :: Nil => (None, queryArgs, None) + } // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => @@ -594,7 +613,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C clusterByClause :: distributeByClause :: limitClause :: - lateralViewClause :: Nil) = { + lateralViewClause :: + windowClause :: Nil) = { getClauses( Seq( "TOK_INSERT_INTO", @@ -612,15 +632,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_CLUSTERBY", "TOK_DISTRIBUTEBY", "TOK_LIMIT", - "TOK_LATERAL_VIEW"), + "TOK_LATERAL_VIEW", + "WINDOW"), singleInsert) } - + val relations = fromClause match { case Some(f) => nodeToRelation(f) - case None => NoRelation + case None => OneRowRelation } - + val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) @@ -655,7 +676,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C AttributeReference("value", StringType)()), true) } - def matchSerDe(clause: Seq[ASTNode]) = clause match { + def matchSerDe(clause: Seq[ASTNode]) + : (Seq[(String, String)], String, Seq[(String, String)]) = clause match { case Token("TOK_SERDEPROPS", propsClause) :: Nil => val rowFormat = propsClause.map { case Token(name, Token(value, Nil) :: Nil) => (name, value) @@ -671,7 +693,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val serdeProps = propsClause.map { case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => (name, value) - } + } (Nil, serdeClass, serdeProps) case Nil => (Nil, "", Nil) @@ -703,18 +725,20 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText - Generate( - nodesToGenerator(clauses), - join = true, - outer = false, - Some(alias.toLowerCase), - withWhere) + val (generator, attributes) = nodesToGenerator(clauses) + Generate( + generator, + join = true, + outer = false, + Some(alias.toLowerCase), + attributes.map(UnresolvedAttribute(_)), + withWhere) }.getOrElse(withWhere) // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = + val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq) Seq( groupByClause.map(e => e match { @@ -742,31 +766,34 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Some(Project(selectExpressions, withLateralView))).flatten.head } - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withProject) else withProject - + // Handle HAVING clause. val withHaving = havingClause.map { h => val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } // Note that we added a cast to boolean. If the expression itself is already boolean, // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withDistinct) - }.getOrElse(withDistinct) + Filter(Cast(havingExpr, BooleanType), withProject) + }.getOrElse(withProject) + + // Handle SELECT DISTINCT + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving + // Handle ORDER BY, SORT BY, DISTRIBETU BY, and CLUSTER BY clause. val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withDistinct) case (None, Some(perPartitionOrdering), None, None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withDistinct) case (None, None, Some(partitionExprs), None) => - Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, - Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving)) + RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) case (None, None, None, Some(clusterExprs)) => Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, - Repartition(clusterExprs.getChildren.map(nodeToExpr), withHaving)) - case (None, None, None, None) => withHaving + RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + case (None, None, None, None) => withDistinct case _ => sys.error("Unsupported set of ordering / distribution clauses.") } @@ -775,6 +802,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C .map(Limit(_, withSort)) .getOrElse(withSort) + // Collect all window specifications defined in the WINDOW clause. + val windowDefinitions = windowClause.map(_.getChildren.toSeq.collect { + case Token("TOK_WINDOWDEF", + Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + windowName -> nodesToWindowSpecification(spec) + }.toMap) + // Handle cases like + // window w1 as (partition by p_mfgr order by p_name + // range between 2 preceding and 2 following), + // w2 as w1 + val resolvedCrossReference = windowDefinitions.map { + windowDefMap => windowDefMap.map { + case (windowName, WindowSpecReference(other)) => + (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) + case o => o.asInstanceOf[(String, WindowSpecDefinition)] + } + } + + val withWindowDefinitions = + resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) + // TOK_INSERT_INTO means to add files to the table. // TOK_DESTINATION means to overwrite the table. val resultDestination = @@ -782,12 +830,15 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val overwrite = intoClause.isEmpty nodeToDest( resultDestination, - withLimit, + withWindowDefinitions, overwrite) } // If there are multiple INSERTS just UNION them together into on query. - queries.reduceLeft(Union) + val query = queries.reduceLeft(Union) + + // return With plan if there is CTE + cteRelations.map(With(query, _)).getOrElse(query) case Token("TOK_UNION", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right)) @@ -808,12 +859,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText - Generate( - nodesToGenerator(clauses), - join = true, - outer = isOuter.nonEmpty, - Some(alias.toLowerCase), - nodeToRelation(relationClause)) + val (generator, attributes) = nodesToGenerator(clauses) + Generate( + generator, + join = true, + outer = isOuter.nonEmpty, + Some(alias.toLowerCase), + attributes.map(UnresolvedAttribute(_)), + nodeToRelation(relationClause)) /* All relations, possibly with aliases or sampling clauses. */ case Token("TOK_TABREF", clauses) => @@ -858,13 +911,13 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), s"Sampling fraction ($fraction) must be on interval [0, 100]") - Sample(fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt, + Sample(0.0, fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt, relation) case Token("TOK_TABLEBUCKETSAMPLE", Token(numerator, Nil) :: Token(denominator, Nil) :: Nil) => val fraction = numerator.toDouble / denominator.toDouble - Sample(fraction, withReplacement = false, (math.random * 1000).toInt, relation) + Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) case a: ASTNode => throw new NotImplementedError( s"""No parse rules for sampling clause: ${a.getType}, text: ${a.getText} : @@ -977,7 +1030,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite) + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.getChildren.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") @@ -1006,7 +1079,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") } - protected val escapedIdentifier = "`([^`]+)`".r /** Strips backticks from ident if present */ protected def cleanIdentifier(ident: String): String = ident match { @@ -1056,7 +1128,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => - UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + UnresolvedAttribute(qualifierName :+ cleanIdentifier(attr)) case other => UnresolvedGetField(other, attr) } @@ -1195,13 +1267,33 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C /* Other functions */ case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) => CreateArray(children.map(nodeToExpr)) - case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand + case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand() + case Token("TOK_FUNCTION", Token(RAND(), Nil) :: seed :: Nil) => Rand(seed.toString.toLong) case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => - Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) + Substring(nodeToExpr(string), nodeToExpr(pos), Literal.create(Integer.MAX_VALUE, IntegerType)) case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr)) + /* Window Functions */ + case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) => + val function = UnresolvedWindowFunction(name, args.map(nodeToExpr)) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => + // Safe to use Literal(1)? + val function = UnresolvedWindowFunction(name, Literal(1) :: Nil) + nodesToWindowSpecification(spec) match { + case reference: WindowSpecReference => + UnresolvedWindowExpression(function, reference) + case definition: WindowSpecDefinition => + WindowExpression(function, definition) + } + /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => UnresolvedFunction(name, args.map(nodeToExpr)) @@ -1209,9 +1301,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C UnresolvedFunction(name, UnresolvedStar(None) :: Nil) /* Literals */ - case Token("TOK_NULL", Nil) => Literal(null, NullType) - case Token(TRUE(), Nil) => Literal(true, BooleanType) - case Token(FALSE(), Nil) => Literal(false, BooleanType) + case Token("TOK_NULL", Nil) => Literal.create(null, NullType) + case Token(TRUE(), Nil) => Literal.create(true, BooleanType) + case Token(FALSE(), Nil) => Literal.create(false, BooleanType) case Token("TOK_STRINGLITERALSEQUENCE", strings) => Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString) @@ -1222,21 +1314,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C try { if (ast.getText.endsWith("L")) { // Literal bigint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) } else if (ast.getText.endsWith("S")) { // Literal smallint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) } else if (ast.getText.endsWith("Y")) { // Literal tinyint. - v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) + v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { // Literal decimal val strVal = ast.getText.stripSuffix("D").stripSuffix("B") v = Literal(Decimal(strVal)) } else { - v = Literal(ast.getText.toDouble, DoubleType) - v = Literal(ast.getText.toLong, LongType) - v = Literal(ast.getText.toInt, IntegerType) + v = Literal.create(ast.getText.toDouble, DoubleType) + v = Literal.create(ast.getText.toLong, LongType) + v = Literal.create(ast.getText.toInt, IntegerType) } } catch { case nfe: NumberFormatException => // Do nothing @@ -1264,9 +1356,92 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C """.stripMargin) } + def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { + case Token(windowName, Nil) :: Nil => + // Refer to a window spec defined in the window clause. + WindowSpecReference(windowName) + case Nil => + // OVER() + WindowSpecDefinition( + partitionSpec = Nil, + orderSpec = Nil, + frameSpecification = UnspecifiedFrame) + case spec => + val (partitionClause :: rowFrame :: rangeFrame :: Nil) = + getClauses( + Seq( + "TOK_PARTITIONINGSPEC", + "TOK_WINDOWRANGE", + "TOK_WINDOWVALUES"), + spec) + + // Handle Partition By and Order By. + val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => + val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = + getClauses( + Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), + partitionAndOrdering.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + + (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { + case (Some(partitionByExpr), Some(orderByExpr), None) => + (partitionByExpr.getChildren.map(nodeToExpr), + orderByExpr.getChildren.map(nodeToSortOrder)) + case (Some(partitionByExpr), None, None) => + (partitionByExpr.getChildren.map(nodeToExpr), Nil) + case (None, Some(orderByExpr), None) => + (Nil, orderByExpr.getChildren.map(nodeToSortOrder)) + case (None, None, Some(clusterByExpr)) => + val expressions = clusterByExpr.getChildren.map(nodeToExpr) + (expressions, expressions.map(SortOrder(_, Ascending))) + case _ => + throw new NotImplementedError( + s"""No parse rules for Node ${partitionAndOrdering.getName} + """.stripMargin) + } + }.getOrElse { + (Nil, Nil) + } + + // Handle Window Frame + val windowFrame = + if (rowFrame.isEmpty && rangeFrame.isEmpty) { + UnspecifiedFrame + } else { + val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) + def nodeToBoundary(node: Node): FrameBoundary = node match { + case Token("preceding", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedPreceding else ValuePreceding(count.toInt) + case Token("following", Token(count, Nil) :: Nil) => + if (count == "unbounded") UnboundedFollowing else ValueFollowing(count.toInt) + case Token("current", Nil) => CurrentRow + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} + """.stripMargin) + } + + rowFrame.orElse(rangeFrame).map { frame => + frame.getChildren.toList match { + case precedingNode :: followingNode :: Nil => + SpecifiedWindowFrame( + frameType, + nodeToBoundary(precedingNode), + nodeToBoundary(followingNode)) + case precedingNode :: Nil => + SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) + case _ => + throw new NotImplementedError( + s"""No parse rules for the Window Frame based on Node ${frame.getName} + """.stripMargin) + } + }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) + } + + WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) + } val explode = "(?i)explode".r - def nodesToGenerator(nodes: Seq[Node]): Generator = { + def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { val function = nodes.head val attributes = nodes.flatMap { @@ -1276,13 +1451,17 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C function match { case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => - Explode(attributes, nodeToExpr(child)) + (Explode(nodeToExpr(child)), attributes) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => - HiveGenericUdtf( - new HiveFunctionWrapper(functionName), - attributes, - children.map(nodeToExpr)) + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $functionName")) + val functionClassName = functionInfo.getFunctionClass.getName + + (HiveGenericUdtf( + new HiveFunctionWrapper(functionClassName), + children.map(nodeToExpr)), attributes) case a: ASTNode => throw new NotImplementedError( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index e63cea60457d9..be9249a8b1f44 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,24 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.catalyst.expressions.Row - import scala.collection.JavaConversions._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.expressions.{Row, _} import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.sources.DescribeCommand -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, CreateTableUsing} +import org.apache.spark.sql.sources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} import org.apache.spark.sql.types.StringType @@ -58,9 +55,9 @@ private[hive] trait HiveStrategies { @Experimental object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: DataFrame) { - def lowerCase = DataFrame(s.sqlContext, s.logicalPlan) + def lowerCase: DataFrame = DataFrame(s.sqlContext, s.logicalPlan) - def addPartitioningAttributes(attrs: Seq[Attribute]) = { + def addPartitioningAttributes(attrs: Seq[Attribute]): DataFrame = { // Don't add the partitioning key if its already present in the data. if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) { s @@ -75,7 +72,7 @@ private[hive] trait HiveStrategies { } implicit class PhysicalPlanHacks(originalPlan: SparkPlan) { - def fakeOutput(newOutput: Seq[Attribute]) = + def fakeOutput(newOutput: Seq[Attribute]): OutputFaker = OutputFaker( originalPlan.output.map(a => newOutput.find(a.name.toLowerCase == _.name.toLowerCase) @@ -122,16 +119,16 @@ private[hive] trait HiveStrategies { val inputData = new GenericMutableRow(relation.partitionKeys.size) val pruningCondition = if (codegenEnabled) { - GeneratePredicate(castedPredicate) + GeneratePredicate.generate(castedPredicate) } else { - InterpretedPredicate(castedPredicate) + InterpretedPredicate.create(castedPredicate) } val partitions = relation.hiveQlPartitions.filter { part => val partitionValues = part.getValues var i = 0 while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) + inputData(i) = CatalystTypeConverters.convertToCatalyst(partitionValues(i)) i += 1 } pruningCondition(inputData) @@ -184,12 +181,14 @@ private[hive] trait HiveStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil - case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) => + table, partition, planLater(child), overwrite, ifNotExists) :: Nil + case hive.InsertIntoHiveTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil + table, partition, planLater(child), overwrite, ifNotExists) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index f22c9eaeedc7d..e556c74ffb015 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} @@ -35,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils +import org.apache.spark.util.Utils /** * A trait for subclasses that handle table scans. @@ -76,7 +77,9 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + Class.forName( + relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + .asInstanceOf[Class[Deserializer]], filterOpt = None) /** @@ -116,7 +119,7 @@ class HadoopTableReader( val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) + HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } deserializedHadoopRDD @@ -142,7 +145,46 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => + + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists + def verifyPartitionPath( + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): + Map[HivePartition, Class[_ <: Deserializer]] = { + if (!sc.conf.verifyPartitionPath) { + partitionToDeserializer + } else { + var existPathSet = collection.mutable.Set[String]() + var pathPatternSet = collection.mutable.Set[String]() + partitionToDeserializer.filter { + case (partition, partDeserializer) => + def updateExistPathSetByPathPattern(pathPatternStr: String) { + val pathPattern = new Path(pathPatternStr) + val fs = pathPattern.getFileSystem(sc.hiveconf) + val matches = fs.globStatus(pathPattern) + matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) + } + // convert /demo/data/year/month/day to /demo/data/*/*/*/ + def getPathPatternByPath(parNum: Int, tempPath: Path): String = { + var path = tempPath + for (i <- (1 to parNum)) path = path.getParent + val tails = (1 to parNum).map(_ => "*").mkString("/", "/", "/") + path.toString + tails + } + + val partPath = HiveShim.getDataLocationPath(partition) + val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size(); + var pathPatternStr = getPathPatternByPath(partNum, partPath) + if (!pathPatternSet.contains(pathPatternStr)) { + pathPatternSet += pathPatternStr + updateExistPathSetByPathPattern(pathPatternStr) + } + existPathSet.contains(partPath.toString) + } + } + } + + val hivePartitionRDDs = verifyPartitionPath(partitionToDeserializer) + .map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = HiveShim.getDataLocationPath(partition) val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) @@ -175,7 +217,7 @@ class HadoopTableReader( relation.partitionKeys.contains(attr) } - def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow) = { + def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow): Unit = { partitionKeyAttrs.foreach { case (attr, ordinal) => val partOrdinal = relation.partitionKeys.indexOf(attr) row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) @@ -189,9 +231,13 @@ class HadoopTableReader( val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) + // get the table deserializer + val tableSerDe = tableDesc.getDeserializerClass.newInstance() + tableSerDe.initialize(hconf, tableDesc.getProperties) // fill the non partition key attributes - HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, mutableRow) + HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, + mutableRow, tableSerDe) } }.toSeq @@ -261,25 +307,36 @@ private[hive] object HadoopTableReader extends HiveInspectors { * Transform all given raw `Writable`s into `Row`s. * * @param iterator Iterator of all `Writable`s to be transformed - * @param deserializer The `Deserializer` associated with the input `Writable` + * @param rawDeser The `Deserializer` associated with the input `Writable` * @param nonPartitionKeyAttrs Attributes that should be filled together with their corresponding * positions in the output schema * @param mutableRow A reusable `MutableRow` that should be filled + * @param tableDeser Table Deserializer * @return An `Iterator[Row]` transformed from `iterator` */ def fillObject( iterator: Iterator[Writable], - deserializer: Deserializer, + rawDeser: Deserializer, nonPartitionKeyAttrs: Seq[(Attribute, Int)], - mutableRow: MutableRow): Iterator[Row] = { + mutableRow: MutableRow, + tableDeser: Deserializer): Iterator[Row] = { + + val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) { + rawDeser.getObjectInspector.asInstanceOf[StructObjectInspector] + } else { + HiveShim.getConvertedOI( + rawDeser.getObjectInspector, + tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector] + } - val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => soi.getStructFieldRef(attr.name) -> ordinal }.unzip - // Builds specific unwrappers ahead of time according to object inspector types to avoid pattern - // matching and branching costs per row. + /** + * Builds specific unwrappers ahead of time according to object inspector + * types to avoid pattern matching and branching costs per row. + */ val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { _.getFieldObjectInspector match { case oi: BooleanObjectInspector => @@ -316,9 +373,11 @@ private[hive] object HadoopTableReader extends HiveInspectors { } } + val converter = ObjectInspectorConverters.getConverter(rawDeser.getObjectInspector, soi) + // Map each tuple to a row object iterator.map { value => - val raw = deserializer.deserialize(value) + val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 while (i < fieldRefs.length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala new file mode 100644 index 0000000000000..a863aa77cb7e0 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala @@ -0,0 +1,149 @@ +/* + * 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.sql.hive.client + +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} + +case class HiveDatabase( + name: String, + location: String) + +abstract class TableType { val name: String } +case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } +case object IndexTable extends TableType { override val name = "INDEX_TABLE" } +case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } +case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } + +case class HiveStorageDescriptor( + location: String, + inputFormat: String, + outputFormat: String, + serde: String) + +case class HivePartition( + values: Seq[String], + storage: HiveStorageDescriptor) + +case class HiveColumn(name: String, hiveType: String, comment: String) +case class HiveTable( + specifiedDatabase: Option[String], + name: String, + schema: Seq[HiveColumn], + partitionColumns: Seq[HiveColumn], + properties: Map[String, String], + serdeProperties: Map[String, String], + tableType: TableType, + location: Option[String] = None, + inputFormat: Option[String] = None, + outputFormat: Option[String] = None, + serde: Option[String] = None) { + + @transient + private[client] var client: ClientInterface = _ + + private[client] def withClient(ci: ClientInterface): this.type = { + client = ci + this + } + + def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) + + def isPartitioned: Boolean = partitionColumns.nonEmpty + + def getAllPartitions: Seq[HivePartition] = client.getAllPartitions(this) + + // Hive does not support backticks when passing names to the client. + def qualifiedName: String = s"$database.$name" +} + +/** + * An externally visible interface to the Hive client. This interface is shared across both the + * internal and external classloaders for a given version of Hive and thus must expose only + * shared classes. + */ +trait ClientInterface { + /** + * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will + * result in one string. + */ + def runSqlHive(sql: String): Seq[String] + + /** Returns the names of all tables in the given database. */ + def listTables(dbName: String): Seq[String] + + /** Returns the name of the active database. */ + def currentDatabase: String + + /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ + def getDatabase(name: String): HiveDatabase = { + getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) + } + + /** Returns the metadata for a given database, or None if it doesn't exist. */ + def getDatabaseOption(name: String): Option[HiveDatabase] + + /** Returns the specified table, or throws [[NoSuchTableException]]. */ + def getTable(dbName: String, tableName: String): HiveTable = { + getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) + } + + /** Returns the metadata for the specified table or None if it doens't exist. */ + def getTableOption(dbName: String, tableName: String): Option[HiveTable] + + /** Creates a table with the given metadata. */ + def createTable(table: HiveTable): Unit + + /** Updates the given table with new metadata. */ + def alterTable(table: HiveTable): Unit + + /** Creates a new database with the given name. */ + def createDatabase(database: HiveDatabase): Unit + + /** Returns all partitions for the given table. */ + def getAllPartitions(hTable: HiveTable): Seq[HivePartition] + + /** Loads a static partition into an existing table. */ + def loadPartition( + loadPath: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering + replace: Boolean, + holdDDLTime: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean): Unit + + /** Loads data into an existing table. */ + def loadTable( + loadPath: String, // TODO URI + tableName: String, + replace: Boolean, + holdDDLTime: Boolean): Unit + + /** Loads new dynamic partitions into an existing table. */ + def loadDynamicPartitions( + loadPath: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering + replace: Boolean, + numDP: Int, + holdDDLTime: Boolean, + listBucketingEnabled: Boolean): Unit + + /** Used for testing only. Removes all metadata from this instance of Hive. */ + def reset(): Unit +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala new file mode 100644 index 0000000000000..ea52fea037f1f --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -0,0 +1,395 @@ +/* + * 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.sql.hive.client + +import java.io.{BufferedReader, InputStreamReader, File, PrintStream} +import java.net.URI +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConversions._ +import scala.language.reflectiveCalls + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.api.Database +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.ql.metadata +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.Driver + +import org.apache.spark.Logging +import org.apache.spark.sql.execution.QueryExecutionException + + +/** + * A class that wraps the HiveClient and converts its responses to externally visible classes. + * Note that this class is typically loaded with an internal classloader for each instantiation, + * allowing it to interact directly with a specific isolated version of Hive. Loading this class + * with the isolated classloader however will result in it only being visible as a ClientInterface, + * not a ClientWrapper. + * + * This class needs to interact with multiple versions of Hive, but will always be compiled with + * the 'native', execution version of Hive. Therefore, any places where hive breaks compatibility + * must use reflection after matching on `version`. + * + * @param version the version of hive used when pick function calls that are not compatible. + * @param config a collection of configuration options that will be added to the hive conf before + * opening the hive client. + */ +class ClientWrapper( + version: HiveVersion, + config: Map[String, String]) + extends ClientInterface + with Logging + with ReflectionMagic { + + private val conf = new HiveConf(classOf[SessionState]) + config.foreach { case (k, v) => + logDebug(s"Hive Config: $k=$v") + conf.set(k, v) + } + + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. + private val outputBuffer = new java.io.OutputStream { + var pos: Int = 0 + var buffer = new Array[Int](10240) + def write(i: Int): Unit = { + buffer(pos) = i + pos = (pos + 1) % buffer.size + } + + override def toString: String = { + val (end, start) = buffer.splitAt(pos) + val input = new java.io.InputStream { + val iterator = (start ++ end).iterator + + def read(): Int = if (iterator.hasNext) iterator.next() else -1 + } + val reader = new BufferedReader(new InputStreamReader(input)) + val stringBuilder = new StringBuilder + var line = reader.readLine() + while(line != null) { + stringBuilder.append(line) + stringBuilder.append("\n") + line = reader.readLine() + } + stringBuilder.toString() + } + } + + val state = { + val original = Thread.currentThread().getContextClassLoader + Thread.currentThread().setContextClassLoader(getClass.getClassLoader) + val ret = try { + val newState = new SessionState(conf) + SessionState.start(newState) + newState.out = new PrintStream(outputBuffer, true, "UTF-8") + newState.err = new PrintStream(outputBuffer, true, "UTF-8") + newState + } finally { + Thread.currentThread().setContextClassLoader(original) + } + ret + } + + private val client = Hive.get(conf) + + /** + * Runs `f` with ThreadLocal session state and classloaders configured for this version of hive. + */ + private def withHiveState[A](f: => A): A = synchronized { + val original = Thread.currentThread().getContextClassLoader + Thread.currentThread().setContextClassLoader(getClass.getClassLoader) + Hive.set(client) + version match { + case hive.v12 => + classOf[SessionState] + .callStatic[SessionState, SessionState]("start", state) + case hive.v13 => + classOf[SessionState] + .callStatic[SessionState, SessionState]("setCurrentSessionState", state) + } + val ret = try f finally { + Thread.currentThread().setContextClassLoader(original) + } + ret + } + + override def currentDatabase: String = withHiveState { + state.getCurrentDatabase + } + + override def createDatabase(database: HiveDatabase): Unit = withHiveState { + client.createDatabase( + new Database( + database.name, + "", + new File(database.location).toURI.toString, + new java.util.HashMap), + true) + } + + override def getDatabaseOption(name: String): Option[HiveDatabase] = withHiveState { + Option(client.getDatabase(name)).map { d => + HiveDatabase( + name = d.getName, + location = d.getLocationUri) + } + } + + override def getTableOption( + dbName: String, + tableName: String): Option[HiveTable] = withHiveState { + + logDebug(s"Looking up $dbName.$tableName") + + val hiveTable = Option(client.getTable(dbName, tableName, false)) + val converted = hiveTable.map { h => + + HiveTable( + name = h.getTableName, + specifiedDatabase = Option(h.getDbName), + schema = h.getCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), + partitionColumns = h.getPartCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), + properties = h.getParameters.toMap, + serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, + tableType = ManagedTable, // TODO + location = version match { + case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString) + case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString) + }, + inputFormat = Option(h.getInputFormatClass).map(_.getName), + outputFormat = Option(h.getOutputFormatClass).map(_.getName), + serde = Option(h.getSerializationLib)).withClient(this) + } + converted + } + + private def toInputFormat(name: String) = + Class.forName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] + + private def toOutputFormat(name: String) = + Class.forName(name) + .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] + + private def toQlTable(table: HiveTable): metadata.Table = { + val qlTable = new metadata.Table(table.database, table.name) + + qlTable.setFields(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + qlTable.setPartCols( + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } + table.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } + version match { + case hive.v12 => + table.location.map(new URI(_)).foreach(u => qlTable.call[URI, Unit]("setDataLocation", u)) + case hive.v13 => + table.location + .map(new org.apache.hadoop.fs.Path(_)) + .foreach(qlTable.call[Path, Unit]("setDataLocation", _)) + } + table.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) + table.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) + table.serde.foreach(qlTable.setSerializationLib) + + qlTable + } + + override def createTable(table: HiveTable): Unit = withHiveState { + val qlTable = toQlTable(table) + client.createTable(qlTable) + } + + override def alterTable(table: HiveTable): Unit = withHiveState { + val qlTable = toQlTable(table) + client.alterTable(table.qualifiedName, qlTable) + } + + override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { + val qlTable = toQlTable(hTable) + val qlPartitions = version match { + case hive.v12 => + client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsForPruner", qlTable) + case hive.v13 => + client.call[metadata.Table, Set[metadata.Partition]]("getAllPartitionsOf", qlTable) + } + qlPartitions.map(_.getTPartition).map { p => + HivePartition( + values = Option(p.getValues).map(_.toSeq).getOrElse(Seq.empty), + storage = HiveStorageDescriptor( + location = p.getSd.getLocation, + inputFormat = p.getSd.getInputFormat, + outputFormat = p.getSd.getOutputFormat, + serde = p.getSd.getSerdeInfo.getSerializationLib)) + }.toSeq + } + + override def listTables(dbName: String): Seq[String] = withHiveState { + client.getAllTables + } + + /** + * Runs the specified SQL query using Hive. + */ + override def runSqlHive(sql: String): Seq[String] = { + val maxResults = 100000 + val results = runHive(sql, maxResults) + // It is very confusing when you only get back some of the results... + if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") + results + } + + /** + * Execute the command using Hive and return the results as a sequence. Each element + * in the sequence is one row. + */ + protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState { + logDebug(s"Running hiveql '$cmd'") + if (cmd.toLowerCase.startsWith("set")) { logDebug(s"Changing config: $cmd") } + try { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc: CommandProcessor = version match { + case hive.v12 => + classOf[CommandProcessorFactory] + .callStatic[String, HiveConf, CommandProcessor]("get", cmd_1, conf) + case hive.v13 => + classOf[CommandProcessorFactory] + .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf) + } + + proc match { + case driver: Driver => + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.close() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + + val results = version match { + case hive.v12 => + val res = new JArrayList[String] + driver.call[JArrayList[String], Boolean]("getResults", res) + res.toSeq + case hive.v13 => + val res = new JArrayList[Object] + driver.call[JArrayList[Object], Boolean]("getResults", res) + res.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + driver.close() + results + + case _ => + if (state.out != null) { + state.out.println(tokens(0) + " " + cmd_1) + } + Seq(proc.run(cmd_1).getResponseCode.toString) + } + } catch { + case e: Exception => + logError( + s""" + |====================== + |HIVE FAILURE OUTPUT + |====================== + |${outputBuffer.toString} + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e + } + } + + def loadPartition( + loadPath: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], + replace: Boolean, + holdDDLTime: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean): Unit = withHiveState { + + client.loadPartition( + new Path(loadPath), // TODO: Use URI + tableName, + partSpec, + replace, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } + + def loadTable( + loadPath: String, // TODO URI + tableName: String, + replace: Boolean, + holdDDLTime: Boolean): Unit = withHiveState { + client.loadTable( + new Path(loadPath), + tableName, + replace, + holdDDLTime) + } + + def loadDynamicPartitions( + loadPath: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], + replace: Boolean, + numDP: Int, + holdDDLTime: Boolean, + listBucketingEnabled: Boolean): Unit = withHiveState { + client.loadDynamicPartitions( + new Path(loadPath), + tableName, + partSpec, + replace, + numDP, + holdDDLTime, + listBucketingEnabled) + } + + def reset(): Unit = withHiveState { + client.getAllTables("default").foreach { t => + logDebug(s"Deleting table $t") + val table = client.getTable("default", t) + client.getIndexes("default", t, 255).foreach { index => + client.dropIndex("default", t, index.getIndexName, true) + } + if (!table.isIndexTable) { + client.dropTable("default", t) + } + } + client.getAllDatabases.filterNot(_ == "default").foreach { db => + logDebug(s"Dropping Database: $db") + client.dropDatabase(db, true, false, true) + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala new file mode 100644 index 0000000000000..710dbca6e3c66 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -0,0 +1,172 @@ +/* + * 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.sql.hive.client + +import java.io.File +import java.net.URLClassLoader +import java.util + +import scala.language.reflectiveCalls +import scala.util.Try + +import org.apache.commons.io.{FileUtils, IOUtils} + +import org.apache.spark.Logging +import org.apache.spark.deploy.SparkSubmitUtils + +import org.apache.spark.sql.catalyst.util.quietly + +/** Factory for `IsolatedClientLoader` with specific versions of hive. */ +object IsolatedClientLoader { + /** + * Creates isolated Hive client loaders by downloading the requested version from maven. + */ + def forVersion( + version: String, + config: Map[String, String] = Map.empty): IsolatedClientLoader = synchronized { + val resolvedVersion = hiveVersion(version) + val files = resolvedVersions.getOrElseUpdate(resolvedVersion, downloadVersion(resolvedVersion)) + new IsolatedClientLoader(hiveVersion(version), files, config) + } + + def hiveVersion(version: String): HiveVersion = version match { + case "12" | "0.12" | "0.12.0" => hive.v12 + case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 + } + + private def downloadVersion(version: HiveVersion): Seq[File] = { + val hiveArtifacts = + (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++ + (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil)) + .map(a => s"org.apache.hive:$a:${version.fullVersion}") :+ + "com.google.guava:guava:14.0.1" :+ + "org.apache.hadoop:hadoop-client:2.4.0" :+ + "mysql:mysql-connector-java:5.1.12" + + val classpath = quietly { + SparkSubmitUtils.resolveMavenCoordinates( + hiveArtifacts.mkString(","), + Some("http://www.datanucleus.org/downloads/maven2"), + None) + } + val allFiles = classpath.split(",").map(new File(_)).toSet + + // TODO: Remove copy logic. + val tempDir = File.createTempFile("hive", "v" + version.toString) + tempDir.delete() + tempDir.mkdir() + + allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) + tempDir.listFiles() + } + + private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[File]] +} + +/** + * Creates a Hive `ClientInterface` using a classloader that works according to the following rules: + * - Shared classes: Java, Scala, logging, and Spark classes are delegated to `baseClassLoader` + * allowing the results of calls to the `ClientInterface` to be visible externally. + * - Hive classes: new instances are loaded from `execJars`. These classes are not + * accessible externally due to their custom loading. + * - ClientWrapper: a new copy is created for each instance of `IsolatedClassLoader`. + * This new instance is able to see a specific version of hive without using reflection. Since + * this is a unique instance, it is not visible externally other than as a generic + * `ClientInterface`, unless `isolationOn` is set to `false`. + * + * @param version The version of hive on the classpath. used to pick specific function signatures + * that are not compatibile accross versions. + * @param execJars A collection of jar files that must include hive and hadoop. + * @param config A set of options that will be added to the HiveConf of the constructed client. + * @param isolationOn When true, custom versions of barrier classes will be constructed. Must be + * true unless loading the version of hive that is on Sparks classloader. + * @param rootClassLoader The system root classloader. Must not know about hive classes. + * @param baseClassLoader The spark classloader that is used to load shared classes. + * + */ +class IsolatedClientLoader( + val version: HiveVersion, + val execJars: Seq[File] = Seq.empty, + val config: Map[String, String] = Map.empty, + val isolationOn: Boolean = true, + val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, + val baseClassLoader: ClassLoader = Thread.currentThread().getContextClassLoader) + extends Logging { + + // Check to make sure that the root classloader does not know about Hive. + assert(Try(baseClassLoader.loadClass("org.apache.hive.HiveConf")).isFailure) + + /** All jars used by the hive specific classloader. */ + protected def allJars = execJars.map(_.toURI.toURL).toArray + + protected def isSharedClass(name: String): Boolean = + name.contains("slf4j") || + name.contains("log4j") || + name.startsWith("org.apache.spark.") || + name.startsWith("scala.") || + name.startsWith("com.google") || + name.startsWith("java.lang.") || + name.startsWith("java.net") + + /** True if `name` refers to a spark class that must see specific version of Hive. */ + protected def isBarrierClass(name: String): Boolean = + name.startsWith("org.apache.spark.sql.hive.execution.PairSerDe") || + name.startsWith(classOf[ClientWrapper].getName) || + name.startsWith(classOf[ReflectionMagic].getName) + + protected def classToPath(name: String): String = + name.replaceAll("\\.", "/") + ".class" + + /** The classloader that is used to load an isolated version of Hive. */ + protected val classLoader: ClassLoader = new URLClassLoader(allJars, rootClassLoader) { + override def loadClass(name: String, resolve: Boolean): Class[_] = { + val loaded = findLoadedClass(name) + if (loaded == null) doLoadClass(name, resolve) else loaded + } + + def doLoadClass(name: String, resolve: Boolean): Class[_] = { + val classFileName = name.replaceAll("\\.", "/") + ".class" + if (isBarrierClass(name) && isolationOn) { + val bytes = IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName)) + logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}") + defineClass(name, bytes, 0, bytes.length) + } else if (!isSharedClass(name)) { + logDebug(s"hive class: $name - ${getResource(classToPath(name))}") + super.loadClass(name, resolve) + } else { + logDebug(s"shared class: $name") + baseClassLoader.loadClass(name) + } + } + } + + // Pre-reflective instantiation setup. + logDebug("Initializing the logger to avoid disaster...") + Thread.currentThread.setContextClassLoader(classLoader) + + /** The isolated client interface to Hive. */ + val client: ClientInterface = try { + classLoader + .loadClass(classOf[ClientWrapper].getName) + .getConstructors.head + .newInstance(version, config) + .asInstanceOf[ClientInterface] + } finally { + Thread.currentThread.setContextClassLoader(baseClassLoader) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala new file mode 100644 index 0000000000000..90d03049356b5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala @@ -0,0 +1,200 @@ +/* + * 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.sql.hive.client + +import scala.reflect._ + +/** + * Provides implicit functions on any object for calling methods reflectively. + */ +protected trait ReflectionMagic { + /** code for InstanceMagic + println( + (1 to 22).map { n => + def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ") + val types = repeat(n => s"A$n <: AnyRef : ClassTag") + val inArgs = repeat(n => s"a$n: A$n") + val erasure = repeat(n => s"classTag[A$n].erasure") + val outArgs = repeat(n => s"a$n") + s"""|def call[$types, R](name: String, $inArgs): R = { + | clazz.getMethod(name, $erasure).invoke(a, $outArgs).asInstanceOf[R] + |}""".stripMargin + }.mkString("\n") + ) + */ + + // scalastyle:off + protected implicit class InstanceMagic(a: Any) { + private val clazz = a.getClass + + def call[R](name: String): R = { + clazz.getMethod(name).invoke(a).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = { + clazz.getMethod(name, classTag[A1].erasure).invoke(a, a1).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(a, a1, a2).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(a, a1, a2, a3).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(a, a1, a2, a3, a4).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(a, a1, a2, a3, a4, a5).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(a, a1, a2, a3, a4, a5, a6).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R] + } + def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = { + clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R] + } + } + + /** code for StaticMagic + println( + (1 to 22).map { n => + def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ") + val types = repeat(n => s"A$n <: AnyRef : ClassTag") + val inArgs = repeat(n => s"a$n: A$n") + val erasure = repeat(n => s"classTag[A$n].erasure") + val outArgs = repeat(n => s"a$n") + s"""|def callStatic[$types, R](name: String, $inArgs): R = { + | c.getDeclaredMethod(name, $erasure).invoke(c, $outArgs).asInstanceOf[R] + |}""".stripMargin + }.mkString("\n") + ) + */ + + protected implicit class StaticMagic(c: Class[_]) { + def callStatic[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = { + c.getDeclaredMethod(name, classTag[A1].erasure).invoke(c, a1).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(c, a1, a2).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(c, a1, a2, a3).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(c, a1, a2, a3, a4).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(c, a1, a2, a3, a4, a5).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(c, a1, a2, a3, a4, a5, a6).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R] + } + def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = { + c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R] + } + } + // scalastyle:on +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala new file mode 100644 index 0000000000000..7db9200d47440 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -0,0 +1,33 @@ +/* + * 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.sql.hive + +/** Support for interacting with different versions of the HiveMetastoreClient */ +package object client { + private[client] abstract class HiveVersion(val fullVersion: String, val hasBuiltinsJar: Boolean) + + // scalastyle:off + private[client] object hive { + case object v10 extends HiveVersion("0.10.0", true) + case object v11 extends HiveVersion("0.11.0", false) + case object v12 extends HiveVersion("0.12.0", false) + case object v13 extends HiveVersion("0.13.1", false) + } + // scalastyle:on + +} \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index a0c91cbc4e86f..76a1965f3cb25 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -45,7 +45,7 @@ case class CreateTableAsSelect( allowExisting: Boolean, desc: Option[CreateTableDesc]) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { // Create Hive Table @@ -67,7 +67,7 @@ case class CreateTableAsSelect( new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") } } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index d0510aa342796..6fce69b58b85e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -37,7 +37,7 @@ case class DescribeHiveTableCommand( override val output: Seq[Attribute], isExtended: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala index 9636da206087f..60a9bb630d0d9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala @@ -26,9 +26,9 @@ import org.apache.spark.sql.types.StringType private[hive] case class HiveNativeCommand(sql: String) extends RunnableCommand { - override def output = + override def output: Seq[AttributeReference] = Seq(AttributeReference("result", StringType, nullable = false)()) - override def run(sqlContext: SQLContext) = + override def run(sqlContext: SQLContext): Seq[Row] = sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 5b3cf2861e8ef..0a5f19eee7105 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ @@ -128,11 +129,11 @@ case class HiveTableScan( } } - override def execute() = if (!relation.hiveQlTable.isPartitioned) { + override def execute(): RDD[Row] = if (!relation.hiveQlTable.isPartitioned) { hadoopReader.makeRDDForTable(relation.hiveQlTable) } else { hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) } - override def output = attributes + override def output: Seq[Attribute] = attributes } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index ba5c8e028a151..89995a91b1a92 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{UnaryNode, SparkPlan} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} @@ -45,12 +45,13 @@ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan, - overwrite: Boolean) extends UnaryNode with HiveInspectors { + overwrite: Boolean, + ifNotExists: Boolean) extends UnaryNode with HiveInspectors { @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) - @transient private lazy val db = Hive.get(sc.hiveconf) + @transient private lazy val catalog = sc.catalog private def newSerializer(tableDesc: TableDesc): Serializer = { val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] @@ -58,7 +59,7 @@ case class InsertIntoHiveTable( serializer } - def output = child.output + def output: Seq[Attribute] = child.output def saveAsHiveFile( rdd: RDD[Row], @@ -72,7 +73,6 @@ case class InsertIntoHiveTable( val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName assert(outputFileFormatClassName != null, "Output format class not set") conf.value.set("mapred.output.format.class", outputFileFormatClassName) - conf.value.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( conf.value, @@ -200,38 +200,55 @@ case class InsertIntoHiveTable( orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - db.validatePartitionNameCharacters(partVals) + catalog.synchronized { + catalog.client.validatePartitionNameCharacters(partVals) + } // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false if (numDynamicPartitions > 0) { - db.loadDynamicPartitions( - outputPath, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - numDynamicPartitions, - holdDDLTime, - isSkewedStoreAsSubdir - ) + catalog.synchronized { + catalog.client.loadDynamicPartitions( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + numDynamicPartitions, + holdDDLTime, + isSkewedStoreAsSubdir) + } } else { - db.loadPartition( + // scalastyle:off + // ifNotExists is only valid with static partition, refer to + // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries + // scalastyle:on + val oldPart = catalog.synchronized { + catalog.client.getPartition( + catalog.client.getTable(qualifiedTableName), partitionSpec, false) + } + if (oldPart == null || !ifNotExists) { + catalog.synchronized { + catalog.client.loadPartition( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } + } + } + } else { + catalog.synchronized { + catalog.client.loadTable( outputPath, qualifiedTableName, - orderedPartitionSpec, overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + holdDDLTime) } - } else { - db.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) } // Invalidate the cache. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0c9aee33985bc..3eddda3b28c66 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader} -import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.io.{BufferedReader, DataInputStream, DataOutputStream, EOFException, InputStreamReader} import java.util.Properties import scala.collection.JavaConversions._ @@ -27,12 +26,14 @@ import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils /** @@ -51,9 +52,9 @@ case class ScriptTransformation( ioschema: HiveScriptIOSchema)(@transient sc: HiveContext) extends UnaryNode { - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs: Seq[HiveContext] = sc :: Nil - def execute() = { + def execute(): RDD[Row] = { child.execute().mapPartitions { iter => val cmd = List("/bin/bash", "-c", script) val builder = new ProcessBuilder(cmd) @@ -120,14 +121,13 @@ case class ScriptTransformation( if (outputSerde == null) { val prevLine = curLine curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"))) .asInstanceOf[Array[Any]]) } else { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2)) .asInstanceOf[Array[Any]]) } } else { @@ -145,20 +145,29 @@ case class ScriptTransformation( val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) - iter - .map(outputProjection) - .foreach { row => - if (inputSerde == null) { - val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), - ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") - - outputStream.write(data) - } else { - val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) - prepareWritable(writable).write(dataOutputStream) + // Put the write(output to the pipeline) into a single thread + // and keep the collector as remain in the main thread. + // otherwise it will causes deadlock if the data size greater than + // the pipeline / buffer capacity. + new Thread(new Runnable() { + override def run(): Unit = { + iter + .map(outputProjection) + .foreach { row => + if (inputSerde == null) { + val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + + outputStream.write(data) + } else { + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + prepareWritable(writable).write(dataOutputStream) + } } + outputStream.close() } - outputStream.close() + }).start() + iterator } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 63ad145a6a980..a40a1e53117cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -22,11 +22,11 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Analyzes the given table in the current database to generate statistics, which will be @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType private[hive] case class AnalyzeTable(tableName: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { sqlContext.asInstanceOf[HiveContext].analyze(tableName) Seq.empty[Row] } @@ -52,18 +52,19 @@ case class DropTable( tableName: String, ifExists: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] val ifExistsClause = if (ifExists) "IF EXISTS " else "" try { hiveContext.cacheManager.tryUncacheQuery(hiveContext.table(tableName)) } catch { - // This table's metadata is not in + // This table's metadata is not in Hive metastore (e.g. the table does not exist). case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException => + case _: org.apache.spark.sql.catalyst.analysis.NoSuchTableException => // Other Throwables can be caused by users providing wrong parameters in OPTIONS // (e.g. invalid paths). We catch it and log a warning message. // Users should be able to drop such kinds of tables regardless if there is an error. - case e: Throwable => log.warn(s"${e.getMessage}") + case e: Throwable => log.warn(s"${e.getMessage}", e) } hiveContext.invalidateTable(tableName) hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") @@ -75,18 +76,24 @@ case class DropTable( private[hive] case class AddJar(path: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("result", IntegerType, false) :: Nil) + schema.toAttributes + } + + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) - Seq.empty[Row] + Seq(Row(0)) } } private[hive] case class AddFile(path: String) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD FILE $path") hiveContext.sparkContext.addFile(path) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 4a702d96563d5..fd0b6f058595d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -17,27 +17,27 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.spark.sql.AnalysisException import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory -import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Generate, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.analysis.MultiAlias -import org.apache.spark.sql.catalyst.errors.TreeNodeException /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -45,7 +45,7 @@ import scala.collection.JavaConversions._ private[hive] abstract class HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { - def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) + def getFunctionInfo(name: String): FunctionInfo = FunctionRegistry.getFunctionInfo(name) def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is @@ -66,7 +66,7 @@ private[hive] abstract class HiveFunctionRegistry } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), Nil, children) + HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } @@ -78,7 +78,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre type EvaluatedType = Any type UDFType = UDF - def nullable = true + override def nullable: Boolean = true @transient lazy val function = funcWrapper.createFunction[UDFType]() @@ -96,7 +96,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre udfType != null && udfType.deterministic() } - override def foldable = isUDFDeterministic && children.forall(_.foldable) + override def foldable: Boolean = isUDFDeterministic && children.forall(_.foldable) // Create parameter converters @transient @@ -110,7 +110,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre method.getGenericReturnType(), ObjectInspectorOptions.JAVA) @transient - protected lazy val cached = new Array[AnyRef](children.length) + protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) // TODO: Finish input output types. override def eval(input: Row): Any = { @@ -120,17 +120,19 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre returnInspector) } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } } // Adapter from Catalyst ExpressionResult to Hive DeferredObject private[hive] class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject with HiveInspectors { private var func: () => Any = _ - def set(func: () => Any) { + def set(func: () => Any): Unit = { this.func = func } - override def prepare(i: Int) = {} + override def prepare(i: Int): Unit = {} override def get(): AnyRef = wrap(func(), oi) } @@ -139,7 +141,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr type UDFType = GenericUDF type EvaluatedType = Any - def nullable = true + override def nullable: Boolean = true @transient lazy val function = funcWrapper.createFunction[UDFType]() @@ -158,7 +160,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr (udfType != null && udfType.deterministic()) } - override def foldable = + override def foldable: Boolean = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] @transient @@ -182,7 +184,222 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr unwrap(function.evaluate(deferedObjects), returnInspector) } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } +} + +/** + * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. + */ +private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case p: LogicalPlan if !p.childrenResolved => p + + // We are resolving WindowExpressions at here. When we get here, we have already + // replaced those WindowSpecReferences. + case p: LogicalPlan => + p transformExpressions { + case WindowExpression( + UnresolvedWindowFunction(name, children), + windowSpec: WindowSpecDefinition) => + // First, let's find the window function info. + val windowFunctionInfo: WindowFunctionInfo = + Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( + throw new AnalysisException(s"Couldn't find window function $name")) + + // Get the class of this function. + // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use + // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. + val functionClass = windowFunctionInfo.getfInfo().getFunctionClass + val newChildren = + // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit + // input parameters and requires implicit parameters, which + // are expressions in Order By clause. + if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { + if (children.nonEmpty) { + throw new AnalysisException(s"$name does not take input parameters.") + } + windowSpec.orderSpec.map(_.child) + } else { + children + } + + // If the class is UDAF, we need to use UDAFBridge. + val isUDAFBridgeRequired = + if (classOf[UDAF].isAssignableFrom(functionClass)) { + true + } else { + false + } + + // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of + // HiveWindowFunction. + val windowFunction = + HiveWindowFunction( + new HiveFunctionWrapper(functionClass.getName), + windowFunctionInfo.isPivotResult, + isUDAFBridgeRequired, + newChildren) + + // Second, check if the specified window function can accept window definition. + windowSpec.frameSpecification match { + case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => + // This Hive window function does not support user-speficied window frame. + throw new AnalysisException( + s"Window function $name does not take a frame specification.") + case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && + windowFunctionInfo.isPivotResult => + // These two should not be true at the same time when a window frame is defined. + // If so, throw an exception. + throw new AnalysisException(s"Could not handle Hive window function $name because " + + s"it supports both a user specified window frame and pivot result.") + case _ => // OK + } + // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs + // a window frame specification to work. + val newWindowSpec = windowSpec.frameSpecification match { + case UnspecifiedFrame => + val newWindowFrame = + SpecifiedWindowFrame.defaultWindowFrame( + windowSpec.orderSpec.nonEmpty, + windowFunctionInfo.isSupportsWindow) + WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) + case _ => windowSpec + } + + // Finally, we create a WindowExpression with the resolved window function and + // specified window spec. + WindowExpression(windowFunction, newWindowSpec) + } + } +} + +/** + * A [[WindowFunction]] implementation wrapping Hive's window function. + * @param funcWrapper The wrapper for the Hive Window Function. + * @param pivotResult If it is true, the Hive function will return a list of values representing + * the values of the added columns. Otherwise, a single value is returned for + * current row. + * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's + * createFunction is UDAF, we need to use GenericUDAFBridge to wrap + * it as a GenericUDAFResolver2. + * @param children Input parameters. + */ +private[hive] case class HiveWindowFunction( + funcWrapper: HiveFunctionWrapper, + pivotResult: Boolean, + isUDAFBridgeRequired: Boolean, + children: Seq[Expression]) extends WindowFunction + with HiveInspectors { + + // Hive window functions are based on GenericUDAFResolver2. + type UDFType = GenericUDAFResolver2 + + @transient + protected lazy val resolver: GenericUDAFResolver2 = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) + } else { + funcWrapper.createFunction[GenericUDAFResolver2]() + } + + @transient + protected lazy val inputInspectors = children.map(toInspector).toArray + + // The GenericUDAFEvaluator used to evaluate the window function. + @transient + protected lazy val evaluator: GenericUDAFEvaluator = { + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The object inspector of values returned from the Hive window function. + @transient + protected lazy val returnInspector = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + def dataType: DataType = + if (!pivotResult) { + inspectorToDataType(returnInspector) + } else { + // If pivotResult is true, we should take the element type out as the data type of this + // function. + inspectorToDataType(returnInspector) match { + case ArrayType(dt, _) => dt + case _ => + sys.error( + s"error resolve the data type of window function ${funcWrapper.functionClassName}") + } + } + + def nullable: Boolean = true + + override type EvaluatedType = Any + + override def eval(input: Row): Any = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + @transient + lazy val inputProjection = new InterpretedProjection(children) + + @transient + private var hiveEvaluatorBuffer: AggregationBuffer = _ + // Output buffer. + private var outputBuffer: Any = _ + + override def init(): Unit = { + evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) + } + + // Reset the hiveEvaluatorBuffer and outputPosition + override def reset(): Unit = { + // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. + // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. + // However, RowNumberBuffer.init does not really reset this buffer. + hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer + evaluator.reset(hiveEvaluatorBuffer) + } + + override def prepareInputParameters(input: Row): AnyRef = { + wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length)) + } + // Add input parameters for a single row. + override def update(input: AnyRef): Unit = { + evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) + } + + override def batchUpdate(inputs: Array[AnyRef]): Unit = { + var i = 0 + while (i < inputs.length) { + evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + i += 1 + } + } + + override def evaluate(): Unit = { + outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) + } + + override def get(index: Int): Any = { + if (!pivotResult) { + // if pivotResult is false, we will get a single value for all rows in the frame. + outputBuffer + } else { + // if pivotResult is true, we will get a Seq having the same size with the size + // of the window frame. At here, we will return the result at the position of + // index in the output buffer. + outputBuffer.asInstanceOf[Seq[Any]].get(index) + } + } + + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } + + override def newInstance: WindowFunction = + new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) } private[hive] case class HiveGenericUdaf( @@ -209,9 +426,11 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } - def newInstance() = new HiveUdafFunction(funcWrapper, children, this) + def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this) } /** It is used as a wrapper for the hive functions which uses UDAF interface */ @@ -240,10 +459,11 @@ private[hive] case class HiveUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" + } - def newInstance() = - new HiveUdafFunction(funcWrapper, children, this, true) + def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this, true) } /** @@ -259,7 +479,6 @@ private[hive] case class HiveUdaf( */ private[hive] case class HiveGenericUdtf( funcWrapper: HiveFunctionWrapper, - aliasNames: Seq[String], children: Seq[Expression]) extends Generator with HiveInspectors { @@ -275,23 +494,8 @@ private[hive] case class HiveGenericUdtf( @transient protected lazy val udtInput = new Array[AnyRef](children.length) - protected lazy val outputDataTypes = outputInspector.getAllStructFieldRefs.map { - field => inspectorToDataType(field.getFieldObjectInspector) - } - - override protected def makeOutput() = { - // Use column names when given, otherwise _c1, _c2, ... _cn. - if (aliasNames.size == outputDataTypes.size) { - aliasNames.zip(outputDataTypes).map { - case (attrName, attrDataType) => - AttributeReference(attrName, attrDataType, nullable = true)() - } - } else { - outputDataTypes.zipWithIndex.map { - case (attrDataType, i) => - AttributeReference(s"_c$i", attrDataType, nullable = true)() - } - } + lazy val elementTypes = outputInspector.getAllStructFieldRefs.map { + field => (inspectorToDataType(field.getFieldObjectInspector), true) } override def eval(input: Row): TraversableOnce[Row] = { @@ -314,29 +518,15 @@ private[hive] case class HiveGenericUdtf( collected += unwrap(input, outputInspector).asInstanceOf[Row] } - def collectRows() = { + def collectRows(): Seq[Row] = { val toCollect = collected collected = new ArrayBuffer[Row] toCollect } } - override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" -} - -/** - * Resolve Udtfs Alias. - */ -private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan) = plan transform { - case p @ Project(projectList, _) - if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 1 => - throw new TreeNodeException(p, "only single Generator supported for SELECT clause") - - case Project(Seq(Alias(udtf @ HiveGenericUdtf(_, _, _), name)), child) => - Generate(udtf.copy(aliasNames = Seq(name)), join = false, outer = false, None, child) - case Project(Seq(MultiAlias(udtf @ HiveGenericUdtf(_, _, _), names)), child) => - Generate(udtf.copy(aliasNames = names), join = false, outer = false, None, child) + override def toString: String = { + s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } } @@ -366,9 +556,8 @@ private[hive] case class HiveUdafFunction( private val returnInspector = function.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) - // Cast required to avoid type inference selecting a deprecated Hive API. private val buffer = - function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] + function.getNewAggregationBuffer override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index f136e43acc8f2..8398da268174d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive -import java.io.IOException import java.text.NumberFormat import java.util.Date @@ -118,19 +117,7 @@ private[hive] class SparkHiveWriterContainer( } protected def commit() { - if (committer.needsTaskCommit(taskContext)) { - try { - committer.commitTask(taskContext) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - committer.abortTask(taskContext) - throw e - } - } else { - logInfo("No need to commit output of task: " + taID.value) - } + SparkHadoopMapRedUtil.commitTask(committer, taskContext, jobID, splitID, attemptID) } private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { @@ -213,7 +200,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( .zip(row.toSeq.takeRight(dynamicPartColNames.length)) .map { case (col, rawVal) => val string = if (rawVal == null) null else String.valueOf(rawVal) - val colString = + val colString = if (string == null || string.isEmpty) { defaultPartName } else { @@ -222,7 +209,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( s"/$col=$colString" }.mkString - def newWriter = { + def newWriter(): FileSinkOperator.RecordWriter = { val newFileSinkDesc = new FileSinkDesc( fileSinkConf.getDirName + dynamicPartPath, fileSinkConf.getTableInfo, @@ -246,6 +233,6 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( Reporter.NULL) } - writers.getOrElseUpdate(dynamicPartPath, newWriter) + writers.getOrElseUpdate(dynamicPartPath, newWriter()) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index b4aee78046383..edeab5158df62 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -23,6 +23,7 @@ import java.util.{Set => JavaSet} import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table +import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -106,7 +107,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { /** Fewer partitions to speed up testing. */ protected[sql] override lazy val conf: SQLConf = new SQLConf { override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + + // TODO as in unit test, conf.clear() probably be called, all of the value will be cleared. + // The super.getConf(SQLConf.DIALECT) is "sql" by default, we need to set it as "hiveql" + override def dialect: String = super.getConf(SQLConf.DIALECT, "hiveql") } } @@ -153,10 +157,15 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { val describedTable = "DESCRIBE (\\w+)".r + val vs = new VariableSubstitution() + + // we should substitute variables in hql to pass the text to parseSql() as a parameter. + // Hive parser need substituted text. HiveContext.sql() does this but return a DataFrame, + // while we need a logicalPlan so we cannot reuse that. protected[hive] class HiveQLQueryExecution(hql: String) - extends this.QueryExecution(HiveQl.parseSql(hql)) { - def hiveExec() = runSqlHive(hql) - override def toString = hql + "\n" + super.toString + extends this.QueryExecution(HiveQl.parseSql(vs.substitute(hiveconf, hql))) { + def hiveExec(): Seq[String] = runSqlHive(hql) + override def toString: String = hql + "\n" + super.toString } /** @@ -179,14 +188,16 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. - analyzer(logical) + analyzer.execute(logical) } } case class TestTable(name: String, commands: (()=>Unit)*) protected[hive] implicit class SqlCmd(sql: String) { - def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit + def cmd: () => Unit = { + () => new HiveQLQueryExecution(sql).stringResult(): Unit + } } /** @@ -194,7 +205,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { * demand when a query are run against it. */ lazy val testTables = new mutable.HashMap[String, TestTable]() - def registerTestTable(testTable: TestTable) = testTables += (testTable.name -> testTable) + + def registerTestTable(testTable: TestTable): Unit = { + testTables += (testTable.name -> testTable) + } // The test tables that are defined in the Hive QTestUtil. // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -251,12 +265,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |WITH SERDEPROPERTIES ('field.delim'='\\t') """.stripMargin.cmd, "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("sales", - s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) - |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java index efd34df293c88..f33210ebdae1b 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java @@ -17,10 +17,10 @@ package org.apache.spark.sql.hive.execution; -import org.apache.hadoop.hive.ql.exec.UDF; - import java.util.List; -import org.apache.commons.lang.StringUtils; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.ql.exec.UDF; public class UDFListString extends UDF { diff --git a/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 new file mode 100644 index 0000000000000..f6ba75da254ca --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 @@ -0,0 +1,3 @@ +5 +5 +5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 new file mode 100644 index 0000000000000..ca7b591095e28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 @@ -0,0 +1,4 @@ +val_4 +val_5 +val_5 +val_5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 new file mode 100644 index 0000000000000..b8626c4cff284 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 @@ -0,0 +1 @@ +4 diff --git a/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 b/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/Specify the udtf output-0-d1f244bce64f22b34ad5bf9fd360b632 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-0-7ac701cf43e73e9e416888e4df694348 b/sql/hive/src/test/resources/golden/insert table with generator with column name-0-7ac701cf43e73e9e416888e4df694348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-1-5cdf9d51fc0e105e365d82e7611e37f3 b/sql/hive/src/test/resources/golden/insert table with generator with column name-1-5cdf9d51fc0e105e365d82e7611e37f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 b/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator with column name-2-f963396461294e06cb7cafe22a1419e4 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-0-46bdb27b3359dc81d8c246b9f69d4b82 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-0-46bdb27b3359dc81d8c246b9f69d4b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-1-cdf6989f3b055257f1692c3bbd80dc73 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-1-cdf6989f3b055257f1692c3bbd80dc73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 new file mode 100644 index 0000000000000..0c7520f2090dd --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator with multiple column names-2-ab3954b69d7a991bc801a509c3166cc5 @@ -0,0 +1,3 @@ +86 val_86 +238 val_238 +311 val_311 diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-0-7ac701cf43e73e9e416888e4df694348 b/sql/hive/src/test/resources/golden/insert table with generator without column name-0-7ac701cf43e73e9e416888e4df694348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-1-26599718c322ff4f9740040c066d8292 b/sql/hive/src/test/resources/golden/insert table with generator without column name-1-26599718c322ff4f9740040c066d8292 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 b/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert table with generator without column name-2-f963396461294e06cb7cafe22a1419e4 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea +++ b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 +++ b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 new file mode 100644 index 0000000000000..c6f275a0db131 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 @@ -0,0 +1 @@ +0.0 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 1. testWindowing-0-327a8cd39fe30255ff492ee86f660522 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 10. testHavingWithWindowingCondRankNoGBY-0-fef4bf638d52a9a601845347010602fd @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 new file mode 100644 index 0000000000000..921679cdcf569 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 11. testFirstLast-0-86bb9c97d92fdcd941bcb5143513e2e6 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 34 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 6 +Manufacturer#1 almond antique chartreuse lavender yellow 34 34 2 28 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6 2 42 +Manufacturer#1 almond aquamarine burnished black steel 28 28 34 42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 42 6 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 14 2 +Manufacturer#2 almond antique violet turquoise frosted 40 40 14 25 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 14 18 +Manufacturer#2 almond aquamarine rose maroon antique 25 25 40 18 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 18 2 18 +Manufacturer#3 almond antique chartreuse khaki white 17 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 19 17 45 +Manufacturer#3 almond antique misty red olive 1 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 45 19 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 10 10 27 +Manufacturer#4 almond antique violet mint lemon 39 39 10 7 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 10 12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 39 12 +Manufacturer#4 almond azure aquamarine papaya violet 12 12 27 12 +Manufacturer#5 almond antique blue firebrick mint 31 31 31 2 +Manufacturer#5 almond antique medium spring khaki 6 6 31 46 +Manufacturer#5 almond antique sky peru orange 2 2 31 23 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 6 23 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 2 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 new file mode 100644 index 0000000000000..09e30c7c57349 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 12. testFirstLastWithWhere-0-84345a9f685ba63b87caa4bb16b122b5 @@ -0,0 +1,5 @@ +Manufacturer#3 almond antique chartreuse khaki white 17 1 17 17 19 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 14 17 1 +Manufacturer#3 almond antique metallic orange dim 19 3 19 17 45 +Manufacturer#3 almond antique misty red olive 1 4 1 14 45 +Manufacturer#3 almond antique olive coral navajo 45 5 45 19 45 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f new file mode 100644 index 0000000000000..01ee88ff23302 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 13. testSumWindow-0-6cfc8840d3a4469b0fe11d63182cb59f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 2 +Manufacturer#1 almond antique burnished rose metallic 2 44 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 34 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 6 +Manufacturer#1 almond aquamarine burnished black steel 28 110 28 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 40 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 2 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 25 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 18 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 14 +Manufacturer#3 almond antique metallic orange dim 19 96 19 +Manufacturer#3 almond antique misty red olive 1 79 1 +Manufacturer#3 almond antique olive coral navajo 45 65 45 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 39 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 27 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 7 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 12 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 6 +Manufacturer#5 almond antique sky peru orange 2 108 2 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 46 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 new file mode 100644 index 0000000000000..c78eb640c9c27 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 14. testNoSortClause-0-88d96a526d3cae6ed8168c5b228974d1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 +Manufacturer#3 almond antique metallic orange dim 19 3 3 +Manufacturer#3 almond antique misty red olive 1 4 4 +Manufacturer#3 almond antique olive coral navajo 45 5 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 +Manufacturer#4 almond antique violet mint lemon 39 2 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 +Manufacturer#5 almond antique medium spring khaki 6 2 2 +Manufacturer#5 almond antique sky peru orange 2 3 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e new file mode 100644 index 0000000000000..050138ccf04ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 15. testExpressions-0-11f6c13cf2710ce7054654cca136e73e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 0.0 1 2 2.0 0.0 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 0.4 2 3 12.666666666666666 15.084944665313014 2 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 0.6 2 4 11.0 13.379088160259652 2 6 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 0.8 3 5 14.4 13.763720427268202 2 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 1.0 3 6 19.0 16.237815945091466 2 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 0.0 1 1 14.0 0.0 4 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 0.25 1 2 27.0 13.0 4 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 0.5 2 3 18.666666666666668 15.86050300449376 4 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 0.75 2 4 20.25 14.00669482783144 4 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 1.0 3 5 19.8 12.560254774486067 4 18 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 0.0 1 1 17.0 0.0 2 17 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 0.25 1 2 15.5 1.5 2 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 0.5 2 3 16.666666666666668 2.0548046676563256 2 19 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 0.75 2 4 12.75 7.013380069552769 2 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 1.0 3 5 19.2 14.344336861632886 2 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 0.0 1 1 10.0 0.0 0 10 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 0.25 1 2 24.5 14.5 0 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 0.5 2 3 25.333333333333332 11.897712198383164 0 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 0.75 2 4 20.75 13.007209539328564 0 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 1.0 3 5 19.0 12.149074038789951 0 12 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 0.0 1 1 31.0 0.0 1 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 0.25 1 2 18.5 12.5 1 6 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 0.5 2 3 13.0 12.832251036613439 1 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 0.75 2 4 21.25 18.102140757380052 1 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 1.0 3 5 21.6 16.206171663906314 1 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a new file mode 100644 index 0000000000000..c10888852b504 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 16. testMultipleWindows-0-efd1476255eeb1b1961149144f574b7a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 0.3333333333333333 4 4 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 0.5 38 34 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 0.6666666666666666 44 10 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 0.8333333333333334 72 28 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 1.0 114 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 0.2 14 14 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 0.4 54 40 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 0.6 56 2 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 0.8 81 25 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 1.0 99 32 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 0.2 17 31 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 0.4 31 14 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 0.6 50 50 17 +Manufacturer#3 almond antique misty red olive 1 4 4 0.8 51 1 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 1.0 96 45 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 0.2 10 17 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 0.4 49 39 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 0.6 76 27 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 0.8 83 7 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 1.0 95 29 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 0.2 31 31 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 0.4 37 8 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 0.6 39 2 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 0.8 85 46 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 1.0 108 23 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d new file mode 100644 index 0000000000000..b1309a497d68e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 17. testCountStar-0-1b1fc185c8fddf68e58e92f29052ab2d @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique burnished rose metallic 2 2 2 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 3 2 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 4 2 +Manufacturer#1 almond aquamarine burnished black steel 28 5 5 34 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 6 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 14 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 14 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 14 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 40 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 2 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 17 +Manufacturer#3 almond antique metallic orange dim 19 3 3 17 +Manufacturer#3 almond antique misty red olive 1 4 4 14 +Manufacturer#3 almond antique olive coral navajo 45 5 5 19 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 10 +Manufacturer#4 almond antique violet mint lemon 39 2 2 10 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 10 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 39 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 27 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 31 +Manufacturer#5 almond antique medium spring khaki 6 2 2 31 +Manufacturer#5 almond antique sky peru orange 2 3 3 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a new file mode 100644 index 0000000000000..52d2ee8d0cd3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 18. testUDAFs-0-6974e5959e41a661e09db18547fef58a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4100.06 1173.15 1753.76 1366.6866666666667 +Manufacturer#1 almond antique burnished rose metallic 2 5702.650000000001 1173.15 1753.76 1425.6625000000001 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 1173.15 1753.76 1423.4140000000002 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 7576.58 1173.15 1753.76 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 6403.43 1414.42 1753.76 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 4649.67 1414.42 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 5523.360000000001 1690.68 2031.98 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 7222.02 1690.68 2031.98 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 1690.68 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 7232.9400000000005 1698.66 2031.98 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5432.24 1698.66 2031.98 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 4272.34 1190.27 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 6195.32 1190.27 1922.98 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 7532.61 1190.27 1922.98 1506.522 +Manufacturer#3 almond antique misty red olive 1 5860.929999999999 1190.27 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 4670.66 1337.29 1922.98 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 4202.35 1206.26 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 6047.27 1206.26 1844.92 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 7337.620000000001 1206.26 1844.92 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 5716.950000000001 1206.26 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 4341.530000000001 1206.26 1844.92 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 5190.08 1611.66 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 6208.18 1018.1 1789.69 1552.045 +Manufacturer#5 almond antique sky peru orange 2 7672.66 1018.1 1789.69 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 5882.970000000001 1018.1 1788.73 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4271.3099999999995 1018.1 1788.73 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 new file mode 100644 index 0000000000000..6461642d34a21 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 19. testUDAFsWithGBY-0-67d15ee5915ac64a738fd4b60d75eb35 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 4529.5 1173.15 1173.15 1509.8333333333333 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 5943.92 1753.76 1753.76 1485.98 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 7576.58 1602.59 1602.59 1515.316 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 6403.43 1414.42 1414.42 1600.8575 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 4649.67 1632.66 1632.66 1549.89 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 5523.360000000001 1690.68 1690.68 1841.1200000000001 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 7222.02 1800.7 1800.7 1805.505 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 8923.62 2031.98 2031.98 1784.7240000000002 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 7232.9400000000005 1698.66 1698.66 1808.2350000000001 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5432.24 1701.6 1701.6 1810.7466666666667 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 4272.34 1671.68 1671.68 1424.1133333333335 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 6195.32 1190.27 1190.27 1548.83 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 7532.61 1410.39 1410.39 1506.522 +Manufacturer#3 almond antique misty red olive 1 1922.98 5860.929999999999 1922.98 1922.98 1465.2324999999998 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 4670.66 1337.29 1337.29 1556.8866666666665 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 4202.35 1620.67 1620.67 1400.7833333333335 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 6047.27 1375.42 1375.42 1511.8175 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 7337.620000000001 1206.26 1206.26 1467.5240000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 5716.950000000001 1844.92 1844.92 1429.2375000000002 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 4341.530000000001 1290.35 1290.35 1447.176666666667 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 5190.08 1789.69 1789.69 1730.0266666666666 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 6208.18 1611.66 1611.66 1552.045 +Manufacturer#5 almond antique sky peru orange 2 1788.73 7672.66 1788.73 1788.73 1534.532 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 5882.970000000001 1018.1 1018.1 1470.7425000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 4271.3099999999995 1464.48 1464.48 1423.7699999999998 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 2. testGroupByWithPartitioning-0-cb5618b1e626f3a9d4a030b508b5d251 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 new file mode 100644 index 0000000000000..1f7e8a5d67036 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995 +Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007 +Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502 +Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658 +Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995 +Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995 +Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676 +Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446 +Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003 +Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664 +Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993 +Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111 +Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779 +Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125 +Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995 +Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975 +Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881 +Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353 +Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999 +Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002 +Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad new file mode 100644 index 0000000000000..e7c39f454fb37 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1753.76,"y":1.0}] 121152.0 1 +Manufacturer#1 almond antique burnished rose metallic 2 [{"x":1173.15,"y":2.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 115872.0 2 +Manufacturer#1 almond antique chartreuse lavender yellow 34 [{"x":1173.15,"y":2.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1753.76,"y":1.0}] 110592.0 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 [{"x":1173.15,"y":1.0},{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86428.0 4 +Manufacturer#1 almond aquamarine burnished black steel 28 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0},{"x":1753.76,"y":1.0}] 86098.0 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 [{"x":1414.42,"y":1.0},{"x":1602.59,"y":1.0},{"x":1632.66,"y":1.0}] 86428.0 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 [{"x":1690.68,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 1 +Manufacturer#2 almond antique violet turquoise frosted 40 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 139825.5 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 [{"x":1690.68,"y":1.0},{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":1800.7,"y":1.0},{"x":2031.98,"y":1.0}] 169347.0 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 [{"x":1698.66,"y":1.0},{"x":1701.6,"y":1.0},{"x":2031.98,"y":1.0}] 146985.0 5 +Manufacturer#3 almond antique chartreuse khaki white 17 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0}] 90681.0 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 [{"x":1190.27,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 65831.5 2 +Manufacturer#3 almond antique metallic orange dim 19 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1671.68,"y":1.0},{"x":1922.98,"y":1.0}] 90681.0 3 +Manufacturer#3 almond antique misty red olive 1 [{"x":1190.27,"y":1.0},{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 76690.0 4 +Manufacturer#3 almond antique olive coral navajo 45 [{"x":1337.29,"y":1.0},{"x":1410.39,"y":1.0},{"x":1922.98,"y":1.0}] 112398.0 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0}] 48427.0 1 +Manufacturer#4 almond antique violet mint lemon 39 [{"x":1206.26,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 46844.0 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1620.67,"y":1.0},{"x":1844.92,"y":1.0}] 45261.0 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1375.42,"y":1.0},{"x":1844.92,"y":1.0}] 39309.0 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 [{"x":1206.26,"y":1.0},{"x":1290.35,"y":1.0},{"x":1844.92,"y":1.0}] 33357.0 5 +Manufacturer#5 almond antique blue firebrick mint 31 [{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 155733.0 1 +Manufacturer#5 almond antique medium spring khaki 6 [{"x":1018.1,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 99201.0 2 +Manufacturer#5 almond antique sky peru orange 2 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0},{"x":1789.69,"y":1.0}] 78486.0 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1611.66,"y":1.0},{"x":1788.73,"y":1.0}] 60577.5 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 [{"x":1018.1,"y":1.0},{"x":1464.48,"y":1.0},{"x":1788.73,"y":1.0}] 78486.0 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a new file mode 100644 index 0000000000000..dc83c9fffe932 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 24. testLateralViews-0-dea06072f0a64fe4537fae854944ed5a @@ -0,0 +1,78 @@ +Manufacturer#1 almond antique burnished rose metallic 1 2 2 +Manufacturer#1 almond antique burnished rose metallic 1 2 4 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 2 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique burnished rose metallic 3 2 6 +Manufacturer#1 almond antique salmon chartreuse burlywood 1 6 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 2 6 14 +Manufacturer#1 almond antique salmon chartreuse burlywood 3 6 18 +Manufacturer#1 almond aquamarine burnished black steel 1 28 40 +Manufacturer#1 almond aquamarine burnished black steel 2 28 62 +Manufacturer#1 almond aquamarine burnished black steel 3 28 84 +Manufacturer#1 almond antique chartreuse lavender yellow 1 34 90 +Manufacturer#1 almond antique chartreuse lavender yellow 2 34 96 +Manufacturer#1 almond antique chartreuse lavender yellow 3 34 102 +Manufacturer#1 almond aquamarine pink moccasin thistle 1 42 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 2 42 118 +Manufacturer#1 almond aquamarine pink moccasin thistle 3 42 126 +Manufacturer#2 almond aquamarine midnight light salmon 1 2 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 4 +Manufacturer#2 almond aquamarine midnight light salmon 3 2 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 14 18 +Manufacturer#2 almond antique violet chocolate turquoise 2 14 30 +Manufacturer#2 almond antique violet chocolate turquoise 3 14 42 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 1 18 46 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 2 18 50 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 3 18 54 +Manufacturer#2 almond aquamarine rose maroon antique 1 25 61 +Manufacturer#2 almond aquamarine rose maroon antique 2 25 68 +Manufacturer#2 almond aquamarine rose maroon antique 3 25 75 +Manufacturer#2 almond antique violet turquoise frosted 1 40 90 +Manufacturer#2 almond antique violet turquoise frosted 2 40 105 +Manufacturer#2 almond antique violet turquoise frosted 3 40 120 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique misty red olive 2 1 2 +Manufacturer#3 almond antique misty red olive 3 1 3 +Manufacturer#3 almond antique forest lavender goldenrod 1 14 16 +Manufacturer#3 almond antique forest lavender goldenrod 2 14 29 +Manufacturer#3 almond antique forest lavender goldenrod 3 14 42 +Manufacturer#3 almond antique chartreuse khaki white 1 17 45 +Manufacturer#3 almond antique chartreuse khaki white 2 17 48 +Manufacturer#3 almond antique chartreuse khaki white 3 17 51 +Manufacturer#3 almond antique metallic orange dim 1 19 53 +Manufacturer#3 almond antique metallic orange dim 2 19 55 +Manufacturer#3 almond antique metallic orange dim 3 19 57 +Manufacturer#3 almond antique olive coral navajo 1 45 83 +Manufacturer#3 almond antique olive coral navajo 2 45 109 +Manufacturer#3 almond antique olive coral navajo 3 45 135 +Manufacturer#4 almond aquamarine yellow dodger mint 1 7 7 +Manufacturer#4 almond aquamarine yellow dodger mint 2 7 14 +Manufacturer#4 almond aquamarine yellow dodger mint 3 7 21 +Manufacturer#4 almond antique gainsboro frosted violet 1 10 24 +Manufacturer#4 almond antique gainsboro frosted violet 2 10 27 +Manufacturer#4 almond antique gainsboro frosted violet 3 10 30 +Manufacturer#4 almond azure aquamarine papaya violet 1 12 32 +Manufacturer#4 almond azure aquamarine papaya violet 2 12 34 +Manufacturer#4 almond azure aquamarine papaya violet 3 12 36 +Manufacturer#4 almond aquamarine floral ivory bisque 1 27 51 +Manufacturer#4 almond aquamarine floral ivory bisque 2 27 66 +Manufacturer#4 almond aquamarine floral ivory bisque 3 27 81 +Manufacturer#4 almond antique violet mint lemon 1 39 93 +Manufacturer#4 almond antique violet mint lemon 2 39 105 +Manufacturer#4 almond antique violet mint lemon 3 39 117 +Manufacturer#5 almond antique sky peru orange 1 2 2 +Manufacturer#5 almond antique sky peru orange 2 2 4 +Manufacturer#5 almond antique sky peru orange 3 2 6 +Manufacturer#5 almond antique medium spring khaki 1 6 10 +Manufacturer#5 almond antique medium spring khaki 2 6 14 +Manufacturer#5 almond antique medium spring khaki 3 6 18 +Manufacturer#5 almond azure blanched chiffon midnight 1 23 35 +Manufacturer#5 almond azure blanched chiffon midnight 2 23 52 +Manufacturer#5 almond azure blanched chiffon midnight 3 23 69 +Manufacturer#5 almond antique blue firebrick mint 1 31 77 +Manufacturer#5 almond antique blue firebrick mint 2 31 85 +Manufacturer#5 almond antique blue firebrick mint 3 31 93 +Manufacturer#5 almond aquamarine dodger light gainsboro 1 46 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 2 46 123 +Manufacturer#5 almond aquamarine dodger light gainsboro 3 46 138 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 26. testGroupByHavingWithSWQAndAlias-0-b996a664b06e5741c08079d5c38241bc @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 new file mode 100644 index 0000000000000..b2a91ba727a72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 27. testMultipleRangeWindows-0-227e080e337d734dd88ff814b3b412e4 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique burnished rose metallic 2 4 10 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 10 6 +Manufacturer#1 almond aquamarine burnished black steel 28 28 62 +Manufacturer#1 almond antique chartreuse lavender yellow 34 62 76 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 42 +Manufacturer#2 almond aquamarine midnight light salmon 2 2 2 +Manufacturer#2 almond antique violet chocolate turquoise 14 14 32 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 32 43 +Manufacturer#2 almond aquamarine rose maroon antique 25 43 25 +Manufacturer#2 almond antique violet turquoise frosted 40 40 40 +Manufacturer#3 almond antique misty red olive 1 1 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 14 50 +Manufacturer#3 almond antique chartreuse khaki white 17 31 36 +Manufacturer#3 almond antique metallic orange dim 19 50 19 +Manufacturer#3 almond antique olive coral navajo 45 45 45 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7 29 +Manufacturer#4 almond antique gainsboro frosted violet 10 17 22 +Manufacturer#4 almond azure aquamarine papaya violet 12 29 12 +Manufacturer#4 almond aquamarine floral ivory bisque 27 27 27 +Manufacturer#4 almond antique violet mint lemon 39 39 39 +Manufacturer#5 almond antique sky peru orange 2 2 8 +Manufacturer#5 almond antique medium spring khaki 6 8 6 +Manufacturer#5 almond azure blanched chiffon midnight 23 23 54 +Manufacturer#5 almond antique blue firebrick mint 31 54 31 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 46 46 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 28. testPartOrderInUDAFInvoke-0-25912ae7d18c91cc09e17e57968fb5db @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a new file mode 100644 index 0000000000000..5bcb0fa941d65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 29. testPartOrderInWdwDef-0-88945892370ccbc1125a927a3d55342a @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 +Manufacturer#1 almond antique burnished rose metallic 2 44 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 +Manufacturer#1 almond aquamarine burnished black steel 28 110 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 +Manufacturer#2 almond antique violet turquoise frosted 40 81 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 +Manufacturer#3 almond antique chartreuse khaki white 17 50 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 +Manufacturer#3 almond antique metallic orange dim 19 96 +Manufacturer#3 almond antique misty red olive 1 79 +Manufacturer#3 almond antique olive coral navajo 45 65 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 +Manufacturer#4 almond antique violet mint lemon 39 83 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 +Manufacturer#5 almond antique blue firebrick mint 31 39 +Manufacturer#5 almond antique medium spring khaki 6 85 +Manufacturer#5 almond antique sky peru orange 2 108 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 new file mode 100644 index 0000000000000..2c30e652aa26d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 3. testGroupByHavingWithSWQ-0-a5a5339330a6a6660d32ccb0cc5d7100 @@ -0,0 +1,25 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1 1 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 2 2 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 3 3 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 4 4 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 5 5 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1 1 14 0 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 2 2 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 3 3 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 4 4 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 5 5 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1 1 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 2 2 14 -3 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 3 3 19 5 +Manufacturer#3 almond antique misty red olive 1 1922.98 4 4 1 -18 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 5 5 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1 1 10 0 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 2 2 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 3 3 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 4 4 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 5 5 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1 1 31 0 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 2 2 6 -25 +Manufacturer#5 almond antique sky peru orange 2 1788.73 3 3 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 4 4 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 5 5 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada new file mode 100644 index 0000000000000..698a44349d2a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 30. testDefaultPartitioningSpecRules-0-fa80b09c99e3c1487de48ea71a88dada @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 38 4 +Manufacturer#1 almond antique burnished rose metallic 2 44 4 +Manufacturer#1 almond antique chartreuse lavender yellow 34 72 38 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 112 44 +Manufacturer#1 almond aquamarine burnished black steel 28 110 72 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 76 114 +Manufacturer#2 almond antique violet chocolate turquoise 14 56 14 +Manufacturer#2 almond antique violet turquoise frosted 40 81 54 +Manufacturer#2 almond aquamarine midnight light salmon 2 99 56 +Manufacturer#2 almond aquamarine rose maroon antique 25 85 81 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 45 99 +Manufacturer#3 almond antique chartreuse khaki white 17 50 17 +Manufacturer#3 almond antique forest lavender goldenrod 14 51 31 +Manufacturer#3 almond antique metallic orange dim 19 96 50 +Manufacturer#3 almond antique misty red olive 1 79 51 +Manufacturer#3 almond antique olive coral navajo 45 65 96 +Manufacturer#4 almond antique gainsboro frosted violet 10 76 10 +Manufacturer#4 almond antique violet mint lemon 39 83 49 +Manufacturer#4 almond aquamarine floral ivory bisque 27 95 76 +Manufacturer#4 almond aquamarine yellow dodger mint 7 85 83 +Manufacturer#4 almond azure aquamarine papaya violet 12 46 95 +Manufacturer#5 almond antique blue firebrick mint 31 39 31 +Manufacturer#5 almond antique medium spring khaki 6 85 37 +Manufacturer#5 almond antique sky peru orange 2 108 39 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 77 85 +Manufacturer#5 almond azure blanched chiffon midnight 23 71 108 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 36. testRankWithPartitioning-0-45ccbaf0ee083858f7661c66b11d4768 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a new file mode 100644 index 0000000000000..9c0ca6c7a00ba --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 37. testPartitioningVariousForms-0-3436e50214f9afdec84334e10faa931a @@ -0,0 +1,26 @@ +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#1 8749.73 1173.15 1753.76 1458.29 6 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#2 8923.62 1690.68 2031.98 1784.72 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#3 7532.61 1190.27 1922.98 1506.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#4 7337.62 1206.26 1844.92 1467.52 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 +Manufacturer#5 7672.66 1018.1 1789.69 1534.53 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e new file mode 100644 index 0000000000000..fc27df2f2b648 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 38. testPartitioningVariousForms2-0-cba9d84a6b1bb5e36595338d4602377e @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 1173.15 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 1173.15 1173.15 +Manufacturer#1 almond antique chartreuse lavender yellow 34 1753.76 1753.76 1753.76 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 1602.59 1602.59 1602.59 +Manufacturer#1 almond aquamarine burnished black steel 28 1414.42 1414.42 1414.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 1632.66 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 1690.68 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 1800.7 1800.7 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 2031.98 2031.98 +Manufacturer#2 almond aquamarine rose maroon antique 25 1698.66 1698.66 1698.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 1701.6 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 1671.68 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 1190.27 1190.27 1190.27 +Manufacturer#3 almond antique metallic orange dim 19 1410.39 1410.39 1410.39 +Manufacturer#3 almond antique misty red olive 1 1922.98 1922.98 1922.98 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 1337.29 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 1620.67 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 1375.42 1375.42 +Manufacturer#4 almond aquamarine floral ivory bisque 27 1206.26 1206.26 1206.26 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 1844.92 1844.92 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 1290.35 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 1789.69 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 1611.66 1611.66 1611.66 +Manufacturer#5 almond antique sky peru orange 2 1788.73 1788.73 1788.73 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 1018.1 1018.1 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 1464.48 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 new file mode 100644 index 0000000000000..e5a541f56f6f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 39. testUDFOnOrderCols-0-7647562850dd367ef1e6c63117805423 @@ -0,0 +1,26 @@ +Manufacturer#1 LARGE BRUSHED STEEL ARGE BRUSHED STEEL 1 +Manufacturer#1 LARGE BURNISHED STEEL ARGE BURNISHED STEEL 2 +Manufacturer#1 PROMO BURNISHED NICKEL ROMO BURNISHED NICKEL 3 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 PROMO PLATED TIN ROMO PLATED TIN 4 +Manufacturer#1 STANDARD ANODIZED STEEL TANDARD ANODIZED STEEL 6 +Manufacturer#2 ECONOMY POLISHED STEEL CONOMY POLISHED STEEL 1 +Manufacturer#2 MEDIUM ANODIZED COPPER EDIUM ANODIZED COPPER 2 +Manufacturer#2 MEDIUM BURNISHED COPPER EDIUM BURNISHED COPPER 3 +Manufacturer#2 SMALL POLISHED NICKEL MALL POLISHED NICKEL 4 +Manufacturer#2 STANDARD PLATED TIN TANDARD PLATED TIN 5 +Manufacturer#3 ECONOMY PLATED COPPER CONOMY PLATED COPPER 1 +Manufacturer#3 MEDIUM BURNISHED BRASS EDIUM BURNISHED BRASS 2 +Manufacturer#3 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#3 PROMO ANODIZED TIN ROMO ANODIZED TIN 4 +Manufacturer#3 STANDARD POLISHED STEEL TANDARD POLISHED STEEL 5 +Manufacturer#4 ECONOMY BRUSHED COPPER CONOMY BRUSHED COPPER 1 +Manufacturer#4 SMALL BRUSHED BRASS MALL BRUSHED BRASS 2 +Manufacturer#4 SMALL PLATED STEEL MALL PLATED STEEL 3 +Manufacturer#4 PROMO POLISHED STEEL ROMO POLISHED STEEL 4 +Manufacturer#4 STANDARD ANODIZED TIN TANDARD ANODIZED TIN 5 +Manufacturer#5 LARGE BRUSHED BRASS ARGE BRUSHED BRASS 1 +Manufacturer#5 ECONOMY BURNISHED STEEL CONOMY BURNISHED STEEL 2 +Manufacturer#5 MEDIUM BURNISHED TIN EDIUM BURNISHED TIN 3 +Manufacturer#5 SMALL PLATED BRASS MALL PLATED BRASS 4 +Manufacturer#5 STANDARD BURNISHED TIN TANDARD BURNISHED TIN 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c new file mode 100644 index 0000000000000..bf8e620a304af --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 4. testCount-0-e6e97e884327df86f16b870527ec026c @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique chartreuse lavender yellow 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 +Manufacturer#1 almond aquamarine burnished black steel 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 +Manufacturer#2 almond antique violet chocolate turquoise 1 +Manufacturer#2 almond antique violet turquoise frosted 2 +Manufacturer#2 almond aquamarine midnight light salmon 3 +Manufacturer#2 almond aquamarine rose maroon antique 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 +Manufacturer#3 almond antique chartreuse khaki white 1 +Manufacturer#3 almond antique forest lavender goldenrod 2 +Manufacturer#3 almond antique metallic orange dim 3 +Manufacturer#3 almond antique misty red olive 4 +Manufacturer#3 almond antique olive coral navajo 5 +Manufacturer#4 almond antique gainsboro frosted violet 1 +Manufacturer#4 almond antique violet mint lemon 2 +Manufacturer#4 almond aquamarine floral ivory bisque 3 +Manufacturer#4 almond aquamarine yellow dodger mint 4 +Manufacturer#4 almond azure aquamarine papaya violet 5 +Manufacturer#5 almond antique blue firebrick mint 1 +Manufacturer#5 almond antique medium spring khaki 2 +Manufacturer#5 almond antique sky peru orange 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 +Manufacturer#5 almond azure blanched chiffon midnight 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 new file mode 100644 index 0000000000000..1e29df62901d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 40. testNoBetweenForRows-0-99007f45b6406869e048b0e4eb9213f1 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 4272.34 +Manufacturer#3 almond antique misty red olive 1 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 new file mode 100644 index 0000000000000..a620479fe4064 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 41. testNoBetweenForRange-0-d81a591e90950de291d2f133793e9283 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique burnished rose metallic 2 2346.3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 3948.8900000000003 +Manufacturer#1 almond aquamarine burnished black steel 28 5363.31 +Manufacturer#1 almond antique chartreuse lavender yellow 34 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 8749.730000000001 +Manufacturer#2 almond aquamarine midnight light salmon 2 2031.98 +Manufacturer#2 almond antique violet chocolate turquoise 14 3722.66 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5424.26 +Manufacturer#2 almond aquamarine rose maroon antique 25 7122.92 +Manufacturer#2 almond antique violet turquoise frosted 40 8923.62 +Manufacturer#3 almond antique misty red olive 1 1922.98 +Manufacturer#3 almond antique forest lavender goldenrod 14 3113.25 +Manufacturer#3 almond antique chartreuse khaki white 17 4784.93 +Manufacturer#3 almond antique metallic orange dim 19 6195.320000000001 +Manufacturer#3 almond antique olive coral navajo 45 7532.610000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 1844.92 +Manufacturer#4 almond antique gainsboro frosted violet 10 3465.59 +Manufacturer#4 almond azure aquamarine papaya violet 12 4755.9400000000005 +Manufacturer#4 almond aquamarine floral ivory bisque 27 5962.200000000001 +Manufacturer#4 almond antique violet mint lemon 39 7337.620000000001 +Manufacturer#5 almond antique sky peru orange 2 1788.73 +Manufacturer#5 almond antique medium spring khaki 6 3400.3900000000003 +Manufacturer#5 almond azure blanched chiffon midnight 23 4864.870000000001 +Manufacturer#5 almond antique blue firebrick mint 31 6654.560000000001 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 7672.660000000002 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 new file mode 100644 index 0000000000000..74147d2571a15 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 42. testUnboundedFollowingForRows-0-fb8648e82e4dd56d6bdcfd739dd1edf0 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 7576.58 +Manufacturer#1 almond antique chartreuse lavender yellow 34 6403.43 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4649.67 +Manufacturer#1 almond aquamarine burnished black steel 28 3047.08 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond antique violet chocolate turquoise 14 8923.62 +Manufacturer#2 almond antique violet turquoise frosted 40 7232.9400000000005 +Manufacturer#2 almond aquamarine midnight light salmon 2 5432.24 +Manufacturer#2 almond aquamarine rose maroon antique 25 3400.26 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 1701.6 +Manufacturer#3 almond antique chartreuse khaki white 17 7532.61 +Manufacturer#3 almond antique forest lavender goldenrod 14 5860.929999999999 +Manufacturer#3 almond antique metallic orange dim 19 4670.66 +Manufacturer#3 almond antique misty red olive 1 3260.27 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond antique gainsboro frosted violet 10 7337.620000000001 +Manufacturer#4 almond antique violet mint lemon 39 5716.950000000001 +Manufacturer#4 almond aquamarine floral ivory bisque 27 4341.530000000001 +Manufacturer#4 almond aquamarine yellow dodger mint 7 3135.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 1290.35 +Manufacturer#5 almond antique blue firebrick mint 31 7672.66 +Manufacturer#5 almond antique medium spring khaki 6 5882.970000000001 +Manufacturer#5 almond antique sky peru orange 2 4271.3099999999995 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 2482.58 +Manufacturer#5 almond azure blanched chiffon midnight 23 1464.48 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 new file mode 100644 index 0000000000000..49d003b5de139 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 43. testUnboundedFollowingForRange-0-3cd04e5f2398853c4850f4f86142bb39 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique burnished rose metallic 2 8749.730000000001 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 6403.43 +Manufacturer#1 almond aquamarine burnished black steel 28 4800.84 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3386.42 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 1632.66 +Manufacturer#2 almond aquamarine midnight light salmon 2 8923.62 +Manufacturer#2 almond antique violet chocolate turquoise 14 6891.639999999999 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5200.96 +Manufacturer#2 almond aquamarine rose maroon antique 25 3499.36 +Manufacturer#2 almond antique violet turquoise frosted 40 1800.7 +Manufacturer#3 almond antique misty red olive 1 7532.610000000001 +Manufacturer#3 almond antique forest lavender goldenrod 14 5609.63 +Manufacturer#3 almond antique chartreuse khaki white 17 4419.360000000001 +Manufacturer#3 almond antique metallic orange dim 19 2747.6800000000003 +Manufacturer#3 almond antique olive coral navajo 45 1337.29 +Manufacturer#4 almond aquamarine yellow dodger mint 7 7337.620000000001 +Manufacturer#4 almond antique gainsboro frosted violet 10 5492.7 +Manufacturer#4 almond azure aquamarine papaya violet 12 3872.0299999999997 +Manufacturer#4 almond aquamarine floral ivory bisque 27 2581.6800000000003 +Manufacturer#4 almond antique violet mint lemon 39 1375.42 +Manufacturer#5 almond antique sky peru orange 2 7672.660000000002 +Manufacturer#5 almond antique medium spring khaki 6 5883.93 +Manufacturer#5 almond azure blanched chiffon midnight 23 4272.27 +Manufacturer#5 almond antique blue firebrick mint 31 2807.79 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 1018.1 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae new file mode 100644 index 0000000000000..5982c9ee2a4d4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 44. testOverNoPartitionSingleAggregate-0-cb3d2f8c1296044dc2658876bb6103ae @@ -0,0 +1,26 @@ +almond antique blue firebrick mint 1789.69 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique burnished rose metallic 1173.15 1546.78 +almond antique chartreuse khaki white 1671.68 1546.78 +almond antique chartreuse lavender yellow 1753.76 1546.78 +almond antique forest lavender goldenrod 1190.27 1546.78 +almond antique gainsboro frosted violet 1620.67 1546.78 +almond antique medium spring khaki 1611.66 1546.78 +almond antique metallic orange dim 1410.39 1546.78 +almond antique misty red olive 1922.98 1546.78 +almond antique olive coral navajo 1337.29 1546.78 +almond antique salmon chartreuse burlywood 1602.59 1546.78 +almond antique sky peru orange 1788.73 1546.78 +almond antique violet chocolate turquoise 1690.68 1546.78 +almond antique violet mint lemon 1375.42 1546.78 +almond antique violet turquoise frosted 1800.7 1546.78 +almond aquamarine burnished black steel 1414.42 1546.78 +almond aquamarine dodger light gainsboro 1018.1 1546.78 +almond aquamarine floral ivory bisque 1206.26 1546.78 +almond aquamarine midnight light salmon 2031.98 1546.78 +almond aquamarine pink moccasin thistle 1632.66 1546.78 +almond aquamarine rose maroon antique 1698.66 1546.78 +almond aquamarine sandy cyan gainsboro 1701.6 1546.78 +almond aquamarine yellow dodger mint 1844.92 1546.78 +almond azure aquamarine papaya violet 1290.35 1546.78 +almond azure blanched chiffon midnight 1464.48 1546.78 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f new file mode 100644 index 0000000000000..00d41fc0bcd9c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 5. testCountWithWindowingUDAF-0-3bde93728761b780a745c2ce0398aa0f @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 1173.15 2 0 +Manufacturer#1 almond antique burnished rose metallic 1 1 2 1173.15 2346.3 2 0 +Manufacturer#1 almond antique chartreuse lavender yellow 3 2 3 1753.76 4100.06 34 32 +Manufacturer#1 almond antique salmon chartreuse burlywood 4 3 4 1602.59 5702.650000000001 6 -28 +Manufacturer#1 almond aquamarine burnished black steel 5 4 5 1414.42 7117.070000000001 28 22 +Manufacturer#1 almond aquamarine pink moccasin thistle 6 5 6 1632.66 8749.730000000001 42 14 +Manufacturer#2 almond antique violet chocolate turquoise 1 1 1 1690.68 1690.68 14 0 +Manufacturer#2 almond antique violet turquoise frosted 2 2 2 1800.7 3491.38 40 26 +Manufacturer#2 almond aquamarine midnight light salmon 3 3 3 2031.98 5523.360000000001 2 -38 +Manufacturer#2 almond aquamarine rose maroon antique 4 4 4 1698.66 7222.02 25 23 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 5 5 5 1701.6 8923.62 18 -7 +Manufacturer#3 almond antique chartreuse khaki white 1 1 1 1671.68 1671.68 17 0 +Manufacturer#3 almond antique forest lavender goldenrod 2 2 2 1190.27 2861.95 14 -3 +Manufacturer#3 almond antique metallic orange dim 3 3 3 1410.39 4272.34 19 5 +Manufacturer#3 almond antique misty red olive 4 4 4 1922.98 6195.32 1 -18 +Manufacturer#3 almond antique olive coral navajo 5 5 5 1337.29 7532.61 45 44 +Manufacturer#4 almond antique gainsboro frosted violet 1 1 1 1620.67 1620.67 10 0 +Manufacturer#4 almond antique violet mint lemon 2 2 2 1375.42 2996.09 39 29 +Manufacturer#4 almond aquamarine floral ivory bisque 3 3 3 1206.26 4202.35 27 -12 +Manufacturer#4 almond aquamarine yellow dodger mint 4 4 4 1844.92 6047.27 7 -20 +Manufacturer#4 almond azure aquamarine papaya violet 5 5 5 1290.35 7337.620000000001 12 5 +Manufacturer#5 almond antique blue firebrick mint 1 1 1 1789.69 1789.69 31 0 +Manufacturer#5 almond antique medium spring khaki 2 2 2 1611.66 3401.3500000000004 6 -25 +Manufacturer#5 almond antique sky peru orange 3 3 3 1788.73 5190.08 2 -4 +Manufacturer#5 almond aquamarine dodger light gainsboro 4 4 4 1018.1 6208.18 46 44 +Manufacturer#5 almond azure blanched chiffon midnight 5 5 5 1464.48 7672.66 23 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce new file mode 100644 index 0000000000000..98c09e4fe15c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 6. testCountInSubQ-0-73d5274a21d4f4fd51d2a0f1d98516ce @@ -0,0 +1,26 @@ +1 1 2 1173.15 0 +1 1 2 2346.3 0 +3 2 3 4100.06 32 +4 3 4 5702.650000000001 -28 +5 4 5 7117.070000000001 22 +6 5 6 8749.730000000001 14 +1 1 1 1690.68 0 +2 2 2 3491.38 26 +3 3 3 5523.360000000001 -38 +4 4 4 7222.02 23 +5 5 5 8923.62 -7 +1 1 1 1671.68 0 +2 2 2 2861.95 -3 +3 3 3 4272.34 5 +4 4 4 6195.32 -18 +5 5 5 7532.61 44 +1 1 1 1620.67 0 +2 2 2 2996.09 29 +3 3 3 4202.35 -12 +4 4 4 6047.27 -20 +5 5 5 7337.620000000001 5 +1 1 1 1789.69 0 +2 2 2 3401.3500000000004 -25 +3 3 3 5190.08 -4 +4 4 4 6208.18 44 +5 5 5 7672.66 -23 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 new file mode 100644 index 0000000000000..e35257d98382e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 8. testMixedCaseAlias-0-4b1ad2515fb079012467e987f484a722 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique burnished rose metallic 2 1 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 +Manufacturer#1 almond aquamarine burnished black steel 28 5 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 +Manufacturer#2 almond antique violet turquoise frosted 40 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 +Manufacturer#3 almond antique chartreuse khaki white 17 1 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 +Manufacturer#3 almond antique metallic orange dim 19 3 +Manufacturer#3 almond antique misty red olive 1 4 +Manufacturer#3 almond antique olive coral navajo 45 5 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 +Manufacturer#4 almond antique violet mint lemon 39 2 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 +Manufacturer#5 almond antique blue firebrick mint 31 1 +Manufacturer#5 almond antique medium spring khaki 6 2 +Manufacturer#5 almond antique sky peru orange 2 3 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 9. testHavingWithWindowingNoGBY-0-70cdc0555a61ef08534a9ebebb95ebbf @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-3-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 new file mode 100644 index 0000000000000..850c41c8115d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-4-50d0c630159068b5b8ccdeb76493f1f7 @@ -0,0 +1,26 @@ +Manufacturer#1 almond antique burnished rose metallic 2 1 1 1173.15 +Manufacturer#1 almond antique burnished rose metallic 2 1 1 2346.3 +Manufacturer#1 almond antique chartreuse lavender yellow 34 3 2 4100.06 +Manufacturer#1 almond antique salmon chartreuse burlywood 6 4 3 5702.650000000001 +Manufacturer#1 almond aquamarine burnished black steel 28 5 4 7117.070000000001 +Manufacturer#1 almond aquamarine pink moccasin thistle 42 6 5 8749.730000000001 +Manufacturer#2 almond antique violet chocolate turquoise 14 1 1 1690.68 +Manufacturer#2 almond antique violet turquoise frosted 40 2 2 3491.38 +Manufacturer#2 almond aquamarine midnight light salmon 2 3 3 5523.360000000001 +Manufacturer#2 almond aquamarine rose maroon antique 25 4 4 7222.02 +Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 5 5 8923.62 +Manufacturer#3 almond antique chartreuse khaki white 17 1 1 1671.68 +Manufacturer#3 almond antique forest lavender goldenrod 14 2 2 2861.95 +Manufacturer#3 almond antique metallic orange dim 19 3 3 4272.34 +Manufacturer#3 almond antique misty red olive 1 4 4 6195.32 +Manufacturer#3 almond antique olive coral navajo 45 5 5 7532.61 +Manufacturer#4 almond antique gainsboro frosted violet 10 1 1 1620.67 +Manufacturer#4 almond antique violet mint lemon 39 2 2 2996.09 +Manufacturer#4 almond aquamarine floral ivory bisque 27 3 3 4202.35 +Manufacturer#4 almond aquamarine yellow dodger mint 7 4 4 6047.27 +Manufacturer#4 almond azure aquamarine papaya violet 12 5 5 7337.620000000001 +Manufacturer#5 almond antique blue firebrick mint 31 1 1 1789.69 +Manufacturer#5 almond antique medium spring khaki 6 2 2 3401.3500000000004 +Manufacturer#5 almond antique sky peru orange 2 3 3 5190.08 +Manufacturer#5 almond aquamarine dodger light gainsboro 46 4 4 6208.18 +Manufacturer#5 almond azure blanched chiffon midnight 23 5 5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_adjust_rowcontainer_sz-5-3f95cd6f4add7a2d0101fe3dd97e5082 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/windowing_columnPruning-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/windowing_columnPruning-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 b/sql/hive/src/test/resources/golden/windowing_columnPruning-2-374e39786feb745cd70f25be58bfa24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-3-9294b4a22bc396ff2accabd53c5da98b @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 new file mode 100644 index 0000000000000..1b5ae55383a4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-4-445cab062581c449ceffcb368cdf133 @@ -0,0 +1,26 @@ +2 1173.15 +2 2346.3 +34 4100.06 +6 5702.650000000001 +28 7117.070000000001 +42 8749.730000000001 +14 1690.68 +40 3491.38 +2 5523.360000000001 +25 7222.02 +18 8923.62 +17 1671.68 +14 2861.95 +19 4272.34 +1 6195.32 +45 7532.61 +10 1620.67 +39 2996.09 +27 4202.35 +7 6047.27 +12 7337.620000000001 +31 1789.69 +6 3401.3500000000004 +2 5190.08 +46 6208.18 +23 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f new file mode 100644 index 0000000000000..e426c725b0e36 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_columnPruning-5-89110070c761eafb992eb9315128b53f @@ -0,0 +1,26 @@ +Manufacturer#1 1173.15 +Manufacturer#1 2346.3 +Manufacturer#1 4100.06 +Manufacturer#1 5702.650000000001 +Manufacturer#1 7117.070000000001 +Manufacturer#1 8749.730000000001 +Manufacturer#2 1690.68 +Manufacturer#2 3491.38 +Manufacturer#2 5523.360000000001 +Manufacturer#2 7222.02 +Manufacturer#2 8923.62 +Manufacturer#3 1671.68 +Manufacturer#3 2861.95 +Manufacturer#3 4272.34 +Manufacturer#3 6195.32 +Manufacturer#3 7532.61 +Manufacturer#4 1620.67 +Manufacturer#4 2996.09 +Manufacturer#4 4202.35 +Manufacturer#4 6047.27 +Manufacturer#4 7337.620000000001 +Manufacturer#5 1789.69 +Manufacturer#5 3401.3500000000004 +Manufacturer#5 5190.08 +Manufacturer#5 6208.18 +Manufacturer#5 7672.66 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f new file mode 100644 index 0000000000000..acc4f3bc2a2d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 1-0-12a92d8800e0da8b515ba3eaf6a7fd0f @@ -0,0 +1,1049 @@ + 1 4294967354 + 2 8589934811 + 3 12884902227 +alice allen 1 4294967487 +alice allen 2 8589935012 +alice allen 3 12884902543 +alice brown 1 4294967355 +alice carson 1 4294967370 +alice davidson 1 4294967517 +alice falkner 1 4294967316 +alice garcia 1 4294967369 +alice hernandez 1 4294967314 +alice hernandez 2 8589934613 +alice johnson 1 4294967424 +alice king 1 4294967387 +alice king 2 8589934903 +alice king 3 12884902449 +alice laertes 1 4294967519 +alice laertes 2 8589935038 +alice miller 1 4294967324 +alice nixon 1 4294967484 +alice nixon 2 8589934894 +alice nixon 3 12884902307 +alice ovid 1 4294967412 +alice polk 1 4294967366 +alice quirinius 1 4294967505 +alice quirinius 2 8589935054 +alice robinson 1 4294967502 +alice robinson 2 8589934947 +alice steinbeck 1 4294967549 +alice steinbeck 2 8589934913 +alice steinbeck 3 12884902387 +alice underhill 1 4294967441 +alice van buren 1 4294967428 +alice xylophone 1 4294967519 +alice xylophone 2 8589934877 +alice xylophone 3 12884902240 +alice zipper 1 4294967380 +alice zipper 2 8589934919 +alice zipper 3 12884902439 +bob brown 1 4294967431 +bob brown 2 8589934853 +bob brown 3 12884902280 +bob carson 1 4294967408 +bob davidson 1 4294967435 +bob davidson 2 8589934939 +bob davidson 3 12884902293 +bob ellison 1 4294967530 +bob ellison 2 8589934966 +bob ellison 3 12884902328 +bob ellison 4 17179869672 +bob falkner 1 4294967464 +bob garcia 1 4294967435 +bob garcia 2 8589934804 +bob garcia 3 12884902148 +bob garcia 4 17179869587 +bob garcia 5 21474836905 +bob hernandez 1 4294967500 +bob ichabod 1 4294967424 +bob king 1 4294967443 +bob king 2 8589934740 +bob king 3 12884902279 +bob laertes 1 4294967472 +bob laertes 2 8589934852 +bob miller 1 4294967349 +bob ovid 1 4294967401 +bob ovid 2 8589934801 +bob ovid 3 12884902313 +bob ovid 4 17179869708 +bob polk 1 4294967337 +bob quirinius 1 4294967346 +bob steinbeck 1 4294967342 +bob van buren 1 4294967422 +bob white 1 4294967493 +bob white 2 8589934855 +bob xylophone 1 4294967407 +bob xylophone 2 8589934872 +bob young 1 4294967413 +bob zipper 1 4294967416 +bob zipper 2 8589934769 +bob zipper 3 12884902068 +calvin allen 1 4294967373 +calvin brown 1 4294967530 +calvin brown 2 8589934967 +calvin brown 3 12884902378 +calvin carson 1 4294967534 +calvin davidson 1 4294967437 +calvin davidson 2 8589934905 +calvin ellison 1 4294967480 +calvin falkner 1 4294967305 +calvin falkner 2 8589934723 +calvin falkner 3 12884902155 +calvin falkner 4 17179869455 +calvin falkner 5 21474836800 +calvin falkner 6 25769804250 +calvin garcia 1 4294967492 +calvin hernandez 1 4294967341 +calvin johnson 1 4294967546 +calvin laertes 1 4294967499 +calvin laertes 2 8589934930 +calvin nixon 1 4294967488 +calvin nixon 2 8589934788 +calvin nixon 3 12884902200 +calvin ovid 1 4294967343 +calvin ovid 2 8589934881 +calvin ovid 3 12884902210 +calvin ovid 4 17179869559 +calvin polk 1 4294967416 +calvin quirinius 1 4294967532 +calvin quirinius 2 8589935053 +calvin robinson 1 4294967326 +calvin steinbeck 1 4294967417 +calvin steinbeck 2 8589934891 +calvin steinbeck 3 12884902396 +calvin thompson 1 4294967346 +calvin thompson 2 8589934859 +calvin underhill 1 4294967478 +calvin van buren 1 4294967300 +calvin van buren 2 8589934808 +calvin white 1 4294967304 +calvin white 2 8589934848 +calvin xylophone 1 4294967299 +calvin xylophone 2 8589934675 +calvin xylophone 3 12884902133 +calvin young 1 4294967410 +calvin young 2 8589934752 +calvin zipper 1 4294967441 +calvin zipper 2 8589934960 +david allen 1 4294967381 +david allen 2 8589934752 +david brown 1 4294967544 +david brown 2 8589934870 +david davidson 1 4294967487 +david davidson 2 8589934952 +david davidson 3 12884902474 +david davidson 4 17179869819 +david ellison 1 4294967477 +david ellison 2 8589934963 +david ellison 3 12884902426 +david hernandez 1 4294967324 +david ichabod 1 4294967385 +david ichabod 2 8589934872 +david laertes 1 4294967385 +david nixon 1 4294967381 +david ovid 1 4294967396 +david ovid 2 8589934839 +david quirinius 1 4294967375 +david quirinius 2 8589934905 +david quirinius 3 12884902362 +david robinson 1 4294967465 +david robinson 2 8589934933 +david thompson 1 4294967361 +david underhill 1 4294967331 +david underhill 2 8589934715 +david underhill 3 12884902185 +david van buren 1 4294967438 +david van buren 2 8589934747 +david white 1 4294967428 +david xylophone 1 8589934898 +david xylophone 1 8589934898 +david xylophone 3 12884902378 +david young 1 4294967296 +david young 2 8589934601 +ethan allen 1 4294967351 +ethan brown 1 4294967477 +ethan brown 2 8589934897 +ethan brown 3 12884902217 +ethan brown 4 17179869548 +ethan brown 5 21474836951 +ethan brown 6 25769804375 +ethan carson 1 4294967352 +ethan ellison 1 4294967514 +ethan ellison 2 8589934923 +ethan falkner 1 4294967318 +ethan falkner 2 8589934779 +ethan garcia 1 4294967310 +ethan hernandez 1 4294967349 +ethan johnson 1 4294967357 +ethan king 1 4294967413 +ethan laertes 1 4294967402 +ethan laertes 2 8589934859 +ethan laertes 3 12884902390 +ethan laertes 4 17179869880 +ethan laertes 5 21474837302 +ethan laertes 6 25769804603 +ethan laertes 7 30064771974 +ethan miller 1 4294967352 +ethan nixon 1 4294967499 +ethan ovid 1 4294967452 +ethan polk 1 4294967329 +ethan polk 2 8589934711 +ethan polk 3 12884902253 +ethan polk 4 17179869732 +ethan quirinius 1 4294967501 +ethan quirinius 2 8589934852 +ethan quirinius 3 12884902200 +ethan robinson 1 4294967353 +ethan robinson 2 8589934855 +ethan underhill 1 4294967466 +ethan van buren 1 4294967511 +ethan white 1 4294967445 +ethan white 2 8589934872 +ethan xylophone 1 4294967543 +ethan zipper 1 4294967462 +ethan zipper 2 8589934815 +fred davidson 1 4294967512 +fred davidson 2 8589934936 +fred davidson 3 12884902424 +fred ellison 1 4294967470 +fred ellison 2 8589934901 +fred ellison 3 12884902294 +fred falkner 1 4294967340 +fred falkner 2 8589934887 +fred falkner 3 12884902187 +fred hernandez 1 4294967365 +fred ichabod 1 4294967342 +fred ichabod 2 8589934831 +fred johnson 1 4294967373 +fred king 1 4294967346 +fred king 2 8589934766 +fred laertes 1 4294967351 +fred miller 1 4294967490 +fred nixon 1 4294967514 +fred nixon 2 8589934811 +fred nixon 3 12884902293 +fred nixon 4 17179869668 +fred polk 1 4294967332 +fred polk 2 8589934775 +fred polk 3 12884902233 +fred polk 4 17179869740 +fred quirinius 1 4294967426 +fred quirinius 2 8589934951 +fred robinson 1 4294967461 +fred steinbeck 1 4294967411 +fred steinbeck 2 8589934740 +fred steinbeck 3 12884902212 +fred underhill 1 4294967387 +fred van buren 1 4294967431 +fred van buren 2 8589934812 +fred van buren 3 12884902338 +fred van buren 4 17179869801 +fred white 1 4294967434 +fred young 1 4294967495 +fred young 2 8589934980 +fred zipper 1 4294967447 +gabriella allen 1 4294967405 +gabriella allen 2 8589934939 +gabriella brown 1 4294967543 +gabriella brown 2 8589934946 +gabriella carson 1 4294967540 +gabriella davidson 1 4294967507 +gabriella ellison 1 4294967393 +gabriella ellison 2 8589934733 +gabriella falkner 1 4294967378 +gabriella falkner 2 8589934901 +gabriella falkner 3 12884902335 +gabriella garcia 1 4294967419 +gabriella hernandez 1 4294967481 +gabriella hernandez 2 8589934943 +gabriella ichabod 1 4294967337 +gabriella ichabod 2 8589934725 +gabriella ichabod 3 12884902062 +gabriella ichabod 4 17179869382 +gabriella ichabod 5 21474836880 +gabriella king 1 4294967434 +gabriella king 2 8589934827 +gabriella laertes 1 4294967410 +gabriella miller 1 4294967363 +gabriella ovid 1 4294967482 +gabriella ovid 2 8589935004 +gabriella polk 1 4294967410 +gabriella polk 2 8589934712 +gabriella steinbeck 1 4294967500 +gabriella steinbeck 2 8589934935 +gabriella thompson 1 4294967299 +gabriella thompson 2 8589934711 +gabriella thompson 3 12884902196 +gabriella van buren 1 4294967457 +gabriella van buren 2 8589934927 +gabriella white 1 4294967335 +gabriella young 1 4294967493 +gabriella young 2 8589934924 +gabriella zipper 1 4294967357 +gabriella zipper 2 8589934867 +holly allen 1 4294967327 +holly brown 1 4294967321 +holly brown 2 8589934659 +holly falkner 1 4294967324 +holly hernandez 1 4294967378 +holly hernandez 2 8589934921 +holly hernandez 3 12884902465 +holly hernandez 4 17179869773 +holly ichabod 1 4294967342 +holly ichabod 2 8589934800 +holly ichabod 3 12884902129 +holly johnson 1 4294967517 +holly johnson 2 8589934897 +holly johnson 3 12884902432 +holly king 1 4294967392 +holly king 2 8589934753 +holly laertes 1 4294967406 +holly miller 1 4294967388 +holly nixon 1 4294967383 +holly nixon 2 8589934707 +holly polk 1 4294967398 +holly polk 2 8589934832 +holly robinson 1 4294967532 +holly thompson 1 4294967529 +holly thompson 2 8589934868 +holly thompson 3 12884902242 +holly underhill 1 4294967383 +holly underhill 2 8589934894 +holly underhill 3 12884902330 +holly underhill 4 17179869856 +holly van buren 1 4294967539 +holly white 1 4294967320 +holly white 2 8589934735 +holly xylophone 1 4294967435 +holly young 1 4294967487 +holly young 2 8589934987 +holly zipper 1 4294967337 +holly zipper 2 8589934846 +irene allen 1 4294967518 +irene brown 1 4294967434 +irene brown 2 8589934862 +irene brown 3 12884902272 +irene carson 1 4294967473 +irene ellison 1 4294967379 +irene ellison 2 8589934797 +irene falkner 1 4294967404 +irene falkner 2 8589934952 +irene garcia 1 4294967498 +irene garcia 2 8589934869 +irene garcia 3 12884902192 +irene ichabod 1 4294967529 +irene ichabod 2 8589935038 +irene johnson 1 4294967468 +irene laertes 1 4294967481 +irene laertes 2 8589934780 +irene laertes 3 12884902116 +irene miller 1 4294967387 +irene nixon 1 4294967323 +irene nixon 2 8589934824 +irene nixon 3 12884902362 +irene ovid 1 4294967499 +irene ovid 2 8589934870 +irene ovid 3 12884902230 +irene polk 1 4294967521 +irene polk 2 8589934930 +irene polk 3 12884902395 +irene polk 4 17179869941 +irene polk 5 21474837237 +irene quirinius 1 4294967365 +irene quirinius 2 8589934751 +irene quirinius 3 12884902141 +irene robinson 1 4294967347 +irene steinbeck 1 4294967549 +irene thompson 1 4294967479 +irene underhill 1 4294967371 +irene underhill 2 8589934753 +irene van buren 1 4294967439 +irene van buren 2 8589934906 +irene xylophone 1 4294967436 +jessica brown 1 4294967496 +jessica carson 1 4294967389 +jessica carson 2 8589934897 +jessica carson 3 12884902345 +jessica davidson 1 4294967325 +jessica davidson 2 8589934709 +jessica davidson 3 12884902098 +jessica davidson 4 17179869569 +jessica ellison 1 4294967316 +jessica ellison 2 8589934721 +jessica falkner 1 4294967549 +jessica garcia 1 4294967540 +jessica garcia 2 8589935041 +jessica ichabod 1 4294967413 +jessica johnson 1 4294967497 +jessica johnson 2 8589934870 +jessica miller 1 4294967495 +jessica nixon 1 4294967311 +jessica nixon 2 8589934754 +jessica ovid 1 4294967330 +jessica ovid 2 8589934855 +jessica polk 1 4294967403 +jessica quirinius 1 4294967523 +jessica quirinius 2 8589934942 +jessica quirinius 3 12884902388 +jessica quirinius 4 17179869696 +jessica robinson 1 4294967542 +jessica thompson 1 4294967449 +jessica thompson 2 8589934763 +jessica underhill 1 4294967541 +jessica underhill 2 8589934844 +jessica underhill 3 12884902153 +jessica van buren 1 4294967344 +jessica white 1 4294967482 +jessica white 2 8589934929 +jessica white 3 12884902378 +jessica white 4 17179869687 +jessica white 5 21474837086 +jessica xylophone 1 4294967421 +jessica young 1 4294967382 +jessica young 2 8589934903 +jessica zipper 1 4294967334 +jessica zipper 2 8589934785 +jessica zipper 3 12884902157 +katie allen 1 4294967443 +katie brown 1 4294967420 +katie davidson 1 4294967459 +katie ellison 1 4294967486 +katie ellison 2 8589934861 +katie falkner 1 4294967362 +katie garcia 1 4294967306 +katie garcia 2 8589934680 +katie hernandez 1 4294967451 +katie ichabod 1 4294967330 +katie ichabod 2 8589934742 +katie ichabod 3 12884902209 +katie king 1 4294967339 +katie king 2 8589934760 +katie king 3 12884902199 +katie miller 1 4294967425 +katie miller 2 8589934859 +katie nixon 1 4294967500 +katie ovid 1 4294967519 +katie polk 1 4294967384 +katie polk 2 8589934926 +katie robinson 1 4294967310 +katie van buren 1 4294967335 +katie van buren 2 8589934647 +katie white 1 4294967337 +katie white 2 8589934643 +katie xylophone 1 4294967486 +katie young 1 4294967349 +katie young 2 8589934681 +katie young 3 12884902225 +katie zipper 1 4294967354 +katie zipper 2 8589934766 +luke allen 1 4294967533 +luke allen 2 8589934836 +luke allen 3 12884902346 +luke allen 4 17179869863 +luke allen 5 21474837208 +luke brown 1 4294967473 +luke davidson 1 4294967550 +luke davidson 2 8589934904 +luke ellison 1 4294967322 +luke ellison 2 8589934675 +luke ellison 3 12884902103 +luke falkner 1 4294967359 +luke falkner 2 8589934782 +luke garcia 1 4294967304 +luke garcia 2 8589934683 +luke ichabod 1 4294967324 +luke ichabod 2 8589934766 +luke johnson 1 4294967527 +luke johnson 2 8589934987 +luke johnson 3 12884902342 +luke laertes 1 4294967505 +luke laertes 2 8589935011 +luke laertes 3 12884902497 +luke laertes 4 17179869806 +luke laertes 5 21474837193 +luke miller 1 4294967497 +luke ovid 1 4294967492 +luke ovid 2 8589934901 +luke polk 1 4294967545 +luke polk 2 8589934873 +luke quirinius 1 4294967320 +luke robinson 1 4294967299 +luke robinson 2 8589934606 +luke thompson 1 4294967521 +luke underhill 1 4294967393 +luke underhill 2 8589934803 +luke underhill 3 12884902122 +luke van buren 1 4294967424 +luke white 1 4294967505 +luke xylophone 1 4294967382 +luke zipper 1 4294967353 +mike allen 1 4294967466 +mike brown 1 4294967369 +mike carson 1 4294967477 +mike carson 2 8589934934 +mike carson 3 12884902482 +mike davidson 1 4294967501 +mike davidson 2 8589934965 +mike ellison 1 4294967353 +mike ellison 2 8589934747 +mike ellison 3 12884902282 +mike ellison 4 17179869806 +mike ellison 5 21474837309 +mike falkner 1 4294967301 +mike garcia 1 4294967428 +mike garcia 2 8589934826 +mike garcia 3 12884902289 +mike hernandez 1 4294967316 +mike hernandez 2 8589934800 +mike ichabod 1 4294967494 +mike king 1 4294967323 +mike king 2 8589934848 +mike king 3 12884902248 +mike king 4 17179869595 +mike king 5 21474837046 +mike king 6 25769804478 +mike miller 1 4294967449 +mike nixon 1 4294967527 +mike nixon 2 8589935004 +mike polk 1 4294967389 +mike polk 2 8589934848 +mike polk 3 12884902351 +mike quirinius 1 4294967422 +mike steinbeck 1 4294967519 +mike steinbeck 2 8589934827 +mike steinbeck 3 12884902316 +mike steinbeck 4 17179869850 +mike van buren 1 4294967544 +mike van buren 2 8589935061 +mike white 1 4294967336 +mike white 2 8589934882 +mike white 3 12884902374 +mike white 4 17179869843 +mike young 1 4294967453 +mike young 2 8589934804 +mike young 3 12884902198 +mike zipper 1 4294967402 +mike zipper 2 8589934727 +mike zipper 3 12884902228 +nick allen 1 4294967507 +nick allen 2 8589934807 +nick brown 1 4294967334 +nick davidson 1 4294967357 +nick ellison 1 4294967397 +nick ellison 2 8589934699 +nick falkner 1 4294967480 +nick falkner 2 8589934923 +nick garcia 1 4294967384 +nick garcia 2 8589934797 +nick garcia 3 12884902319 +nick ichabod 1 4294967388 +nick ichabod 2 8589934758 +nick ichabod 3 12884902225 +nick johnson 1 4294967398 +nick johnson 2 8589934809 +nick laertes 1 4294967389 +nick miller 1 4294967550 +nick nixon 1 4294967482 +nick ovid 1 4294967488 +nick polk 1 4294967551 +nick quirinius 1 4294967316 +nick quirinius 2 8589934612 +nick robinson 1 4294967409 +nick robinson 2 8589934731 +nick steinbeck 1 4294967355 +nick thompson 1 4294967401 +nick underhill 1 4294967527 +nick van buren 1 4294967303 +nick xylophone 1 4294967460 +nick young 1 4294967405 +nick young 2 8589934917 +nick zipper 1 4294967430 +nick zipper 2 8589934796 +oscar allen 1 4294967500 +oscar brown 1 4294967331 +oscar carson 1 4294967460 +oscar carson 2 8589934904 +oscar carson 3 12884902286 +oscar carson 4 17179869599 +oscar carson 5 21474836960 +oscar davidson 1 4294967482 +oscar ellison 1 8589934740 +oscar ellison 1 8589934740 +oscar falkner 1 4294967526 +oscar garcia 1 4294967301 +oscar hernandez 1 4294967343 +oscar hernandez 2 8589934843 +oscar ichabod 1 4294967513 +oscar ichabod 2 8589934837 +oscar ichabod 3 12884902165 +oscar ichabod 4 17179869569 +oscar johnson 1 4294967418 +oscar johnson 2 8589934763 +oscar king 1 4294967465 +oscar king 2 8589934936 +oscar king 3 12884902469 +oscar laertes 1 4294967425 +oscar laertes 2 8589934876 +oscar laertes 3 12884902426 +oscar laertes 4 17179869786 +oscar nixon 1 4294967532 +oscar ovid 1 4294967508 +oscar ovid 2 8589934910 +oscar ovid 3 12884902418 +oscar polk 1 4294967325 +oscar polk 2 8589934713 +oscar quirinius 1 4294967416 +oscar quirinius 2 8589934932 +oscar quirinius 3 12884902390 +oscar quirinius 4 17179869763 +oscar robinson 1 4294967355 +oscar robinson 2 8589934681 +oscar robinson 3 12884902031 +oscar robinson 4 17179869383 +oscar steinbeck 1 4294967548 +oscar thompson 1 4294967453 +oscar thompson 2 8589934824 +oscar thompson 3 12884902197 +oscar thompson 4 17179869496 +oscar underhill 1 4294967374 +oscar van buren 1 4294967520 +oscar van buren 2 8589934990 +oscar van buren 3 12884902490 +oscar white 1 4294967454 +oscar white 2 8589934761 +oscar white 3 12884902163 +oscar white 4 17179869512 +oscar xylophone 1 4294967400 +oscar xylophone 2 8589934806 +oscar xylophone 3 12884902124 +oscar zipper 1 4294967449 +oscar zipper 2 8589934969 +oscar zipper 3 12884902458 +priscilla brown 1 4294967369 +priscilla brown 2 8589934897 +priscilla brown 3 12884902360 +priscilla carson 1 4294967489 +priscilla carson 2 8589934838 +priscilla carson 3 12884902270 +priscilla ichabod 1 4294967379 +priscilla ichabod 2 8589934926 +priscilla johnson 1 4294967535 +priscilla johnson 2 8589935003 +priscilla johnson 3 12884902308 +priscilla johnson 4 17179869707 +priscilla johnson 5 21474837167 +priscilla king 1 4294967385 +priscilla nixon 1 4294967388 +priscilla nixon 2 8589934849 +priscilla ovid 1 4294967528 +priscilla ovid 2 8589935035 +priscilla polk 1 4294967434 +priscilla quirinius 1 4294967347 +priscilla thompson 1 4294967497 +priscilla underhill 1 4294967520 +priscilla underhill 2 8589934853 +priscilla van buren 1 4294967318 +priscilla van buren 2 8589934809 +priscilla van buren 3 12884902351 +priscilla white 1 4294967419 +priscilla xylophone 1 4294967503 +priscilla xylophone 2 8589934956 +priscilla xylophone 3 12884902406 +priscilla young 1 4294967401 +priscilla young 2 8589934931 +priscilla zipper 1 4294967516 +priscilla zipper 2 8589934950 +quinn allen 1 4294967339 +quinn allen 2 8589934881 +quinn brown 1 4294967335 +quinn brown 2 8589934651 +quinn brown 3 12884902099 +quinn davidson 1 4294967478 +quinn davidson 2 8589934849 +quinn davidson 3 12884902238 +quinn davidson 4 17179869565 +quinn ellison 1 4294967392 +quinn ellison 2 8589934907 +quinn garcia 1 4294967344 +quinn garcia 2 8589934882 +quinn garcia 3 12884902395 +quinn garcia 4 17179869868 +quinn ichabod 1 4294967405 +quinn king 1 4294967538 +quinn king 2 8589934996 +quinn laertes 1 4294967533 +quinn laertes 2 8589934977 +quinn laertes 3 12884902524 +quinn nixon 1 4294967432 +quinn ovid 1 4294967340 +quinn quirinius 1 4294967347 +quinn robinson 1 4294967365 +quinn steinbeck 1 4294967358 +quinn steinbeck 2 8589934810 +quinn thompson 1 4294967488 +quinn thompson 2 8589934888 +quinn underhill 1 4294967307 +quinn underhill 2 8589934744 +quinn underhill 3 12884902278 +quinn van buren 1 4294967362 +quinn young 1 4294967392 +quinn zipper 1 4294967521 +quinn zipper 2 8589934944 +rachel allen 1 4294967334 +rachel allen 2 8589934713 +rachel brown 1 4294967451 +rachel brown 2 8589934886 +rachel brown 3 12884902325 +rachel brown 4 17179869632 +rachel brown 5 21474836938 +rachel carson 1 4294967461 +rachel carson 2 8589934777 +rachel davidson 1 4294967387 +rachel ellison 1 4294967423 +rachel falkner 1 4294967544 +rachel falkner 2 8589934892 +rachel falkner 3 12884902350 +rachel falkner 4 17179869809 +rachel johnson 1 4294967541 +rachel king 1 4294967442 +rachel king 2 8589934771 +rachel laertes 1 4294967446 +rachel laertes 2 8589934804 +rachel ovid 1 4294967481 +rachel ovid 2 8589934832 +rachel polk 1 4294967335 +rachel quirinius 1 4294967297 +rachel robinson 1 4294967344 +rachel robinson 2 8589934807 +rachel robinson 3 12884902135 +rachel thompson 1 4294967518 +rachel thompson 2 8589934881 +rachel thompson 3 12884902306 +rachel underhill 1 4294967382 +rachel white 1 4294967457 +rachel white 2 8589934793 +rachel young 1 4294967391 +rachel zipper 1 4294967434 +rachel zipper 2 8589934813 +sarah carson 1 4294967503 +sarah carson 2 8589934822 +sarah carson 3 12884902167 +sarah ellison 1 4294967542 +sarah falkner 1 4294967525 +sarah falkner 2 8589934974 +sarah garcia 1 4294967391 +sarah garcia 2 8589934849 +sarah garcia 3 12884902247 +sarah ichabod 1 4294967370 +sarah ichabod 2 8589934909 +sarah johnson 1 4294967433 +sarah johnson 2 8589934926 +sarah johnson 3 12884902235 +sarah johnson 4 17179869559 +sarah king 1 4294967496 +sarah king 2 8589935039 +sarah miller 1 4294967458 +sarah ovid 1 4294967350 +sarah robinson 1 4294967419 +sarah robinson 2 8589934917 +sarah steinbeck 1 4294967456 +sarah white 1 4294967514 +sarah white 2 8589934882 +sarah xylophone 1 4294967355 +sarah young 1 4294967442 +sarah zipper 1 4294967432 +tom brown 1 4294967432 +tom brown 2 8589934950 +tom carson 1 4294967388 +tom carson 2 8589934693 +tom carson 3 12884902227 +tom davidson 1 4294967507 +tom ellison 1 4294967487 +tom ellison 2 8589934844 +tom ellison 3 12884902188 +tom falkner 1 4294967382 +tom falkner 2 8589934837 +tom hernandez 1 8589934733 +tom hernandez 1 8589934733 +tom ichabod 1 4294967445 +tom johnson 1 4294967492 +tom johnson 2 8589934923 +tom king 1 4294967331 +tom laertes 1 4294967431 +tom laertes 2 8589934744 +tom miller 1 4294967366 +tom miller 2 8589934723 +tom miller 3 12884902078 +tom nixon 1 4294967506 +tom ovid 1 4294967512 +tom polk 1 4294967329 +tom polk 2 8589934869 +tom quirinius 1 4294967507 +tom quirinius 2 8589934823 +tom robinson 1 4294967457 +tom robinson 2 8589935008 +tom robinson 3 12884902462 +tom robinson 4 17179869770 +tom steinbeck 1 4294967447 +tom van buren 1 4294967374 +tom van buren 2 8589934703 +tom van buren 3 12884902195 +tom white 1 4294967413 +tom young 1 4294967539 +tom young 2 8589935074 +tom zipper 1 4294967526 +ulysses brown 1 4294967537 +ulysses carson 1 4294967323 +ulysses carson 2 8589934815 +ulysses carson 3 12884902127 +ulysses carson 4 17179869485 +ulysses davidson 1 4294967467 +ulysses ellison 1 4294967442 +ulysses garcia 1 4294967470 +ulysses hernandez 1 4294967449 +ulysses hernandez 2 8589934995 +ulysses hernandez 3 12884902393 +ulysses ichabod 1 4294967353 +ulysses ichabod 2 8589934728 +ulysses johnson 1 4294967432 +ulysses king 1 4294967537 +ulysses laertes 1 4294967391 +ulysses laertes 2 8589934938 +ulysses laertes 3 12884902431 +ulysses miller 1 4294967373 +ulysses miller 2 8589934808 +ulysses nixon 1 4294967296 +ulysses ovid 1 4294967394 +ulysses polk 1 4294967509 +ulysses polk 2 8589934960 +ulysses polk 3 12884902440 +ulysses polk 4 17179869745 +ulysses quirinius 1 4294967449 +ulysses robinson 1 4294967531 +ulysses steinbeck 1 4294967303 +ulysses steinbeck 2 8589934788 +ulysses thompson 1 4294967389 +ulysses underhill 1 4294967544 +ulysses underhill 2 8589934949 +ulysses underhill 3 12884902275 +ulysses underhill 4 17179869726 +ulysses underhill 5 21474837190 +ulysses underhill 6 25769804570 +ulysses underhill 7 30064771927 +ulysses van buren 1 4294967439 +ulysses white 1 4294967429 +ulysses white 2 8589934878 +ulysses xylophone 1 4294967524 +ulysses xylophone 2 8589935025 +ulysses xylophone 3 12884902473 +ulysses young 1 4294967427 +ulysses young 2 8589934763 +ulysses young 3 12884902154 +victor allen 1 4294967450 +victor allen 2 8589934776 +victor brown 1 4294967521 +victor brown 2 8589934864 +victor brown 3 12884902170 +victor brown 4 17179869625 +victor davidson 1 4294967419 +victor davidson 2 8589934720 +victor davidson 3 12884902156 +victor ellison 1 4294967362 +victor ellison 2 8589934831 +victor hernandez 1 4294967428 +victor hernandez 2 8589934733 +victor hernandez 3 12884902062 +victor hernandez 4 17179869402 +victor hernandez 5 21474836874 +victor johnson 1 4294967496 +victor johnson 2 8589934824 +victor johnson 3 12884902246 +victor king 1 4294967401 +victor king 2 8589934884 +victor laertes 1 4294967407 +victor laertes 2 8589934862 +victor miller 1 4294967410 +victor nixon 1 4294967424 +victor nixon 2 8589934803 +victor ovid 1 4294967355 +victor polk 1 4294967333 +victor quirinius 1 4294967520 +victor quirinius 2 8589934846 +victor robinson 1 4294967440 +victor robinson 2 8589934930 +victor steinbeck 1 4294967390 +victor steinbeck 2 8589934707 +victor steinbeck 3 12884902037 +victor thompson 1 4294967319 +victor van buren 1 4294967365 +victor van buren 2 8589934906 +victor white 1 4294967403 +victor white 2 8589934862 +victor xylophone 1 4294967331 +victor xylophone 2 8589934864 +victor xylophone 3 12884902262 +victor xylophone 4 17179869633 +victor xylophone 5 21474837062 +victor young 1 4294967337 +victor zipper 1 4294967428 +wendy allen 1 4294967473 +wendy allen 2 8589934989 +wendy allen 3 12884902367 +wendy brown 1 4294967337 +wendy brown 2 8589934817 +wendy ellison 1 4294967475 +wendy ellison 2 8589934989 +wendy falkner 1 4294967313 +wendy falkner 2 8589934810 +wendy falkner 3 12884902236 +wendy garcia 1 4294967394 +wendy garcia 2 8589934775 +wendy garcia 3 12884902088 +wendy garcia 4 17179869400 +wendy hernandez 1 4294967299 +wendy ichabod 1 4294967516 +wendy king 1 4294967420 +wendy king 2 8589934811 +wendy king 3 12884902252 +wendy laertes 1 4294967519 +wendy laertes 2 8589934939 +wendy laertes 3 12884902315 +wendy miller 1 4294967478 +wendy miller 2 8589934957 +wendy nixon 1 4294967407 +wendy nixon 2 8589934901 +wendy ovid 1 4294967464 +wendy ovid 2 8589934894 +wendy polk 1 4294967434 +wendy polk 2 8589934824 +wendy quirinius 1 4294967334 +wendy quirinius 2 8589934782 +wendy robinson 1 4294967302 +wendy robinson 2 8589934613 +wendy robinson 3 12884901977 +wendy steinbeck 1 4294967444 +wendy thompson 1 4294967301 +wendy thompson 2 8589934621 +wendy underhill 1 4294967540 +wendy underhill 2 8589934993 +wendy underhill 3 12884902410 +wendy van buren 1 4294967488 +wendy van buren 2 8589934835 +wendy white 1 4294967490 +wendy xylophone 1 4294967488 +wendy xylophone 2 8589934939 +wendy young 1 4294967395 +wendy young 2 8589934708 +xavier allen 1 4294967304 +xavier allen 2 8589934743 +xavier allen 3 12884902129 +xavier brown 1 4294967546 +xavier brown 2 8589935074 +xavier brown 3 12884902532 +xavier carson 1 4294967547 +xavier carson 2 8589934862 +xavier davidson 1 4294967361 +xavier davidson 2 8589934760 +xavier davidson 3 12884902204 +xavier ellison 1 4294967441 +xavier ellison 2 8589934914 +xavier garcia 1 4294967465 +xavier hernandez 1 4294967383 +xavier hernandez 2 8589934743 +xavier hernandez 3 12884902274 +xavier ichabod 1 4294967511 +xavier ichabod 2 8589934950 +xavier johnson 1 4294967507 +xavier johnson 2 8589934898 +xavier king 1 4294967456 +xavier king 2 8589934758 +xavier laertes 1 4294967450 +xavier ovid 1 4294967403 +xavier polk 1 4294967506 +xavier polk 2 8589934925 +xavier polk 3 12884902406 +xavier polk 4 17179869906 +xavier quirinius 1 4294967383 +xavier quirinius 2 8589934748 +xavier quirinius 3 12884902060 +xavier quirinius 4 17179869562 +xavier thompson 1 4294967444 +xavier underhill 1 4294967332 +xavier white 1 4294967473 +xavier white 2 8589934952 +xavier xylophone 1 4294967499 +xavier zipper 1 4294967547 +yuri allen 1 4294967528 +yuri allen 2 8589935079 +yuri brown 1 4294967433 +yuri brown 2 8589934960 +yuri carson 1 4294967317 +yuri carson 2 8589934851 +yuri ellison 1 4294967299 +yuri ellison 2 8589934697 +yuri falkner 1 4294967368 +yuri falkner 2 8589934891 +yuri garcia 1 4294967362 +yuri hernandez 1 4294967367 +yuri johnson 1 4294967421 +yuri johnson 2 8589934877 +yuri johnson 3 12884902361 +yuri king 1 4294967376 +yuri laertes 1 4294967402 +yuri laertes 2 8589934924 +yuri nixon 1 4294967400 +yuri nixon 2 8589934706 +yuri polk 1 4294967391 +yuri polk 2 8589934861 +yuri polk 3 12884902167 +yuri quirinius 1 4294967398 +yuri quirinius 2 8589934768 +yuri quirinius 3 12884902081 +yuri steinbeck 1 4294967535 +yuri steinbeck 2 8589934873 +yuri thompson 1 4294967447 +yuri underhill 1 4294967499 +yuri underhill 2 8589934900 +yuri white 1 4294967341 +yuri xylophone 1 4294967420 +zach allen 1 4294967507 +zach brown 1 4294967316 +zach brown 2 8589934728 +zach brown 3 12884902099 +zach brown 4 17179869452 +zach brown 5 21474836769 +zach carson 1 4294967463 +zach ellison 1 4294967471 +zach falkner 1 4294967362 +zach falkner 2 8589934717 +zach garcia 1 4294967481 +zach garcia 2 8589934854 +zach garcia 3 12884902240 +zach garcia 4 17179869723 +zach ichabod 1 4294967539 +zach ichabod 2 8589934912 +zach king 1 4294967424 +zach king 2 8589934956 +zach king 3 12884902458 +zach miller 1 4294967442 +zach miller 2 8589934772 +zach miller 3 12884902163 +zach ovid 1 4294967412 +zach ovid 2 8589934775 +zach ovid 3 12884902244 +zach ovid 4 17179869574 +zach quirinius 1 4294967299 +zach robinson 1 4294967325 +zach steinbeck 1 4294967469 +zach steinbeck 2 8589934834 +zach thompson 1 4294967405 +zach thompson 2 8589934730 +zach underhill 1 4294967496 +zach white 1 4294967501 +zach xylophone 1 4294967452 +zach xylophone 2 8589934755 +zach young 1 4294967297 +zach zipper 1 4294967497 +zach zipper 2 8589934855 +zach zipper 3 12884902222 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 new file mode 100644 index 0000000000000..f47923618a1aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 3-0-455e41d9949a2d22bab634fd8e42f2b1 @@ -0,0 +1 @@ +bob steinbeck 65637 9.699999809265137 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 4-0-cfad06ae8eba6b047d32a6a61dd59392 @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 new file mode 100644 index 0000000000000..5308b2eb457e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 5-0-d7ca7a61377cef3a9f721a28afdae012 @@ -0,0 +1 @@ +bob steinbeck 9.699999809265137 1 diff --git a/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f new file mode 100644 index 0000000000000..f41eaa259cec0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_multipartitioning.q (deterministic) 6-0-287bcc7679822bc7b684532b267bf11f @@ -0,0 +1 @@ +bob steinbeck 1 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b new file mode 100644 index 0000000000000..8150409e62d33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-0-36217f6074daaacddb9fcb50a3f4fb5b @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 2 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 2 +alice hernandez 1 +alice johnson 2 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 3 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 2 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob hernandez 1 +bob ichabod 1 +bob king 2 +bob king 1 +bob king 1 +bob laertes 2 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 2 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 2 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 2 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 2 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 3 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 2 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 2 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 1 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 3 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 2 +ethan brown 2 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 3 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 2 +ethan nixon 1 +ethan ovid 1 +ethan polk 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 2 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 2 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 1 +fred johnson 1 +fred king 2 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 2 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 3 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 2 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 2 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 2 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 2 +holly polk 1 +holly robinson 2 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 1 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 2 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 2 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 2 +irene underhill 2 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 1 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 2 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 2 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 2 +jessica ovid 1 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 1 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 2 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 2 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 2 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke brown 2 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 2 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 2 +luke ovid 2 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 1 +luke van buren 2 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 2 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 2 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 1 +mike ichabod 1 +mike king 2 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike miller 1 +mike nixon 2 +mike nixon 1 +mike polk 2 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 2 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 1 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 2 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 2 +nick quirinius 2 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 2 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 2 +nick zipper 1 +oscar allen 1 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 2 +oscar ellison 2 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 2 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 2 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 1 +oscar polk 1 +oscar polk 1 +oscar quirinius 2 +oscar quirinius 2 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 2 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 2 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 2 +oscar zipper 1 +oscar zipper 1 +priscilla brown 2 +priscilla brown 1 +priscilla brown 1 +priscilla carson 2 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 2 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 2 +priscilla nixon 2 +priscilla ovid 2 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 2 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 2 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 2 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 2 +quinn steinbeck 1 +quinn steinbeck 1 +quinn thompson 1 +quinn thompson 1 +quinn underhill 2 +quinn underhill 1 +quinn underhill 1 +quinn van buren 1 +quinn young 2 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 3 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel carson 2 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 2 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 1 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 1 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 1 +tom king 1 +tom laertes 2 +tom laertes 1 +tom miller 2 +tom miller 1 +tom miller 1 +tom nixon 2 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom robinson 1 +tom steinbeck 3 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 2 +tom young 1 +tom young 1 +tom zipper 3 +ulysses brown 1 +ulysses carson 4 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 2 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 2 +ulysses king 1 +ulysses laertes 2 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 2 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 2 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 1 +victor ellison 2 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 2 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 1 +victor nixon 1 +victor nixon 1 +victor ovid 2 +victor polk 2 +victor quirinius 1 +victor quirinius 1 +victor robinson 2 +victor robinson 1 +victor steinbeck 2 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 2 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 2 +wendy falkner 1 +wendy falkner 1 +wendy garcia 2 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 2 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 2 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 2 +wendy thompson 1 +wendy underhill 2 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 1 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 2 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 2 +xavier quirinius 2 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 2 +xavier white 1 +xavier xylophone 1 +xavier zipper 1 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 2 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 1 +zach brown 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 2 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 2 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 2 +zach king 1 +zach king 1 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 1 +zach steinbeck 1 +zach steinbeck 1 +zach thompson 2 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 2 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 new file mode 100644 index 0000000000000..275772e1f643a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-1-9ee79e711248dd6e0a6ce27e439e55f4 @@ -0,0 +1,1049 @@ +65791 calvin nixon +65791 katie garcia +65790 fred nixon +65790 victor polk +65790 yuri ellison +65789 NULL +65789 alice laertes +65789 gabriella king +65789 katie zipper +65789 oscar zipper +65789 quinn davidson +65789 wendy brown +65788 irene brown +65788 oscar zipper +65788 rachel king +65788 xavier thompson +65787 alice laertes +65787 david laertes +65787 katie ichabod +65787 ulysses king +65786 bob carson +65786 quinn king +65786 victor xylophone +65786 xavier allen +65786 xavier davidson +65785 sarah carson +65785 sarah johnson +65784 holly young +65784 jessica ellison +65784 jessica ovid +65784 jessica zipper +65784 quinn van buren +65783 david allen +65783 fred ellison +65783 irene nixon +65783 mike ichabod +65783 wendy miller +65783 zach garcia +65782 tom miller +65782 ulysses underhill +65782 victor nixon +65782 yuri white +65782 zach ovid +65781 ethan carson +65781 luke laertes +65781 quinn brown +65780 holly hernandez +65780 holly zipper +65780 wendy falkner +65779 gabriella ichabod +65779 irene brown +65779 irene underhill +65779 ulysses xylophone +65779 wendy hernandez +65779 yuri nixon +65779 zach ellison +65778 jessica davidson +65778 katie allen +65778 oscar van buren +65778 oscar white +65778 rachel johnson +65778 victor quirinius +65778 yuri polk +65778 yuri quirinius +65777 ethan garcia +65777 irene garcia +65777 katie polk +65777 rachel thompson +65776 NULL +65776 luke garcia +65776 luke quirinius +65776 priscilla ovid +65776 sarah king +65775 bob ellison +65775 calvin steinbeck +65775 ethan laertes +65775 luke robinson +65775 oscar polk +65774 calvin young +65774 irene quirinius +65774 katie brown +65774 oscar hernandez +65774 sarah ichabod +65774 zach king +65773 NULL +65773 calvin garcia +65773 irene polk +65773 jessica davidson +65773 nick ichabod +65773 nick zipper +65773 quinn ovid +65773 zach falkner +65772 oscar garcia +65771 ethan underhill +65771 yuri hernandez +65771 zach brown +65770 alice nixon +65770 gabriella ellison +65769 mike king +65769 nick steinbeck +65769 oscar nixon +65769 rachel zipper +65769 xavier quirinius +65769 zach ichabod +65768 jessica davidson +65767 tom robinson +65767 victor davidson +65767 xavier king +65766 fred davidson +65766 luke underhill +65766 ulysses carson +65766 wendy nixon +65766 xavier brown +65765 bob polk +65765 gabriella brown +65765 wendy allen +65764 alice hernandez +65764 alice robinson +65764 luke ellison +65763 calvin van buren +65763 ethan brown +65763 priscilla johnson +65763 tom ichabod +65763 tom miller +65762 david xylophone +65762 priscilla xylophone +65762 quinn allen +65762 sarah johnson +65762 tom carson +65761 david ovid +65760 NULL +65760 ethan king +65760 priscilla johnson +65760 wendy van buren +65759 alice xylophone +65759 ethan quirinius +65759 fred steinbeck +65759 sarah robinson +65759 xavier johnson +65758 alice van buren +65758 ethan white +65758 irene johnson +65758 jessica underhill +65758 rachel laertes +65757 bob quirinius +65757 nick quirinius +65757 tom van buren +65757 xavier carson +65757 zach brown +65756 gabriella falkner +65756 mike polk +65756 nick allen +65756 yuri xylophone +65755 alice johnson +65755 david ellison +65755 fred polk +65755 gabriella hernandez +65755 irene van buren +65755 jessica nixon +65755 mike ellison +65755 nick van buren +65755 zach miller +65754 luke johnson +65754 xavier quirinius +65753 rachel robinson +65753 yuri garcia +65752 bob miller +65752 oscar zipper +65751 calvin quirinius +65751 katie king +65751 mike allen +65751 mike quirinius +65751 mike white +65751 zach zipper +65750 mike white +65750 nick thompson +65750 oscar quirinius +65750 priscilla polk +65750 rachel brown +65749 david allen +65749 gabriella polk +65749 mike ellison +65749 sarah carson +65749 wendy thompson +65748 david davidson +65748 oscar laertes +65747 calvin falkner +65747 fred steinbeck +65747 priscilla zipper +65747 quinn underhill +65747 rachel falkner +65747 wendy falkner +65747 yuri falkner +65746 +65746 katie robinson +65746 luke garcia +65746 tom brown +65746 zach steinbeck +65745 oscar steinbeck +65745 oscar xylophone +65745 priscilla nixon +65745 victor laertes +65744 alice allen +65744 priscilla ichabod +65743 jessica carson +65743 oscar ichabod +65743 sarah falkner +65743 tom brown +65742 jessica brown +65742 jessica ellison +65742 wendy polk +65741 calvin thompson +65740 irene van buren +65740 mike zipper +65740 rachel quirinius +65739 gabriella van buren +65739 victor robinson +65739 wendy nixon +65738 sarah ichabod +65738 wendy robinson +65738 wendy young +65738 xavier king +65737 holly brown +65737 irene falkner +65737 jessica white +65737 quinn garcia +65737 wendy laertes +65736 fred young +65736 quinn laertes +65736 rachel allen +65736 victor ellison +65735 oscar laertes +65735 tom falkner +65735 ulysses thompson +65734 calvin falkner +65734 mike garcia +65733 gabriella van buren +65733 holly nixon +65733 luke underhill +65733 oscar allen +65733 xavier polk +65733 yuri ellison +65732 mike steinbeck +65732 tom carson +65732 tom ellison +65731 alice robinson +65731 priscilla carson +65731 tom johnson +65731 ulysses ovid +65730 bob king +65730 calvin polk +65730 gabriella ichabod +65730 rachel brown +65729 fred quirinius +65728 priscilla king +65728 victor brown +65727 ethan brown +65727 fred polk +65727 jessica white +65727 priscilla underhill +65727 sarah white +65726 mike garcia +65726 ulysses hernandez +65726 wendy quirinius +65725 zach thompson +65724 NULL +65724 alice king +65724 alice steinbeck +65724 mike ellison +65724 nick ovid +65724 priscilla zipper +65723 nick allen +65723 rachel white +65722 fred falkner +65722 jessica underhill +65722 luke ichabod +65721 ethan falkner +65721 jessica zipper +65721 luke laertes +65721 luke underhill +65721 mike carson +65721 oscar laertes +65721 ulysses ichabod +65720 calvin nixon +65720 calvin thompson +65720 gabriella young +65720 holly polk +65720 mike king +65719 bob brown +65719 holly ichabod +65719 ulysses ellison +65719 ulysses young +65718 jessica ichabod +65718 oscar johnson +65718 victor hernandez +65718 zach ovid +65717 holly hernandez +65717 mike nixon +65717 quinn garcia +65717 ulysses davidson +65717 ulysses polk +65716 ethan zipper +65716 holly xylophone +65716 jessica garcia +65716 nick falkner +65716 sarah king +65716 victor xylophone +65715 alice hernandez +65715 david young +65715 ethan polk +65715 oscar falkner +65715 priscilla brown +65714 NULL +65714 ethan laertes +65714 luke allen +65713 bob ellison +65713 nick nixon +65713 ulysses carson +65713 ulysses ichabod +65713 zach garcia +65712 NULL +65712 katie king +65712 luke davidson +65712 quinn garcia +65712 zach carson +65711 bob zipper +65711 fred miller +65711 holly nixon +65711 katie ellison +65711 wendy ovid +65711 zach zipper +65710 luke polk +65710 mike young +65709 bob laertes +65709 calvin laertes +65709 oscar laertes +65708 gabriella laertes +65708 priscilla van buren +65707 bob garcia +65707 mike steinbeck +65706 bob ellison +65706 bob xylophone +65706 luke allen +65706 ulysses xylophone +65706 wendy ichabod +65705 david ellison +65705 mike white +65705 priscilla johnson +65705 tom ellison +65704 ethan brown +65704 holly king +65704 jessica young +65704 tom steinbeck +65704 victor steinbeck +65704 zach falkner +65703 holly underhill +65703 mike ellison +65703 victor davidson +65703 xavier underhill +65702 NULL +65702 bob brown +65702 bob zipper +65702 ulysses quirinius +65701 alice allen +65701 mike steinbeck +65701 oscar thompson +65700 luke allen +65700 quinn laertes +65700 tom quirinius +65699 ethan brown +65699 ethan van buren +65699 irene laertes +65699 mike young +65699 nick garcia +65699 sarah white +65698 gabriella thompson +65698 nick polk +65697 NULL +65697 NULL +65697 holly miller +65697 oscar quirinius +65697 oscar thompson +65697 xavier davidson +65696 alice xylophone +65696 jessica davidson +65696 luke allen +65696 oscar xylophone +65695 oscar king +65695 rachel young +65695 wendy ellison +65695 yuri quirinius +65694 david brown +65694 holly underhill +65694 victor quirinius +65694 zach brown +65693 bob hernandez +65693 bob young +65693 david brown +65693 holly hernandez +65693 tom polk +65693 ulysses polk +65693 victor brown +65692 holly johnson +65692 tom robinson +65691 calvin ovid +65691 ethan nixon +65691 ethan robinson +65691 fred underhill +65691 holly white +65691 irene polk +65691 oscar white +65691 rachel brown +65690 fred van buren +65690 jessica quirinius +65689 oscar ovid +65689 wendy thompson +65688 bob steinbeck +65688 victor steinbeck +65687 gabriella ichabod +65687 jessica underhill +65687 mike zipper +65687 quinn thompson +65686 bob king +65686 bob zipper +65686 david quirinius +65686 luke ichabod +65685 ethan robinson +65685 gabriella hernandez +65685 katie garcia +65685 sarah ellison +65685 victor hernandez +65685 victor nixon +65684 priscilla brown +65684 victor laertes +65684 wendy van buren +65683 NULL +65683 mike king +65683 tom laertes +65682 calvin quirinius +65682 ethan brown +65682 katie ellison +65681 gabriella allen +65681 luke laertes +65681 oscar quirinius +65681 ulysses laertes +65681 wendy ellison +65681 xavier polk +65680 NULL +65680 alice nixon +65680 gabriella ovid +65680 jessica carson +65680 ulysses nixon +65680 zach zipper +65679 bob garcia +65679 wendy underhill +65678 bob falkner +65678 victor xylophone +65678 wendy king +65677 alice allen +65677 fred van buren +65677 mike brown +65677 nick xylophone +65677 ulysses underhill +65677 zach robinson +65676 bob davidson +65676 bob laertes +65676 tom ovid +65676 xavier johnson +65675 david hernandez +65675 david nixon +65675 holly falkner +65675 quinn steinbeck +65675 rachel robinson +65675 sarah zipper +65675 tom polk +65675 victor allen +65674 gabriella falkner +65673 nick johnson +65673 quinn brown +65673 quinn underhill +65673 rachel ovid +65673 wendy brown +65672 nick laertes +65672 nick underhill +65672 rachel zipper +65672 tom white +65672 victor king +65671 fred ellison +65671 fred falkner +65671 zach white +65670 david robinson +65670 jessica zipper +65670 luke van buren +65670 oscar ovid +65670 quinn steinbeck +65669 NULL +65669 alice king +65669 calvin hernandez +65669 katie polk +65669 nick miller +65669 oscar van buren +65668 luke ellison +65667 bob brown +65667 irene nixon +65667 oscar brown +65667 tom falkner +65666 +65666 david underhill +65666 fred van buren +65665 rachel brown +65664 NULL +65664 bob davidson +65664 david ichabod +65664 ethan laertes +65664 irene robinson +65664 mike carson +65664 priscilla young +65664 victor king +65663 calvin underhill +65663 jessica johnson +65663 priscilla carson +65663 zach ichabod +65662 ethan allen +65662 katie ovid +65662 oscar johnson +65662 ulysses carson +65662 ulysses polk +65662 victor ovid +65661 david van buren +65661 luke xylophone +65661 mike falkner +65661 priscilla van buren +65661 victor johnson +65660 holly ichabod +65660 priscilla johnson +65660 victor thompson +65659 david robinson +65659 gabriella king +65659 luke davidson +65659 mike king +65659 mike zipper +65659 nick brown +65659 nick zipper +65659 yuri underhill +65658 NULL +65658 alice zipper +65658 calvin allen +65658 calvin johnson +65658 jessica garcia +65658 quinn davidson +65658 sarah ovid +65658 ulysses brown +65658 ulysses miller +65658 yuri king +65657 ethan falkner +65657 holly zipper +65657 irene ovid +65657 luke ovid +65657 priscilla white +65656 david davidson +65656 irene ovid +65656 jessica xylophone +65656 luke laertes +65656 oscar ichabod +65656 xavier ellison +65655 calvin falkner +65655 yuri laertes +65654 alice carson +65654 alice quirinius +65654 gabriella falkner +65654 nick young +65654 oscar robinson +65654 quinn robinson +65654 rachel falkner +65654 tom laertes +65654 yuri johnson +65653 calvin ellison +65653 holly underhill +65653 ulysses polk +65653 wendy xylophone +65652 NULL +65652 tom ellison +65652 victor johnson +65651 NULL +65651 ethan laertes +65651 ethan laertes +65651 irene garcia +65651 mike young +65650 irene ellison +65650 oscar white +65650 sarah steinbeck +65650 ulysses underhill +65650 ulysses xylophone +65650 victor xylophone +65649 irene underhill +65649 priscilla quirinius +65649 quinn ellison +65649 tom quirinius +65648 alice nixon +65648 calvin brown +65648 sarah carson +65648 xavier ellison +65647 irene allen +65647 mike ellison +65646 bob ovid +65646 xavier brown +65646 xavier ovid +65645 jessica quirinius +65645 katie miller +65645 ulysses hernandez +65644 alice king +65644 calvin ovid +65644 jessica white +65644 katie van buren +65644 sarah young +65644 ulysses hernandez +65644 yuri carson +65643 david davidson +65643 ethan polk +65643 ethan zipper +65643 gabriella ichabod +65643 mike davidson +65643 mike hernandez +65643 oscar robinson +65643 priscilla underhill +65643 zach king +65642 gabriella thompson +65641 fred laertes +65641 sarah garcia +65641 tom miller +65641 xavier hernandez +65640 david underhill +65639 wendy garcia +65638 fred nixon +65638 luke polk +65638 rachel carson +65637 alice underhill +65637 david davidson +65637 fred davidson +65637 gabriella davidson +65637 oscar carson +65637 rachel laertes +65637 sarah garcia +65637 wendy garcia +65636 +65636 irene polk +65636 wendy allen +65635 alice steinbeck +65635 alice zipper +65635 ulysses white +65634 NULL +65634 calvin white +65634 holly underhill +65634 sarah falkner +65633 NULL +65633 holly polk +65633 jessica nixon +65633 oscar quirinius +65632 alice falkner +65632 zach quirinius +65631 fred ellison +65630 rachel brown +65630 xavier hernandez +65629 jessica quirinius +65629 priscilla carson +65629 victor young +65629 xavier carson +65628 bob ovid +65628 ethan ovid +65628 irene ichabod +65628 oscar hernandez +65628 oscar robinson +65628 xavier quirinius +65627 alice miller +65627 holly johnson +65627 luke falkner +65627 yuri polk +65626 ethan polk +65626 holly robinson +65626 tom young +65626 yuri johnson +65625 david xylophone +65625 fred ichabod +65625 katie white +65625 ulysses garcia +65624 calvin steinbeck +65624 calvin xylophone +65624 rachel carson +65624 tom van buren +65624 yuri brown +65623 alice quirinius +65623 jessica miller +65623 oscar ichabod +65623 quinn zipper +65623 tom van buren +65623 victor brown +65623 wendy young +65622 nick davidson +65622 rachel king +65622 wendy robinson +65622 xavier ichabod +65622 zach xylophone +65622 zach young +65621 quinn underhill +65621 ulysses young +65620 nick garcia +65620 oscar thompson +65620 quinn quirinius +65620 victor white +65620 victor xylophone +65620 wendy quirinius +65619 calvin brown +65619 gabriella polk +65619 oscar king +65619 ulysses miller +65619 ulysses robinson +65619 ulysses steinbeck +65618 gabriella ovid +65618 irene laertes +65618 katie king +65618 oscar ovid +65618 quinn thompson +65617 fred van buren +65617 gabriella carson +65617 sarah johnson +65617 ulysses underhill +65616 calvin steinbeck +65616 xavier ichabod +65615 alice ovid +65615 david quirinius +65615 irene quirinius +65615 katie nixon +65614 wendy king +65614 xavier quirinius +65614 xavier white +65613 xavier zipper +65612 irene miller +65612 victor hernandez +65612 wendy white +65612 yuri polk +65611 ethan johnson +65611 fred zipper +65611 irene carson +65611 nick quirinius +65610 tom king +65610 victor steinbeck +65610 wendy garcia +65610 yuri carson +65610 zach ovid +65609 sarah robinson +65608 katie van buren +65608 mike van buren +65608 quinn ichabod +65608 zach underhill +65607 katie miller +65607 luke falkner +65607 mike polk +65607 priscilla xylophone +65607 yuri allen +65607 yuri allen +65606 bob white +65606 gabriella white +65606 oscar carson +65606 victor white +65606 xavier allen +65606 zach allen +65605 holly king +65604 katie zipper +65604 oscar davidson +65604 wendy laertes +65604 zach brown +65603 alice davidson +65603 ethan miller +65603 katie davidson +65603 katie young +65603 mike garcia +65602 NULL +65602 calvin laertes +65602 ethan laertes +65602 fred steinbeck +65602 jessica young +65602 xavier brown +65601 priscilla ovid +65601 sarah xylophone +65601 tom robinson +65600 gabriella thompson +65600 jessica polk +65600 nick robinson +65600 rachel allen +65599 fred quirinius +65599 luke johnson +65599 nick garcia +65599 oscar xylophone +65599 ulysses underhill +65598 ulysses van buren +65598 victor zipper +65597 ethan ellison +65597 nick ellison +65597 quinn davidson +65596 NULL +65596 calvin zipper +65596 david ellison +65596 irene ichabod +65596 wendy laertes +65595 bob white +65595 holly hernandez +65595 luke brown +65595 oscar ellison +65595 oscar ichabod +65595 quinn ellison +65594 gabriella ellison +65594 oscar robinson +65594 ulysses underhill +65594 victor robinson +65593 oscar white +65593 zach xylophone +65592 calvin xylophone +65591 alice zipper +65591 nick ichabod +65591 priscilla ichabod +65591 rachel underhill +65590 NULL +65590 katie falkner +65590 oscar van buren +65590 xavier garcia +65590 yuri underhill +65589 ethan white +65589 gabriella zipper +65589 irene ovid +65589 oscar king +65589 wendy xylophone +65588 bob van buren +65588 david ichabod +65588 mike miller +65588 tom hernandez +65588 victor van buren +65587 bob garcia +65587 luke johnson +65587 mike king +65587 victor allen +65587 xavier white +65586 david young +65586 irene brown +65586 priscilla brown +65586 wendy allen +65586 xavier laertes +65585 alice garcia +65585 bob garcia +65585 ethan ellison +65585 nick ellison +65585 priscilla thompson +65584 jessica carson +65584 jessica van buren +65584 jessica white +65583 bob xylophone +65583 nick ichabod +65583 yuri brown +65583 yuri steinbeck +65582 holly johnson +65582 mike carson +65582 victor van buren +65582 zach miller +65581 gabriella steinbeck +65581 irene quirinius +65581 luke allen +65581 nick robinson +65581 nick young +65581 wendy robinson +65580 alice steinbeck +65580 alice xylophone +65580 irene xylophone +65579 irene polk +65579 luke ovid +65579 quinn nixon +65579 sarah garcia +65579 wendy ovid +65578 calvin robinson +65578 fred king +65578 holly thompson +65578 katie ichabod +65578 quinn king +65578 rachel davidson +65578 victor hernandez +65577 holly white +65576 calvin falkner +65576 calvin ovid +65576 fred polk +65576 luke robinson +65575 calvin falkner +65575 irene steinbeck +65575 luke zipper +65575 zach king +65574 gabriella steinbeck +65574 priscilla nixon +65574 rachel thompson +65573 victor ellison +65573 victor hernandez +65573 yuri nixon +65572 calvin davidson +65572 calvin young +65572 katie young +65572 oscar ellison +65572 quinn garcia +65571 bob king +65571 irene polk +65571 katie ichabod +65571 mike steinbeck +65570 NULL +65570 bob ovid +65570 fred polk +65570 luke ellison +65570 mike hernandez +65570 yuri quirinius +65569 nick falkner +65568 bob ichabod +65568 holly thompson +65568 jessica thompson +65567 katie xylophone +65566 gabriella garcia +65566 rachel white +65565 katie young +65565 quinn young +65564 alice polk +65564 calvin carson +65564 calvin white +65564 ethan hernandez +65564 ethan quirinius +65564 jessica thompson +65564 katie hernandez +65563 calvin zipper +65563 priscilla young +65563 xavier davidson +65563 yuri steinbeck +65562 calvin falkner +65562 ethan xylophone +65562 luke white +65562 quinn allen +65562 rachel polk +65562 wendy polk +65561 bob davidson +65561 ethan polk +65561 jessica robinson +65560 fred white +65560 jessica johnson +65560 oscar thompson +65560 ulysses steinbeck +65560 zach brown +65559 NULL +65559 ethan laertes +65559 gabriella ichabod +65559 gabriella zipper +65559 irene garcia +65558 fred robinson +65557 fred hernandez +65557 nick johnson +65556 oscar underhill +65556 xavier hernandez +65556 yuri falkner +65556 zach garcia +65556 zach steinbeck +65555 fred nixon +65554 gabriella miller +65554 rachel falkner +65553 calvin van buren +65553 david van buren +65553 irene nixon +65553 luke laertes +65553 oscar carson +65552 NULL +65552 irene ellison +65552 oscar polk +65552 wendy falkner +65552 zach miller +65551 fred young +65551 ulysses underhill +65551 wendy underhill +65550 ethan quirinius +65550 fred davidson +65550 holly young +65550 jessica ovid +65550 quinn brown +65550 quinn laertes +65550 tom johnson +65549 bob garcia +65549 bob ovid +65549 fred ichabod +65549 fred king +65549 jessica white +65549 ulysses laertes +65549 victor davidson +65549 victor miller +65548 calvin ovid +65548 gabriella allen +65548 holly ichabod +65548 priscilla johnson +65548 quinn zipper +65548 tom hernandez +65548 wendy king +65547 bob ellison +65547 jessica quirinius +65547 mike davidson +65547 xavier allen +65546 katie white +65545 mike king +65545 tom carson +65545 victor brown +65544 calvin davidson +65544 calvin nixon +65544 david ovid +65544 irene thompson +65544 ulysses young +65544 xavier polk +65544 xavier xylophone +65544 zach ovid +65543 fred johnson +65543 sarah johnson +65542 fred falkner +65542 holly thompson +65542 luke miller +65542 mike white +65542 tom davidson +65541 calvin brown +65541 ethan brown +65541 holly brown +65541 jessica falkner +65541 rachel thompson +65541 tom zipper +65541 wendy underhill +65541 xavier polk +65541 yuri johnson +65540 rachel falkner +65539 gabriella young +65539 holly laertes +65539 oscar carson +65538 irene laertes +65538 mike polk +65538 tom robinson +65537 NULL +65537 david quirinius +65537 rachel ovid +65537 ulysses laertes +65537 zach garcia +65536 calvin xylophone +65536 david thompson +65536 irene falkner +65536 ulysses johnson +65536 victor johnson +65536 wendy miller +65536 yuri thompson diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 new file mode 100644 index 0000000000000..62d71abc6fc7d --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -0,0 +1,1049 @@ +65536 NULL +65536 16.85 +65536 32.68 +65536 33.45 +65536 58.86 +65536 75.7 +65536 83.48 +65537 NULL +65537 4.49 +65537 11.87 +65537 51.91 +65537 99.34 +65538 NULL +65538 9.53 +65538 30.27 +65539 NULL +65539 58.85 +65539 96.64 +65540 NULL +65541 NULL +65541 9.04 +65541 14.94 +65541 15.85 +65541 27.89 +65541 35.38 +65541 72.33 +65541 89.14 +65541 98.87 +65542 NULL +65542 42.85 +65542 73.72 +65542 73.93 +65542 84.63 +65543 NULL +65543 21.59 +65544 NULL +65544 26.94 +65544 32.03 +65544 48.84 +65544 65.77 +65544 65.85 +65544 73.08 +65544 79.21 +65545 NULL +65545 34.65 +65545 65.81 +65546 NULL +65547 NULL +65547 17.71 +65547 62.31 +65547 83.21 +65548 NULL +65548 52.94 +65548 53.52 +65548 57.61 +65548 58.51 +65548 75.39 +65548 77.24 +65549 NULL +65549 13.3 +65549 28.93 +65549 50.6 +65549 55.04 +65549 64.91 +65549 76.06 +65549 80.09 +65550 NULL +65550 1.75 +65550 8.46 +65550 33.01 +65550 57.63 +65550 91.38 +65550 96.9 +65551 NULL +65551 39.43 +65551 73.93 +65552 NULL +65552 3.43 +65552 18.11 +65552 48.96 +65552 82.21 +65553 NULL +65553 25.31 +65553 29.62 +65553 71.07 +65553 72.16 +65554 NULL +65554 0.24 +65555 NULL +65556 NULL +65556 9.96 +65556 45.81 +65556 50.42 +65556 95.12 +65557 NULL +65557 21.14 +65558 NULL +65559 NULL +65559 29.55 +65559 56.06 +65559 73.94 +65559 83.5 +65560 NULL +65560 16.86 +65560 21.81 +65560 64.38 +65560 71.59 +65561 NULL +65561 32.86 +65561 47.71 +65562 NULL +65562 26.86 +65562 45.94 +65562 54.94 +65562 69.41 +65562 72.04 +65563 NULL +65563 14.36 +65563 33.29 +65563 39.96 +65564 NULL +65564 9.47 +65564 15.82 +65564 26.97 +65564 30.78 +65564 90.34 +65564 98.36 +65565 NULL +65565 81.72 +65566 NULL +65566 7.8 +65567 NULL +65568 NULL +65568 21.79 +65568 58.66 +65569 NULL +65570 NULL +65570 17.09 +65570 18.2 +65570 25.57 +65570 45.23 +65570 76.8 +65571 NULL +65571 26.64 +65571 40.68 +65571 82.5 +65572 NULL +65572 22.64 +65572 43.49 +65572 70.91 +65572 93.48 +65573 NULL +65573 53.56 +65573 96.32 +65574 NULL +65574 31.28 +65574 38.54 +65575 NULL +65575 17 +65575 32.85 +65575 83.4 +65576 NULL +65576 2.04 +65576 4.88 +65576 66.55 +65577 NULL +65578 NULL +65578 16.01 +65578 41.1 +65578 51.36 +65578 54.35 +65578 58.78 +65578 75.62 +65579 NULL +65579 21.36 +65579 33.37 +65579 73.48 +65579 91.42 +65580 NULL +65580 56.33 +65580 81.42 +65581 NULL +65581 29.74 +65581 45.48 +65581 56.59 +65581 60.88 +65581 88.09 +65582 NULL +65582 1.23 +65582 9.35 +65582 96.6 +65583 NULL +65583 28.07 +65583 50.57 +65583 57.67 +65584 NULL +65584 6.88 +65584 65.78 +65585 NULL +65585 31.23 +65585 37.34 +65585 39.32 +65585 50.38 +65586 NULL +65586 1.15 +65586 18.65 +65586 87.57 +65586 94.25 +65587 NULL +65587 5.83 +65587 11.86 +65587 53.84 +65587 94.47 +65588 NULL +65588 64.63 +65588 69.48 +65588 76.44 +65588 98.33 +65589 NULL +65589 49.49 +65589 72.3 +65589 74.83 +65589 94.73 +65590 NULL +65590 13.51 +65590 22.13 +65590 42.53 +65590 92.71 +65591 NULL +65591 9.85 +65591 11.43 +65591 60.78 +65592 NULL +65593 NULL +65593 35.15 +65594 NULL +65594 35.28 +65594 60.57 +65594 65.61 +65595 NULL +65595 8.76 +65595 67.56 +65595 72.7 +65595 89.6 +65595 90.24 +65596 NULL +65596 12.72 +65596 54.75 +65596 85.74 +65596 94.35 +65597 NULL +65597 37.41 +65597 69.05 +65598 NULL +65598 63.3 +65599 NULL +65599 0.56 +65599 4.93 +65599 41.61 +65599 76.29 +65600 NULL +65600 36.57 +65600 43.03 +65600 92.92 +65601 NULL +65601 26.54 +65601 37.93 +65602 NULL +65602 8.13 +65602 47.16 +65602 83.67 +65602 86.23 +65602 95.58 +65603 NULL +65603 41.44 +65603 45.63 +65603 69.26 +65603 80.24 +65604 NULL +65604 63.36 +65604 63.65 +65604 85.91 +65605 NULL +65606 NULL +65606 7.51 +65606 24.8 +65606 57.69 +65606 67.94 +65606 87.16 +65607 NULL +65607 9.67 +65607 36.58 +65607 71.75 +65607 75.86 +65607 91.52 +65608 NULL +65608 48.9 +65608 69.42 +65608 87.9 +65609 NULL +65610 NULL +65610 7.59 +65610 11.99 +65610 36.77 +65610 39.74 +65611 NULL +65611 21.21 +65611 25.92 +65611 64.89 +65612 NULL +65612 16.05 +65612 25.1 +65612 52.64 +65613 NULL +65614 NULL +65614 1.42 +65614 94.47 +65615 NULL +65615 10.79 +65615 39.4 +65615 99.88 +65616 NULL +65616 75.2 +65617 NULL +65617 18.51 +65617 47.45 +65617 64.9 +65618 NULL +65618 10.06 +65618 16.6 +65618 81.99 +65618 88.38 +65619 NULL +65619 27.32 +65619 32.64 +65619 34.72 +65619 36.48 +65619 36.59 +65620 NULL +65620 6.85 +65620 8.16 +65620 29.14 +65620 64.65 +65620 81.28 +65621 NULL +65621 95.14 +65622 NULL +65622 28.37 +65622 50.08 +65622 74.31 +65622 88.6 +65622 93.7 +65623 NULL +65623 30.83 +65623 31.22 +65623 39.74 +65623 48.51 +65623 95.58 +65623 97.2 +65624 NULL +65624 58.02 +65624 65.31 +65624 70.08 +65624 93.3 +65625 NULL +65625 20.61 +65625 42.86 +65625 55.06 +65626 NULL +65626 63.54 +65626 64.61 +65626 75.15 +65627 NULL +65627 19.65 +65627 61.89 +65627 93.29 +65628 NULL +65628 14.83 +65628 30.43 +65628 37.8 +65628 74.31 +65628 83.26 +65629 NULL +65629 19.33 +65629 58.81 +65629 72.9 +65630 NULL +65630 72.13 +65631 NULL +65632 NULL +65632 88.51 +65633 NULL +65633 59.56 +65633 72.54 +65633 81.02 +65634 NULL +65634 57.09 +65634 64.36 +65634 99.34 +65635 NULL +65635 64.99 +65635 82.29 +65636 NULL +65636 21.15 +65636 86.29 +65637 NULL +65637 16.89 +65637 26.78 +65637 29.34 +65637 35.51 +65637 44.32 +65637 48.88 +65637 93.41 +65638 NULL +65638 11.2 +65638 19.13 +65639 NULL +65640 NULL +65641 NULL +65641 26.02 +65641 84.27 +65641 91.46 +65642 NULL +65643 NULL +65643 22.05 +65643 50.79 +65643 52.56 +65643 61.29 +65643 71.29 +65643 80.96 +65643 92.24 +65643 93.11 +65644 NULL +65644 1.97 +65644 30.25 +65644 58.05 +65644 87.31 +65644 89.95 +65644 96.45 +65645 NULL +65645 3.95 +65645 63.22 +65646 NULL +65646 17.92 +65646 27.34 +65647 NULL +65647 58.03 +65648 NULL +65648 0.08 +65648 17.66 +65648 64.06 +65649 NULL +65649 8.69 +65649 43.92 +65649 91.03 +65650 NULL +65650 23.55 +65650 59.55 +65650 85.89 +65650 89.12 +65650 90.77 +65651 NULL +65651 24.25 +65651 58.25 +65651 74.13 +65651 84.42 +65652 NULL +65652 55.04 +65652 73.61 +65653 NULL +65653 3.81 +65653 52.23 +65653 85.09 +65654 NULL +65654 8.91 +65654 11.64 +65654 26.73 +65654 29.85 +65654 37.74 +65654 37.8 +65654 53.55 +65654 88.23 +65655 NULL +65655 77.41 +65656 NULL +65656 14 +65656 14.96 +65656 53.27 +65656 64.44 +65656 82.67 +65657 NULL +65657 11.93 +65657 26.4 +65657 64.39 +65657 65.01 +65658 NULL +65658 2.63 +65658 20.69 +65658 42.93 +65658 46.61 +65658 60.94 +65658 66.53 +65658 68.85 +65658 77.66 +65658 92.67 +65659 NULL +65659 8.95 +65659 46.57 +65659 53.8 +65659 94.3 +65659 94.69 +65659 95.71 +65659 99.87 +65660 NULL +65660 28.05 +65660 62.82 +65661 NULL +65661 5.24 +65661 8.06 +65661 26.8 +65661 68.98 +65662 NULL +65662 59.92 +65662 76.11 +65662 76.51 +65662 88.64 +65662 99.18 +65663 NULL +65663 5.42 +65663 78.56 +65663 94.16 +65664 NULL +65664 11.46 +65664 27.6 +65664 34.71 +65664 38.42 +65664 45.4 +65664 55.82 +65664 97.64 +65665 NULL +65666 NULL +65666 32.73 +65666 83.95 +65667 NULL +65667 13.96 +65667 63.9 +65667 97.87 +65668 NULL +65669 NULL +65669 1.76 +65669 16.95 +65669 38.6 +65669 54.25 +65669 93.79 +65670 NULL +65670 5.37 +65670 61.06 +65670 61.54 +65670 92.97 +65671 NULL +65671 8.65 +65671 52.05 +65672 NULL +65672 52.6 +65672 58.1 +65672 64.09 +65672 75.27 +65673 NULL +65673 0.9 +65673 33.27 +65673 43.81 +65673 87.78 +65674 NULL +65675 NULL +65675 4.19 +65675 24.19 +65675 35.33 +65675 35.78 +65675 79.9 +65675 83.09 +65675 87.36 +65676 NULL +65676 8.77 +65676 58.12 +65676 80.13 +65677 NULL +65677 5.06 +65677 25.37 +65677 44.47 +65677 48.79 +65677 87.67 +65678 NULL +65678 8.72 +65678 33.9 +65679 NULL +65679 64.15 +65680 NULL +65680 1.01 +65680 34.08 +65680 54.11 +65680 55.3 +65680 65.88 +65681 NULL +65681 35.45 +65681 41.57 +65681 61.3 +65681 71.17 +65681 75.85 +65682 NULL +65682 67.17 +65682 92.95 +65683 NULL +65683 17.62 +65683 99.56 +65684 NULL +65684 3.51 +65684 67.34 +65685 NULL +65685 38.71 +65685 43.48 +65685 63.27 +65685 87.84 +65685 90.69 +65686 NULL +65686 31.75 +65686 58.87 +65686 98.68 +65687 NULL +65687 3.37 +65687 21.79 +65687 48.73 +65688 NULL +65688 76.21 +65689 NULL +65689 9.12 +65690 NULL +65690 3.43 +65691 NULL +65691 5.01 +65691 6.93 +65691 28.47 +65691 56.02 +65691 58.01 +65691 69.8 +65691 76.98 +65692 NULL +65692 54.76 +65693 NULL +65693 8.38 +65693 32.33 +65693 45.69 +65693 69.32 +65693 71.72 +65693 84.88 +65694 NULL +65694 58.23 +65694 82.24 +65694 88.5 +65695 NULL +65695 57.33 +65695 59.96 +65695 77.09 +65696 NULL +65696 17.35 +65696 40.3 +65696 54.02 +65697 NULL +65697 3.18 +65697 50.01 +65697 67.9 +65697 86.79 +65697 90.16 +65698 NULL +65698 42.98 +65699 NULL +65699 13.29 +65699 38.71 +65699 68.94 +65699 84.79 +65699 88.09 +65700 NULL +65700 2.83 +65700 37.61 +65701 NULL +65701 1.81 +65701 6.35 +65702 NULL +65702 37.6 +65702 55.68 +65702 79.5 +65703 NULL +65703 37.18 +65703 40.81 +65703 90.89 +65704 NULL +65704 16.22 +65704 37.12 +65704 48.48 +65704 54.76 +65704 93.21 +65705 NULL +65705 20.57 +65705 25.89 +65705 65.13 +65706 NULL +65706 3.91 +65706 9.74 +65706 55.94 +65706 72.87 +65707 NULL +65707 76.2 +65708 NULL +65708 1.29 +65709 NULL +65709 5.64 +65709 49.79 +65710 NULL +65710 86.7 +65711 NULL +65711 8.66 +65711 50.26 +65711 71.89 +65711 78.69 +65711 96.1 +65712 NULL +65712 30.27 +65712 34.7 +65712 49.69 +65712 53.65 +65713 NULL +65713 10.94 +65713 39.47 +65713 72.37 +65713 90.91 +65714 NULL +65714 14.85 +65714 47.42 +65715 NULL +65715 39.62 +65715 54.79 +65715 81.28 +65715 89.4 +65716 NULL +65716 9 +65716 10.07 +65716 33.4 +65716 71.53 +65716 85.93 +65717 NULL +65717 1.23 +65717 5.81 +65717 57.61 +65717 80.05 +65718 NULL +65718 63.06 +65718 84.35 +65718 89.67 +65719 NULL +65719 51.13 +65719 66.85 +65719 82.1 +65720 NULL +65720 2.72 +65720 18.8 +65720 22.34 +65720 62.04 +65721 NULL +65721 23.78 +65721 39.19 +65721 55.75 +65721 72.82 +65721 95.12 +65721 95.38 +65722 NULL +65722 1.76 +65722 38.82 +65723 NULL +65723 39.9 +65724 NULL +65724 10.52 +65724 36.05 +65724 50.96 +65724 71.66 +65724 85.52 +65725 NULL +65726 NULL +65726 6 +65726 60.46 +65727 NULL +65727 19.81 +65727 49.19 +65727 87.37 +65727 88.11 +65728 NULL +65728 55.37 +65729 NULL +65730 NULL +65730 1.35 +65730 30.6 +65730 81.44 +65731 NULL +65731 24.48 +65731 61.52 +65731 97.18 +65732 NULL +65732 30.06 +65732 91.15 +65733 NULL +65733 11.44 +65733 20.72 +65733 88.46 +65733 93.45 +65733 99.8 +65734 NULL +65734 31.71 +65735 NULL +65735 12.67 +65735 61.16 +65736 NULL +65736 28.9 +65736 48.54 +65736 86.51 +65737 NULL +65737 3.98 +65737 20.85 +65737 29.92 +65737 80.97 +65738 NULL +65738 30.94 +65738 82.32 +65738 95.1 +65739 NULL +65739 74.77 +65739 92.4 +65740 NULL +65740 7.49 +65740 58.65 +65741 NULL +65742 NULL +65742 6.61 +65742 43.84 +65743 NULL +65743 26.6 +65743 52.65 +65743 62 +65744 NULL +65744 46.98 +65745 NULL +65745 25.19 +65745 66.36 +65745 80.12 +65746 NULL +65746 36.74 +65746 93.21 +65746 97.52 +65746 98.1 +65747 NULL +65747 11.16 +65747 15.07 +65747 21.8 +65747 39.77 +65747 52.77 +65747 71.87 +65748 NULL +65748 29.49 +65749 NULL +65749 15.14 +65749 45 +65749 65.49 +65749 73.24 +65750 NULL +65750 20.91 +65750 83.44 +65750 85.44 +65750 96.85 +65751 NULL +65751 2.96 +65751 9.02 +65751 30.68 +65751 47.81 +65751 78.75 +65752 NULL +65752 47.82 +65753 NULL +65753 86.97 +65754 NULL +65754 54.35 +65755 NULL +65755 11.23 +65755 22.44 +65755 64 +65755 67.54 +65755 76.75 +65755 81.44 +65755 90.08 +65755 96.8 +65756 NULL +65756 1.45 +65756 11.81 +65756 63.51 +65757 NULL +65757 1.86 +65757 9.24 +65757 34.84 +65757 90.09 +65758 NULL +65758 25.62 +65758 56.56 +65758 60.88 +65758 94.9 +65759 NULL +65759 10.63 +65759 14.1 +65759 47.54 +65759 92.81 +65760 NULL +65760 21.14 +65760 27.52 +65760 95.45 +65761 NULL +65762 NULL +65762 5.49 +65762 45.7 +65762 77.96 +65762 87.5 +65763 NULL +65763 0.72 +65763 43.8 +65763 86.43 +65763 87.99 +65764 NULL +65764 31.41 +65764 57.1 +65765 NULL +65765 88.52 +65765 88.56 +65766 NULL +65766 37.06 +65766 66.34 +65766 86.53 +65766 98.9 +65767 NULL +65767 90.88 +65767 95.57 +65768 NULL +65769 NULL +65769 11.45 +65769 38.98 +65769 58.05 +65769 70.52 +65769 91.49 +65770 NULL +65770 51.9 +65771 NULL +65771 6.15 +65771 7.5 +65772 NULL +65773 NULL +65773 3.81 +65773 18.2 +65773 30.49 +65773 47.09 +65773 53.09 +65773 63.26 +65773 76.46 +65774 NULL +65774 45.74 +65774 45.97 +65774 48.8 +65774 56.84 +65774 94.77 +65775 NULL +65775 7.88 +65775 66.56 +65775 66.68 +65775 98.43 +65776 NULL +65776 18.7 +65776 28.47 +65776 49.73 +65776 98.87 +65777 NULL +65777 54.39 +65777 73.79 +65777 82.62 +65778 NULL +65778 7.37 +65778 51.64 +65778 59.03 +65778 62.17 +65778 64.69 +65778 89.51 +65778 95.69 +65779 NULL +65779 11.87 +65779 28.2 +65779 39.48 +65779 45.61 +65779 64.41 +65779 65.24 +65780 NULL +65780 10.95 +65780 38.58 +65781 NULL +65781 70.59 +65781 95.52 +65782 NULL +65782 30.24 +65782 34.31 +65782 76.14 +65782 81.9 +65783 NULL +65783 46.34 +65783 51.08 +65783 52.43 +65783 62.58 +65783 77.4 +65784 NULL +65784 15.7 +65784 31.35 +65784 68.18 +65784 93.95 +65785 NULL +65785 29.61 +65786 NULL +65786 8.99 +65786 29.32 +65786 66.89 +65786 80.94 +65787 NULL +65787 18.78 +65787 31.19 +65787 64.88 +65788 NULL +65788 16.1 +65788 21.81 +65788 25.77 +65789 NULL +65789 20.44 +65789 43.53 +65789 52.49 +65789 83.18 +65789 92.74 +65789 96.9 +65790 NULL +65790 46.91 +65790 84.87 +65791 NULL +65791 4.24 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c new file mode 100644 index 0000000000000..569c1d4e5f7b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-3-34d9ee4120f21d0d0ae914fba0acc60c @@ -0,0 +1,1049 @@ + 7 + 11 + 38 +alice allen 48 +alice allen 78 +alice allen 79 +alice brown 91 +alice carson 28 +alice davidson 88 +alice falkner 117 +alice garcia 106 +alice hernandez 37 +alice hernandez 85 +alice johnson 65 +alice king 109 +alice king 113 +alice king 118 +alice laertes 6 +alice laertes 65 +alice miller 97 +alice nixon 15 +alice nixon 31 +alice nixon 56 +alice ovid 15 +alice polk 90 +alice quirinius 27 +alice quirinius 89 +alice robinson 5 +alice robinson 68 +alice steinbeck 45 +alice steinbeck 50 +alice steinbeck 77 +alice underhill 34 +alice van buren 31 +alice xylophone 0 +alice xylophone 5 +alice xylophone 113 +alice zipper 19 +alice zipper 51 +alice zipper 74 +bob brown 35 +bob brown 61 +bob brown 71 +bob carson 40 +bob davidson 44 +bob davidson 103 +bob davidson 124 +bob ellison 7 +bob ellison 46 +bob ellison 85 +bob ellison 103 +bob falkner 2 +bob garcia 16 +bob garcia 42 +bob garcia 71 +bob garcia 77 +bob garcia 83 +bob hernandez 93 +bob ichabod 72 +bob king 9 +bob king 22 +bob king 81 +bob laertes -1 +bob laertes 105 +bob miller 31 +bob ovid 26 +bob ovid 27 +bob ovid 81 +bob ovid 86 +bob polk 55 +bob quirinius 26 +bob steinbeck 95 +bob van buren 88 +bob white 1 +bob white 16 +bob xylophone -2 +bob xylophone 49 +bob young -1 +bob zipper 36 +bob zipper 78 +bob zipper 92 +calvin allen 98 +calvin brown 81 +calvin brown 87 +calvin brown 121 +calvin carson 105 +calvin davidson 34 +calvin davidson 89 +calvin ellison 34 +calvin falkner -1 +calvin falkner 12 +calvin falkner 46 +calvin falkner 50 +calvin falkner 53 +calvin falkner 101 +calvin garcia 79 +calvin hernandez 22 +calvin johnson 34 +calvin laertes 37 +calvin laertes 100 +calvin nixon 50 +calvin nixon 71 +calvin nixon 72 +calvin ovid -1 +calvin ovid 50 +calvin ovid 65 +calvin ovid 71 +calvin polk 23 +calvin quirinius 5 +calvin quirinius 96 +calvin robinson 0 +calvin steinbeck 13 +calvin steinbeck 14 +calvin steinbeck 33 +calvin thompson 12 +calvin thompson 40 +calvin underhill 19 +calvin van buren 57 +calvin van buren 78 +calvin white 26 +calvin white 80 +calvin xylophone 17 +calvin xylophone 48 +calvin xylophone 78 +calvin young 8 +calvin young 99 +calvin zipper 31 +calvin zipper 46 +david allen 27 +david allen 80 +david brown 75 +david brown 117 +david davidson 11 +david davidson 38 +david davidson 54 +david davidson 74 +david ellison 50 +david ellison 54 +david ellison 120 +david hernandez 72 +david ichabod 6 +david ichabod 55 +david laertes 88 +david nixon 118 +david ovid 61 +david ovid 95 +david quirinius 43 +david quirinius 85 +david quirinius 121 +david robinson 47 +david robinson 59 +david thompson 89 +david underhill 69 +david underhill 87 +david underhill 98 +david van buren 1 +david van buren 38 +david white 93 +david xylophone 0 +david xylophone 22 +david xylophone 82 +david young 2 +david young 79 +ethan allen 24 +ethan brown 3 +ethan brown 29 +ethan brown 55 +ethan brown 64 +ethan brown 84 +ethan brown 108 +ethan carson 83 +ethan ellison 28 +ethan ellison 65 +ethan falkner 45 +ethan falkner 49 +ethan garcia 16 +ethan hernandez 71 +ethan johnson 108 +ethan king 44 +ethan laertes -1 +ethan laertes 27 +ethan laertes 46 +ethan laertes 68 +ethan laertes 81 +ethan laertes 103 +ethan laertes 114 +ethan miller 40 +ethan nixon 18 +ethan ovid 66 +ethan polk 46 +ethan polk 71 +ethan polk 114 +ethan polk 117 +ethan quirinius 16 +ethan quirinius 41 +ethan quirinius 85 +ethan robinson 32 +ethan robinson 34 +ethan underhill 89 +ethan van buren 43 +ethan white 38 +ethan white 51 +ethan xylophone 107 +ethan zipper 37 +ethan zipper 89 +fred davidson 18 +fred davidson 18 +fred davidson 77 +fred ellison -3 +fred ellison 44 +fred ellison 116 +fred falkner 18 +fred falkner 33 +fred falkner 100 +fred hernandez 36 +fred ichabod 1 +fred ichabod 77 +fred johnson 62 +fred king 33 +fred king 92 +fred laertes 17 +fred miller NULL +fred nixon 33 +fred nixon 36 +fred nixon 102 +fred nixon 111 +fred polk -2 +fred polk 39 +fred polk 60 +fred polk 85 +fred quirinius 25 +fred quirinius 124 +fred robinson 89 +fred steinbeck 79 +fred steinbeck 118 +fred steinbeck 119 +fred underhill 122 +fred van buren 4 +fred van buren 24 +fred van buren 63 +fred van buren 106 +fred white 97 +fred young 33 +fred young 103 +fred zipper 66 +gabriella allen 114 +gabriella allen 119 +gabriella brown 25 +gabriella brown 92 +gabriella carson 112 +gabriella davidson 45 +gabriella ellison 21 +gabriella ellison 101 +gabriella falkner 14 +gabriella falkner 66 +gabriella falkner 77 +gabriella garcia 110 +gabriella hernandez 20 +gabriella hernandez 36 +gabriella ichabod 17 +gabriella ichabod 66 +gabriella ichabod 71 +gabriella ichabod 90 +gabriella ichabod 91 +gabriella king 100 +gabriella king 115 +gabriella laertes 50 +gabriella miller 35 +gabriella ovid 38 +gabriella ovid 89 +gabriella polk 42 +gabriella polk 105 +gabriella steinbeck 18 +gabriella steinbeck 115 +gabriella thompson 45 +gabriella thompson 70 +gabriella thompson 88 +gabriella van buren 5 +gabriella van buren 117 +gabriella white 37 +gabriella young 48 +gabriella young 107 +gabriella zipper 57 +gabriella zipper 71 +holly allen 63 +holly brown 50 +holly brown 117 +holly falkner NULL +holly hernandez 31 +holly hernandez 43 +holly hernandez 48 +holly hernandez 100 +holly ichabod 28 +holly ichabod 53 +holly ichabod 83 +holly johnson 60 +holly johnson 112 +holly johnson 121 +holly king 90 +holly king 114 +holly laertes 13 +holly miller 28 +holly nixon -2 +holly nixon 120 +holly polk 54 +holly polk 124 +holly robinson 1 +holly thompson 28 +holly thompson 33 +holly thompson 83 +holly underhill 5 +holly underhill 31 +holly underhill 101 +holly underhill 113 +holly van buren 19 +holly white 18 +holly white 90 +holly xylophone 18 +holly young 32 +holly young 72 +holly zipper 78 +holly zipper 88 +irene allen 56 +irene brown 78 +irene brown 93 +irene brown 108 +irene carson 0 +irene ellison -3 +irene ellison 40 +irene falkner 13 +irene falkner 104 +irene garcia 34 +irene garcia 62 +irene garcia 124 +irene ichabod 83 +irene ichabod 112 +irene johnson 88 +irene laertes 9 +irene laertes 54 +irene laertes 60 +irene miller 108 +irene nixon -1 +irene nixon 12 +irene nixon 101 +irene ovid 26 +irene ovid 32 +irene ovid 53 +irene polk 18 +irene polk 92 +irene polk 99 +irene polk 109 +irene polk 116 +irene quirinius 7 +irene quirinius 76 +irene quirinius 97 +irene robinson 51 +irene steinbeck 46 +irene thompson 10 +irene underhill 27 +irene underhill 63 +irene van buren 17 +irene van buren 104 +irene xylophone 18 +jessica brown 117 +jessica carson 3 +jessica carson 13 +jessica carson 88 +jessica davidson 11 +jessica davidson 28 +jessica davidson 89 +jessica davidson 124 +jessica ellison 38 +jessica ellison 50 +jessica falkner 71 +jessica garcia 25 +jessica garcia 43 +jessica ichabod 104 +jessica johnson 31 +jessica johnson 69 +jessica miller 74 +jessica nixon 22 +jessica nixon 120 +jessica ovid 47 +jessica ovid 73 +jessica polk 118 +jessica quirinius 0 +jessica quirinius 87 +jessica quirinius 105 +jessica quirinius 114 +jessica robinson 15 +jessica thompson 1 +jessica thompson 77 +jessica underhill 32 +jessica underhill 46 +jessica underhill 83 +jessica van buren 54 +jessica white 5 +jessica white 30 +jessica white 45 +jessica white 65 +jessica white 98 +jessica xylophone 67 +jessica young 61 +jessica young 123 +jessica zipper 27 +jessica zipper 33 +jessica zipper 54 +katie allen 114 +katie brown 39 +katie davidson 35 +katie ellison 5 +katie ellison 58 +katie falkner 15 +katie garcia 49 +katie garcia 65 +katie hernandez 83 +katie ichabod 9 +katie ichabod 75 +katie ichabod 104 +katie king 44 +katie king 59 +katie king 93 +katie miller 23 +katie miller 117 +katie nixon 43 +katie ovid 81 +katie polk 17 +katie polk 85 +katie robinson 92 +katie van buren 25 +katie van buren 88 +katie white 34 +katie white 86 +katie xylophone 84 +katie young 2 +katie young 24 +katie young 70 +katie zipper 25 +katie zipper 87 +luke allen 7 +luke allen 44 +luke allen 62 +luke allen 100 +luke allen 114 +luke brown 112 +luke davidson 51 +luke davidson 84 +luke ellison 35 +luke ellison 40 +luke ellison 86 +luke falkner 59 +luke falkner 97 +luke garcia 51 +luke garcia 100 +luke ichabod 42 +luke ichabod 123 +luke johnson 9 +luke johnson 17 +luke johnson 53 +luke laertes 66 +luke laertes 73 +luke laertes 76 +luke laertes 101 +luke laertes 118 +luke miller 93 +luke ovid 43 +luke ovid 70 +luke polk 53 +luke polk 88 +luke quirinius 82 +luke robinson 0 +luke robinson 114 +luke thompson 51 +luke underhill 2 +luke underhill 109 +luke underhill 119 +luke van buren 43 +luke white 110 +luke xylophone 15 +luke zipper 10 +mike allen 0 +mike brown 88 +mike carson 12 +mike carson 17 +mike carson 122 +mike davidson 9 +mike davidson 110 +mike ellison 5 +mike ellison 50 +mike ellison 70 +mike ellison 94 +mike ellison 95 +mike falkner 61 +mike garcia 2 +mike garcia 68 +mike garcia 110 +mike hernandez 91 +mike hernandez 106 +mike ichabod 18 +mike king 4 +mike king 58 +mike king 83 +mike king 96 +mike king 103 +mike king 118 +mike miller 51 +mike nixon 97 +mike nixon 106 +mike polk 6 +mike polk 65 +mike polk 119 +mike quirinius 22 +mike steinbeck 75 +mike steinbeck 85 +mike steinbeck 101 +mike steinbeck 116 +mike van buren 16 +mike van buren 111 +mike white -1 +mike white 22 +mike white 45 +mike white 61 +mike young 37 +mike young 53 +mike young 72 +mike zipper 27 +mike zipper 76 +mike zipper 106 +nick allen 8 +nick allen 57 +nick brown 114 +nick davidson 84 +nick ellison 10 +nick ellison 107 +nick falkner 83 +nick falkner 86 +nick garcia 53 +nick garcia 69 +nick garcia 108 +nick ichabod 59 +nick ichabod 71 +nick ichabod 84 +nick johnson 47 +nick johnson 88 +nick laertes 17 +nick miller 101 +nick nixon 43 +nick ovid 42 +nick polk 1 +nick quirinius 22 +nick quirinius 36 +nick robinson 48 +nick robinson 54 +nick steinbeck 33 +nick thompson 73 +nick underhill 122 +nick van buren 53 +nick xylophone 80 +nick young 6 +nick young 60 +nick zipper 3 +nick zipper 21 +oscar allen 58 +oscar brown 80 +oscar carson 10 +oscar carson 27 +oscar carson 36 +oscar carson 72 +oscar carson 88 +oscar davidson 14 +oscar ellison 50 +oscar ellison 74 +oscar falkner 96 +oscar garcia 44 +oscar hernandez 1 +oscar hernandez 93 +oscar ichabod 20 +oscar ichabod 28 +oscar ichabod 69 +oscar ichabod 120 +oscar johnson 44 +oscar johnson 53 +oscar king 67 +oscar king 71 +oscar king 81 +oscar laertes 4 +oscar laertes 28 +oscar laertes 53 +oscar laertes 63 +oscar nixon 58 +oscar ovid 3 +oscar ovid 27 +oscar ovid 47 +oscar polk 8 +oscar polk 112 +oscar quirinius NULL +oscar quirinius 0 +oscar quirinius 17 +oscar quirinius 114 +oscar robinson 16 +oscar robinson 42 +oscar robinson 59 +oscar robinson 93 +oscar steinbeck 51 +oscar thompson 44 +oscar thompson 44 +oscar thompson 60 +oscar thompson 66 +oscar underhill 86 +oscar van buren 40 +oscar van buren 51 +oscar van buren 114 +oscar white 2 +oscar white 20 +oscar white 49 +oscar white 58 +oscar xylophone 18 +oscar xylophone 73 +oscar xylophone 74 +oscar zipper 0 +oscar zipper 23 +oscar zipper 95 +priscilla brown 51 +priscilla brown 75 +priscilla brown 97 +priscilla carson 16 +priscilla carson 52 +priscilla carson 124 +priscilla ichabod 117 +priscilla ichabod 122 +priscilla johnson 5 +priscilla johnson 17 +priscilla johnson 62 +priscilla johnson 77 +priscilla johnson 117 +priscilla king 43 +priscilla nixon 61 +priscilla nixon 66 +priscilla ovid 46 +priscilla ovid 118 +priscilla polk 45 +priscilla quirinius 83 +priscilla thompson 82 +priscilla underhill 117 +priscilla underhill 122 +priscilla van buren 0 +priscilla van buren 22 +priscilla van buren 102 +priscilla white 88 +priscilla xylophone 8 +priscilla xylophone 90 +priscilla xylophone 109 +priscilla young 17 +priscilla young 113 +priscilla zipper 27 +priscilla zipper 35 +quinn allen 27 +quinn allen 114 +quinn brown 70 +quinn brown 88 +quinn brown 117 +quinn davidson 93 +quinn davidson 93 +quinn davidson 109 +quinn davidson 121 +quinn ellison 83 +quinn ellison 116 +quinn garcia 78 +quinn garcia 104 +quinn garcia 110 +quinn garcia 120 +quinn ichabod 60 +quinn king 14 +quinn king 46 +quinn laertes -2 +quinn laertes 65 +quinn laertes 95 +quinn nixon 11 +quinn ovid 123 +quinn quirinius 94 +quinn robinson 60 +quinn steinbeck 82 +quinn steinbeck 122 +quinn thompson 41 +quinn thompson 60 +quinn underhill 19 +quinn underhill 28 +quinn underhill 34 +quinn van buren 18 +quinn young 15 +quinn zipper 44 +quinn zipper 103 +rachel allen 76 +rachel allen 122 +rachel brown 23 +rachel brown 56 +rachel brown 71 +rachel brown 101 +rachel brown 108 +rachel carson 27 +rachel carson 74 +rachel davidson 84 +rachel ellison 51 +rachel falkner -2 +rachel falkner 43 +rachel falkner 72 +rachel falkner 104 +rachel johnson 32 +rachel king 84 +rachel king 95 +rachel laertes 37 +rachel laertes 106 +rachel ovid 5 +rachel ovid 31 +rachel polk 79 +rachel quirinius 108 +rachel robinson 24 +rachel robinson 41 +rachel robinson 91 +rachel thompson -3 +rachel thompson -2 +rachel thompson 74 +rachel underhill 11 +rachel white 108 +rachel white 119 +rachel young 77 +rachel zipper 16 +rachel zipper 116 +sarah carson 41 +sarah carson 58 +sarah carson 119 +sarah ellison 14 +sarah falkner 112 +sarah falkner 123 +sarah garcia 72 +sarah garcia 91 +sarah garcia 98 +sarah ichabod 38 +sarah ichabod 80 +sarah johnson 5 +sarah johnson 51 +sarah johnson 69 +sarah johnson 116 +sarah king 13 +sarah king 120 +sarah miller 31 +sarah ovid 122 +sarah robinson 26 +sarah robinson 35 +sarah steinbeck 30 +sarah white 11 +sarah white 32 +sarah xylophone 28 +sarah young 120 +sarah zipper 107 +tom brown 27 +tom brown 89 +tom carson 11 +tom carson 70 +tom carson 123 +tom davidson 72 +tom ellison 28 +tom ellison 118 +tom ellison 120 +tom falkner 11 +tom falkner 35 +tom hernandez -3 +tom hernandez 118 +tom ichabod 19 +tom johnson 42 +tom johnson 82 +tom king 59 +tom laertes 33 +tom laertes 54 +tom miller 9 +tom miller 48 +tom miller 94 +tom nixon 45 +tom ovid 68 +tom polk 70 +tom polk 107 +tom quirinius 10 +tom quirinius 38 +tom robinson 52 +tom robinson 104 +tom robinson 109 +tom robinson 115 +tom steinbeck 113 +tom van buren 5 +tom van buren 48 +tom van buren 63 +tom white 81 +tom young 13 +tom young 99 +tom zipper 31 +ulysses brown 46 +ulysses carson 5 +ulysses carson 26 +ulysses carson 55 +ulysses carson 109 +ulysses davidson 18 +ulysses ellison 61 +ulysses garcia 12 +ulysses hernandez 9 +ulysses hernandez 22 +ulysses hernandez 53 +ulysses ichabod 32 +ulysses ichabod 99 +ulysses johnson 41 +ulysses king 2 +ulysses laertes 40 +ulysses laertes 51 +ulysses laertes 95 +ulysses miller 23 +ulysses miller 85 +ulysses nixon 92 +ulysses ovid 31 +ulysses polk 28 +ulysses polk 74 +ulysses polk 86 +ulysses polk 89 +ulysses quirinius 7 +ulysses robinson 79 +ulysses steinbeck 6 +ulysses steinbeck 45 +ulysses thompson 24 +ulysses underhill 6 +ulysses underhill 27 +ulysses underhill 42 +ulysses underhill 51 +ulysses underhill 93 +ulysses underhill 98 +ulysses underhill 111 +ulysses van buren 58 +ulysses white 67 +ulysses white 109 +ulysses xylophone 47 +ulysses xylophone 105 +ulysses xylophone 123 +ulysses young 61 +ulysses young 86 +ulysses young 89 +victor allen 2 +victor allen 17 +victor brown 0 +victor brown 23 +victor brown 60 +victor brown 64 +victor davidson 42 +victor davidson 89 +victor davidson 123 +victor ellison 35 +victor ellison 84 +victor hernandez 1 +victor hernandez 17 +victor hernandez 91 +victor hernandez 94 +victor hernandez 116 +victor johnson 34 +victor johnson 53 +victor johnson 57 +victor king 59 +victor king 112 +victor laertes 18 +victor laertes 118 +victor miller 79 +victor nixon 50 +victor nixon 104 +victor ovid 120 +victor polk 106 +victor quirinius 77 +victor quirinius 85 +victor robinson 29 +victor robinson 105 +victor steinbeck 20 +victor steinbeck 92 +victor steinbeck 100 +victor thompson 124 +victor van buren 41 +victor van buren 71 +victor white 15 +victor white 49 +victor xylophone -3 +victor xylophone 41 +victor xylophone 43 +victor xylophone 54 +victor xylophone 91 +victor young 24 +victor zipper 3 +wendy allen 25 +wendy allen 38 +wendy allen 95 +wendy brown 92 +wendy brown 119 +wendy ellison 53 +wendy ellison 103 +wendy falkner 23 +wendy falkner 28 +wendy falkner 58 +wendy garcia 3 +wendy garcia 48 +wendy garcia 60 +wendy garcia 99 +wendy hernandez 53 +wendy ichabod 87 +wendy king -2 +wendy king 45 +wendy king 124 +wendy laertes 26 +wendy laertes 51 +wendy laertes 72 +wendy miller 51 +wendy miller 105 +wendy nixon 5 +wendy nixon 25 +wendy ovid 17 +wendy ovid 85 +wendy polk 44 +wendy polk 99 +wendy quirinius 77 +wendy quirinius 88 +wendy robinson -3 +wendy robinson 71 +wendy robinson 97 +wendy steinbeck 37 +wendy thompson 28 +wendy thompson 31 +wendy underhill 58 +wendy underhill 82 +wendy underhill 120 +wendy van buren 27 +wendy van buren 82 +wendy white 63 +wendy xylophone 53 +wendy xylophone 119 +wendy young 66 +wendy young 112 +xavier allen 18 +xavier allen 41 +xavier allen 106 +xavier brown 10 +xavier brown 63 +xavier brown 108 +xavier carson 20 +xavier carson 57 +xavier davidson 21 +xavier davidson 24 +xavier davidson 106 +xavier ellison 0 +xavier ellison 53 +xavier garcia 42 +xavier hernandez 9 +xavier hernandez 80 +xavier hernandez 114 +xavier ichabod 20 +xavier ichabod 58 +xavier johnson 44 +xavier johnson 85 +xavier king 26 +xavier king 107 +xavier laertes 60 +xavier ovid 3 +xavier polk 29 +xavier polk 83 +xavier polk 91 +xavier polk 122 +xavier quirinius 27 +xavier quirinius 35 +xavier quirinius 39 +xavier quirinius 111 +xavier thompson 2 +xavier underhill 102 +xavier white 8 +xavier white 56 +xavier xylophone 24 +xavier zipper 48 +yuri allen 31 +yuri allen 121 +yuri brown 101 +yuri brown 106 +yuri carson 1 +yuri carson 36 +yuri ellison -1 +yuri ellison 43 +yuri falkner 31 +yuri falkner 96 +yuri garcia 49 +yuri hernandez 92 +yuri johnson 1 +yuri johnson 2 +yuri johnson 111 +yuri king 44 +yuri laertes 84 +yuri laertes 115 +yuri nixon 5 +yuri nixon 111 +yuri polk 13 +yuri polk 49 +yuri polk 115 +yuri quirinius 24 +yuri quirinius 28 +yuri quirinius 90 +yuri steinbeck 8 +yuri steinbeck 65 +yuri thompson 42 +yuri underhill 10 +yuri underhill 66 +yuri white 73 +yuri xylophone 63 +zach allen 35 +zach brown 7 +zach brown 15 +zach brown 37 +zach brown 61 +zach brown 94 +zach carson 114 +zach ellison 16 +zach falkner 70 +zach falkner 115 +zach garcia -2 +zach garcia 59 +zach garcia 68 +zach garcia 97 +zach ichabod 14 +zach ichabod 73 +zach king 66 +zach king 70 +zach king 81 +zach miller 4 +zach miller 9 +zach miller 73 +zach ovid 61 +zach ovid 68 +zach ovid 77 +zach ovid 114 +zach quirinius 79 +zach robinson 69 +zach steinbeck 6 +zach steinbeck 122 +zach thompson 75 +zach thompson 95 +zach underhill 123 +zach white 58 +zach xylophone 19 +zach xylophone 85 +zach young 11 +zach zipper 68 +zach zipper 100 +zach zipper 101 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 new file mode 100644 index 0000000000000..86ca4e49d21bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-4-dfd39236756a3951bc1ec354799d69e4 @@ -0,0 +1,1049 @@ + + + +alice allen +alice allen +alice allen +alice brown +alice carson +alice davidson +alice falkner +alice garcia +alice hernandez +alice hernandez +alice johnson +alice king +alice king +alice king +alice laertes +alice laertes +alice miller +alice nixon +alice nixon +alice nixon +alice ovid +alice polk +alice quirinius +alice quirinius +alice robinson +alice robinson +alice steinbeck +alice steinbeck +alice steinbeck +alice underhill +alice van buren +alice xylophone +alice xylophone +alice xylophone +alice zipper +alice zipper +alice zipper +bob brown +bob brown +bob brown +bob carson +bob davidson +bob davidson +bob davidson +bob ellison +bob ellison +bob ellison +bob ellison +bob falkner +bob garcia +bob garcia +bob garcia +bob garcia +bob garcia +bob hernandez +bob ichabod +bob king +bob king +bob king +bob laertes +bob laertes +bob miller +bob ovid +bob ovid +bob ovid +bob ovid +bob polk +bob quirinius +bob steinbeck +bob van buren +bob white +bob white +bob xylophone +bob xylophone +bob young +bob zipper +bob zipper +bob zipper +calvin allen +calvin brown +calvin brown +calvin brown +calvin carson +calvin davidson +calvin davidson +calvin ellison +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin falkner +calvin garcia +calvin hernandez +calvin johnson +calvin laertes +calvin laertes +calvin nixon +calvin nixon +calvin nixon +calvin ovid +calvin ovid +calvin ovid +calvin ovid +calvin polk +calvin quirinius +calvin quirinius +calvin robinson +calvin steinbeck +calvin steinbeck +calvin steinbeck +calvin thompson +calvin thompson +calvin underhill +calvin van buren +calvin van buren +calvin white +calvin white +calvin xylophone +calvin xylophone +calvin xylophone +calvin young +calvin young +calvin zipper +calvin zipper +david allen +david allen +david brown +david brown +david davidson +david davidson +david davidson +david davidson +david ellison +david ellison +david ellison +david hernandez +david ichabod +david ichabod +david laertes +david nixon +david ovid +david ovid +david quirinius +david quirinius +david quirinius +david robinson +david robinson +david thompson +david underhill +david underhill +david underhill +david van buren +david van buren +david white +david xylophone +david xylophone +david xylophone +david young +david young +ethan allen +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan brown +ethan carson +ethan ellison +ethan ellison +ethan falkner +ethan falkner +ethan garcia +ethan hernandez +ethan johnson +ethan king +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan laertes +ethan miller +ethan nixon +ethan ovid +ethan polk +ethan polk +ethan polk +ethan polk +ethan quirinius +ethan quirinius +ethan quirinius +ethan robinson +ethan robinson +ethan underhill +ethan van buren +ethan white +ethan white +ethan xylophone +ethan zipper +ethan zipper +fred davidson +fred davidson +fred davidson +fred ellison +fred ellison +fred ellison +fred falkner +fred falkner +fred falkner +fred hernandez +fred ichabod +fred ichabod +fred johnson +fred king +fred king +fred laertes +fred miller +fred nixon +fred nixon +fred nixon +fred nixon +fred polk +fred polk +fred polk +fred polk +fred quirinius +fred quirinius +fred robinson +fred steinbeck +fred steinbeck +fred steinbeck +fred underhill +fred van buren +fred van buren +fred van buren +fred van buren +fred white +fred young +fred young +fred zipper +gabriella allen +gabriella allen +gabriella brown +gabriella brown +gabriella carson +gabriella davidson +gabriella ellison +gabriella ellison +gabriella falkner +gabriella falkner +gabriella falkner +gabriella garcia +gabriella hernandez +gabriella hernandez +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella ichabod +gabriella king +gabriella king +gabriella laertes +gabriella miller +gabriella ovid +gabriella ovid +gabriella polk +gabriella polk +gabriella steinbeck +gabriella steinbeck +gabriella thompson +gabriella thompson +gabriella thompson +gabriella van buren +gabriella van buren +gabriella white +gabriella young +gabriella young +gabriella zipper +gabriella zipper +holly allen +holly brown +holly brown +holly falkner +holly hernandez +holly hernandez +holly hernandez +holly hernandez +holly ichabod +holly ichabod +holly ichabod +holly johnson +holly johnson +holly johnson +holly king +holly king +holly laertes +holly miller +holly nixon +holly nixon +holly polk +holly polk +holly robinson +holly thompson +holly thompson +holly thompson +holly underhill +holly underhill +holly underhill +holly underhill +holly van buren +holly white +holly white +holly xylophone +holly young +holly young +holly zipper +holly zipper +irene allen +irene brown +irene brown +irene brown +irene carson +irene ellison +irene ellison +irene falkner +irene falkner +irene garcia +irene garcia +irene garcia +irene ichabod +irene ichabod +irene johnson +irene laertes +irene laertes +irene laertes +irene miller +irene nixon +irene nixon +irene nixon +irene ovid +irene ovid +irene ovid +irene polk +irene polk +irene polk +irene polk +irene polk +irene quirinius +irene quirinius +irene quirinius +irene robinson +irene steinbeck +irene thompson +irene underhill +irene underhill +irene van buren +irene van buren +irene xylophone +jessica brown +jessica carson +jessica carson +jessica carson +jessica davidson +jessica davidson +jessica davidson +jessica davidson +jessica ellison +jessica ellison +jessica falkner +jessica garcia +jessica garcia +jessica ichabod +jessica johnson +jessica johnson +jessica miller +jessica nixon +jessica nixon +jessica ovid +jessica ovid +jessica polk +jessica quirinius +jessica quirinius +jessica quirinius +jessica quirinius +jessica robinson +jessica thompson +jessica thompson +jessica underhill +jessica underhill +jessica underhill +jessica van buren +jessica white +jessica white +jessica white +jessica white +jessica white +jessica xylophone +jessica young +jessica young +jessica zipper +jessica zipper +jessica zipper +katie allen +katie brown +katie davidson +katie ellison +katie ellison +katie falkner +katie garcia +katie garcia +katie hernandez +katie ichabod +katie ichabod +katie ichabod +katie king +katie king +katie king +katie miller +katie miller +katie nixon +katie ovid +katie polk +katie polk +katie robinson +katie van buren +katie van buren +katie white +katie white +katie xylophone +katie young +katie young +katie young +katie zipper +katie zipper +luke allen +luke allen +luke allen +luke allen +luke allen +luke brown +luke davidson +luke davidson +luke ellison +luke ellison +luke ellison +luke falkner +luke falkner +luke garcia +luke garcia +luke ichabod +luke ichabod +luke johnson +luke johnson +luke johnson +luke laertes +luke laertes +luke laertes +luke laertes +luke laertes +luke miller +luke ovid +luke ovid +luke polk +luke polk +luke quirinius +luke robinson +luke robinson +luke thompson +luke underhill +luke underhill +luke underhill +luke van buren +luke white +luke xylophone +luke zipper +mike allen +mike brown +mike carson +mike carson +mike carson +mike davidson +mike davidson +mike ellison +mike ellison +mike ellison +mike ellison +mike ellison +mike falkner +mike garcia +mike garcia +mike garcia +mike hernandez +mike hernandez +mike ichabod +mike king +mike king +mike king +mike king +mike king +mike king +mike miller +mike nixon +mike nixon +mike polk +mike polk +mike polk +mike quirinius +mike steinbeck +mike steinbeck +mike steinbeck +mike steinbeck +mike van buren +mike van buren +mike white +mike white +mike white +mike white +mike young +mike young +mike young +mike zipper +mike zipper +mike zipper +nick allen +nick allen +nick brown +nick davidson +nick ellison +nick ellison +nick falkner +nick falkner +nick garcia +nick garcia +nick garcia +nick ichabod +nick ichabod +nick ichabod +nick johnson +nick johnson +nick laertes +nick miller +nick nixon +nick ovid +nick polk +nick quirinius +nick quirinius +nick robinson +nick robinson +nick steinbeck +nick thompson +nick underhill +nick van buren +nick xylophone +nick young +nick young +nick zipper +nick zipper +oscar allen +oscar brown +oscar carson +oscar carson +oscar carson +oscar carson +oscar carson +oscar davidson +oscar ellison +oscar ellison +oscar falkner +oscar garcia +oscar hernandez +oscar hernandez +oscar ichabod +oscar ichabod +oscar ichabod +oscar ichabod +oscar johnson +oscar johnson +oscar king +oscar king +oscar king +oscar laertes +oscar laertes +oscar laertes +oscar laertes +oscar nixon +oscar ovid +oscar ovid +oscar ovid +oscar polk +oscar polk +oscar quirinius +oscar quirinius +oscar quirinius +oscar quirinius +oscar robinson +oscar robinson +oscar robinson +oscar robinson +oscar steinbeck +oscar thompson +oscar thompson +oscar thompson +oscar thompson +oscar underhill +oscar van buren +oscar van buren +oscar van buren +oscar white +oscar white +oscar white +oscar white +oscar xylophone +oscar xylophone +oscar xylophone +oscar zipper +oscar zipper +oscar zipper +priscilla brown +priscilla brown +priscilla brown +priscilla carson +priscilla carson +priscilla carson +priscilla ichabod +priscilla ichabod +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla johnson +priscilla king +priscilla nixon +priscilla nixon +priscilla ovid +priscilla ovid +priscilla polk +priscilla quirinius +priscilla thompson +priscilla underhill +priscilla underhill +priscilla van buren +priscilla van buren +priscilla van buren +priscilla white +priscilla xylophone +priscilla xylophone +priscilla xylophone +priscilla young +priscilla young +priscilla zipper +priscilla zipper +quinn allen +quinn allen +quinn brown +quinn brown +quinn brown +quinn davidson +quinn davidson +quinn davidson +quinn davidson +quinn ellison +quinn ellison +quinn garcia +quinn garcia +quinn garcia +quinn garcia +quinn ichabod +quinn king +quinn king +quinn laertes +quinn laertes +quinn laertes +quinn nixon +quinn ovid +quinn quirinius +quinn robinson +quinn steinbeck +quinn steinbeck +quinn thompson +quinn thompson +quinn underhill +quinn underhill +quinn underhill +quinn van buren +quinn young +quinn zipper +quinn zipper +rachel allen +rachel allen +rachel brown +rachel brown +rachel brown +rachel brown +rachel brown +rachel carson +rachel carson +rachel davidson +rachel ellison +rachel falkner +rachel falkner +rachel falkner +rachel falkner +rachel johnson +rachel king +rachel king +rachel laertes +rachel laertes +rachel ovid +rachel ovid +rachel polk +rachel quirinius +rachel robinson +rachel robinson +rachel robinson +rachel thompson +rachel thompson +rachel thompson +rachel underhill +rachel white +rachel white +rachel young +rachel zipper +rachel zipper +sarah carson +sarah carson +sarah carson +sarah ellison +sarah falkner +sarah falkner +sarah garcia +sarah garcia +sarah garcia +sarah ichabod +sarah ichabod +sarah johnson +sarah johnson +sarah johnson +sarah johnson +sarah king +sarah king +sarah miller +sarah ovid +sarah robinson +sarah robinson +sarah steinbeck +sarah white +sarah white +sarah xylophone +sarah young +sarah zipper +tom brown +tom brown +tom carson +tom carson +tom carson +tom davidson +tom ellison +tom ellison +tom ellison +tom falkner +tom falkner +tom hernandez +tom hernandez +tom ichabod +tom johnson +tom johnson +tom king +tom laertes +tom laertes +tom miller +tom miller +tom miller +tom nixon +tom ovid +tom polk +tom polk +tom quirinius +tom quirinius +tom robinson +tom robinson +tom robinson +tom robinson +tom steinbeck +tom van buren +tom van buren +tom van buren +tom white +tom young +tom young +tom zipper +ulysses brown +ulysses carson +ulysses carson +ulysses carson +ulysses carson +ulysses davidson +ulysses ellison +ulysses garcia +ulysses hernandez +ulysses hernandez +ulysses hernandez +ulysses ichabod +ulysses ichabod +ulysses johnson +ulysses king +ulysses laertes +ulysses laertes +ulysses laertes +ulysses miller +ulysses miller +ulysses nixon +ulysses ovid +ulysses polk +ulysses polk +ulysses polk +ulysses polk +ulysses quirinius +ulysses robinson +ulysses steinbeck +ulysses steinbeck +ulysses thompson +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses underhill +ulysses van buren +ulysses white +ulysses white +ulysses xylophone +ulysses xylophone +ulysses xylophone +ulysses young +ulysses young +ulysses young +victor allen +victor allen +victor brown +victor brown +victor brown +victor brown +victor davidson +victor davidson +victor davidson +victor ellison +victor ellison +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor hernandez +victor johnson +victor johnson +victor johnson +victor king +victor king +victor laertes +victor laertes +victor miller +victor nixon +victor nixon +victor ovid +victor polk +victor quirinius +victor quirinius +victor robinson +victor robinson +victor steinbeck +victor steinbeck +victor steinbeck +victor thompson +victor van buren +victor van buren +victor white +victor white +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor xylophone +victor young +victor zipper +wendy allen +wendy allen +wendy allen +wendy brown +wendy brown +wendy ellison +wendy ellison +wendy falkner +wendy falkner +wendy falkner +wendy garcia +wendy garcia +wendy garcia +wendy garcia +wendy hernandez +wendy ichabod +wendy king +wendy king +wendy king +wendy laertes +wendy laertes +wendy laertes +wendy miller +wendy miller +wendy nixon +wendy nixon +wendy ovid +wendy ovid +wendy polk +wendy polk +wendy quirinius +wendy quirinius +wendy robinson +wendy robinson +wendy robinson +wendy steinbeck +wendy thompson +wendy thompson +wendy underhill +wendy underhill +wendy underhill +wendy van buren +wendy van buren +wendy white +wendy xylophone +wendy xylophone +wendy young +wendy young +xavier allen +xavier allen +xavier allen +xavier brown +xavier brown +xavier brown +xavier carson +xavier carson +xavier davidson +xavier davidson +xavier davidson +xavier ellison +xavier ellison +xavier garcia +xavier hernandez +xavier hernandez +xavier hernandez +xavier ichabod +xavier ichabod +xavier johnson +xavier johnson +xavier king +xavier king +xavier laertes +xavier ovid +xavier polk +xavier polk +xavier polk +xavier polk +xavier quirinius +xavier quirinius +xavier quirinius +xavier quirinius +xavier thompson +xavier underhill +xavier white +xavier white +xavier xylophone +xavier zipper +yuri allen +yuri allen +yuri brown +yuri brown +yuri carson +yuri carson +yuri ellison +yuri ellison +yuri falkner +yuri falkner +yuri garcia +yuri hernandez +yuri johnson +yuri johnson +yuri johnson +yuri king +yuri laertes +yuri laertes +yuri nixon +yuri nixon +yuri polk +yuri polk +yuri polk +yuri quirinius +yuri quirinius +yuri quirinius +yuri steinbeck +yuri steinbeck +yuri thompson +yuri underhill +yuri underhill +yuri white +yuri xylophone +zach allen +zach brown +zach brown +zach brown +zach brown +zach brown +zach carson +zach ellison +zach falkner +zach falkner +zach garcia +zach garcia +zach garcia +zach garcia +zach ichabod +zach ichabod +zach king +zach king +zach king +zach miller +zach miller +zach miller +zach ovid +zach ovid +zach ovid +zach ovid +zach quirinius +zach robinson +zach steinbeck +zach steinbeck +zach thompson +zach thompson +zach underhill +zach white +zach xylophone +zach xylophone +zach young +zach zipper +zach zipper +zach zipper diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc new file mode 100644 index 0000000000000..ddb15e338263f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-5-8d0ee3e1605f38214bfad28a5ce897cc @@ -0,0 +1 @@ +10 oscar carson 65549 65549 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a new file mode 100644 index 0000000000000..42e5151fe211b --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-0-b7cb25303831392a51cd996e758ac79a @@ -0,0 +1,1049 @@ +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65536 1 +65537 1 +65537 1 +65537 1 +65537 1 +65537 1 +65538 1 +65538 1 +65538 1 +65539 1 +65539 1 +65539 1 +65540 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65541 1 +65542 1 +65542 1 +65542 1 +65542 1 +65542 1 +65543 1 +65543 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 1 +65544 2 +65545 1 +65545 1 +65545 1 +65546 2 +65547 1 +65547 1 +65547 1 +65547 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 1 +65548 2 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 1 +65549 3 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 1 +65550 2 +65551 1 +65551 1 +65551 1 +65552 1 +65552 1 +65552 1 +65552 1 +65552 2 +65553 1 +65553 1 +65553 1 +65553 1 +65553 1 +65554 1 +65554 1 +65555 2 +65556 1 +65556 1 +65556 1 +65556 1 +65556 1 +65557 1 +65557 1 +65558 1 +65559 1 +65559 1 +65559 1 +65559 1 +65559 1 +65560 1 +65560 1 +65560 1 +65560 2 +65560 2 +65561 1 +65561 2 +65561 2 +65562 1 +65562 1 +65562 1 +65562 1 +65562 1 +65562 2 +65563 1 +65563 1 +65563 1 +65563 1 +65564 1 +65564 1 +65564 1 +65564 1 +65564 2 +65564 2 +65564 2 +65565 1 +65565 1 +65566 1 +65566 2 +65567 1 +65568 1 +65568 1 +65568 1 +65569 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 1 +65570 2 +65571 1 +65571 1 +65571 1 +65571 2 +65572 1 +65572 1 +65572 1 +65572 1 +65572 1 +65573 1 +65573 1 +65573 3 +65574 1 +65574 1 +65574 1 +65575 1 +65575 1 +65575 1 +65575 2 +65576 1 +65576 1 +65576 1 +65576 3 +65577 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 1 +65578 2 +65579 1 +65579 1 +65579 1 +65579 1 +65579 3 +65580 1 +65580 2 +65580 2 +65581 1 +65581 1 +65581 1 +65581 1 +65581 2 +65581 2 +65582 1 +65582 1 +65582 1 +65582 1 +65583 1 +65583 1 +65583 2 +65583 2 +65584 1 +65584 1 +65584 1 +65585 1 +65585 1 +65585 1 +65585 1 +65585 2 +65586 1 +65586 1 +65586 1 +65586 1 +65586 2 +65587 1 +65587 1 +65587 1 +65587 2 +65587 3 +65588 1 +65588 1 +65588 1 +65588 1 +65588 2 +65589 1 +65589 1 +65589 1 +65589 1 +65589 2 +65590 1 +65590 1 +65590 1 +65590 1 +65590 1 +65591 1 +65591 1 +65591 2 +65591 3 +65592 1 +65593 1 +65593 1 +65594 1 +65594 1 +65594 1 +65594 3 +65595 1 +65595 1 +65595 1 +65595 2 +65595 2 +65595 4 +65596 1 +65596 1 +65596 1 +65596 2 +65596 2 +65597 1 +65597 1 +65597 2 +65598 1 +65598 2 +65599 1 +65599 1 +65599 1 +65599 1 +65599 3 +65600 1 +65600 1 +65600 2 +65600 2 +65601 1 +65601 2 +65601 2 +65602 1 +65602 1 +65602 1 +65602 1 +65602 1 +65602 2 +65603 1 +65603 1 +65603 1 +65603 2 +65603 2 +65604 1 +65604 2 +65604 2 +65604 3 +65605 2 +65606 1 +65606 1 +65606 1 +65606 1 +65606 2 +65606 2 +65607 1 +65607 1 +65607 1 +65607 1 +65607 2 +65607 3 +65608 1 +65608 1 +65608 1 +65608 2 +65609 1 +65610 1 +65610 1 +65610 3 +65610 4 +65610 5 +65611 1 +65611 1 +65611 1 +65611 1 +65612 1 +65612 2 +65612 2 +65612 3 +65613 1 +65614 1 +65614 1 +65614 2 +65615 1 +65615 1 +65615 2 +65615 2 +65616 1 +65616 2 +65617 1 +65617 2 +65617 2 +65617 2 +65618 1 +65618 1 +65618 2 +65618 2 +65618 2 +65619 1 +65619 1 +65619 1 +65619 1 +65619 2 +65619 2 +65620 1 +65620 1 +65620 1 +65620 1 +65620 1 +65620 3 +65621 1 +65621 1 +65622 1 +65622 1 +65622 1 +65622 3 +65622 3 +65622 4 +65623 1 +65623 1 +65623 1 +65623 1 +65623 1 +65623 2 +65623 4 +65624 1 +65624 2 +65624 2 +65624 2 +65624 4 +65625 1 +65625 1 +65625 1 +65625 1 +65626 1 +65626 1 +65626 1 +65626 2 +65627 1 +65627 1 +65627 2 +65627 3 +65628 1 +65628 1 +65628 1 +65628 2 +65628 2 +65628 3 +65629 1 +65629 1 +65629 1 +65629 3 +65630 1 +65630 2 +65631 1 +65632 2 +65632 2 +65633 1 +65633 2 +65633 2 +65633 2 +65634 1 +65634 1 +65634 1 +65634 2 +65635 1 +65635 2 +65635 3 +65636 1 +65636 1 +65636 2 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 1 +65637 2 +65637 3 +65638 1 +65638 1 +65638 2 +65639 1 +65640 2 +65641 1 +65641 1 +65641 1 +65641 5 +65642 4 +65643 1 +65643 1 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 2 +65643 4 +65644 1 +65644 1 +65644 1 +65644 1 +65644 2 +65644 2 +65644 3 +65645 1 +65645 1 +65645 1 +65646 1 +65646 1 +65646 1 +65647 1 +65647 2 +65648 1 +65648 1 +65648 2 +65648 2 +65649 1 +65649 1 +65649 2 +65649 2 +65650 1 +65650 1 +65650 1 +65650 1 +65650 2 +65650 2 +65651 1 +65651 1 +65651 2 +65651 2 +65651 3 +65652 1 +65652 1 +65652 2 +65653 1 +65653 2 +65653 2 +65653 3 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 1 +65654 2 +65654 2 +65654 2 +65655 1 +65655 1 +65656 1 +65656 1 +65656 1 +65656 1 +65656 2 +65656 3 +65657 1 +65657 2 +65657 2 +65657 2 +65657 2 +65658 1 +65658 1 +65658 1 +65658 1 +65658 1 +65658 2 +65658 2 +65658 2 +65658 2 +65658 2 +65659 1 +65659 1 +65659 1 +65659 1 +65659 1 +65659 2 +65659 2 +65659 3 +65660 1 +65660 1 +65660 2 +65661 1 +65661 2 +65661 2 +65661 2 +65661 3 +65662 1 +65662 1 +65662 2 +65662 2 +65662 2 +65662 2 +65663 2 +65663 2 +65663 2 +65663 3 +65664 1 +65664 1 +65664 1 +65664 1 +65664 1 +65664 2 +65664 2 +65664 2 +65665 2 +65666 1 +65666 1 +65666 2 +65667 1 +65667 1 +65667 1 +65667 1 +65668 3 +65669 1 +65669 1 +65669 1 +65669 1 +65669 1 +65669 3 +65670 1 +65670 1 +65670 2 +65670 2 +65670 3 +65671 2 +65671 2 +65671 3 +65672 1 +65672 1 +65672 1 +65672 2 +65672 2 +65673 2 +65673 2 +65673 3 +65673 4 +65673 4 +65674 1 +65675 1 +65675 1 +65675 2 +65675 2 +65675 2 +65675 2 +65675 3 +65675 3 +65676 1 +65676 1 +65676 2 +65676 3 +65677 1 +65677 1 +65677 1 +65677 2 +65677 3 +65677 4 +65678 1 +65678 1 +65678 1 +65679 1 +65679 2 +65680 1 +65680 1 +65680 2 +65680 2 +65680 2 +65680 5 +65681 1 +65681 2 +65681 2 +65681 2 +65681 4 +65681 4 +65682 1 +65682 2 +65682 2 +65683 1 +65683 1 +65683 2 +65684 1 +65684 1 +65684 2 +65685 2 +65685 2 +65685 2 +65685 3 +65685 3 +65685 4 +65686 1 +65686 2 +65686 3 +65686 3 +65687 1 +65687 1 +65687 2 +65687 3 +65688 1 +65688 2 +65689 1 +65689 3 +65690 2 +65690 2 +65691 1 +65691 1 +65691 1 +65691 1 +65691 1 +65691 2 +65691 3 +65691 3 +65692 1 +65692 2 +65693 1 +65693 2 +65693 2 +65693 2 +65693 2 +65693 3 +65693 5 +65694 1 +65694 2 +65694 2 +65694 3 +65695 1 +65695 2 +65695 2 +65695 3 +65696 1 +65696 2 +65696 2 +65696 4 +65697 1 +65697 1 +65697 1 +65697 1 +65697 2 +65697 5 +65698 1 +65698 2 +65699 1 +65699 1 +65699 1 +65699 2 +65699 2 +65699 2 +65700 1 +65700 2 +65700 2 +65701 1 +65701 2 +65701 3 +65702 2 +65702 2 +65702 2 +65702 3 +65703 1 +65703 2 +65703 3 +65703 3 +65704 1 +65704 1 +65704 2 +65704 2 +65704 3 +65704 3 +65705 1 +65705 1 +65705 3 +65705 4 +65706 1 +65706 1 +65706 1 +65706 3 +65706 4 +65707 2 +65707 2 +65708 2 +65708 3 +65709 1 +65709 1 +65709 2 +65710 1 +65710 2 +65711 1 +65711 2 +65711 2 +65711 2 +65711 2 +65711 2 +65712 1 +65712 2 +65712 3 +65712 3 +65712 4 +65713 1 +65713 2 +65713 3 +65713 3 +65713 6 +65714 1 +65714 1 +65714 2 +65715 1 +65715 1 +65715 2 +65715 2 +65715 4 +65716 1 +65716 2 +65716 2 +65716 2 +65716 4 +65716 4 +65717 1 +65717 2 +65717 2 +65717 2 +65717 5 +65718 1 +65718 2 +65718 3 +65718 3 +65719 1 +65719 1 +65719 2 +65719 3 +65720 1 +65720 2 +65720 2 +65720 3 +65720 4 +65721 1 +65721 1 +65721 1 +65721 1 +65721 3 +65721 3 +65721 3 +65722 2 +65722 3 +65722 5 +65723 2 +65723 3 +65724 1 +65724 1 +65724 2 +65724 2 +65724 3 +65724 3 +65725 1 +65726 2 +65726 2 +65726 4 +65727 1 +65727 1 +65727 3 +65727 3 +65727 4 +65728 1 +65728 2 +65729 2 +65730 1 +65730 1 +65730 2 +65730 4 +65731 1 +65731 1 +65731 1 +65731 3 +65732 1 +65732 2 +65732 2 +65733 1 +65733 1 +65733 2 +65733 3 +65733 3 +65733 6 +65734 2 +65734 2 +65735 1 +65735 4 +65735 4 +65736 1 +65736 2 +65736 2 +65736 3 +65737 1 +65737 1 +65737 2 +65737 3 +65737 4 +65738 3 +65738 3 +65738 4 +65738 4 +65739 1 +65739 2 +65739 3 +65740 2 +65740 2 +65740 3 +65741 1 +65742 1 +65742 2 +65742 3 +65743 1 +65743 1 +65743 1 +65743 2 +65744 1 +65744 2 +65745 2 +65745 2 +65745 3 +65745 6 +65746 1 +65746 2 +65746 2 +65746 2 +65746 2 +65747 1 +65747 1 +65747 1 +65747 2 +65747 2 +65747 3 +65747 3 +65748 1 +65748 3 +65749 2 +65749 3 +65749 3 +65749 3 +65749 4 +65750 1 +65750 1 +65750 2 +65750 3 +65750 3 +65751 1 +65751 2 +65751 2 +65751 3 +65751 3 +65751 4 +65752 1 +65752 3 +65753 2 +65753 3 +65754 2 +65754 4 +65755 1 +65755 2 +65755 2 +65755 3 +65755 3 +65755 3 +65755 3 +65755 5 +65755 5 +65756 2 +65756 3 +65756 3 +65756 5 +65757 1 +65757 1 +65757 1 +65757 2 +65757 3 +65758 1 +65758 2 +65758 2 +65758 3 +65758 4 +65759 2 +65759 2 +65759 2 +65759 4 +65759 4 +65760 2 +65760 4 +65760 5 +65760 7 +65761 1 +65762 1 +65762 1 +65762 2 +65762 4 +65762 5 +65763 1 +65763 2 +65763 2 +65763 2 +65763 4 +65764 3 +65764 3 +65764 4 +65765 2 +65765 2 +65765 3 +65766 1 +65766 1 +65766 3 +65766 3 +65766 3 +65767 2 +65767 3 +65767 3 +65768 3 +65769 1 +65769 2 +65769 2 +65769 2 +65769 3 +65769 5 +65770 2 +65770 3 +65771 2 +65771 3 +65771 3 +65772 2 +65773 1 +65773 2 +65773 2 +65773 2 +65773 2 +65773 3 +65773 3 +65773 4 +65774 2 +65774 2 +65774 2 +65774 2 +65774 3 +65774 3 +65775 1 +65775 2 +65775 3 +65775 3 +65775 5 +65776 1 +65776 3 +65776 3 +65776 5 +65776 6 +65777 2 +65777 3 +65777 4 +65777 4 +65778 1 +65778 2 +65778 2 +65778 2 +65778 3 +65778 3 +65778 4 +65778 5 +65779 3 +65779 3 +65779 3 +65779 3 +65779 4 +65779 4 +65779 5 +65780 1 +65780 2 +65780 3 +65781 1 +65781 3 +65781 3 +65782 2 +65782 2 +65782 3 +65782 4 +65782 5 +65783 2 +65783 3 +65783 3 +65783 3 +65783 3 +65783 3 +65784 2 +65784 2 +65784 2 +65784 4 +65784 6 +65785 2 +65785 7 +65786 1 +65786 2 +65786 3 +65786 3 +65786 4 +65787 1 +65787 2 +65787 3 +65787 3 +65788 1 +65788 1 +65788 2 +65788 4 +65789 1 +65789 1 +65789 2 +65789 2 +65789 2 +65789 2 +65789 4 +65790 2 +65790 2 +65790 4 +65791 2 +65791 2 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 new file mode 100644 index 0000000000000..dc72606a83db6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-1-a3d352560ac835993001665db6954965 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 1 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 2 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 2 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 2 +alice zipper 1 +alice zipper 1 +alice zipper 2 +bob brown 1 +bob brown 1 +bob brown 1 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob falkner 2 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 2 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 2 +bob zipper 2 +calvin allen 3 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 2 +calvin davidson 1 +calvin davidson 2 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 2 +calvin falkner 2 +calvin falkner 2 +calvin garcia 2 +calvin hernandez 3 +calvin johnson 1 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 2 +calvin ovid 2 +calvin polk 2 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 2 +calvin thompson 1 +calvin thompson 2 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 2 +calvin xylophone 1 +calvin xylophone 2 +calvin xylophone 2 +calvin young 1 +calvin young 2 +calvin zipper 3 +calvin zipper 4 +david allen 1 +david allen 1 +david brown 2 +david brown 3 +david davidson 1 +david davidson 2 +david davidson 3 +david davidson 3 +david ellison 1 +david ellison 2 +david ellison 3 +david hernandez 1 +david ichabod 1 +david ichabod 3 +david laertes 3 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 3 +david robinson 1 +david robinson 4 +david thompson 1 +david underhill 1 +david underhill 2 +david underhill 3 +david van buren 1 +david van buren 2 +david white 1 +david xylophone 1 +david xylophone 1 +david xylophone 2 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 2 +ethan brown 2 +ethan carson 1 +ethan ellison 1 +ethan ellison 2 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 2 +ethan king 2 +ethan laertes 1 +ethan laertes 1 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 3 +ethan miller 1 +ethan nixon 2 +ethan ovid 2 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 2 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 2 +ethan robinson 1 +ethan robinson 2 +ethan underhill 2 +ethan van buren 1 +ethan white 1 +ethan white 2 +ethan xylophone 2 +ethan zipper 1 +ethan zipper 3 +fred davidson 2 +fred davidson 2 +fred davidson 3 +fred ellison 1 +fred ellison 2 +fred ellison 2 +fred falkner 1 +fred falkner 3 +fred falkner 4 +fred hernandez 2 +fred ichabod 2 +fred ichabod 3 +fred johnson 2 +fred king 2 +fred king 2 +fred laertes 2 +fred miller 3 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 3 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 2 +fred quirinius 2 +fred quirinius 3 +fred robinson 2 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 2 +fred van buren 3 +fred van buren 3 +fred van buren 4 +fred white 2 +fred young 1 +fred young 2 +fred zipper 3 +gabriella allen 1 +gabriella allen 3 +gabriella brown 1 +gabriella brown 1 +gabriella carson 2 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 3 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 3 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 2 +gabriella ichabod 4 +gabriella king 1 +gabriella king 2 +gabriella laertes 1 +gabriella miller 2 +gabriella ovid 1 +gabriella ovid 2 +gabriella polk 1 +gabriella polk 2 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 2 +gabriella thompson 3 +gabriella van buren 1 +gabriella van buren 2 +gabriella white 1 +gabriella young 1 +gabriella young 2 +gabriella zipper 1 +gabriella zipper 2 +holly allen 3 +holly brown 2 +holly brown 2 +holly falkner 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 2 +holly hernandez 3 +holly ichabod 1 +holly ichabod 2 +holly ichabod 2 +holly johnson 1 +holly johnson 3 +holly johnson 4 +holly king 2 +holly king 2 +holly laertes 3 +holly miller 2 +holly nixon 1 +holly nixon 2 +holly polk 1 +holly polk 2 +holly robinson 3 +holly thompson 1 +holly thompson 3 +holly thompson 4 +holly underhill 2 +holly underhill 2 +holly underhill 3 +holly underhill 3 +holly van buren 1 +holly white 4 +holly white 4 +holly xylophone 2 +holly young 1 +holly young 2 +holly zipper 1 +holly zipper 4 +irene allen 3 +irene brown 1 +irene brown 2 +irene brown 3 +irene carson 2 +irene ellison 2 +irene ellison 2 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 2 +irene garcia 3 +irene ichabod 1 +irene ichabod 2 +irene johnson 2 +irene laertes 1 +irene laertes 3 +irene laertes 4 +irene miller 1 +irene nixon 1 +irene nixon 3 +irene nixon 3 +irene ovid 2 +irene ovid 2 +irene ovid 2 +irene polk 1 +irene polk 1 +irene polk 2 +irene polk 2 +irene polk 4 +irene quirinius 2 +irene quirinius 3 +irene quirinius 4 +irene robinson 2 +irene steinbeck 1 +irene thompson 1 +irene underhill 2 +irene underhill 3 +irene van buren 2 +irene van buren 3 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 2 +jessica carson 4 +jessica davidson 1 +jessica davidson 2 +jessica davidson 3 +jessica davidson 3 +jessica ellison 1 +jessica ellison 3 +jessica falkner 2 +jessica garcia 1 +jessica garcia 5 +jessica ichabod 2 +jessica johnson 2 +jessica johnson 3 +jessica miller 2 +jessica nixon 2 +jessica nixon 3 +jessica ovid 2 +jessica ovid 3 +jessica polk 5 +jessica quirinius 2 +jessica quirinius 2 +jessica quirinius 3 +jessica quirinius 3 +jessica robinson 1 +jessica thompson 2 +jessica thompson 3 +jessica underhill 2 +jessica underhill 2 +jessica underhill 4 +jessica van buren 2 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 3 +jessica white 4 +jessica xylophone 4 +jessica young 4 +jessica young 4 +jessica zipper 1 +jessica zipper 2 +jessica zipper 4 +katie allen 2 +katie brown 4 +katie davidson 3 +katie ellison 3 +katie ellison 3 +katie falkner 2 +katie garcia 2 +katie garcia 3 +katie hernandez 2 +katie ichabod 2 +katie ichabod 2 +katie ichabod 2 +katie king 1 +katie king 1 +katie king 2 +katie miller 2 +katie miller 3 +katie nixon 5 +katie ovid 1 +katie polk 2 +katie polk 3 +katie robinson 4 +katie van buren 2 +katie van buren 4 +katie white 1 +katie white 2 +katie xylophone 3 +katie young 2 +katie young 2 +katie young 3 +katie zipper 1 +katie zipper 3 +luke allen 2 +luke allen 2 +luke allen 2 +luke allen 3 +luke allen 3 +luke brown 2 +luke davidson 1 +luke davidson 3 +luke ellison 3 +luke ellison 5 +luke ellison 5 +luke falkner 2 +luke falkner 4 +luke garcia 1 +luke garcia 5 +luke ichabod 3 +luke ichabod 3 +luke johnson 1 +luke johnson 2 +luke johnson 3 +luke laertes 2 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke laertes 3 +luke miller 2 +luke ovid 1 +luke ovid 2 +luke polk 2 +luke polk 3 +luke quirinius 2 +luke robinson 1 +luke robinson 4 +luke thompson 1 +luke underhill 2 +luke underhill 3 +luke underhill 5 +luke van buren 2 +luke white 3 +luke xylophone 2 +luke zipper 1 +mike allen 4 +mike brown 4 +mike carson 1 +mike carson 2 +mike carson 4 +mike davidson 3 +mike davidson 4 +mike ellison 2 +mike ellison 3 +mike ellison 3 +mike ellison 4 +mike ellison 4 +mike falkner 1 +mike garcia 1 +mike garcia 2 +mike garcia 3 +mike hernandez 2 +mike hernandez 3 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 3 +mike king 3 +mike king 4 +mike king 4 +mike miller 4 +mike nixon 3 +mike nixon 4 +mike polk 4 +mike polk 5 +mike polk 5 +mike quirinius 3 +mike steinbeck 2 +mike steinbeck 3 +mike steinbeck 3 +mike steinbeck 4 +mike van buren 2 +mike van buren 3 +mike white 3 +mike white 4 +mike white 5 +mike white 6 +mike young 2 +mike young 2 +mike young 4 +mike zipper 1 +mike zipper 4 +mike zipper 6 +nick allen 2 +nick allen 2 +nick brown 5 +nick davidson 1 +nick ellison 3 +nick ellison 4 +nick falkner 2 +nick falkner 3 +nick garcia 2 +nick garcia 4 +nick garcia 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 3 +nick johnson 4 +nick johnson 4 +nick laertes 2 +nick miller 3 +nick nixon 2 +nick ovid 3 +nick polk 4 +nick quirinius 1 +nick quirinius 3 +nick robinson 4 +nick robinson 4 +nick steinbeck 1 +nick thompson 2 +nick underhill 6 +nick van buren 2 +nick xylophone 3 +nick young 3 +nick young 5 +nick zipper 3 +nick zipper 4 +oscar allen 4 +oscar brown 3 +oscar carson 3 +oscar carson 4 +oscar carson 4 +oscar carson 5 +oscar carson 5 +oscar davidson 4 +oscar ellison 1 +oscar ellison 2 +oscar falkner 1 +oscar garcia 5 +oscar hernandez 1 +oscar hernandez 2 +oscar ichabod 2 +oscar ichabod 5 +oscar ichabod 5 +oscar ichabod 6 +oscar johnson 3 +oscar johnson 7 +oscar king 3 +oscar king 4 +oscar king 4 +oscar laertes 1 +oscar laertes 2 +oscar laertes 3 +oscar laertes 3 +oscar nixon 4 +oscar ovid 3 +oscar ovid 3 +oscar ovid 6 +oscar polk 5 +oscar polk 5 +oscar quirinius 2 +oscar quirinius 4 +oscar quirinius 5 +oscar quirinius 6 +oscar robinson 2 +oscar robinson 3 +oscar robinson 5 +oscar robinson 6 +oscar steinbeck 1 +oscar thompson 2 +oscar thompson 3 +oscar thompson 3 +oscar thompson 4 +oscar underhill 2 +oscar van buren 1 +oscar van buren 2 +oscar van buren 5 +oscar white 1 +oscar white 2 +oscar white 5 +oscar white 5 +oscar xylophone 3 +oscar xylophone 3 +oscar xylophone 4 +oscar zipper 2 +oscar zipper 2 +oscar zipper 2 +priscilla brown 2 +priscilla brown 2 +priscilla brown 4 +priscilla carson 3 +priscilla carson 5 +priscilla carson 7 +priscilla ichabod 1 +priscilla ichabod 4 +priscilla johnson 1 +priscilla johnson 2 +priscilla johnson 4 +priscilla johnson 4 +priscilla johnson 6 +priscilla king 3 +priscilla nixon 3 +priscilla nixon 6 +priscilla ovid 3 +priscilla ovid 7 +priscilla polk 4 +priscilla quirinius 3 +priscilla thompson 6 +priscilla underhill 1 +priscilla underhill 4 +priscilla van buren 3 +priscilla van buren 5 +priscilla van buren 5 +priscilla white 1 +priscilla xylophone 2 +priscilla xylophone 3 +priscilla xylophone 3 +priscilla young 5 +priscilla young 8 +priscilla zipper 3 +priscilla zipper 3 +quinn allen 1 +quinn allen 4 +quinn brown 3 +quinn brown 4 +quinn brown 4 +quinn davidson 2 +quinn davidson 4 +quinn davidson 6 +quinn davidson 7 +quinn ellison 3 +quinn ellison 8 +quinn garcia 2 +quinn garcia 3 +quinn garcia 3 +quinn garcia 5 +quinn ichabod 7 +quinn king 1 +quinn king 1 +quinn laertes 2 +quinn laertes 4 +quinn laertes 5 +quinn nixon 4 +quinn ovid 4 +quinn quirinius 5 +quinn robinson 3 +quinn steinbeck 4 +quinn steinbeck 5 +quinn thompson 4 +quinn thompson 6 +quinn underhill 2 +quinn underhill 3 +quinn underhill 7 +quinn van buren 1 +quinn young 2 +quinn zipper 3 +quinn zipper 4 +rachel allen 2 +rachel allen 3 +rachel brown 2 +rachel brown 3 +rachel brown 4 +rachel brown 4 +rachel brown 5 +rachel carson 2 +rachel carson 4 +rachel davidson 7 +rachel ellison 1 +rachel falkner 1 +rachel falkner 3 +rachel falkner 5 +rachel falkner 6 +rachel johnson 9 +rachel king 3 +rachel king 7 +rachel laertes 4 +rachel laertes 6 +rachel ovid 3 +rachel ovid 4 +rachel polk 3 +rachel quirinius 4 +rachel robinson 4 +rachel robinson 4 +rachel robinson 6 +rachel thompson 4 +rachel thompson 5 +rachel thompson 5 +rachel underhill 2 +rachel white 4 +rachel white 5 +rachel young 4 +rachel zipper 1 +rachel zipper 5 +sarah carson 1 +sarah carson 4 +sarah carson 7 +sarah ellison 1 +sarah falkner 4 +sarah falkner 5 +sarah garcia 2 +sarah garcia 2 +sarah garcia 4 +sarah ichabod 3 +sarah ichabod 3 +sarah johnson 3 +sarah johnson 5 +sarah johnson 5 +sarah johnson 6 +sarah king 3 +sarah king 5 +sarah miller 2 +sarah ovid 5 +sarah robinson 5 +sarah robinson 5 +sarah steinbeck 5 +sarah white 4 +sarah white 6 +sarah xylophone 3 +sarah young 5 +sarah zipper 6 +tom brown 2 +tom brown 5 +tom carson 1 +tom carson 3 +tom carson 5 +tom davidson 2 +tom ellison 3 +tom ellison 4 +tom ellison 6 +tom falkner 3 +tom falkner 4 +tom hernandez 1 +tom hernandez 3 +tom ichabod 4 +tom johnson 6 +tom johnson 7 +tom king 3 +tom laertes 3 +tom laertes 3 +tom miller 3 +tom miller 4 +tom miller 5 +tom nixon 4 +tom ovid 3 +tom polk 2 +tom polk 2 +tom quirinius 3 +tom quirinius 5 +tom robinson 2 +tom robinson 3 +tom robinson 3 +tom robinson 5 +tom steinbeck 2 +tom van buren 2 +tom van buren 3 +tom van buren 6 +tom white 5 +tom young 1 +tom young 5 +tom zipper 7 +ulysses brown 2 +ulysses carson 2 +ulysses carson 5 +ulysses carson 6 +ulysses carson 8 +ulysses davidson 3 +ulysses ellison 4 +ulysses garcia 3 +ulysses hernandez 3 +ulysses hernandez 3 +ulysses hernandez 4 +ulysses ichabod 1 +ulysses ichabod 3 +ulysses johnson 5 +ulysses king 2 +ulysses laertes 2 +ulysses laertes 5 +ulysses laertes 6 +ulysses miller 5 +ulysses miller 7 +ulysses nixon 4 +ulysses ovid 4 +ulysses polk 2 +ulysses polk 2 +ulysses polk 3 +ulysses polk 4 +ulysses quirinius 4 +ulysses robinson 1 +ulysses steinbeck 2 +ulysses steinbeck 5 +ulysses thompson 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 3 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 4 +ulysses underhill 5 +ulysses van buren 2 +ulysses white 6 +ulysses white 7 +ulysses xylophone 2 +ulysses xylophone 3 +ulysses xylophone 6 +ulysses young 1 +ulysses young 4 +ulysses young 7 +victor allen 2 +victor allen 3 +victor brown 1 +victor brown 4 +victor brown 5 +victor brown 7 +victor davidson 4 +victor davidson 4 +victor davidson 6 +victor ellison 4 +victor ellison 4 +victor hernandez 2 +victor hernandez 3 +victor hernandez 4 +victor hernandez 4 +victor hernandez 6 +victor johnson 4 +victor johnson 5 +victor johnson 6 +victor king 2 +victor king 6 +victor laertes 3 +victor laertes 5 +victor miller 5 +victor nixon 2 +victor nixon 3 +victor ovid 3 +victor polk 4 +victor quirinius 5 +victor quirinius 5 +victor robinson 5 +victor robinson 5 +victor steinbeck 3 +victor steinbeck 4 +victor steinbeck 5 +victor thompson 6 +victor van buren 5 +victor van buren 6 +victor white 2 +victor white 7 +victor xylophone 4 +victor xylophone 6 +victor xylophone 6 +victor xylophone 8 +victor xylophone 8 +victor young 5 +victor zipper 3 +wendy allen 5 +wendy allen 6 +wendy allen 6 +wendy brown 3 +wendy brown 5 +wendy ellison 3 +wendy ellison 5 +wendy falkner 2 +wendy falkner 4 +wendy falkner 6 +wendy garcia 4 +wendy garcia 4 +wendy garcia 7 +wendy garcia 7 +wendy hernandez 4 +wendy ichabod 4 +wendy king 4 +wendy king 5 +wendy king 7 +wendy laertes 2 +wendy laertes 3 +wendy laertes 5 +wendy miller 4 +wendy miller 4 +wendy nixon 3 +wendy nixon 5 +wendy ovid 5 +wendy ovid 9 +wendy polk 2 +wendy polk 5 +wendy quirinius 3 +wendy quirinius 4 +wendy robinson 5 +wendy robinson 6 +wendy robinson 6 +wendy steinbeck 3 +wendy thompson 4 +wendy thompson 5 +wendy underhill 4 +wendy underhill 5 +wendy underhill 6 +wendy van buren 6 +wendy van buren 6 +wendy white 4 +wendy xylophone 4 +wendy xylophone 6 +wendy young 1 +wendy young 6 +xavier allen 3 +xavier allen 4 +xavier allen 5 +xavier brown 2 +xavier brown 4 +xavier brown 6 +xavier carson 4 +xavier carson 5 +xavier davidson 7 +xavier davidson 8 +xavier davidson 9 +xavier ellison 8 +xavier ellison 8 +xavier garcia 5 +xavier hernandez 5 +xavier hernandez 6 +xavier hernandez 9 +xavier ichabod 3 +xavier ichabod 4 +xavier johnson 2 +xavier johnson 9 +xavier king 3 +xavier king 5 +xavier laertes 4 +xavier ovid 4 +xavier polk 3 +xavier polk 4 +xavier polk 4 +xavier polk 8 +xavier quirinius 3 +xavier quirinius 5 +xavier quirinius 6 +xavier quirinius 6 +xavier thompson 4 +xavier underhill 2 +xavier white 3 +xavier white 3 +xavier xylophone 4 +xavier zipper 3 +yuri allen 2 +yuri allen 3 +yuri brown 2 +yuri brown 3 +yuri carson 5 +yuri carson 6 +yuri ellison 6 +yuri ellison 6 +yuri falkner 6 +yuri falkner 10 +yuri garcia 1 +yuri hernandez 5 +yuri johnson 5 +yuri johnson 5 +yuri johnson 6 +yuri king 7 +yuri laertes 7 +yuri laertes 8 +yuri nixon 3 +yuri nixon 3 +yuri polk 3 +yuri polk 5 +yuri polk 6 +yuri quirinius 3 +yuri quirinius 4 +yuri quirinius 7 +yuri steinbeck 1 +yuri steinbeck 2 +yuri thompson 3 +yuri underhill 4 +yuri underhill 4 +yuri white 8 +yuri xylophone 3 +zach allen 4 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 5 +zach brown 7 +zach carson 5 +zach ellison 2 +zach falkner 4 +zach falkner 6 +zach garcia 4 +zach garcia 5 +zach garcia 7 +zach garcia 8 +zach ichabod 4 +zach ichabod 4 +zach king 4 +zach king 5 +zach king 8 +zach miller 1 +zach miller 3 +zach miller 4 +zach ovid 4 +zach ovid 5 +zach ovid 5 +zach ovid 7 +zach quirinius 8 +zach robinson 5 +zach steinbeck 4 +zach steinbeck 6 +zach thompson 3 +zach thompson 4 +zach underhill 3 +zach white 6 +zach xylophone 3 +zach xylophone 5 +zach young 4 +zach zipper 4 +zach zipper 4 +zach zipper 5 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 new file mode 100644 index 0000000000000..76cbeb254c0e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-2-fafa16c0f7697ca28aeb6f2698799562 @@ -0,0 +1,1049 @@ +0.08 1 +0.1 1 +0.13 1 +0.15 1 +0.27 1 +0.28 1 +0.43 1 +0.52 1 +0.56 1 +0.6 1 +0.61 1 +0.79 1 +0.84 1 +0.98 1 +1.02 1 +1.08 1 +1.08 1 +1.12 1 +1.21 1 +1.25 1 +1.27 1 +1.29 1 +1.31 1 +1.58 1 +1.87 1 +1.91 1 +1.92 1 +2.07 1 +2.18 1 +2.2 1 +2.35 1 +2.6 1 +2.79 1 +2.92 1 +2.96 1 +2.96 1 +2.97 1 +3.0 1 +3.21 1 +3.28 1 +3.33 1 +3.61 1 +3.62 1 +3.82 1 +3.86 1 +3.96 1 +3.97 1 +4.17 1 +4.32 1 +4.35 1 +4.41 1 +4.46 1 +4.47 1 +4.57 1 +4.59 1 +4.71 1 +4.72 1 +4.79 1 +4.8 1 +4.92 1 +5.08 1 +5.24 1 +5.28 1 +5.4 1 +5.44 1 +5.45 1 +5.51 1 +5.54 1 +5.62 1 +5.67 1 +5.85 1 +5.88 1 +6.29 1 +6.55 1 +6.57 1 +6.63 1 +6.67 1 +6.72 1 +6.74 1 +6.84 1 +6.87 1 +7.05 1 +7.06 1 +7.11 1 +7.54 1 +7.56 1 +7.79 1 +7.82 1 +7.96 1 +7.96 1 +7.98 1 +8.07 1 +8.07 1 +8.32 1 +8.37 1 +8.42 1 +8.45 1 +8.45 1 +8.45 1 +8.45 1 +8.57 1 +8.61 1 +8.67 2 +8.71 1 +8.79 1 +8.91 1 +9.04 1 +9.13 1 +9.19 1 +9.22 1 +9.25 1 +9.26 1 +9.35 1 +9.48 1 +9.56 1 +9.57 1 +9.57 1 +9.68 1 +9.7 1 +9.71 1 +9.74 1 +9.8 1 +9.81 1 +9.93 1 +10.09 1 +10.09 1 +10.13 1 +10.16 1 +10.17 1 +10.19 1 +10.2 1 +10.22 1 +10.25 1 +10.26 1 +10.29 1 +10.6 1 +10.66 1 +10.67 1 +10.73 1 +11.15 1 +11.18 1 +11.19 1 +11.22 1 +11.34 1 +11.55 1 +11.57 1 +11.68 1 +11.82 1 +11.89 1 +11.91 1 +12.02 1 +12.16 1 +12.19 1 +12.32 1 +12.42 1 +12.44 1 +12.45 1 +12.46 1 +12.5 1 +12.54 1 +12.85 1 +12.9 1 +13.01 1 +13.1 1 +13.15 1 +13.35 1 +13.87 1 +13.89 1 +13.94 1 +13.99 1 +14.13 1 +14.21 2 +14.3 1 +14.44 1 +14.84 2 +14.92 1 +14.92 1 +14.93 1 +15.1 1 +15.15 1 +15.18 1 +15.22 1 +15.26 2 +15.3 1 +15.37 1 +15.45 1 +15.63 2 +15.75 1 +15.81 1 +15.86 1 +15.9 1 +15.92 1 +16.08 1 +16.09 1 +16.24 1 +16.25 1 +16.48 1 +16.69 1 +16.99 1 +16.99 1 +17.16 1 +17.37 1 +17.74 1 +17.79 1 +17.87 1 +18.2 1 +18.5 1 +18.56 1 +18.63 1 +18.63 1 +18.86 1 +18.89 1 +18.93 1 +19.0 1 +19.03 1 +19.06 1 +19.06 1 +19.13 1 +19.14 1 +19.28 1 +19.69 1 +20.07 1 +20.38 1 +20.64 1 +20.67 1 +20.79 1 +20.81 1 +20.82 1 +20.82 1 +21.18 1 +21.19 1 +21.23 1 +21.28 1 +21.32 1 +21.45 1 +21.49 1 +21.61 1 +21.7 1 +21.8 1 +21.94 1 +22.01 1 +22.08 1 +22.12 1 +22.12 1 +22.25 1 +22.27 1 +22.36 1 +22.68 1 +22.78 1 +22.85 1 +22.85 1 +22.94 1 +23.07 1 +23.13 1 +23.17 1 +23.19 1 +23.44 1 +23.45 1 +23.6 1 +23.77 1 +23.96 1 +24.02 1 +24.28 1 +24.49 1 +24.52 1 +24.73 1 +24.79 1 +24.8 1 +24.83 1 +24.86 1 +25.11 1 +25.28 1 +25.37 1 +25.42 1 +25.55 1 +25.67 1 +25.88 1 +26.08 1 +26.39 1 +26.43 1 +26.47 1 +26.49 1 +26.49 1 +26.64 1 +26.71 2 +26.73 1 +26.76 1 +27.07 1 +27.12 1 +27.3 1 +27.31 1 +27.63 1 +27.66 1 +27.72 2 +27.87 1 +28.11 1 +28.31 1 +28.45 1 +28.5 1 +28.56 1 +28.69 1 +28.71 1 +28.79 1 +28.89 1 +28.95 1 +29.02 1 +29.24 1 +29.36 1 +29.4 1 +29.41 2 +29.54 1 +29.59 1 +29.78 1 +30.25 2 +30.36 1 +30.37 1 +30.61 1 +30.62 1 +30.63 1 +30.65 1 +30.71 1 +30.81 1 +31.01 1 +31.15 1 +31.4 1 +31.61 1 +31.67 1 +31.77 1 +31.86 1 +31.91 1 +32.01 1 +32.18 1 +32.2 1 +32.23 1 +32.25 1 +32.37 1 +32.41 1 +32.47 1 +32.52 1 +32.75 1 +32.89 2 +32.92 1 +33.36 1 +33.52 1 +33.55 1 +33.58 1 +33.67 1 +33.76 1 +33.83 1 +33.85 2 +33.87 1 +34.03 1 +34.21 1 +34.35 1 +34.41 2 +34.58 1 +34.73 1 +34.97 1 +35.0 2 +35.08 1 +35.13 1 +35.17 1 +35.17 1 +35.56 1 +35.62 1 +35.65 1 +35.68 1 +35.72 1 +35.8 1 +35.89 1 +36.22 1 +36.26 1 +36.58 1 +36.7 1 +36.79 1 +36.89 1 +36.95 1 +37.07 2 +37.1 1 +37.14 1 +37.14 1 +37.24 1 +37.59 1 +37.6 1 +37.72 2 +37.78 1 +37.8 1 +37.85 1 +37.9 1 +38.05 1 +38.05 1 +38.3 2 +38.33 1 +38.57 1 +38.62 1 +38.79 1 +38.85 1 +38.88 1 +38.94 1 +39.01 1 +39.03 1 +39.18 1 +39.21 1 +39.34 1 +39.69 1 +39.81 1 +39.82 1 +39.83 1 +39.87 1 +39.9 1 +39.98 1 +40.0 1 +40.04 1 +40.17 1 +40.24 1 +40.42 1 +40.44 1 +40.78 1 +40.8 1 +40.98 1 +41.2 2 +41.29 1 +41.29 1 +41.31 2 +41.34 1 +41.34 1 +41.36 1 +41.44 1 +41.45 2 +41.62 1 +41.68 1 +41.71 1 +41.81 1 +41.85 1 +41.87 1 +41.89 2 +42.24 1 +42.31 1 +42.42 2 +42.48 1 +42.51 1 +42.55 1 +42.56 1 +42.67 2 +42.76 1 +42.85 1 +43.01 1 +43.02 1 +43.13 1 +43.16 1 +43.17 1 +43.19 1 +43.31 1 +43.37 1 +43.57 1 +43.71 1 +43.73 1 +43.92 1 +44.1 1 +44.22 1 +44.27 1 +44.43 1 +44.57 1 +45.06 2 +45.1 1 +45.19 1 +45.19 1 +45.24 1 +45.34 1 +45.35 1 +45.42 1 +45.45 1 +45.56 1 +45.59 1 +45.68 1 +45.92 1 +45.99 2 +46.02 1 +46.09 1 +46.1 1 +46.15 1 +46.18 1 +46.21 1 +46.27 1 +46.43 1 +46.45 1 +46.62 1 +46.8 1 +46.86 1 +46.87 1 +46.88 1 +46.97 2 +47.08 2 +47.27 1 +47.32 2 +47.57 1 +47.59 1 +47.69 1 +47.88 1 +48.01 1 +48.08 1 +48.11 1 +48.15 1 +48.22 1 +48.23 1 +48.25 1 +48.28 1 +48.37 1 +48.45 1 +48.45 1 +48.52 1 +48.59 1 +49.12 1 +49.28 2 +49.44 1 +49.68 1 +49.77 2 +49.78 1 +50.02 3 +50.08 1 +50.09 1 +50.26 1 +50.28 1 +50.31 1 +50.32 1 +50.4 1 +50.41 1 +50.66 1 +50.7 1 +50.83 2 +50.92 2 +50.96 1 +51.25 1 +51.29 1 +51.29 2 +51.72 1 +51.79 2 +51.84 1 +51.85 1 +52.17 1 +52.23 1 +52.44 1 +52.5 1 +52.53 2 +52.72 1 +52.73 1 +52.85 2 +52.87 1 +53.02 1 +53.06 1 +53.18 2 +53.27 1 +53.59 2 +53.78 1 +53.93 1 +53.94 1 +54.1 2 +54.31 1 +54.34 1 +54.43 1 +54.44 1 +54.47 1 +54.73 1 +54.75 1 +54.83 1 +54.99 1 +55.1 1 +55.18 1 +55.2 1 +55.39 1 +55.51 1 +55.63 1 +55.99 1 +56.04 1 +56.07 1 +56.1 1 +56.15 1 +56.33 1 +56.62 1 +56.68 2 +56.81 1 +57.08 1 +57.11 1 +57.12 2 +57.23 1 +57.25 1 +57.29 1 +57.35 1 +57.37 1 +57.46 1 +57.64 1 +57.67 1 +57.89 2 +57.93 1 +58.0 1 +58.08 2 +58.09 2 +58.13 1 +58.43 1 +58.52 1 +58.66 1 +58.67 1 +58.75 1 +58.86 2 +59.07 1 +59.16 1 +59.21 1 +59.34 1 +59.43 1 +59.45 1 +59.45 2 +59.5 1 +59.55 1 +59.61 1 +59.62 1 +59.68 1 +59.68 1 +59.7 1 +59.71 1 +59.83 1 +59.87 1 +59.99 1 +60.02 1 +60.06 1 +60.12 1 +60.13 1 +60.22 2 +60.26 1 +60.26 1 +60.53 1 +60.6 2 +60.71 1 +60.85 1 +61.21 1 +61.7 1 +61.86 1 +61.88 1 +61.92 2 +61.94 1 +62.14 1 +62.2 1 +62.23 1 +62.3 1 +62.39 1 +62.52 1 +62.72 1 +62.74 1 +62.85 2 +62.9 3 +62.92 1 +63.12 1 +63.33 1 +63.35 2 +63.42 1 +63.51 1 +63.9 1 +64.0 1 +64.22 1 +64.25 1 +64.3 1 +64.36 2 +64.46 1 +64.65 2 +64.67 1 +64.77 1 +64.87 1 +64.95 1 +65.02 1 +65.02 1 +65.38 1 +65.43 1 +65.43 1 +65.44 2 +65.55 1 +65.62 1 +65.7 1 +65.72 1 +66.17 1 +66.17 2 +66.36 1 +66.51 1 +66.61 1 +66.61 1 +66.67 1 +66.89 1 +67.12 1 +67.18 1 +67.26 1 +67.38 1 +67.45 1 +67.48 1 +67.59 1 +67.94 1 +67.98 1 +68.01 2 +68.04 1 +68.22 1 +68.25 1 +68.25 1 +68.32 1 +68.41 1 +68.5 1 +68.81 1 +68.85 2 +68.89 1 +68.95 1 +68.96 1 +69.32 2 +69.53 1 +69.74 3 +69.8 2 +69.88 1 +69.96 1 +69.97 1 +70.0 2 +70.04 1 +70.06 1 +70.24 1 +70.35 1 +70.38 1 +70.39 1 +70.52 1 +70.53 1 +70.56 1 +70.85 1 +70.89 1 +70.93 1 +71.01 1 +71.07 2 +71.13 1 +71.19 1 +71.26 1 +71.31 1 +71.32 1 +71.35 1 +71.5 1 +71.54 1 +71.55 3 +71.68 1 +71.68 2 +71.78 2 +71.8 1 +71.89 2 +72.04 1 +72.18 1 +72.51 1 +72.53 2 +72.56 1 +72.62 1 +72.79 1 +72.98 1 +73.18 1 +73.32 1 +73.48 2 +73.63 2 +73.65 1 +73.68 1 +73.88 1 +73.93 1 +74.0 1 +74.02 1 +74.15 1 +74.19 1 +74.19 1 +74.3 1 +74.42 1 +74.45 1 +74.52 1 +74.53 1 +74.59 1 +74.62 1 +74.72 1 +74.78 1 +75.03 2 +75.1 3 +75.19 1 +75.29 1 +75.35 2 +75.42 2 +75.66 2 +75.73 2 +75.83 1 +75.88 1 +76.05 3 +76.1 1 +76.28 1 +76.28 1 +76.33 1 +76.52 1 +76.69 1 +76.7 1 +76.71 1 +76.72 2 +76.72 2 +76.74 1 +76.92 1 +76.93 1 +77.02 3 +77.1 1 +77.36 2 +77.42 1 +77.57 2 +77.66 1 +77.81 1 +77.84 2 +77.89 1 +77.97 1 +78.21 2 +78.26 1 +78.28 1 +78.3 2 +78.31 1 +78.62 1 +78.64 1 +78.73 1 +78.89 1 +78.98 2 +79.12 1 +79.19 1 +79.21 1 +79.38 1 +79.42 1 +79.48 1 +79.48 2 +79.49 2 +79.54 1 +79.55 2 +79.75 1 +79.83 1 +79.96 1 +79.97 2 +79.99 1 +80.23 2 +80.3 1 +80.3 2 +80.46 1 +80.52 1 +80.58 1 +80.6 1 +80.71 1 +80.74 1 +80.84 2 +80.92 1 +80.96 1 +80.97 1 +80.99 1 +81.17 1 +81.32 1 +81.32 1 +81.47 1 +81.58 1 +81.64 1 +81.66 1 +82.24 1 +82.3 1 +82.34 2 +82.41 1 +82.52 1 +82.55 1 +82.56 1 +82.72 1 +82.97 1 +83.08 1 +83.27 1 +83.33 1 +83.4 1 +83.54 2 +83.57 1 +83.58 1 +83.87 1 +83.92 1 +83.93 1 +84.03 1 +84.23 2 +84.31 1 +84.38 1 +84.4 1 +84.69 1 +84.72 1 +84.83 1 +85.0 1 +85.03 2 +85.1 1 +85.14 1 +85.23 1 +85.49 1 +85.49 2 +85.51 2 +85.74 1 +85.76 1 +85.87 1 +85.9 1 +86.0 1 +86.22 1 +86.23 1 +86.63 1 +86.69 1 +86.92 2 +86.93 1 +86.93 1 +87.14 2 +87.22 1 +87.4 1 +87.48 1 +87.57 1 +87.61 1 +87.67 1 +87.83 2 +87.94 1 +87.99 1 +88.02 1 +88.05 1 +88.07 2 +88.17 1 +88.22 1 +88.36 1 +88.47 1 +88.48 1 +88.55 1 +88.77 1 +88.78 1 +88.8 1 +88.91 2 +89.01 2 +89.03 1 +89.1 3 +89.15 2 +89.28 1 +89.38 1 +89.53 1 +89.55 1 +89.55 1 +89.55 1 +89.8 1 +89.81 1 +89.93 1 +90.05 1 +90.05 1 +90.07 1 +90.12 1 +90.2 1 +90.25 1 +90.28 2 +90.35 1 +90.38 1 +90.51 1 +90.56 2 +90.69 1 +90.69 1 +90.73 1 +90.77 1 +91.05 1 +91.16 1 +91.42 1 +91.48 1 +91.53 1 +91.61 1 +91.63 1 +91.78 1 +91.88 1 +91.97 1 +92.05 2 +92.11 2 +92.33 2 +92.37 1 +92.4 2 +92.55 1 +92.61 1 +92.82 1 +92.96 2 +92.98 1 +93.03 1 +93.09 1 +93.11 1 +93.61 1 +93.64 1 +93.73 1 +94.08 1 +94.15 1 +94.25 1 +94.27 1 +94.31 1 +94.33 1 +94.34 1 +94.38 1 +94.43 1 +94.54 1 +94.66 1 +94.68 1 +94.68 1 +94.72 1 +95.07 1 +95.11 1 +95.28 1 +95.33 1 +95.34 1 +95.38 2 +95.48 1 +95.53 1 +95.53 1 +95.81 1 +95.81 2 +95.84 1 +96.09 1 +96.23 1 +96.25 1 +96.29 1 +96.38 1 +96.62 1 +96.68 1 +96.73 1 +96.78 1 +96.91 2 +96.94 1 +97.09 1 +97.24 1 +97.26 1 +97.39 1 +97.46 1 +97.51 2 +97.56 1 +97.57 1 +97.65 2 +97.68 1 +97.71 4 +97.81 1 +97.83 1 +97.87 1 +98.18 2 +98.22 1 +98.23 1 +98.31 1 +98.48 1 +98.51 1 +98.57 1 +98.72 1 +98.96 1 +99.13 1 +99.15 1 +99.21 1 +99.24 1 +99.29 1 +99.36 1 +99.62 1 +99.65 1 +99.67 1 +99.68 1 +99.91 1 +99.92 1 diff --git a/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d new file mode 100644 index 0000000000000..a9ec53c0cb21c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_ntile.q (deterministic)-3-bda0e7c77d6f4712a03389cb5032bc6d @@ -0,0 +1,1049 @@ +0.02 1 +0.21 1 +0.27 1 +0.37 1 +0.37 1 +0.47 1 +0.48 1 +0.52 1 +0.6 1 +0.63 1 +0.63 1 +0.66 1 +0.73 1 +0.74 1 +0.74 1 +0.78 1 +0.8 1 +0.86 1 +0.86 1 +0.98 1 +1.17 1 +1.29 1 +1.3 1 +1.31 1 +1.37 1 +1.42 1 +1.45 1 +1.5 1 +1.53 1 +1.61 1 +1.62 1 +1.69 1 +1.71 1 +1.79 1 +1.98 1 +2.07 1 +2.09 1 +2.13 1 +2.16 1 +2.26 1 +2.29 1 +2.34 1 +2.34 1 +2.36 1 +2.43 1 +2.43 1 +2.52 1 +2.53 1 +2.53 1 +2.61 1 +2.63 1 +2.74 1 +2.75 1 +2.75 1 +2.79 1 +2.8 1 +2.82 1 +2.89 1 +2.89 1 +2.89 1 +2.9 1 +2.92 1 +3.03 1 +3.07 1 +3.07 1 +3.12 1 +3.15 1 +3.17 1 +3.27 1 +3.27 1 +3.29 1 +3.4 1 +3.4 1 +3.4 1 +3.4 1 +3.42 1 +3.66 1 +3.67 1 +3.69 1 +3.71 1 +3.78 1 +3.83 1 +3.86 1 +3.92 1 +3.98 1 +3.99 1 +4.04 1 +4.16 1 +4.25 1 +4.27 1 +4.44 1 +4.48 1 +4.53 1 +4.57 1 +4.58 1 +4.62 1 +4.81 1 +4.82 1 +4.83 1 +4.92 1 +4.95 1 +4.96 1 +4.97 1 +4.98 1 +5.09 1 +5.11 1 +5.19 1 +5.23 1 +5.3 1 +5.31 1 +5.31 1 +5.35 1 +5.42 1 +5.51 1 +5.55 1 +5.58 1 +5.74 1 +5.82 1 +5.84 1 +5.93 1 +5.93 1 +5.96 1 +6.06 1 +6.06 1 +6.21 1 +6.28 1 +6.46 1 +6.52 2 +6.54 1 +6.56 1 +6.57 1 +6.58 1 +6.61 1 +6.62 1 +6.76 1 +6.81 1 +6.81 1 +6.96 1 +6.98 1 +7.02 1 +7.03 1 +7.14 1 +7.18 1 +7.24 1 +7.24 1 +7.31 1 +7.36 1 +7.37 1 +7.45 1 +7.53 1 +7.62 1 +7.66 1 +7.71 1 +7.71 1 +7.8 1 +7.92 1 +8.05 1 +8.09 1 +8.21 1 +8.33 1 +8.33 1 +8.49 1 +8.49 1 +8.52 1 +8.56 1 +8.61 1 +8.62 2 +8.72 1 +8.76 1 +8.79 1 +8.82 1 +8.84 1 +8.95 1 +8.98 1 +9.14 1 +9.19 1 +9.21 1 +9.22 1 +9.26 1 +9.27 1 +9.39 2 +9.4 1 +9.42 1 +9.5 1 +9.51 1 +9.56 1 +9.6 1 +9.61 1 +9.62 1 +9.64 1 +9.81 1 +9.87 1 +9.88 1 +9.93 1 +9.94 1 +9.96 1 +9.99 1 +10.15 1 +10.21 1 +10.22 1 +10.23 1 +10.24 1 +10.36 1 +10.38 1 +10.38 1 +10.41 1 +10.47 1 +10.49 1 +10.49 1 +10.51 1 +10.52 1 +10.7 1 +10.71 1 +10.85 1 +10.99 1 +11.02 1 +11.12 1 +11.12 1 +11.16 1 +11.2 1 +11.26 1 +11.27 1 +11.35 1 +11.35 1 +11.4 1 +11.43 1 +11.44 1 +11.44 1 +11.46 1 +11.48 1 +11.5 1 +11.54 1 +11.63 1 +11.66 1 +11.69 1 +11.83 1 +11.9 1 +11.91 1 +11.96 1 +12.02 1 +12.13 1 +12.14 1 +12.15 1 +12.3 1 +12.3 1 +12.3 2 +12.34 1 +12.35 1 +12.43 1 +12.43 1 +12.64 1 +12.66 1 +12.7 1 +12.72 1 +12.73 1 +12.74 2 +12.82 1 +12.85 1 +13.02 1 +13.04 1 +13.08 1 +13.14 1 +13.2 1 +13.2 1 +13.22 1 +13.23 1 +13.3 1 +13.3 1 +13.44 1 +13.44 1 +13.44 1 +13.49 1 +13.6 1 +13.66 1 +13.71 1 +13.72 1 +13.8 1 +13.83 1 +13.84 1 +13.88 1 +13.95 1 +14.07 1 +14.16 1 +14.17 1 +14.22 1 +14.24 1 +14.26 1 +14.29 1 +14.33 1 +14.39 1 +14.44 1 +14.51 1 +14.51 1 +14.52 1 +14.62 1 +14.69 1 +14.72 1 +14.75 1 +14.83 1 +14.83 1 +14.84 1 +14.9 1 +14.91 1 +14.92 1 +14.99 1 +15.0 1 +15.01 1 +15.09 1 +15.09 1 +15.09 1 +15.1 1 +15.12 1 +15.13 1 +15.16 1 +15.18 1 +15.22 1 +15.27 1 +15.28 1 +15.32 1 +15.38 1 +15.46 1 +15.46 1 +15.51 1 +15.54 1 +15.87 1 +15.94 1 +15.97 1 +15.98 1 +16.04 2 +16.1 1 +16.12 1 +16.13 1 +16.15 1 +16.29 1 +16.35 1 +16.36 1 +16.38 1 +16.4 1 +16.42 1 +16.47 1 +16.49 1 +16.54 1 +16.61 1 +16.66 1 +16.79 1 +16.79 1 +16.82 1 +16.87 1 +16.87 1 +16.9 1 +16.9 1 +16.91 1 +16.92 1 +17.03 1 +17.03 2 +17.08 1 +17.15 1 +17.19 1 +17.29 1 +17.33 1 +17.44 1 +17.46 1 +17.47 1 +17.51 1 +17.52 2 +17.55 1 +17.59 1 +17.63 1 +17.69 1 +17.76 1 +17.86 1 +17.89 1 +17.99 1 +18.09 1 +18.19 1 +18.2 1 +18.28 1 +18.29 1 +18.31 1 +18.34 1 +18.35 1 +18.36 1 +18.38 1 +18.38 1 +18.41 1 +18.47 1 +18.48 1 +18.79 1 +18.82 1 +18.83 1 +18.86 1 +18.86 1 +19.03 1 +19.12 1 +19.15 1 +19.2 1 +19.31 1 +19.32 1 +19.41 1 +19.47 1 +19.47 2 +19.56 1 +19.59 1 +19.63 1 +19.65 1 +19.72 1 +19.72 1 +19.79 1 +19.79 1 +19.85 1 +19.87 1 +19.9 1 +19.92 1 +19.93 1 +19.98 1 +20.02 1 +20.02 1 +20.17 1 +20.19 1 +20.22 1 +20.3 1 +20.3 1 +20.34 1 +20.39 1 +20.42 1 +20.42 1 +20.44 1 +20.55 1 +20.55 1 +20.56 1 +20.56 1 +20.58 1 +20.58 1 +20.64 1 +20.65 1 +20.75 1 +20.76 1 +20.76 1 +20.8 1 +20.82 1 +20.91 1 +20.93 1 +20.94 1 +20.94 1 +20.97 1 +21.0 1 +21.01 1 +21.01 1 +21.02 1 +21.02 1 +21.11 1 +21.11 1 +21.14 1 +21.16 1 +21.26 1 +21.27 1 +21.3 1 +21.3 1 +21.33 1 +21.33 1 +21.38 1 +21.42 1 +21.52 1 +21.53 1 +21.57 1 +21.66 1 +21.67 1 +21.69 1 +21.77 1 +21.81 1 +21.86 1 +21.91 1 +21.95 1 +22.15 1 +22.19 1 +22.19 1 +22.22 1 +22.27 1 +22.27 1 +22.33 1 +22.48 1 +22.6 1 +22.61 1 +22.64 1 +22.68 1 +22.73 1 +22.75 1 +22.94 1 +22.95 1 +23.03 1 +23.07 1 +23.15 1 +23.15 1 +23.18 1 +23.18 1 +23.25 1 +23.27 1 +23.3 1 +23.31 1 +23.45 1 +23.48 1 +23.53 1 +23.55 1 +23.59 1 +23.61 1 +23.63 1 +23.73 1 +23.77 1 +23.78 2 +23.88 1 +23.91 1 +24.03 1 +24.03 1 +24.13 1 +24.17 1 +24.18 1 +24.25 1 +24.35 1 +24.35 2 +24.42 1 +24.46 1 +24.53 2 +24.59 1 +24.61 1 +24.61 1 +24.84 1 +24.85 1 +24.86 1 +24.95 1 +25.01 1 +25.02 1 +25.03 1 +25.08 1 +25.11 1 +25.17 1 +25.32 2 +25.36 1 +25.36 1 +25.43 1 +25.49 1 +25.51 1 +25.51 1 +25.58 1 +25.59 1 +25.63 1 +25.71 1 +25.75 1 +25.8 1 +25.92 1 +25.92 1 +25.95 1 +25.97 2 +26.0 1 +26.17 2 +26.21 1 +26.22 1 +26.24 1 +26.28 1 +26.44 1 +26.55 1 +26.55 1 +26.65 1 +26.67 1 +26.71 1 +26.71 1 +26.73 1 +26.74 1 +26.79 1 +26.84 1 +26.87 1 +27.02 1 +27.14 1 +27.2 1 +27.27 1 +27.29 1 +27.36 1 +27.39 1 +27.4 1 +27.42 1 +27.46 1 +27.54 1 +27.54 1 +27.61 1 +27.62 1 +27.89 1 +28.02 1 +28.1 1 +28.13 1 +28.14 1 +28.15 1 +28.17 1 +28.19 1 +28.29 1 +28.36 1 +28.4 1 +28.42 2 +28.44 1 +28.52 1 +28.52 1 +28.61 1 +28.64 1 +28.68 1 +28.69 1 +28.69 1 +28.71 1 +28.71 2 +28.77 1 +28.77 1 +28.85 1 +28.86 1 +28.91 1 +28.96 1 +28.96 1 +28.98 1 +29.0 1 +29.11 1 +29.19 1 +29.22 1 +29.24 1 +29.25 1 +29.36 1 +29.41 1 +29.46 1 +29.49 1 +29.52 2 +29.62 1 +29.63 1 +29.66 1 +29.73 1 +29.76 1 +29.78 1 +29.88 1 +29.96 1 +30.04 1 +30.04 1 +30.09 1 +30.12 1 +30.13 1 +30.16 1 +30.17 1 +30.21 1 +30.22 1 +30.28 1 +30.35 1 +30.37 2 +30.41 1 +30.41 1 +30.49 1 +30.55 1 +30.55 1 +30.58 1 +30.61 1 +30.62 1 +30.66 1 +30.67 1 +30.78 1 +30.78 1 +30.87 1 +30.89 1 +30.9 1 +30.92 1 +30.98 2 +30.99 1 +31.01 1 +31.16 1 +31.23 1 +31.3 1 +31.33 1 +31.36 1 +31.45 1 +31.46 1 +31.5 1 +31.61 1 +31.63 1 +31.64 1 +31.66 1 +31.68 1 +31.74 1 +31.75 1 +31.76 1 +31.84 1 +32.04 1 +32.06 1 +32.13 1 +32.17 1 +32.2 1 +32.25 1 +32.25 1 +32.31 1 +32.33 1 +32.39 1 +32.56 1 +32.56 1 +32.61 1 +32.74 1 +32.85 1 +32.89 1 +32.98 1 +33.0 1 +33.01 1 +33.02 1 +33.02 1 +33.02 1 +33.11 1 +33.12 1 +33.18 1 +33.18 1 +33.19 1 +33.24 1 +33.3 1 +33.36 1 +33.36 1 +33.38 1 +33.49 1 +33.52 2 +33.6 2 +33.64 1 +33.64 1 +33.66 1 +33.67 1 +33.72 1 +33.76 1 +33.9 1 +34.05 1 +34.06 1 +34.11 1 +34.14 1 +34.15 1 +34.17 1 +34.2 1 +34.41 2 +34.48 2 +34.49 1 +34.52 1 +34.53 1 +34.54 1 +34.62 1 +34.68 1 +34.68 1 +34.72 1 +34.81 1 +34.83 1 +34.84 1 +34.9 1 +34.95 1 +34.95 1 +34.97 1 +34.97 1 +34.98 1 +35.01 1 +35.02 1 +35.1 1 +35.15 1 +35.16 1 +35.23 1 +35.24 1 +35.36 1 +35.36 1 +35.49 1 +35.62 1 +35.68 1 +35.72 1 +35.84 1 +35.85 1 +35.9 1 +36.05 2 +36.09 1 +36.11 1 +36.12 1 +36.13 1 +36.22 1 +36.56 1 +36.57 1 +36.57 1 +36.62 1 +36.7 1 +36.72 1 +36.73 1 +36.84 1 +36.86 1 +36.93 1 +36.93 1 +37.02 1 +37.08 1 +37.12 1 +37.23 2 +37.32 1 +37.34 1 +37.37 1 +37.76 1 +37.77 1 +37.8 1 +37.93 1 +37.94 1 +37.96 1 +38.0 1 +38.04 1 +38.04 1 +38.04 1 +38.05 1 +38.07 1 +38.14 1 +38.22 1 +38.28 1 +38.37 2 +38.39 1 +38.43 1 +38.53 1 +38.6 1 +38.62 1 +38.66 1 +38.67 1 +38.73 1 +38.74 1 +38.92 1 +38.94 1 +39.01 1 +39.04 1 +39.05 1 +39.11 1 +39.27 1 +39.29 1 +39.41 1 +39.46 1 +39.49 1 +39.55 1 +39.57 1 +39.6 1 +39.63 1 +39.73 1 +39.74 1 +39.78 2 +39.8 1 +39.84 1 +39.84 1 +39.85 1 +39.92 1 +40.01 1 +40.04 1 +40.15 1 +40.21 2 +40.22 1 +40.24 1 +40.26 1 +40.27 1 +40.39 2 +40.43 1 +40.46 1 +40.5 1 +40.59 1 +40.6 1 +40.63 1 +40.76 1 +40.79 1 +40.84 1 +40.89 1 +40.91 1 +40.94 1 +40.96 1 +41.02 1 +41.08 1 +41.2 1 +41.24 1 +41.33 1 +41.34 1 +41.34 2 +41.36 1 +41.36 1 +41.37 1 +41.54 1 +41.56 1 +41.69 1 +41.73 1 +41.75 1 +41.75 1 +41.83 1 +41.86 1 +41.9 1 +42.0 2 +42.02 1 +42.03 1 +42.04 1 +42.22 1 +42.3 1 +42.37 1 +42.41 2 +42.47 1 +42.55 1 +42.56 1 +42.57 1 +42.76 1 +42.89 1 +42.96 1 +42.96 1 +43.0 1 +43.04 1 +43.04 3 +43.18 1 +43.23 1 +43.3 1 +43.34 1 +43.34 2 +43.4 1 +43.42 1 +43.58 1 +43.64 1 +43.67 1 +43.76 2 +43.84 1 +43.85 1 +43.92 1 +43.95 1 +43.95 1 +43.96 1 +43.96 1 +44.04 1 +44.11 1 +44.12 1 +44.12 1 +44.22 3 +44.24 1 +44.27 1 +44.32 1 +44.36 1 +44.4 1 +44.57 1 +44.6 1 +44.63 1 +44.66 1 +44.73 1 +44.75 1 +44.8 1 +44.83 1 +44.9 1 +44.92 1 +44.93 1 +45.0 1 +45.02 1 +45.06 1 +45.06 1 +45.09 1 +45.1 1 +45.1 1 +45.11 1 +45.12 2 +45.14 1 +45.14 2 +45.28 2 +45.29 1 +45.46 1 +45.49 1 +45.53 1 +45.53 2 +45.54 1 +45.69 1 +45.71 1 +45.78 1 +45.81 2 +45.86 1 +45.9 1 +45.94 1 +46.03 1 +46.03 1 +46.09 1 +46.18 1 +46.19 1 +46.28 1 +46.3 1 +46.31 1 +46.33 1 +46.36 1 +46.39 1 +46.52 1 +46.53 1 +46.54 1 +46.57 1 +46.59 1 +46.67 1 +46.69 1 +46.73 1 +46.73 1 +46.73 2 +46.74 1 +46.81 1 +46.87 1 +46.88 1 +46.9 1 +46.93 1 +46.98 1 +47.0 1 +47.03 1 +47.03 1 +47.06 1 +47.15 1 +47.22 1 +47.3 1 +47.31 2 +47.37 2 +47.4 1 +47.46 1 +47.49 1 +47.55 1 +47.6 1 +47.66 1 +47.68 1 +47.71 1 +47.72 1 +47.82 1 +47.86 2 +47.91 1 +47.91 2 +47.95 1 +47.98 1 +48.0 1 +48.08 1 +48.5 1 +48.52 1 +48.71 1 +48.78 1 +48.8 1 +48.85 1 +48.89 1 +48.96 1 +48.98 1 +49.04 1 +49.05 1 +49.16 1 +49.21 1 +49.32 1 +49.34 1 +49.34 1 +49.38 1 +49.44 2 +49.45 1 +49.45 1 +49.46 1 +49.46 1 +49.52 1 +49.56 1 +49.59 1 +49.63 1 +49.67 1 +49.69 1 +49.71 2 +49.72 1 +49.73 1 +49.79 1 +49.84 1 +49.85 2 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 new file mode 100644 index 0000000000000..612bdf44c0cdf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-0-2e0cbc2d7c5f16657edacd9e7209e6e7 @@ -0,0 +1,1049 @@ + 1 + 1 + 1 +alice allen 1 +alice allen 1 +alice allen 1 +alice brown 1 +alice carson 1 +alice davidson 1 +alice falkner 1 +alice garcia 2 +alice hernandez 1 +alice hernandez 1 +alice johnson 1 +alice king 1 +alice king 1 +alice king 1 +alice laertes 1 +alice laertes 1 +alice miller 1 +alice nixon 1 +alice nixon 1 +alice nixon 1 +alice ovid 1 +alice polk 1 +alice quirinius 1 +alice quirinius 1 +alice robinson 1 +alice robinson 1 +alice steinbeck 1 +alice steinbeck 1 +alice steinbeck 1 +alice underhill 1 +alice van buren 1 +alice xylophone 1 +alice xylophone 1 +alice xylophone 1 +alice zipper 1 +alice zipper 1 +alice zipper 1 +bob brown 1 +bob brown 1 +bob brown 2 +bob carson 1 +bob davidson 1 +bob davidson 1 +bob davidson 1 +bob ellison 1 +bob ellison 1 +bob ellison 1 +bob ellison 2 +bob falkner 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 1 +bob garcia 2 +bob hernandez 1 +bob ichabod 1 +bob king 1 +bob king 1 +bob king 1 +bob laertes 1 +bob laertes 1 +bob miller 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob ovid 1 +bob polk 1 +bob quirinius 1 +bob steinbeck 1 +bob van buren 1 +bob white 1 +bob white 1 +bob xylophone 1 +bob xylophone 1 +bob young 1 +bob zipper 1 +bob zipper 1 +bob zipper 1 +calvin allen 1 +calvin brown 1 +calvin brown 1 +calvin brown 1 +calvin carson 1 +calvin davidson 1 +calvin davidson 1 +calvin ellison 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin falkner 1 +calvin garcia 1 +calvin hernandez 1 +calvin johnson 2 +calvin laertes 1 +calvin laertes 1 +calvin nixon 1 +calvin nixon 1 +calvin nixon 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin ovid 1 +calvin polk 1 +calvin quirinius 1 +calvin quirinius 1 +calvin robinson 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin steinbeck 1 +calvin thompson 1 +calvin thompson 1 +calvin underhill 1 +calvin van buren 1 +calvin van buren 1 +calvin white 1 +calvin white 1 +calvin xylophone 1 +calvin xylophone 1 +calvin xylophone 1 +calvin young 1 +calvin young 1 +calvin zipper 1 +calvin zipper 1 +david allen 1 +david allen 1 +david brown 1 +david brown 1 +david davidson 1 +david davidson 1 +david davidson 1 +david davidson 2 +david ellison 1 +david ellison 1 +david ellison 1 +david hernandez 1 +david ichabod 1 +david ichabod 1 +david laertes 1 +david nixon 1 +david ovid 1 +david ovid 1 +david quirinius 1 +david quirinius 1 +david quirinius 1 +david robinson 1 +david robinson 1 +david thompson 1 +david underhill 1 +david underhill 1 +david underhill 1 +david van buren 1 +david van buren 1 +david white 2 +david xylophone 1 +david xylophone 1 +david xylophone 1 +david young 1 +david young 1 +ethan allen 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan carson 1 +ethan ellison 1 +ethan ellison 1 +ethan falkner 1 +ethan falkner 1 +ethan garcia 1 +ethan hernandez 1 +ethan johnson 1 +ethan king 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan laertes 1 +ethan miller 1 +ethan nixon 1 +ethan ovid 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan quirinius 1 +ethan quirinius 1 +ethan quirinius 1 +ethan robinson 1 +ethan robinson 1 +ethan underhill 1 +ethan van buren 1 +ethan white 1 +ethan white 1 +ethan xylophone 1 +ethan zipper 1 +ethan zipper 1 +fred davidson 1 +fred davidson 1 +fred davidson 1 +fred ellison 1 +fred ellison 1 +fred ellison 1 +fred falkner 1 +fred falkner 1 +fred falkner 1 +fred hernandez 1 +fred ichabod 1 +fred ichabod 2 +fred johnson 1 +fred king 1 +fred king 1 +fred laertes 1 +fred miller 1 +fred nixon 1 +fred nixon 1 +fred nixon 1 +fred nixon 2 +fred polk 1 +fred polk 1 +fred polk 1 +fred polk 1 +fred quirinius 1 +fred quirinius 1 +fred robinson 1 +fred steinbeck 1 +fred steinbeck 1 +fred steinbeck 1 +fred underhill 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred white 1 +fred young 1 +fred young 1 +fred zipper 1 +gabriella allen 1 +gabriella allen 1 +gabriella brown 1 +gabriella brown 1 +gabriella carson 1 +gabriella davidson 1 +gabriella ellison 1 +gabriella ellison 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella falkner 1 +gabriella garcia 1 +gabriella hernandez 1 +gabriella hernandez 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella ichabod 1 +gabriella king 1 +gabriella king 1 +gabriella laertes 1 +gabriella miller 1 +gabriella ovid 1 +gabriella ovid 1 +gabriella polk 1 +gabriella polk 1 +gabriella steinbeck 1 +gabriella steinbeck 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella thompson 1 +gabriella van buren 1 +gabriella van buren 1 +gabriella white 1 +gabriella young 1 +gabriella young 1 +gabriella zipper 1 +gabriella zipper 1 +holly allen 1 +holly brown 1 +holly brown 1 +holly falkner 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 1 +holly hernandez 2 +holly ichabod 1 +holly ichabod 1 +holly ichabod 1 +holly johnson 1 +holly johnson 1 +holly johnson 1 +holly king 1 +holly king 1 +holly laertes 1 +holly miller 1 +holly nixon 1 +holly nixon 1 +holly polk 1 +holly polk 1 +holly robinson 1 +holly thompson 1 +holly thompson 1 +holly thompson 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly underhill 1 +holly van buren 1 +holly white 1 +holly white 2 +holly xylophone 1 +holly young 1 +holly young 1 +holly zipper 1 +holly zipper 1 +irene allen 1 +irene brown 1 +irene brown 1 +irene brown 1 +irene carson 1 +irene ellison 1 +irene ellison 1 +irene falkner 1 +irene falkner 1 +irene garcia 1 +irene garcia 1 +irene garcia 1 +irene ichabod 1 +irene ichabod 1 +irene johnson 1 +irene laertes 1 +irene laertes 1 +irene laertes 1 +irene miller 1 +irene nixon 1 +irene nixon 1 +irene nixon 1 +irene ovid 1 +irene ovid 1 +irene ovid 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene polk 1 +irene quirinius 1 +irene quirinius 1 +irene quirinius 1 +irene robinson 1 +irene steinbeck 1 +irene thompson 1 +irene underhill 1 +irene underhill 1 +irene van buren 1 +irene van buren 1 +irene xylophone 2 +jessica brown 2 +jessica carson 1 +jessica carson 1 +jessica carson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica davidson 1 +jessica ellison 1 +jessica ellison 1 +jessica falkner 1 +jessica garcia 1 +jessica garcia 1 +jessica ichabod 1 +jessica johnson 1 +jessica johnson 1 +jessica miller 1 +jessica nixon 1 +jessica nixon 1 +jessica ovid 1 +jessica ovid 2 +jessica polk 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica quirinius 1 +jessica robinson 1 +jessica thompson 1 +jessica thompson 3 +jessica underhill 1 +jessica underhill 1 +jessica underhill 1 +jessica van buren 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica white 1 +jessica xylophone 1 +jessica young 1 +jessica young 1 +jessica zipper 1 +jessica zipper 1 +jessica zipper 1 +katie allen 1 +katie brown 1 +katie davidson 1 +katie ellison 1 +katie ellison 1 +katie falkner 1 +katie garcia 1 +katie garcia 1 +katie hernandez 1 +katie ichabod 1 +katie ichabod 1 +katie ichabod 1 +katie king 1 +katie king 1 +katie king 1 +katie miller 1 +katie miller 1 +katie nixon 1 +katie ovid 1 +katie polk 1 +katie polk 1 +katie robinson 1 +katie van buren 1 +katie van buren 1 +katie white 1 +katie white 1 +katie xylophone 1 +katie young 1 +katie young 1 +katie young 1 +katie zipper 1 +katie zipper 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 1 +luke allen 2 +luke brown 1 +luke davidson 1 +luke davidson 1 +luke ellison 1 +luke ellison 1 +luke ellison 1 +luke falkner 1 +luke falkner 1 +luke garcia 1 +luke garcia 1 +luke ichabod 1 +luke ichabod 1 +luke johnson 1 +luke johnson 1 +luke johnson 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke laertes 1 +luke miller 1 +luke ovid 1 +luke ovid 1 +luke polk 1 +luke polk 1 +luke quirinius 1 +luke robinson 1 +luke robinson 1 +luke thompson 1 +luke underhill 1 +luke underhill 1 +luke underhill 2 +luke van buren 1 +luke white 1 +luke xylophone 1 +luke zipper 1 +mike allen 1 +mike brown 1 +mike carson 1 +mike carson 1 +mike carson 1 +mike davidson 1 +mike davidson 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike ellison 1 +mike falkner 1 +mike garcia 1 +mike garcia 1 +mike garcia 1 +mike hernandez 1 +mike hernandez 2 +mike ichabod 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 1 +mike king 2 +mike miller 1 +mike nixon 1 +mike nixon 1 +mike polk 1 +mike polk 1 +mike polk 1 +mike quirinius 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike steinbeck 1 +mike van buren 1 +mike van buren 1 +mike white 1 +mike white 1 +mike white 1 +mike white 2 +mike young 1 +mike young 1 +mike young 1 +mike zipper 1 +mike zipper 1 +mike zipper 1 +nick allen 1 +nick allen 1 +nick brown 1 +nick davidson 1 +nick ellison 1 +nick ellison 1 +nick falkner 1 +nick falkner 1 +nick garcia 1 +nick garcia 1 +nick garcia 1 +nick ichabod 1 +nick ichabod 1 +nick ichabod 1 +nick johnson 1 +nick johnson 1 +nick laertes 1 +nick miller 1 +nick nixon 1 +nick ovid 1 +nick polk 1 +nick quirinius 1 +nick quirinius 1 +nick robinson 1 +nick robinson 1 +nick steinbeck 1 +nick thompson 1 +nick underhill 1 +nick van buren 1 +nick xylophone 1 +nick young 1 +nick young 1 +nick zipper 1 +nick zipper 1 +oscar allen 2 +oscar brown 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar carson 1 +oscar davidson 1 +oscar ellison 1 +oscar ellison 1 +oscar falkner 1 +oscar garcia 1 +oscar hernandez 1 +oscar hernandez 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar ichabod 1 +oscar johnson 1 +oscar johnson 1 +oscar king 1 +oscar king 1 +oscar king 2 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar laertes 1 +oscar nixon 1 +oscar ovid 1 +oscar ovid 1 +oscar ovid 2 +oscar polk 1 +oscar polk 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar quirinius 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar robinson 1 +oscar steinbeck 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 1 +oscar thompson 2 +oscar underhill 1 +oscar van buren 1 +oscar van buren 1 +oscar van buren 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar white 1 +oscar xylophone 1 +oscar xylophone 1 +oscar xylophone 1 +oscar zipper 1 +oscar zipper 1 +oscar zipper 1 +priscilla brown 1 +priscilla brown 1 +priscilla brown 1 +priscilla carson 1 +priscilla carson 1 +priscilla carson 1 +priscilla ichabod 1 +priscilla ichabod 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla johnson 1 +priscilla king 1 +priscilla nixon 1 +priscilla nixon 2 +priscilla ovid 1 +priscilla ovid 1 +priscilla polk 1 +priscilla quirinius 1 +priscilla thompson 1 +priscilla underhill 1 +priscilla underhill 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla van buren 1 +priscilla white 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla xylophone 1 +priscilla young 1 +priscilla young 1 +priscilla zipper 1 +priscilla zipper 1 +quinn allen 1 +quinn allen 1 +quinn brown 1 +quinn brown 1 +quinn brown 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn davidson 1 +quinn ellison 1 +quinn ellison 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn garcia 1 +quinn ichabod 1 +quinn king 1 +quinn king 1 +quinn laertes 1 +quinn laertes 1 +quinn laertes 1 +quinn nixon 1 +quinn ovid 1 +quinn quirinius 1 +quinn robinson 1 +quinn steinbeck 1 +quinn steinbeck 4 +quinn thompson 1 +quinn thompson 2 +quinn underhill 1 +quinn underhill 1 +quinn underhill 2 +quinn van buren 1 +quinn young 1 +quinn zipper 1 +quinn zipper 1 +rachel allen 1 +rachel allen 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 1 +rachel brown 2 +rachel carson 1 +rachel carson 1 +rachel davidson 1 +rachel ellison 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel falkner 1 +rachel johnson 1 +rachel king 1 +rachel king 1 +rachel laertes 1 +rachel laertes 1 +rachel ovid 1 +rachel ovid 1 +rachel polk 1 +rachel quirinius 1 +rachel robinson 1 +rachel robinson 1 +rachel robinson 1 +rachel thompson 1 +rachel thompson 1 +rachel thompson 1 +rachel underhill 1 +rachel white 1 +rachel white 1 +rachel young 1 +rachel zipper 1 +rachel zipper 1 +sarah carson 1 +sarah carson 1 +sarah carson 1 +sarah ellison 2 +sarah falkner 1 +sarah falkner 1 +sarah garcia 1 +sarah garcia 1 +sarah garcia 2 +sarah ichabod 1 +sarah ichabod 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah johnson 1 +sarah king 1 +sarah king 1 +sarah miller 1 +sarah ovid 1 +sarah robinson 1 +sarah robinson 1 +sarah steinbeck 1 +sarah white 1 +sarah white 1 +sarah xylophone 1 +sarah young 1 +sarah zipper 1 +tom brown 1 +tom brown 1 +tom carson 1 +tom carson 1 +tom carson 1 +tom davidson 1 +tom ellison 1 +tom ellison 1 +tom ellison 1 +tom falkner 1 +tom falkner 1 +tom hernandez 1 +tom hernandez 1 +tom ichabod 1 +tom johnson 1 +tom johnson 2 +tom king 1 +tom laertes 1 +tom laertes 1 +tom miller 1 +tom miller 1 +tom miller 2 +tom nixon 1 +tom ovid 1 +tom polk 1 +tom polk 1 +tom quirinius 1 +tom quirinius 1 +tom robinson 1 +tom robinson 1 +tom robinson 2 +tom robinson 2 +tom steinbeck 2 +tom van buren 1 +tom van buren 1 +tom van buren 1 +tom white 1 +tom young 1 +tom young 2 +tom zipper 1 +ulysses brown 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses carson 1 +ulysses davidson 1 +ulysses ellison 1 +ulysses garcia 1 +ulysses hernandez 1 +ulysses hernandez 1 +ulysses hernandez 2 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses johnson 1 +ulysses king 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses laertes 1 +ulysses miller 1 +ulysses miller 1 +ulysses nixon 1 +ulysses ovid 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 1 +ulysses polk 2 +ulysses quirinius 1 +ulysses robinson 1 +ulysses steinbeck 1 +ulysses steinbeck 1 +ulysses thompson 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses underhill 1 +ulysses van buren 1 +ulysses white 1 +ulysses white 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses xylophone 1 +ulysses young 1 +ulysses young 1 +ulysses young 1 +victor allen 1 +victor allen 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor brown 1 +victor davidson 1 +victor davidson 1 +victor davidson 2 +victor ellison 1 +victor ellison 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor hernandez 1 +victor johnson 1 +victor johnson 1 +victor johnson 1 +victor king 1 +victor king 1 +victor laertes 1 +victor laertes 1 +victor miller 2 +victor nixon 1 +victor nixon 1 +victor ovid 1 +victor polk 1 +victor quirinius 1 +victor quirinius 1 +victor robinson 1 +victor robinson 1 +victor steinbeck 1 +victor steinbeck 1 +victor steinbeck 1 +victor thompson 1 +victor van buren 1 +victor van buren 1 +victor white 1 +victor white 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 1 +victor xylophone 2 +victor young 1 +victor zipper 1 +wendy allen 1 +wendy allen 1 +wendy allen 1 +wendy brown 1 +wendy brown 1 +wendy ellison 1 +wendy ellison 1 +wendy falkner 1 +wendy falkner 1 +wendy falkner 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy garcia 1 +wendy hernandez 1 +wendy ichabod 1 +wendy king 1 +wendy king 1 +wendy king 1 +wendy laertes 1 +wendy laertes 1 +wendy laertes 1 +wendy miller 1 +wendy miller 1 +wendy nixon 1 +wendy nixon 1 +wendy ovid 1 +wendy ovid 1 +wendy polk 1 +wendy polk 1 +wendy quirinius 1 +wendy quirinius 1 +wendy robinson 1 +wendy robinson 1 +wendy robinson 1 +wendy steinbeck 1 +wendy thompson 1 +wendy thompson 1 +wendy underhill 1 +wendy underhill 1 +wendy underhill 1 +wendy van buren 1 +wendy van buren 1 +wendy white 1 +wendy xylophone 1 +wendy xylophone 1 +wendy young 1 +wendy young 3 +xavier allen 1 +xavier allen 1 +xavier allen 1 +xavier brown 1 +xavier brown 1 +xavier brown 1 +xavier carson 1 +xavier carson 1 +xavier davidson 1 +xavier davidson 1 +xavier davidson 1 +xavier ellison 1 +xavier ellison 1 +xavier garcia 1 +xavier hernandez 1 +xavier hernandez 1 +xavier hernandez 1 +xavier ichabod 1 +xavier ichabod 1 +xavier johnson 1 +xavier johnson 1 +xavier king 1 +xavier king 1 +xavier laertes 1 +xavier ovid 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier polk 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier quirinius 1 +xavier thompson 1 +xavier underhill 1 +xavier white 1 +xavier white 1 +xavier xylophone 1 +xavier zipper 2 +yuri allen 1 +yuri allen 1 +yuri brown 1 +yuri brown 1 +yuri carson 1 +yuri carson 1 +yuri ellison 1 +yuri ellison 1 +yuri falkner 1 +yuri falkner 1 +yuri garcia 1 +yuri hernandez 1 +yuri johnson 1 +yuri johnson 1 +yuri johnson 1 +yuri king 1 +yuri laertes 1 +yuri laertes 1 +yuri nixon 1 +yuri nixon 1 +yuri polk 1 +yuri polk 1 +yuri polk 1 +yuri quirinius 1 +yuri quirinius 1 +yuri quirinius 1 +yuri steinbeck 1 +yuri steinbeck 1 +yuri thompson 1 +yuri underhill 1 +yuri underhill 1 +yuri white 1 +yuri xylophone 1 +zach allen 2 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach brown 1 +zach carson 1 +zach ellison 1 +zach falkner 1 +zach falkner 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach garcia 1 +zach ichabod 1 +zach ichabod 1 +zach king 1 +zach king 1 +zach king 2 +zach miller 1 +zach miller 1 +zach miller 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach ovid 1 +zach quirinius 1 +zach robinson 2 +zach steinbeck 2 +zach steinbeck 2 +zach thompson 1 +zach thompson 1 +zach underhill 1 +zach white 1 +zach xylophone 1 +zach xylophone 1 +zach young 1 +zach zipper 1 +zach zipper 1 +zach zipper 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 new file mode 100644 index 0000000000000..22a6f27253dcf --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-1-5c5f373e325115d710a7a23fcb1626f1 @@ -0,0 +1,1049 @@ +zach zipper 4 +zach zipper 3 +zach zipper 1 +zach young 4 +zach xylophone 4 +zach xylophone 1 +zach white 1 +zach underhill 1 +zach thompson 2 +zach thompson 2 +zach steinbeck 5 +zach steinbeck 1 +zach robinson 1 +zach quirinius 3 +zach ovid 5 +zach ovid 4 +zach ovid 3 +zach ovid 1 +zach miller 5 +zach miller 3 +zach miller 1 +zach king 6 +zach king 4 +zach king 1 +zach ichabod 3 +zach ichabod 2 +zach garcia 6 +zach garcia 3 +zach garcia 1 +zach garcia 1 +zach falkner 2 +zach falkner 1 +zach ellison 4 +zach carson 3 +zach brown 5 +zach brown 4 +zach brown 3 +zach brown 2 +zach brown 1 +zach allen 4 +yuri xylophone 3 +yuri white 2 +yuri underhill 6 +yuri underhill 4 +yuri thompson 4 +yuri steinbeck 6 +yuri steinbeck 2 +yuri quirinius 4 +yuri quirinius 3 +yuri quirinius 1 +yuri polk 4 +yuri polk 3 +yuri polk 2 +yuri nixon 3 +yuri nixon 2 +yuri laertes 3 +yuri laertes 1 +yuri king 5 +yuri johnson 4 +yuri johnson 3 +yuri johnson 1 +yuri hernandez 4 +yuri garcia 3 +yuri falkner 7 +yuri falkner 3 +yuri ellison 1 +yuri ellison 1 +yuri carson 7 +yuri carson 4 +yuri brown 3 +yuri brown 1 +yuri allen 3 +yuri allen 2 +xavier zipper 1 +xavier xylophone 1 +xavier white 3 +xavier white 3 +xavier underhill 2 +xavier thompson 3 +xavier quirinius 6 +xavier quirinius 5 +xavier quirinius 2 +xavier quirinius 1 +xavier polk 5 +xavier polk 3 +xavier polk 3 +xavier polk 3 +xavier ovid 5 +xavier laertes 4 +xavier king 3 +xavier king 1 +xavier johnson 3 +xavier johnson 1 +xavier ichabod 2 +xavier ichabod 2 +xavier hernandez 3 +xavier hernandez 1 +xavier hernandez 1 +xavier garcia 4 +xavier ellison 1 +xavier ellison 1 +xavier davidson 5 +xavier davidson 4 +xavier davidson 1 +xavier carson 5 +xavier carson 3 +xavier brown 4 +xavier brown 2 +xavier brown 2 +xavier allen 6 +xavier allen 3 +xavier allen 1 +wendy young 8 +wendy young 2 +wendy xylophone 6 +wendy xylophone 4 +wendy white 5 +wendy van buren 2 +wendy van buren 2 +wendy underhill 6 +wendy underhill 5 +wendy underhill 4 +wendy thompson 5 +wendy thompson 2 +wendy steinbeck 1 +wendy robinson 5 +wendy robinson 3 +wendy robinson 2 +wendy quirinius 6 +wendy quirinius 4 +wendy polk 2 +wendy polk 2 +wendy ovid 4 +wendy ovid 1 +wendy nixon 3 +wendy nixon 1 +wendy miller 2 +wendy miller 1 +wendy laertes 3 +wendy laertes 3 +wendy laertes 1 +wendy king 5 +wendy king 4 +wendy king 1 +wendy ichabod 3 +wendy hernandez 1 +wendy garcia 7 +wendy garcia 5 +wendy garcia 4 +wendy garcia 1 +wendy falkner 3 +wendy falkner 1 +wendy falkner 1 +wendy ellison 2 +wendy ellison 1 +wendy brown 5 +wendy brown 2 +wendy allen 6 +wendy allen 2 +wendy allen 2 +victor zipper 3 +victor young 1 +victor xylophone 6 +victor xylophone 6 +victor xylophone 2 +victor xylophone 1 +victor xylophone 1 +victor white 2 +victor white 1 +victor van buren 4 +victor van buren 4 +victor thompson 2 +victor steinbeck 5 +victor steinbeck 2 +victor steinbeck 1 +victor robinson 4 +victor robinson 2 +victor quirinius 3 +victor quirinius 1 +victor polk 3 +victor ovid 2 +victor nixon 6 +victor nixon 4 +victor miller 1 +victor laertes 4 +victor laertes 3 +victor king 6 +victor king 1 +victor johnson 2 +victor johnson 2 +victor johnson 1 +victor hernandez 6 +victor hernandez 4 +victor hernandez 3 +victor hernandez 1 +victor hernandez 1 +victor ellison 7 +victor ellison 4 +victor davidson 6 +victor davidson 2 +victor davidson 2 +victor brown 4 +victor brown 3 +victor brown 2 +victor brown 1 +victor allen 4 +victor allen 2 +ulysses young 7 +ulysses young 6 +ulysses young 3 +ulysses xylophone 6 +ulysses xylophone 3 +ulysses xylophone 2 +ulysses white 6 +ulysses white 2 +ulysses van buren 3 +ulysses underhill 8 +ulysses underhill 4 +ulysses underhill 3 +ulysses underhill 2 +ulysses underhill 2 +ulysses underhill 1 +ulysses underhill 1 +ulysses thompson 5 +ulysses steinbeck 3 +ulysses steinbeck 1 +ulysses robinson 5 +ulysses quirinius 8 +ulysses polk 6 +ulysses polk 4 +ulysses polk 1 +ulysses polk 1 +ulysses ovid 3 +ulysses nixon 1 +ulysses miller 3 +ulysses miller 2 +ulysses laertes 5 +ulysses laertes 4 +ulysses laertes 2 +ulysses king 2 +ulysses johnson 5 +ulysses ichabod 1 +ulysses ichabod 1 +ulysses hernandez 6 +ulysses hernandez 3 +ulysses hernandez 2 +ulysses garcia 2 +ulysses ellison 2 +ulysses davidson 8 +ulysses carson 4 +ulysses carson 3 +ulysses carson 2 +ulysses carson 1 +ulysses brown 3 +tom zipper 5 +tom young 2 +tom young 1 +tom white 1 +tom van buren 5 +tom van buren 2 +tom van buren 1 +tom steinbeck 4 +tom robinson 8 +tom robinson 4 +tom robinson 3 +tom robinson 2 +tom quirinius 5 +tom quirinius 1 +tom polk 3 +tom polk 2 +tom ovid 2 +tom nixon 5 +tom miller 1 +tom miller 1 +tom miller 1 +tom laertes 4 +tom laertes 2 +tom king 1 +tom johnson 8 +tom johnson 1 +tom ichabod 1 +tom hernandez 3 +tom hernandez 2 +tom falkner 3 +tom falkner 2 +tom ellison 5 +tom ellison 3 +tom ellison 1 +tom davidson 7 +tom carson 3 +tom carson 3 +tom carson 1 +tom brown 4 +tom brown 2 +sarah zipper 1 +sarah young 1 +sarah xylophone 2 +sarah white 4 +sarah white 3 +sarah steinbeck 6 +sarah robinson 3 +sarah robinson 2 +sarah ovid 1 +sarah miller 1 +sarah king 3 +sarah king 2 +sarah johnson 7 +sarah johnson 6 +sarah johnson 4 +sarah johnson 2 +sarah ichabod 4 +sarah ichabod 3 +sarah garcia 2 +sarah garcia 2 +sarah garcia 2 +sarah falkner 7 +sarah falkner 1 +sarah ellison 1 +sarah carson 6 +sarah carson 4 +sarah carson 4 +rachel zipper 8 +rachel zipper 5 +rachel young 3 +rachel white 2 +rachel white 2 +rachel underhill 2 +rachel thompson 5 +rachel thompson 4 +rachel thompson 3 +rachel robinson 10 +rachel robinson 3 +rachel robinson 1 +rachel quirinius 5 +rachel polk 4 +rachel ovid 5 +rachel ovid 4 +rachel laertes 1 +rachel laertes 1 +rachel king 3 +rachel king 1 +rachel johnson 1 +rachel falkner 8 +rachel falkner 5 +rachel falkner 5 +rachel falkner 2 +rachel ellison 6 +rachel davidson 6 +rachel carson 7 +rachel carson 2 +rachel brown 5 +rachel brown 4 +rachel brown 3 +rachel brown 3 +rachel brown 1 +rachel allen 5 +rachel allen 1 +quinn zipper 2 +quinn zipper 2 +quinn young 2 +quinn van buren 2 +quinn underhill 7 +quinn underhill 6 +quinn underhill 2 +quinn thompson 5 +quinn thompson 2 +quinn steinbeck 3 +quinn steinbeck 2 +quinn robinson 2 +quinn quirinius 5 +quinn ovid 6 +quinn nixon 3 +quinn laertes 2 +quinn laertes 2 +quinn laertes 1 +quinn king 2 +quinn king 1 +quinn ichabod 1 +quinn garcia 6 +quinn garcia 3 +quinn garcia 2 +quinn garcia 1 +quinn ellison 7 +quinn ellison 5 +quinn davidson 7 +quinn davidson 4 +quinn davidson 3 +quinn davidson 2 +quinn brown 5 +quinn brown 3 +quinn brown 2 +quinn allen 5 +quinn allen 2 +priscilla zipper 5 +priscilla zipper 2 +priscilla young 4 +priscilla young 1 +priscilla xylophone 7 +priscilla xylophone 2 +priscilla xylophone 1 +priscilla white 4 +priscilla van buren 3 +priscilla van buren 3 +priscilla van buren 2 +priscilla underhill 5 +priscilla underhill 4 +priscilla thompson 2 +priscilla quirinius 4 +priscilla polk 5 +priscilla ovid 4 +priscilla ovid 1 +priscilla nixon 2 +priscilla nixon 1 +priscilla king 4 +priscilla johnson 4 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 2 +priscilla johnson 1 +priscilla ichabod 3 +priscilla ichabod 2 +priscilla carson 6 +priscilla carson 5 +priscilla carson 4 +priscilla brown 5 +priscilla brown 5 +priscilla brown 3 +oscar zipper 4 +oscar zipper 4 +oscar zipper 2 +oscar xylophone 7 +oscar xylophone 5 +oscar xylophone 3 +oscar white 5 +oscar white 5 +oscar white 3 +oscar white 2 +oscar van buren 5 +oscar van buren 3 +oscar van buren 2 +oscar underhill 1 +oscar thompson 6 +oscar thompson 3 +oscar thompson 3 +oscar thompson 2 +oscar steinbeck 7 +oscar robinson 7 +oscar robinson 3 +oscar robinson 3 +oscar robinson 1 +oscar quirinius 3 +oscar quirinius 3 +oscar quirinius 2 +oscar quirinius 1 +oscar polk 2 +oscar polk 2 +oscar ovid 4 +oscar ovid 2 +oscar ovid 1 +oscar nixon 1 +oscar laertes 6 +oscar laertes 4 +oscar laertes 3 +oscar laertes 2 +oscar king 4 +oscar king 2 +oscar king 1 +oscar johnson 6 +oscar johnson 3 +oscar ichabod 3 +oscar ichabod 3 +oscar ichabod 1 +oscar ichabod 1 +oscar hernandez 6 +oscar hernandez 6 +oscar garcia 4 +oscar falkner 2 +oscar ellison 2 +oscar ellison 1 +oscar davidson 1 +oscar carson 4 +oscar carson 2 +oscar carson 2 +oscar carson 1 +oscar carson 1 +oscar brown 4 +oscar allen 2 +nick zipper 7 +nick zipper 5 +nick young 4 +nick young 2 +nick xylophone 2 +nick van buren 2 +nick underhill 2 +nick thompson 2 +nick steinbeck 4 +nick robinson 3 +nick robinson 1 +nick quirinius 5 +nick quirinius 1 +nick polk 5 +nick ovid 6 +nick nixon 4 +nick miller 2 +nick laertes 3 +nick johnson 4 +nick johnson 4 +nick ichabod 3 +nick ichabod 3 +nick ichabod 1 +nick garcia 5 +nick garcia 4 +nick garcia 4 +nick falkner 3 +nick falkner 1 +nick ellison 3 +nick ellison 2 +nick davidson 4 +nick brown 3 +nick allen 5 +nick allen 4 +mike zipper 4 +mike zipper 4 +mike zipper 1 +mike young 3 +mike young 1 +mike young 1 +mike white 9 +mike white 7 +mike white 5 +mike white 2 +mike van buren 2 +mike van buren 1 +mike steinbeck 4 +mike steinbeck 2 +mike steinbeck 2 +mike steinbeck 1 +mike quirinius 7 +mike polk 4 +mike polk 2 +mike polk 2 +mike nixon 3 +mike nixon 2 +mike miller 1 +mike king 6 +mike king 5 +mike king 4 +mike king 3 +mike king 1 +mike king 1 +mike ichabod 3 +mike hernandez 2 +mike hernandez 1 +mike garcia 3 +mike garcia 2 +mike garcia 1 +mike falkner 2 +mike ellison 6 +mike ellison 5 +mike ellison 3 +mike ellison 1 +mike ellison 1 +mike davidson 5 +mike davidson 5 +mike carson 9 +mike carson 4 +mike carson 3 +mike brown 2 +mike allen 3 +luke zipper 2 +luke xylophone 1 +luke white 1 +luke van buren 2 +luke underhill 2 +luke underhill 2 +luke underhill 1 +luke thompson 3 +luke robinson 6 +luke robinson 1 +luke quirinius 3 +luke polk 3 +luke polk 1 +luke ovid 3 +luke ovid 1 +luke miller 4 +luke laertes 4 +luke laertes 2 +luke laertes 2 +luke laertes 2 +luke laertes 1 +luke johnson 4 +luke johnson 2 +luke johnson 1 +luke ichabod 4 +luke ichabod 1 +luke garcia 5 +luke garcia 2 +luke falkner 4 +luke falkner 2 +luke ellison 3 +luke ellison 2 +luke ellison 1 +luke davidson 2 +luke davidson 2 +luke brown 5 +luke allen 5 +luke allen 2 +luke allen 1 +luke allen 1 +luke allen 1 +katie zipper 1 +katie zipper 1 +katie young 11 +katie young 6 +katie young 1 +katie xylophone 1 +katie white 5 +katie white 3 +katie van buren 6 +katie van buren 4 +katie robinson 2 +katie polk 5 +katie polk 2 +katie ovid 3 +katie nixon 1 +katie miller 1 +katie miller 1 +katie king 7 +katie king 5 +katie king 4 +katie ichabod 6 +katie ichabod 2 +katie ichabod 1 +katie hernandez 1 +katie garcia 4 +katie garcia 3 +katie falkner 4 +katie ellison 5 +katie ellison 4 +katie davidson 1 +katie brown 6 +katie allen 1 +jessica zipper 7 +jessica zipper 6 +jessica zipper 1 +jessica young 4 +jessica young 3 +jessica xylophone 3 +jessica white 8 +jessica white 6 +jessica white 3 +jessica white 1 +jessica white 1 +jessica van buren 1 +jessica underhill 5 +jessica underhill 3 +jessica underhill 2 +jessica thompson 3 +jessica thompson 2 +jessica robinson 2 +jessica quirinius 4 +jessica quirinius 4 +jessica quirinius 3 +jessica quirinius 1 +jessica polk 4 +jessica ovid 2 +jessica ovid 1 +jessica nixon 3 +jessica nixon 2 +jessica miller 5 +jessica johnson 4 +jessica johnson 3 +jessica ichabod 5 +jessica garcia 4 +jessica garcia 3 +jessica falkner 2 +jessica ellison 5 +jessica ellison 2 +jessica davidson 5 +jessica davidson 2 +jessica davidson 2 +jessica davidson 1 +jessica carson 4 +jessica carson 2 +jessica carson 1 +jessica brown 3 +irene xylophone 3 +irene van buren 2 +irene van buren 1 +irene underhill 5 +irene underhill 1 +irene thompson 6 +irene steinbeck 1 +irene robinson 1 +irene quirinius 6 +irene quirinius 5 +irene quirinius 5 +irene polk 3 +irene polk 2 +irene polk 2 +irene polk 1 +irene polk 1 +irene ovid 6 +irene ovid 6 +irene ovid 5 +irene nixon 4 +irene nixon 4 +irene nixon 1 +irene miller 6 +irene laertes 5 +irene laertes 3 +irene laertes 3 +irene johnson 2 +irene ichabod 7 +irene ichabod 1 +irene garcia 4 +irene garcia 2 +irene garcia 2 +irene falkner 5 +irene falkner 2 +irene ellison 4 +irene ellison 3 +irene carson 1 +irene brown 4 +irene brown 4 +irene brown 1 +irene allen 2 +holly zipper 3 +holly zipper 3 +holly young 2 +holly young 2 +holly xylophone 1 +holly white 3 +holly white 1 +holly van buren 4 +holly underhill 6 +holly underhill 3 +holly underhill 3 +holly underhill 2 +holly thompson 2 +holly thompson 1 +holly thompson 1 +holly robinson 2 +holly polk 7 +holly polk 4 +holly nixon 5 +holly nixon 1 +holly miller 4 +holly laertes 5 +holly king 4 +holly king 1 +holly johnson 2 +holly johnson 2 +holly johnson 2 +holly ichabod 4 +holly ichabod 4 +holly ichabod 2 +holly hernandez 9 +holly hernandez 3 +holly hernandez 3 +holly hernandez 2 +holly falkner 6 +holly brown 3 +holly brown 2 +holly allen 1 +gabriella zipper 5 +gabriella zipper 1 +gabriella young 3 +gabriella young 1 +gabriella white 3 +gabriella van buren 3 +gabriella van buren 1 +gabriella thompson 5 +gabriella thompson 5 +gabriella thompson 5 +gabriella steinbeck 4 +gabriella steinbeck 1 +gabriella polk 4 +gabriella polk 4 +gabriella ovid 2 +gabriella ovid 1 +gabriella miller 1 +gabriella laertes 4 +gabriella king 3 +gabriella king 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 3 +gabriella ichabod 2 +gabriella ichabod 1 +gabriella hernandez 9 +gabriella hernandez 5 +gabriella garcia 2 +gabriella falkner 4 +gabriella falkner 3 +gabriella falkner 2 +gabriella ellison 3 +gabriella ellison 1 +gabriella davidson 2 +gabriella carson 1 +gabriella brown 6 +gabriella brown 3 +gabriella allen 5 +gabriella allen 3 +fred zipper 1 +fred young 2 +fred young 1 +fred white 3 +fred van buren 7 +fred van buren 1 +fred van buren 1 +fred van buren 1 +fred underhill 4 +fred steinbeck 4 +fred steinbeck 2 +fred steinbeck 1 +fred robinson 3 +fred quirinius 7 +fred quirinius 4 +fred polk 7 +fred polk 6 +fred polk 4 +fred polk 2 +fred nixon 7 +fred nixon 5 +fred nixon 1 +fred nixon 1 +fred miller 1 +fred laertes 4 +fred king 6 +fred king 3 +fred johnson 4 +fred ichabod 3 +fred ichabod 2 +fred hernandez 1 +fred falkner 4 +fred falkner 3 +fred falkner 3 +fred ellison 5 +fred ellison 2 +fred ellison 1 +fred davidson 2 +fred davidson 2 +fred davidson 1 +ethan zipper 2 +ethan zipper 1 +ethan xylophone 3 +ethan white 5 +ethan white 2 +ethan van buren 1 +ethan underhill 1 +ethan robinson 3 +ethan robinson 1 +ethan quirinius 6 +ethan quirinius 2 +ethan quirinius 1 +ethan polk 3 +ethan polk 1 +ethan polk 1 +ethan polk 1 +ethan ovid 2 +ethan nixon 7 +ethan miller 5 +ethan laertes 4 +ethan laertes 4 +ethan laertes 3 +ethan laertes 2 +ethan laertes 2 +ethan laertes 2 +ethan laertes 1 +ethan king 1 +ethan johnson 1 +ethan hernandez 3 +ethan garcia 8 +ethan falkner 2 +ethan falkner 1 +ethan ellison 6 +ethan ellison 4 +ethan carson 6 +ethan brown 4 +ethan brown 3 +ethan brown 3 +ethan brown 1 +ethan brown 1 +ethan brown 1 +ethan allen 4 +david young 4 +david young 1 +david xylophone 6 +david xylophone 4 +david xylophone 1 +david white 2 +david van buren 3 +david van buren 2 +david underhill 7 +david underhill 4 +david underhill 1 +david thompson 1 +david robinson 3 +david robinson 2 +david quirinius 4 +david quirinius 4 +david quirinius 2 +david ovid 4 +david ovid 3 +david nixon 1 +david laertes 4 +david ichabod 6 +david ichabod 3 +david hernandez 7 +david ellison 5 +david ellison 3 +david ellison 3 +david davidson 4 +david davidson 3 +david davidson 1 +david davidson 1 +david brown 6 +david brown 2 +david allen 5 +david allen 2 +calvin zipper 9 +calvin zipper 3 +calvin young 3 +calvin young 1 +calvin xylophone 6 +calvin xylophone 3 +calvin xylophone 1 +calvin white 1 +calvin white 1 +calvin van buren 9 +calvin van buren 1 +calvin underhill 4 +calvin thompson 3 +calvin thompson 2 +calvin steinbeck 6 +calvin steinbeck 3 +calvin steinbeck 3 +calvin robinson 2 +calvin quirinius 4 +calvin quirinius 3 +calvin polk 2 +calvin ovid 5 +calvin ovid 4 +calvin ovid 3 +calvin ovid 1 +calvin nixon 7 +calvin nixon 3 +calvin nixon 2 +calvin laertes 3 +calvin laertes 1 +calvin johnson 2 +calvin hernandez 1 +calvin garcia 3 +calvin falkner 8 +calvin falkner 4 +calvin falkner 4 +calvin falkner 3 +calvin falkner 2 +calvin falkner 1 +calvin ellison 3 +calvin davidson 1 +calvin davidson 1 +calvin carson 1 +calvin brown 5 +calvin brown 3 +calvin brown 1 +calvin allen 1 +bob zipper 4 +bob zipper 1 +bob zipper 1 +bob young 1 +bob xylophone 3 +bob xylophone 2 +bob white 3 +bob white 1 +bob van buren 3 +bob steinbeck 2 +bob quirinius 4 +bob polk 2 +bob ovid 7 +bob ovid 2 +bob ovid 2 +bob ovid 1 +bob miller 1 +bob laertes 5 +bob laertes 1 +bob king 3 +bob king 3 +bob king 2 +bob ichabod 1 +bob hernandez 1 +bob garcia 4 +bob garcia 3 +bob garcia 2 +bob garcia 1 +bob garcia 1 +bob falkner 6 +bob ellison 3 +bob ellison 2 +bob ellison 1 +bob ellison 1 +bob davidson 5 +bob davidson 2 +bob davidson 2 +bob carson 3 +bob brown 8 +bob brown 6 +bob brown 2 +alice zipper 2 +alice zipper 1 +alice zipper 1 +alice xylophone 2 +alice xylophone 2 +alice xylophone 1 +alice van buren 2 +alice underhill 2 +alice steinbeck 7 +alice steinbeck 3 +alice steinbeck 1 +alice robinson 4 +alice robinson 1 +alice quirinius 6 +alice quirinius 4 +alice polk 1 +alice ovid 2 +alice nixon 2 +alice nixon 2 +alice nixon 1 +alice miller 2 +alice laertes 3 +alice laertes 2 +alice king 8 +alice king 4 +alice king 2 +alice johnson 5 +alice hernandez 8 +alice hernandez 8 +alice garcia 1 +alice falkner 5 +alice davidson 2 +alice carson 1 +alice brown 5 +alice allen 5 +alice allen 5 +alice allen 4 + 5 + 4 + 3 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 new file mode 100644 index 0000000000000..c38e7bbabc21e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-2-ac487cc1b94130bf9ce00e07c7075f65 @@ -0,0 +1,1049 @@ + 0.25047801147227533 + 0.47992351816443596 + 0.6197718631178707 +alice allen 0.7300380228136882 +alice allen 0.8954372623574145 +alice allen 0.9216061185468452 +alice brown 0.22053231939163498 +alice carson 0.2889733840304182 +alice davidson 0.8593155893536122 +alice falkner 0.08604206500956023 +alice garcia 0.2870722433460076 +alice hernandez 0.011472275334608031 +alice hernandez 0.07604562737642585 +alice johnson 0.5181644359464627 +alice king 0.3652007648183556 +alice king 0.8536121673003803 +alice king 0.9771863117870723 +alice laertes 0.870722433460076 +alice laertes 0.870722433460076 +alice miller 0.12045889101338432 +alice nixon 0.4372623574144487 +alice nixon 0.47036328871892924 +alice nixon 0.768642447418738 +alice ovid 0.4665391969407266 +alice polk 0.279467680608365 +alice quirinius 0.8432122370936902 +alice quirinius 0.9923518164435946 +alice robinson 0.5722433460076045 +alice robinson 0.7984790874524715 +alice steinbeck 0.27151051625239003 +alice steinbeck 0.739961759082218 +alice steinbeck 0.9923954372623575 +alice underhill 0.5513307984790875 +alice van buren 0.4923954372623574 +alice xylophone 0.2376425855513308 +alice xylophone 0.26806083650190116 +alice xylophone 0.8776290630975143 +alice zipper 0.33460076045627374 +alice zipper 0.8814531548757171 +alice zipper 0.9445506692160612 +bob brown 0.5038022813688213 +bob brown 0.5066921606118547 +bob brown 0.5372848948374761 +bob carson 0.43346007604562736 +bob davidson 0.21673003802281368 +bob davidson 0.5285171102661597 +bob davidson 0.8413001912045889 +bob ellison 0.2045889101338432 +bob ellison 0.26577437858508607 +bob ellison 0.5793499043977055 +bob ellison 0.9144486692015209 +bob falkner 0.6940726577437859 +bob garcia 0.08555133079847908 +bob garcia 0.17680608365019013 +bob garcia 0.2887189292543021 +bob garcia 0.5418250950570342 +bob garcia 0.5736137667304015 +bob hernandez 0.7813688212927756 +bob ichabod 0.5200764818355641 +bob king 0.0076481835564053535 +bob king 0.5627376425855514 +bob king 0.9524714828897338 +bob laertes 0.32887189292543023 +bob laertes 0.6825095057034221 +bob miller 0.19771863117870722 +bob ovid 0.40304182509505704 +bob ovid 0.40344168260038243 +bob ovid 0.42065009560229444 +bob ovid 0.8403041825095057 +bob polk 0.15019011406844107 +bob quirinius 0.1844106463878327 +bob steinbeck 0.16920152091254753 +bob van buren 0.5086042065009561 +bob white 0.26045627376425856 +bob white 0.7623574144486692 +bob xylophone 0.4474187380497132 +bob xylophone 0.6539923954372624 +bob young 0.4722753346080306 +bob zipper 0.009505703422053232 +bob zipper 0.24091778202676864 +bob zipper 0.4600760456273764 +calvin allen 0.30975143403441685 +calvin brown 0.4448669201520912 +calvin brown 0.5361216730038023 +calvin brown 0.9196940726577438 +calvin carson 0.9315589353612167 +calvin davidson 0.5869980879541109 +calvin davidson 0.6653992395437263 +calvin ellison 0.6977186311787072 +calvin falkner 0.02091254752851711 +calvin falkner 0.03824091778202677 +calvin falkner 0.21223709369024857 +calvin falkner 0.46577946768060835 +calvin falkner 0.5114068441064639 +calvin falkner 0.5950570342205324 +calvin garcia 0.7896749521988528 +calvin hernandez 0.16730038022813687 +calvin johnson 0.9790874524714829 +calvin laertes 0.5487571701720841 +calvin laertes 0.8145315487571702 +calvin nixon 0.019120458891013385 +calvin nixon 0.4467680608365019 +calvin nixon 0.7395437262357415 +calvin ovid 0.14531548757170173 +calvin ovid 0.17490494296577946 +calvin ovid 0.19961977186311788 +calvin ovid 0.9407265774378585 +calvin polk 0.4619771863117871 +calvin quirinius 0.8802281368821293 +calvin quirinius 0.9254302103250478 +calvin robinson 0.13193116634799235 +calvin steinbeck 0.4818355640535373 +calvin steinbeck 0.7418738049713193 +calvin steinbeck 0.8060836501901141 +calvin thompson 0.2179732313575526 +calvin thompson 0.8422053231939164 +calvin underhill 0.7495219885277247 +calvin van buren 0.022813688212927757 +calvin van buren 0.8508604206500956 +calvin white 0.04182509505703422 +calvin white 0.9674952198852772 +calvin xylophone 0.011406844106463879 +calvin xylophone 0.3193116634799235 +calvin xylophone 0.6634799235181644 +calvin young 0.1988527724665392 +calvin young 0.4391634980988593 +calvin zipper 0.5532319391634981 +calvin zipper 0.8726235741444867 +david allen 0.30019120458891013 +david allen 0.3326959847036329 +david brown 0.1338432122370937 +david brown 0.9694072657743786 +david davidson 0.21414913957934992 +david davidson 0.655893536121673 +david davidson 0.7319391634980988 +david davidson 0.8878326996197718 +david ellison 0.6863117870722434 +david ellison 0.6883365200764818 +david ellison 0.7243346007604563 +david hernandez 0.12237093690248566 +david ichabod 0.35564053537284895 +david ichabod 0.7338403041825095 +david laertes 0.3575525812619503 +david nixon 0.33460803059273425 +david ovid 0.3916349809885932 +david ovid 0.6022944550669216 +david quirinius 0.3155893536121673 +david quirinius 0.6577437858508605 +david quirinius 0.9163498098859315 +david robinson 0.6673003802281369 +david robinson 0.6998087954110899 +david thompson 0.25285171102661597 +david underhill 0.1586998087954111 +david underhill 0.35181644359464626 +david underhill 0.7189292543021033 +david van buren 0.05927342256214149 +david van buren 0.5889101338432122 +david white 0.49429657794676807 +david xylophone 0.4875717017208413 +david xylophone 0.6901140684410646 +david xylophone 0.7571701720841301 +david young 0.0019011406844106464 +david young 0.040152963671128104 +ethan allen 0.20532319391634982 +ethan brown 0.10707456978967496 +ethan brown 0.13307984790874525 +ethan brown 0.4340344168260038 +ethan brown 0.4752851711026616 +ethan brown 0.5219885277246654 +ethan brown 0.745697896749522 +ethan carson 0.20912547528517111 +ethan ellison 0.45124282982791586 +ethan ellison 0.8680688336520076 +ethan falkner 0.0994263862332696 +ethan falkner 0.6845124282982792 +ethan garcia 0.06653992395437262 +ethan hernandez 0.2237093690248566 +ethan johnson 0.2300380228136882 +ethan king 0.47418738049713194 +ethan laertes 0.022944550669216062 +ethan laertes 0.2908745247148289 +ethan laertes 0.42638623326959846 +ethan laertes 0.48098859315589354 +ethan laertes 0.6596558317399618 +ethan laertes 0.7839388145315488 +ethan laertes 0.9201520912547528 +ethan miller 0.23709369024856597 +ethan nixon 0.8164435946462715 +ethan ovid 0.6121673003802282 +ethan polk 0.12167300380228137 +ethan polk 0.3384321223709369 +ethan polk 0.6920152091254753 +ethan polk 0.9619771863117871 +ethan quirinius 0.19391634980988592 +ethan quirinius 0.23135755258126195 +ethan quirinius 0.7908745247148289 +ethan robinson 0.24282982791587 +ethan robinson 0.8003802281368821 +ethan underhill 0.6615969581749049 +ethan van buren 0.8365019011406845 +ethan white 0.48859315589353614 +ethan white 0.5741444866920152 +ethan xylophone 0.9695817490494296 +ethan zipper 0.21102661596958175 +ethan zipper 0.6425855513307985 +fred davidson 0.5239005736137667 +fred davidson 0.7414448669201521 +fred davidson 0.8604206500956023 +fred ellison 0.3977055449330784 +fred ellison 0.5506692160611855 +fred ellison 0.7208413001912046 +fred falkner 0.024714828897338403 +fred falkner 0.19120458891013384 +fred falkner 0.9809885931558935 +fred hernandez 0.2734225621414914 +fred ichabod 0.17110266159695817 +fred ichabod 0.780114722753346 +fred johnson 0.30038022813688214 +fred king 0.2198852772466539 +fred king 0.47718631178707227 +fred laertes 0.2332695984703633 +fred miller 0.7858508604206501 +fred nixon 0.005703422053231939 +fred nixon 0.31749049429657794 +fred nixon 0.7648183556405354 +fred nixon 0.8460076045627376 +fred polk 0.16252390057361377 +fred polk 0.564638783269962 +fred polk 0.6273764258555133 +fred polk 0.8155893536121673 +fred quirinius 0.4866920152091255 +fred quirinius 0.8973384030418251 +fred robinson 0.6387832699619772 +fred steinbeck 0.14722753346080306 +fred steinbeck 0.4627151051625239 +fred steinbeck 0.7265774378585086 +fred underhill 0.35361216730038025 +fred van buren 0.3365200764818356 +fred van buren 0.5057034220532319 +fred van buren 0.6463878326996197 +fred van buren 0.904397705544933 +fred white 0.5171102661596958 +fred young 0.7705544933078394 +fred young 0.7992351816443595 +fred zipper 0.615678776290631 +gabriella allen 0.4435946462715105 +gabriella allen 0.9334600760456274 +gabriella brown 0.4359464627151052 +gabriella brown 0.9636711281070746 +gabriella carson 0.9562737642585551 +gabriella davidson 0.8174904942965779 +gabriella ellison 0.1931166347992352 +gabriella ellison 0.38022813688212925 +gabriella falkner 0.3231939163498099 +gabriella falkner 0.5659655831739961 +gabriella falkner 0.8948374760994264 +gabriella garcia 0.4695817490494297 +gabriella hernandez 0.6444866920152091 +gabriella hernandez 0.7015209125475285 +gabriella ichabod 0.09125475285171103 +gabriella ichabod 0.1520912547528517 +gabriella ichabod 0.1835564053537285 +gabriella ichabod 0.372848948374761 +gabriella ichabod 0.8107074569789675 +gabriella king 0.39961759082217974 +gabriella king 0.5190114068441065 +gabriella laertes 0.4569789674952199 +gabriella miller 0.26996197718631176 +gabriella ovid 0.7091254752851711 +gabriella ovid 0.8897338403041825 +gabriella polk 0.030418250950570342 +gabriella polk 0.44106463878326996 +gabriella steinbeck 0.5755258126195029 +gabriella steinbeck 0.8221797323135756 +gabriella thompson 0.013307984790874524 +gabriella thompson 0.44866920152091255 +gabriella thompson 0.7224334600760456 +gabriella van buren 0.6216730038022814 +gabriella van buren 0.6730038022813688 +gabriella white 0.17208413001912046 +gabriella young 0.5076045627376425 +gabriella young 0.7934990439770554 +gabriella zipper 0.23193916349809887 +gabriella zipper 0.8565965583173997 +holly allen 0.11596958174904944 +holly brown 0.11281070745697896 +holly brown 0.155893536121673 +holly falkner 0.124282982791587 +holly hernandez 0.055449330783938815 +holly hernandez 0.32509505703422054 +holly hernandez 0.97131931166348 +holly hernandez 0.9714828897338403 +holly ichabod 0.12357414448669202 +holly ichabod 0.17300380228136883 +holly ichabod 0.629277566539924 +holly johnson 0.33078393881453155 +holly johnson 0.8612167300380228 +holly johnson 0.9391634980988594 +holly king 0.25475285171102663 +holly king 0.3745247148288973 +holly laertes 0.42775665399239543 +holly miller 0.37476099426386233 +holly nixon 0.10076045627376426 +holly nixon 0.34608030592734224 +holly polk 0.40535372848948376 +holly polk 0.5209125475285171 +holly robinson 0.9273422562141491 +holly thompson 0.1596958174904943 +holly thompson 0.311787072243346 +holly thompson 0.9125475285171103 +holly underhill 0.3479923518164436 +holly underhill 0.5812619502868069 +holly underhill 0.8384030418250951 +holly underhill 0.903041825095057 +holly van buren 0.9464627151051626 +holly white 0.1089866156787763 +holly white 0.4780114722753346 +holly xylophone 0.5304182509505704 +holly young 0.7357414448669202 +holly young 0.8240917782026769 +holly zipper 0.15399239543726237 +holly zipper 0.8546845124282982 +irene allen 0.8738049713193117 +irene brown 0.4588910133843212 +irene brown 0.49619771863117873 +irene brown 0.5678776290630975 +irene carson 0.6844106463878327 +irene ellison 0.32504780114722753 +irene ellison 0.48565965583174 +irene falkner 0.41825095057034223 +irene falkner 0.9866920152091255 +irene garcia 0.11663479923518165 +irene garcia 0.29277566539923955 +irene garcia 0.8126195028680688 +irene ichabod 0.8307984790874525 +irene ichabod 0.9177820267686424 +irene johnson 0.7112810707456979 +irene laertes 0.01338432122370937 +irene laertes 0.1482889733840304 +irene laertes 0.7034220532319392 +irene miller 0.367112810707457 +irene nixon 0.11854684512428298 +irene nixon 0.7927756653992395 +irene nixon 0.9426386233269598 +irene ovid 0.24714828897338403 +irene ovid 0.30210325047801145 +irene ovid 0.779467680608365 +irene polk 0.0038022813688212928 +irene polk 0.45315487571701724 +irene polk 0.6577946768060836 +irene polk 0.8891013384321224 +irene polk 0.9789674952198852 +irene quirinius 0.27533460803059273 +irene quirinius 0.35946462715105165 +irene quirinius 0.384321223709369 +irene robinson 0.18631178707224336 +irene steinbeck 0.9942965779467681 +irene thompson 0.6939163498098859 +irene underhill 0.30401529636711283 +irene underhill 0.3403041825095057 +irene van buren 0.5908221797323135 +irene van buren 0.6634980988593155 +irene xylophone 0.5342205323193916 +jessica brown 0.7680608365019012 +jessica carson 0.3574144486692015 +jessica carson 0.6195028680688337 +jessica carson 0.8269961977186312 +jessica davidson 0.10646387832699619 +jessica davidson 0.34790874524714827 +jessica davidson 0.3593155893536122 +jessica davidson 0.6768060836501901 +jessica ellison 0.0779467680608365 +jessica ellison 0.42015209125475284 +jessica falkner 0.994263862332696 +jessica garcia 0.8279158699808795 +jessica garcia 0.9581749049429658 +jessica ichabod 0.45627376425855515 +jessica johnson 0.30228136882129275 +jessica johnson 0.8049713193116634 +jessica miller 0.8011472275334608 +jessica nixon 0.06500956022944551 +jessica nixon 0.6042065009560229 +jessica ovid 0.15105162523900573 +jessica ovid 0.8992395437262357 +jessica polk 0.4378585086042065 +jessica quirinius 0.058935361216730035 +jessica quirinius 0.4714828897338403 +jessica quirinius 0.5760456273764258 +jessica quirinius 0.8935361216730038 +jessica robinson 0.9638783269961977 +jessica thompson 0.08221797323135756 +jessica thompson 0.5893536121673004 +jessica underhill 0.034220532319391636 +jessica underhill 0.06118546845124283 +jessica underhill 0.9541108986615678 +jessica van buren 0.20650095602294455 +jessica white 0.06273764258555133 +jessica white 0.4149139579349904 +jessica white 0.5798479087452472 +jessica white 0.591254752851711 +jessica white 0.7667304015296367 +jessica xylophone 0.5009560229445507 +jessica young 0.3403441682600382 +jessica young 0.8821292775665399 +jessica zipper 0.14068441064638784 +jessica zipper 0.2984790874524715 +jessica zipper 0.6007604562737643 +katie allen 0.5665399239543726 +katie brown 0.49521988527724664 +katie davidson 0.6730401529636711 +katie ellison 0.3173996175908222 +katie ellison 0.7262357414448669 +katie falkner 0.2676864244741874 +katie garcia 0.049429657794676805 +katie garcia 0.3135755258126195 +katie hernandez 0.6026615969581749 +katie ichabod 0.15296367112810708 +katie ichabod 0.4684512428298279 +katie ichabod 0.7055449330783938 +katie king 0.16159695817490494 +katie king 0.502868068833652 +katie king 0.5927342256214149 +katie miller 0.5228136882129277 +katie miller 0.5296367112810707 +katie nixon 0.7832699619771863 +katie ovid 0.8795411089866156 +katie polk 0.35372848948374763 +katie polk 0.9657794676806084 +katie robinson 0.06844106463878327 +katie van buren 0.06883365200764818 +katie van buren 0.1739961759082218 +katie white 0.045889101338432124 +katie white 0.18546845124282982 +katie xylophone 0.7281368821292775 +katie young 0.16443594646271512 +katie young 0.20152091254752852 +katie young 0.9732313575525813 +katie zipper 0.21863117870722434 +katie zipper 0.4505703422053232 +luke allen 0.03612167300380228 +luke allen 0.21606118546845124 +luke allen 0.8346007604562737 +luke allen 0.8631178707224335 +luke allen 0.9311663479923518 +luke brown 0.7304015296367112 +luke davidson 0.25239005736137665 +luke davidson 0.9961977186311787 +luke ellison 0.1147227533460803 +luke ellison 0.2447418738049713 +luke ellison 0.49809885931558934 +luke falkner 0.24524714828897337 +luke falkner 0.5124282982791587 +luke garcia 0.03441682600382409 +luke garcia 0.32695984703632885 +luke ichabod 0.10266159695817491 +luke ichabod 0.5551330798479087 +luke johnson 0.25430210325047803 +luke johnson 0.6787762906309751 +luke johnson 0.9082217973231358 +luke laertes 0.06309751434034416 +luke laertes 0.3690248565965583 +luke laertes 0.7743785850860421 +luke laertes 0.8079847908745247 +luke laertes 0.811787072243346 +luke miller 0.8068833652007649 +luke ovid 0.435361216730038 +luke ovid 0.7547528517110266 +luke polk 0.13957934990439771 +luke polk 0.9770554493307839 +luke quirinius 0.09315589353612168 +luke robinson 0.015209125475285171 +luke robinson 0.053231939163498096 +luke thompson 0.8840304182509505 +luke underhill 0.08745247148288973 +luke underhill 0.40152963671128106 +luke underhill 0.4608030592734226 +luke van buren 0.4847908745247148 +luke white 0.8098859315589354 +luke xylophone 0.34220532319391633 +luke zipper 0.21292775665399238 +mike allen 0.7036328871892925 +mike brown 0.29063097514340347 +mike carson 0.623574144486692 +mike carson 0.7476099426386233 +mike carson 0.9885931558935361 +mike davidson 0.6520912547528517 +mike davidson 0.8298279158699808 +mike ellison 0.24665391969407266 +mike ellison 0.3821292775665399 +mike ellison 0.8355640535372849 +mike ellison 0.8986615678776291 +mike ellison 0.94106463878327 +mike falkner 0.0248565965583174 +mike garcia 0.39543726235741444 +mike garcia 0.5391969407265774 +mike garcia 0.6482889733840305 +mike hernandez 0.07984790874524715 +mike hernandez 0.7186311787072244 +mike ichabod 0.7642585551330798 +mike king 0.09695817490494296 +mike king 0.188212927756654 +mike king 0.4049429657794677 +mike king 0.5544933078393881 +mike king 0.6045627376425855 +mike king 0.9011406844106464 +mike miller 0.621414913957935 +mike nixon 0.688212927756654 +mike nixon 0.9068441064638784 +mike polk 0.3612167300380228 +mike polk 0.6749521988527725 +mike polk 0.8374760994263862 +mike quirinius 0.5105162523900574 +mike steinbeck 0.05736137667304015 +mike steinbeck 0.747148288973384 +mike steinbeck 0.8745247148288974 +mike steinbeck 0.9330783938814532 +mike van buren 0.8650190114068441 +mike van buren 0.973384030418251 +mike white 0.17782026768642448 +mike white 0.7151051625239006 +mike white 0.7566539923954373 +mike white 0.9808795411089866 +mike young 0.20722433460076045 +mike young 0.3840304182509506 +mike young 0.6405353728489483 +mike zipper 0.12810707456978968 +mike zipper 0.42829827915869984 +mike zipper 0.7946768060836502 +nick allen 0.021032504780114723 +nick allen 0.847036328871893 +nick brown 0.14258555133079848 +nick davidson 0.26003824091778205 +nick ellison 0.028680688336520075 +nick ellison 0.3935361216730038 +nick falkner 0.5684410646387833 +nick falkner 0.7590822179732314 +nick garcia 0.34980988593155893 +nick garcia 0.45817490494296575 +nick garcia 0.892925430210325 +nick ichabod 0.2944550669216061 +nick ichabod 0.37667304015296366 +nick ichabod 0.7074569789674953 +nick johnson 0.3973384030418251 +nick johnson 0.4646271510516252 +nick laertes 0.36311787072243346 +nick miller 0.9961759082217974 +nick nixon 0.7110266159695817 +nick ovid 0.7762906309751434 +nick polk 1.0 +nick quirinius 0.0019120458891013384 +nick quirinius 0.08795411089866156 +nick robinson 0.09505703422053231 +nick robinson 0.45506692160611856 +nick steinbeck 0.2224334600760456 +nick thompson 0.4225621414913958 +nick underhill 0.9101338432122371 +nick van buren 0.03802281368821293 +nick xylophone 0.6806883365200764 +nick young 0.4220532319391635 +nick young 0.8623326959847036 +nick zipper 0.2829827915869981 +nick zipper 0.5468451242829828 +oscar allen 0.785171102661597 +oscar brown 0.13498098859315588 +oscar carson 0.07224334600760456 +oscar carson 0.25665399239543724 +oscar carson 0.3422562141491396 +oscar carson 0.6061185468451242 +oscar carson 0.6826003824091779 +oscar davidson 0.7129277566539924 +oscar ellison 0.036328871892925434 +oscar ellison 0.5831739961759083 +oscar falkner 0.9049429657794676 +oscar garcia 0.02676864244741874 +oscar hernandez 0.20076481835564053 +oscar hernandez 0.7870722433460076 +oscar ichabod 0.12619502868068833 +oscar ichabod 0.14149139579349904 +oscar ichabod 0.4416826003824092 +oscar ichabod 0.8661567877629063 +oscar johnson 0.1806083650190114 +oscar johnson 0.467680608365019 +oscar king 0.6596958174904943 +oscar king 0.6787072243346007 +oscar king 0.9258555133079848 +oscar laertes 0.24904942965779467 +oscar laertes 0.5315487571701721 +oscar laertes 0.6328871892925431 +oscar laertes 0.9980988593155894 +oscar nixon 0.9292543021032504 +oscar ovid 0.43021032504780116 +oscar ovid 0.8288973384030418 +oscar ovid 0.8527724665391969 +oscar polk 0.10836501901140684 +oscar polk 0.37858508604206503 +oscar quirinius 0.3041825095057034 +oscar quirinius 0.46387832699619774 +oscar quirinius 0.6311787072243346 +oscar quirinius 0.8555133079847909 +oscar robinson 0.11216730038022814 +oscar robinson 0.22433460076045628 +oscar robinson 0.2294455066921606 +oscar robinson 0.2390057361376673 +oscar steinbeck 0.9904942965779467 +oscar thompson 0.015296367112810707 +oscar thompson 0.2946768060836502 +oscar thompson 0.3060836501901141 +oscar thompson 0.6140684410646388 +oscar underhill 0.31368821292775667 +oscar van buren 0.722753346080306 +oscar van buren 0.7889733840304183 +oscar van buren 0.8833652007648184 +oscar white 0.055133079847908745 +oscar white 0.22562141491395793 +oscar white 0.4321223709369025 +oscar white 0.6443594646271511 +oscar xylophone 0.10133843212237094 +oscar xylophone 0.4187380497131931 +oscar xylophone 0.4296577946768061 +oscar zipper 0.6233269598470363 +oscar zipper 0.7490494296577946 +oscar zipper 0.8783269961977186 +priscilla brown 0.2925430210325048 +priscilla brown 0.6501901140684411 +priscilla brown 0.9120458891013384 +priscilla carson 0.22753346080305928 +priscilla carson 0.5564053537284895 +priscilla carson 0.7820267686424475 +priscilla ichabod 0.3269961977186312 +priscilla ichabod 0.9828897338403042 +priscilla johnson 0.04206500956022945 +priscilla johnson 0.4011406844106464 +priscilla johnson 0.6368821292775665 +priscilla johnson 0.7131931166347992 +priscilla johnson 0.9429657794676806 +priscilla king 0.3517110266159696 +priscilla nixon 0.38049713193116635 +priscilla nixon 0.6864244741873805 +priscilla ovid 0.8193916349809885 +priscilla ovid 0.9139579349904398 +priscilla polk 0.5697896749521989 +priscilla quirinius 0.22179732313575526 +priscilla thompson 0.7737642585551331 +priscilla underhill 0.1682600382409178 +priscilla underhill 0.8852772466539197 +priscilla van buren 0.10325047801147227 +priscilla van buren 0.7877629063097514 +priscilla van buren 0.9598470363288719 +priscilla white 0.4894837476099426 +priscilla xylophone 0.596958174904943 +priscilla xylophone 0.6159695817490495 +priscilla xylophone 0.8393881453154876 +priscilla young 0.41064638783269963 +priscilla young 0.9182509505703422 +priscilla zipper 0.5247148288973384 +priscilla zipper 0.8574144486692015 +quinn allen 0.1634980988593156 +quinn allen 0.9617590822179732 +quinn brown 0.08986615678776291 +quinn brown 0.17590822179732313 +quinn brown 0.5836501901140685 +quinn davidson 0.11787072243346007 +quinn davidson 0.30592734225621415 +quinn davidson 0.3650190114068441 +quinn davidson 0.751434034416826 +quinn ellison 0.376425855513308 +quinn ellison 0.8517110266159695 +quinn garcia 0.17870722433460076 +quinn garcia 0.7323135755258127 +quinn garcia 0.844106463878327 +quinn garcia 0.9486692015209125 +quinn ichabod 0.42395437262357416 +quinn king 0.6653919694072657 +quinn king 0.9505703422053232 +quinn laertes 0.6080305927342257 +quinn laertes 0.9277566539923955 +quinn laertes 0.9847908745247148 +quinn nixon 0.5133079847908745 +quinn ovid 0.16539923954372623 +quinn quirinius 0.19011406844106463 +quinn robinson 0.27756653992395436 +quinn steinbeck 0.23954372623574144 +quinn steinbeck 0.6367112810707457 +quinn thompson 0.4068441064638783 +quinn thompson 0.7782026768642447 +quinn underhill 0.05353728489483748 +quinn underhill 0.5380228136882129 +quinn underhill 0.9349904397705545 +quinn van buren 0.2623574144486692 +quinn young 0.37832699619771865 +quinn zipper 0.51434034416826 +quinn zipper 0.8859315589353612 +rachel allen 0.1701720841300191 +rachel allen 0.3288973384030418 +rachel brown 0.04780114722753346 +rachel brown 0.057034220532319393 +rachel brown 0.532319391634981 +rachel brown 0.5946462715105163 +rachel brown 0.6064638783269962 +rachel carson 0.09177820267686425 +rachel carson 0.6406844106463878 +rachel davidson 0.37093690248565964 +rachel ellison 0.5162523900573613 +rachel falkner 0.1958174904942966 +rachel falkner 0.6330798479087453 +rachel falkner 0.6768642447418738 +rachel falkner 0.9751434034416826 +rachel johnson 0.9560229445506692 +rachel king 0.12547528517110265 +rachel king 0.6003824091778203 +rachel laertes 0.2638623326959847 +rachel laertes 0.5779467680608364 +rachel ovid 0.23518164435946462 +rachel ovid 0.7053231939163498 +rachel polk 0.14638783269961977 +rachel quirinius 0.0076045627376425855 +rachel robinson 0.14340344168260039 +rachel robinson 0.2084130019120459 +rachel robinson 0.6902485659655831 +rachel thompson 0.2718631178707224 +rachel thompson 0.5334608030592735 +rachel thompson 0.875717017208413 +rachel underhill 0.344106463878327 +rachel white 0.17973231357552583 +rachel white 0.6615678776290631 +rachel young 0.3862332695984704 +rachel zipper 0.33079847908745247 +rachel zipper 0.5717017208413002 +sarah carson 0.08935361216730038 +sarah carson 0.18250950570342206 +sarah carson 0.8041825095057035 +sarah ellison 0.967680608365019 +sarah falkner 0.6252390057361377 +sarah falkner 0.9024856596558317 +sarah garcia 0.3881453154875717 +sarah garcia 0.4072657743785851 +sarah garcia 0.6673040152963671 +sarah ichabod 0.29636711281070743 +sarah ichabod 0.9483747609942639 +sarah johnson 0.06463878326996197 +sarah johnson 0.10456273764258556 +sarah johnson 0.5640535372848948 +sarah johnson 0.7954110898661568 +sarah king 0.8030592734225621 +sarah king 0.9655831739961759 +sarah miller 0.6692160611854685 +sarah ovid 0.20342205323193915 +sarah robinson 0.47338403041825095 +sarah robinson 0.7775665399239544 +sarah steinbeck 0.6520076481835564 +sarah white 0.28517110266159695 +sarah white 0.8479087452471483 +sarah xylophone 0.25621414913957935 +sarah young 0.5570342205323194 +sarah zipper 0.5583173996175909 +tom brown 0.5602294455066922 +tom brown 0.8669201520912547 +tom carson 0.045627376425855515 +tom carson 0.35551330798479086 +tom carson 0.935361216730038 +tom davidson 0.8212927756653993 +tom ellison 0.21032504780114722 +tom ellison 0.26195028680688337 +tom ellison 0.7376425855513308 +tom falkner 0.3441682600382409 +tom falkner 0.6481835564053537 +tom hernandez 0.0038240917782026767 +tom hernandez 0.5399239543726235 +tom ichabod 0.6137667304015296 +tom johnson 0.5525812619502868 +tom johnson 0.7915869980879541 +tom king 0.16061185468451242 +tom laertes 0.0745697896749522 +tom laertes 0.5095057034220533 +tom miller 0.2262357414448669 +tom miller 0.2338403041825095 +tom miller 0.2813688212927757 +tom nixon 0.8451242829827916 +tom ovid 0.864244741873805 +tom polk 0.1491395793499044 +tom polk 0.9521988527724665 +tom quirinius 0.09369024856596558 +tom quirinius 0.8489483747609943 +tom robinson 0.060836501901140684 +tom robinson 0.6254752851711026 +tom robinson 0.6462715105162524 +tom robinson 0.9980879541108987 +tom steinbeck 0.5817490494296578 +tom van buren 0.12737642585551331 +tom van buren 0.3154875717017208 +tom van buren 0.7585551330798479 +tom white 0.47609942638623326 +tom young 0.9369024856596558 +tom young 0.9543726235741445 +tom zipper 0.9063097514340345 +ulysses brown 0.9448669201520913 +ulysses carson 0.07034220532319392 +ulysses carson 0.09885931558935361 +ulysses carson 0.2414448669201521 +ulysses carson 0.7604562737642585 +ulysses davidson 0.7093690248565966 +ulysses ellison 0.55893536121673 +ulysses garcia 0.7246653919694073 +ulysses hernandez 0.4091778202676864 +ulysses hernandez 0.627151051625239 +ulysses hernandez 0.982791586998088 +ulysses ichabod 0.21482889733840305 +ulysses ichabod 0.3193916349809886 +ulysses johnson 0.5621414913957935 +ulysses king 0.9467680608365019 +ulysses laertes 0.390057361376673 +ulysses laertes 0.7973231357552581 +ulysses laertes 0.9866156787762906 +ulysses miller 0.31166347992351817 +ulysses miller 0.5774378585086042 +ulysses nixon 0.0057361376673040155 +ulysses ovid 0.38593155893536124 +ulysses polk 0.04752851711026616 +ulysses polk 0.6083650190114068 +ulysses polk 0.7609942638623327 +ulysses polk 0.8326996197718631 +ulysses quirinius 0.6290630975143403 +ulysses robinson 0.9235181644359465 +ulysses steinbeck 0.039923954372623575 +ulysses steinbeck 0.7724665391969407 +ulysses thompson 0.3824091778202677 +ulysses underhill 0.11406844106463879 +ulysses underhill 0.23574144486692014 +ulysses underhill 0.3365019011406844 +ulysses underhill 0.42585551330798477 +ulysses underhill 0.6102661596958175 +ulysses underhill 0.6959847036328872 +ulysses underhill 0.9752851711026616 +ulysses van buren 0.5437262357414449 +ulysses white 0.5 +ulysses white 0.5931558935361216 +ulysses xylophone 0.5855513307984791 +ulysses xylophone 0.8317399617590823 +ulysses xylophone 0.9005736137667304 +ulysses young 0.18164435946462715 +ulysses young 0.3919694072657744 +ulysses young 0.49049429657794674 +victor allen 0.13575525812619502 +victor allen 0.6309751434034416 +victor brown 0.0497131931166348 +victor brown 0.20267686424474188 +victor brown 0.6178707224334601 +victor brown 0.8910133843212237 +victor davidson 0.026615969581749048 +victor davidson 0.491395793499044 +victor davidson 0.5850860420650096 +victor ellison 0.26425855513307983 +victor ellison 0.6692015209125475 +victor hernandez 0.04397705544933078 +victor hernandez 0.12927756653992395 +victor hernandez 0.1950286806883365 +victor hernandez 0.5411089866156787 +victor hernandez 0.7284894837476099 +victor johnson 0.11977186311787072 +victor johnson 0.4828897338403042 +victor johnson 0.7699619771863118 +victor king 0.41254752851711024 +victor king 0.714828897338403 +victor laertes 0.43155893536121676 +victor laertes 0.6500956022944551 +victor miller 0.4429657794676806 +victor nixon 0.33269961977186313 +victor nixon 0.5258126195028681 +victor ovid 0.22813688212927757 +victor polk 0.13878326996197718 +victor quirinius 0.13766730401529637 +victor quirinius 0.887189292543021 +victor robinson 0.5494296577946768 +victor robinson 0.7509505703422054 +victor steinbeck 0.08365019011406843 +victor steinbeck 0.15487571701720843 +victor steinbeck 0.3669201520912547 +victor thompson 0.10516252390057361 +victor van buren 0.27724665391969405 +victor van buren 0.9579349904397706 +victor white 0.41634980988593157 +victor white 0.6349809885931559 +victor xylophone 0.13688212927756654 +victor xylophone 0.3078393881453155 +victor xylophone 0.4110898661567878 +victor xylophone 0.5449330783938815 +victor xylophone 0.9296577946768061 +victor young 0.18738049713193117 +victor zipper 0.5430210325047801 +wendy allen 0.3231357552581262 +wendy allen 0.734225621414914 +wendy allen 0.869980879541109 +wendy brown 0.18929254302103252 +wendy brown 0.6996197718631179 +wendy ellison 0.7437858508604207 +wendy ellison 0.8498098859315589 +wendy falkner 0.07648183556405354 +wendy falkner 0.5353728489483748 +wendy falkner 0.7756653992395437 +wendy garcia 0.07074569789674952 +wendy garcia 0.0741444866920152 +wendy garcia 0.33840304182509506 +wendy garcia 0.38783269961977185 +wendy hernandez 0.017110266159695818 +wendy ichabod 0.8718929254302104 +wendy king 0.37072243346007605 +wendy king 0.497131931166348 +wendy king 0.5965583173996176 +wendy laertes 0.32122370936902483 +wendy laertes 0.49904397705544934 +wendy laertes 0.876425855513308 +wendy miller 0.7533460803059273 +wendy miller 0.7552581261950286 +wendy nixon 0.44933078393881454 +wendy nixon 0.7661596958174905 +wendy ovid 0.5019011406844106 +wendy ovid 0.6978967495219885 +wendy polk 0.3688212927756654 +wendy polk 0.526615969581749 +wendy quirinius 0.1444866920152091 +wendy quirinius 0.5874524714828897 +wendy robinson 0.030592734225621414 +wendy robinson 0.06692160611854685 +wendy robinson 0.27566539923954375 +wendy steinbeck 0.5703422053231939 +wendy thompson 0.028517110266159697 +wendy thompson 0.11089866156787763 +wendy underhill 0.4837476099426386 +wendy underhill 0.6424474187380497 +wendy underhill 0.9600760456273765 +wendy van buren 0.1920152091254753 +wendy van buren 0.7433460076045627 +wendy white 0.752851711026616 +wendy xylophone 0.6347992351816444 +wendy xylophone 0.7452471482889734 +wendy young 0.07839388145315487 +wendy young 0.3897338403041825 +xavier allen 0.043726235741444866 +xavier allen 0.361376673040153 +xavier allen 0.5456273764258555 +xavier brown 0.6711281070745698 +xavier brown 0.9158699808795411 +xavier brown 0.9847036328871893 +xavier carson 0.0841300191204589 +xavier carson 0.988527724665392 +xavier davidson 0.2585551330798479 +xavier davidson 0.4168260038240918 +xavier davidson 0.609942638623327 +xavier ellison 0.5984703632887189 +xavier ellison 0.7361376673040153 +xavier garcia 0.7017208413001912 +xavier hernandez 0.2509505703422053 +xavier hernandez 0.34990439770554493 +xavier hernandez 0.9220532319391636 +xavier ichabod 0.5475285171102662 +xavier ichabod 0.858508604206501 +xavier johnson 0.3938814531548757 +xavier johnson 0.8231939163498099 +xavier king 0.03231939163498099 +xavier king 0.6539196940726577 +xavier laertes 0.5988593155893536 +xavier ovid 0.4397705544933078 +xavier polk 0.4933078393881453 +xavier polk 0.762906309751434 +xavier polk 0.8136882129277566 +xavier polk 0.8260038240917782 +xavier quirinius 0.07265774378585087 +xavier quirinius 0.27915869980879543 +xavier quirinius 0.34600760456273766 +xavier quirinius 0.8022813688212928 +xavier thompson 0.6118546845124283 +xavier underhill 0.16634799235181644 +xavier white 0.6958174904942965 +xavier white 0.7380497131931166 +xavier xylophone 0.8183556405353728 +xavier zipper 0.9904397705544933 +yuri allen 0.9106463878326996 +yuri allen 1.0 +yuri brown 0.5152091254752852 +yuri brown 0.908745247148289 +yuri carson 0.09560229445506692 +yuri carson 0.9372623574144486 +yuri ellison 0.017208413001912046 +yuri ellison 0.39923954372623577 +yuri falkner 0.28680688336520077 +yuri falkner 0.8967495219885278 +yuri garcia 0.2661596958174905 +yuri hernandez 0.28489483747609945 +yuri johnson 0.5047801147227533 +yuri johnson 0.655831739961759 +yuri johnson 0.720532319391635 +yuri king 0.32129277566539927 +yuri laertes 0.4144486692015209 +yuri laertes 0.8916349809885932 +yuri nixon 0.05162523900573614 +yuri nixon 0.40874524714828897 +yuri polk 0.051330798479087454 +yuri polk 0.39579349904397704 +yuri polk 0.6749049429657795 +yuri quirinius 0.08030592734225621 +yuri quirinius 0.2982791586998088 +yuri quirinius 0.4130019120458891 +yuri steinbeck 0.15779467680608364 +yuri steinbeck 0.9388145315487572 +yuri thompson 0.6175908221797323 +yuri underhill 0.42447418738049714 +yuri underhill 0.8202676864244742 +yuri white 0.19694072657743786 +yuri xylophone 0.4790874524714829 +zach allen 0.8250950570342205 +zach brown 0.0817490494296578 +zach brown 0.09751434034416825 +zach brown 0.248565965583174 +zach brown 0.2965779467680608 +zach brown 0.4524714828897338 +zach carson 0.6921606118546845 +zach ellison 0.6806083650190115 +zach falkner 0.25812619502868067 +zach falkner 0.2695984703632887 +zach garcia 0.30798479087452474 +zach garcia 0.3632887189292543 +zach garcia 0.7072243346007605 +zach garcia 0.7167300380228137 +zach ichabod 0.30988593155893535 +zach ichabod 0.9502868068833652 +zach king 0.5277246653919694 +zach king 0.8336520076481836 +zach king 0.9239543726235742 +zach miller 0.15678776290630975 +zach miller 0.3726235741444867 +zach miller 0.5608365019011406 +zach ovid 0.1311787072243346 +zach ovid 0.2737642585551331 +zach ovid 0.4543726235741445 +zach ovid 0.6711026615969582 +zach quirinius 0.019011406844106463 +zach robinson 0.11026615969581749 +zach steinbeck 0.28107074569789675 +zach steinbeck 0.7170172084130019 +zach thompson 0.13001912045889102 +zach thompson 0.44550669216061184 +zach underhill 0.7718631178707225 +zach white 0.7965779467680608 +zach xylophone 0.032504780114722756 +zach xylophone 0.638623326959847 +zach young 0.009560229445506692 +zach zipper 0.24334600760456274 +zach zipper 0.2832699619771863 +zach zipper 0.8087954110898662 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 new file mode 100644 index 0000000000000..1e0cf03db63a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 1-3-b82dfa24123047be4b4e3c27c3997d34 @@ -0,0 +1,1049 @@ +zach zipper 0.0 +zach zipper 0.0 +zach zipper 0.0 +zach young 0.0 +zach xylophone 0.0 +zach xylophone 0.0 +zach white 0.0 +zach underhill 0.0 +zach thompson 0.0 +zach thompson 0.0 +zach steinbeck 0.0 +zach steinbeck 0.0 +zach robinson 0.0 +zach quirinius 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach ovid 0.0 +zach miller 0.0 +zach miller 0.0 +zach miller 0.0 +zach king 0.0 +zach king 0.0 +zach king 0.0 +zach ichabod 0.0 +zach ichabod 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach garcia 0.0 +zach falkner 0.0 +zach falkner 0.0 +zach ellison 0.0 +zach carson 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach brown 0.0 +zach allen 0.0 +yuri xylophone 0.0 +yuri white 0.0 +yuri underhill 0.0 +yuri underhill 0.0 +yuri thompson 0.0 +yuri steinbeck 0.0 +yuri steinbeck 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri quirinius 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri polk 0.0 +yuri nixon 0.0 +yuri nixon 0.0 +yuri laertes 0.0 +yuri laertes 0.0 +yuri king 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri johnson 0.0 +yuri hernandez 0.0 +yuri garcia 0.0 +yuri falkner 0.0 +yuri falkner 0.0 +yuri ellison 0.0 +yuri ellison 0.0 +yuri carson 0.0 +yuri carson 0.0 +yuri brown 0.0 +yuri brown 0.0 +yuri allen 0.0 +yuri allen 0.0 +xavier zipper 1.0 +xavier xylophone 0.0 +xavier white 0.0 +xavier white 0.0 +xavier underhill 0.0 +xavier thompson 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier quirinius 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier polk 0.0 +xavier ovid 0.0 +xavier laertes 0.0 +xavier king 0.0 +xavier king 0.0 +xavier johnson 0.0 +xavier johnson 0.0 +xavier ichabod 0.0 +xavier ichabod 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier hernandez 0.0 +xavier garcia 0.0 +xavier ellison 0.0 +xavier ellison 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier davidson 0.0 +xavier carson 1.0 +xavier carson 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier brown 0.0 +xavier allen 0.0 +xavier allen 0.0 +xavier allen 0.0 +wendy young 1.0 +wendy young 0.0 +wendy xylophone 0.0 +wendy xylophone 0.0 +wendy white 0.0 +wendy van buren 0.0 +wendy van buren 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy underhill 0.0 +wendy thompson 0.0 +wendy thompson 0.0 +wendy steinbeck 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy robinson 0.0 +wendy quirinius 0.0 +wendy quirinius 0.0 +wendy polk 0.0 +wendy polk 0.0 +wendy ovid 0.0 +wendy ovid 0.0 +wendy nixon 0.0 +wendy nixon 0.0 +wendy miller 0.0 +wendy miller 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy laertes 0.0 +wendy king 0.0 +wendy king 0.0 +wendy king 0.0 +wendy ichabod 0.0 +wendy hernandez 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy garcia 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy falkner 0.0 +wendy ellison 0.0 +wendy ellison 0.0 +wendy brown 0.0 +wendy brown 0.0 +wendy allen 0.0 +wendy allen 0.0 +wendy allen 0.0 +victor zipper 0.0 +victor young 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor xylophone 0.0 +victor white 1.0 +victor white 0.0 +victor van buren 0.0 +victor van buren 0.0 +victor thompson 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor steinbeck 0.0 +victor robinson 0.0 +victor robinson 0.0 +victor quirinius 0.0 +victor quirinius 0.0 +victor polk 0.0 +victor ovid 0.0 +victor nixon 0.0 +victor nixon 0.0 +victor miller 0.0 +victor laertes 0.0 +victor laertes 0.0 +victor king 0.0 +victor king 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor johnson 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor hernandez 0.0 +victor ellison 0.0 +victor ellison 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor davidson 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor brown 0.0 +victor allen 0.0 +victor allen 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses young 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses xylophone 0.0 +ulysses white 0.0 +ulysses white 0.0 +ulysses van buren 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses underhill 0.0 +ulysses thompson 1.0 +ulysses steinbeck 0.0 +ulysses steinbeck 0.0 +ulysses robinson 0.0 +ulysses quirinius 0.0 +ulysses polk 1.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses polk 0.0 +ulysses ovid 0.0 +ulysses nixon 0.0 +ulysses miller 0.0 +ulysses miller 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses laertes 0.0 +ulysses king 0.0 +ulysses johnson 0.0 +ulysses ichabod 0.0 +ulysses ichabod 0.0 +ulysses hernandez 1.0 +ulysses hernandez 0.0 +ulysses hernandez 0.0 +ulysses garcia 0.0 +ulysses ellison 1.0 +ulysses davidson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses carson 0.0 +ulysses brown 0.0 +tom zipper 0.0 +tom young 0.0 +tom young 0.0 +tom white 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom van buren 0.0 +tom steinbeck 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom robinson 0.0 +tom quirinius 0.0 +tom quirinius 0.0 +tom polk 0.0 +tom polk 0.0 +tom ovid 0.0 +tom nixon 0.0 +tom miller 0.0 +tom miller 0.0 +tom miller 0.0 +tom laertes 0.0 +tom laertes 0.0 +tom king 0.0 +tom johnson 0.0 +tom johnson 0.0 +tom ichabod 0.0 +tom hernandez 0.0 +tom hernandez 0.0 +tom falkner 0.0 +tom falkner 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom ellison 0.0 +tom davidson 0.0 +tom carson 0.0 +tom carson 0.0 +tom carson 0.0 +tom brown 0.0 +tom brown 0.0 +sarah zipper 0.0 +sarah young 0.0 +sarah xylophone 0.0 +sarah white 0.0 +sarah white 0.0 +sarah steinbeck 0.0 +sarah robinson 0.0 +sarah robinson 0.0 +sarah ovid 0.0 +sarah miller 0.0 +sarah king 0.0 +sarah king 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah johnson 0.0 +sarah ichabod 0.0 +sarah ichabod 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah garcia 0.0 +sarah falkner 0.0 +sarah falkner 0.0 +sarah ellison 0.0 +sarah carson 0.0 +sarah carson 0.0 +sarah carson 0.0 +rachel zipper 0.0 +rachel zipper 0.0 +rachel young 0.0 +rachel white 0.0 +rachel white 0.0 +rachel underhill 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel thompson 0.0 +rachel robinson 1.0 +rachel robinson 0.0 +rachel robinson 0.0 +rachel quirinius 0.0 +rachel polk 0.0 +rachel ovid 0.0 +rachel ovid 0.0 +rachel laertes 0.0 +rachel laertes 0.0 +rachel king 0.0 +rachel king 0.0 +rachel johnson 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel falkner 0.0 +rachel ellison 0.0 +rachel davidson 0.0 +rachel carson 0.0 +rachel carson 0.0 +rachel brown 1.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel brown 0.0 +rachel allen 0.0 +rachel allen 0.0 +quinn zipper 0.0 +quinn zipper 0.0 +quinn young 0.0 +quinn van buren 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn underhill 0.0 +quinn thompson 0.0 +quinn thompson 0.0 +quinn steinbeck 0.0 +quinn steinbeck 0.0 +quinn robinson 0.0 +quinn quirinius 0.0 +quinn ovid 0.0 +quinn nixon 0.0 +quinn laertes 1.0 +quinn laertes 0.0 +quinn laertes 0.0 +quinn king 1.0 +quinn king 0.0 +quinn ichabod 0.0 +quinn garcia 1.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn garcia 0.0 +quinn ellison 0.0 +quinn ellison 0.0 +quinn davidson 1.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn davidson 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn brown 0.0 +quinn allen 1.0 +quinn allen 0.0 +priscilla zipper 0.0 +priscilla zipper 0.0 +priscilla young 0.0 +priscilla young 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla xylophone 0.0 +priscilla white 1.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla van buren 0.0 +priscilla underhill 0.0 +priscilla underhill 0.0 +priscilla thompson 0.0 +priscilla quirinius 0.0 +priscilla polk 0.0 +priscilla ovid 0.0 +priscilla ovid 0.0 +priscilla nixon 0.0 +priscilla nixon 0.0 +priscilla king 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla johnson 0.0 +priscilla ichabod 0.0 +priscilla ichabod 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla carson 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +priscilla brown 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar zipper 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar xylophone 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar white 0.0 +oscar van buren 1.0 +oscar van buren 0.0 +oscar van buren 0.0 +oscar underhill 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar thompson 0.0 +oscar steinbeck 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar robinson 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar quirinius 0.0 +oscar polk 1.0 +oscar polk 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar ovid 0.0 +oscar nixon 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar laertes 0.0 +oscar king 0.0 +oscar king 0.0 +oscar king 0.0 +oscar johnson 0.0 +oscar johnson 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar ichabod 0.0 +oscar hernandez 0.0 +oscar hernandez 0.0 +oscar garcia 0.0 +oscar falkner 1.0 +oscar ellison 0.0 +oscar ellison 0.0 +oscar davidson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar carson 0.0 +oscar brown 0.0 +oscar allen 0.0 +nick zipper 0.0 +nick zipper 0.0 +nick young 1.0 +nick young 0.0 +nick xylophone 0.0 +nick van buren 0.0 +nick underhill 0.0 +nick thompson 0.0 +nick steinbeck 0.0 +nick robinson 0.0 +nick robinson 0.0 +nick quirinius 0.0 +nick quirinius 0.0 +nick polk 0.0 +nick ovid 0.0 +nick nixon 0.0 +nick miller 0.0 +nick laertes 0.0 +nick johnson 0.0 +nick johnson 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick ichabod 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick garcia 0.0 +nick falkner 0.0 +nick falkner 0.0 +nick ellison 0.0 +nick ellison 0.0 +nick davidson 0.0 +nick brown 0.0 +nick allen 0.0 +nick allen 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike zipper 0.0 +mike young 0.0 +mike young 0.0 +mike young 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike white 0.0 +mike van buren 0.0 +mike van buren 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike steinbeck 0.0 +mike quirinius 0.0 +mike polk 0.0 +mike polk 0.0 +mike polk 0.0 +mike nixon 0.0 +mike nixon 0.0 +mike miller 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike king 0.0 +mike ichabod 0.0 +mike hernandez 0.0 +mike hernandez 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike garcia 0.0 +mike falkner 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike ellison 0.0 +mike davidson 0.0 +mike davidson 0.0 +mike carson 0.0 +mike carson 0.0 +mike carson 0.0 +mike brown 0.0 +mike allen 0.0 +luke zipper 0.0 +luke xylophone 0.0 +luke white 0.0 +luke van buren 0.0 +luke underhill 1.0 +luke underhill 0.0 +luke underhill 0.0 +luke thompson 0.0 +luke robinson 0.0 +luke robinson 0.0 +luke quirinius 0.0 +luke polk 0.0 +luke polk 0.0 +luke ovid 0.0 +luke ovid 0.0 +luke miller 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke laertes 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke johnson 0.0 +luke ichabod 0.0 +luke ichabod 0.0 +luke garcia 0.0 +luke garcia 0.0 +luke falkner 0.0 +luke falkner 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke ellison 0.0 +luke davidson 0.0 +luke davidson 0.0 +luke brown 0.0 +luke allen 1.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +luke allen 0.0 +katie zipper 1.0 +katie zipper 0.0 +katie young 1.0 +katie young 0.0 +katie young 0.0 +katie xylophone 0.0 +katie white 0.0 +katie white 0.0 +katie van buren 0.0 +katie van buren 0.0 +katie robinson 0.0 +katie polk 0.0 +katie polk 0.0 +katie ovid 0.0 +katie nixon 1.0 +katie miller 0.0 +katie miller 0.0 +katie king 0.0 +katie king 0.0 +katie king 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie ichabod 0.0 +katie hernandez 0.0 +katie garcia 0.0 +katie garcia 0.0 +katie falkner 0.0 +katie ellison 0.0 +katie ellison 0.0 +katie davidson 0.0 +katie brown 0.0 +katie allen 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica zipper 0.0 +jessica young 0.0 +jessica young 0.0 +jessica xylophone 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica white 0.0 +jessica van buren 0.0 +jessica underhill 1.0 +jessica underhill 0.0 +jessica underhill 0.0 +jessica thompson 0.0 +jessica thompson 0.0 +jessica robinson 0.0 +jessica quirinius 1.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica quirinius 0.0 +jessica polk 0.0 +jessica ovid 0.0 +jessica ovid 0.0 +jessica nixon 0.0 +jessica nixon 0.0 +jessica miller 0.0 +jessica johnson 0.0 +jessica johnson 0.0 +jessica ichabod 1.0 +jessica garcia 0.0 +jessica garcia 0.0 +jessica falkner 0.0 +jessica ellison 0.0 +jessica ellison 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica davidson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica carson 0.0 +jessica brown 0.0 +irene xylophone 0.0 +irene van buren 0.0 +irene van buren 0.0 +irene underhill 0.0 +irene underhill 0.0 +irene thompson 0.0 +irene steinbeck 0.0 +irene robinson 0.0 +irene quirinius 0.5 +irene quirinius 0.0 +irene quirinius 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene polk 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene ovid 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene nixon 0.0 +irene miller 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene laertes 0.0 +irene johnson 0.0 +irene ichabod 0.0 +irene ichabod 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene garcia 0.0 +irene falkner 0.0 +irene falkner 0.0 +irene ellison 0.0 +irene ellison 0.0 +irene carson 0.0 +irene brown 0.0 +irene brown 0.0 +irene brown 0.0 +irene allen 0.0 +holly zipper 1.0 +holly zipper 0.0 +holly young 0.0 +holly young 0.0 +holly xylophone 0.0 +holly white 0.0 +holly white 0.0 +holly van buren 0.0 +holly underhill 1.0 +holly underhill 0.0 +holly underhill 0.0 +holly underhill 0.0 +holly thompson 1.0 +holly thompson 0.0 +holly thompson 0.0 +holly robinson 0.0 +holly polk 0.0 +holly polk 0.0 +holly nixon 0.0 +holly nixon 0.0 +holly miller 1.0 +holly laertes 0.0 +holly king 0.0 +holly king 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly johnson 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly ichabod 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly hernandez 0.0 +holly falkner 0.0 +holly brown 0.0 +holly brown 0.0 +holly allen 0.0 +gabriella zipper 0.0 +gabriella zipper 0.0 +gabriella young 0.0 +gabriella young 0.0 +gabriella white 0.0 +gabriella van buren 0.0 +gabriella van buren 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella thompson 0.0 +gabriella steinbeck 0.0 +gabriella steinbeck 0.0 +gabriella polk 0.0 +gabriella polk 0.0 +gabriella ovid 0.0 +gabriella ovid 0.0 +gabriella miller 0.0 +gabriella laertes 0.0 +gabriella king 0.0 +gabriella king 0.0 +gabriella ichabod 1.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella ichabod 0.0 +gabriella hernandez 1.0 +gabriella hernandez 0.0 +gabriella garcia 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella falkner 0.0 +gabriella ellison 0.0 +gabriella ellison 0.0 +gabriella davidson 0.0 +gabriella carson 0.0 +gabriella brown 0.0 +gabriella brown 0.0 +gabriella allen 0.0 +gabriella allen 0.0 +fred zipper 0.0 +fred young 0.0 +fred young 0.0 +fred white 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred van buren 0.0 +fred underhill 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred steinbeck 0.0 +fred robinson 1.0 +fred quirinius 0.0 +fred quirinius 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred polk 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred nixon 0.0 +fred miller 0.0 +fred laertes 0.0 +fred king 0.0 +fred king 0.0 +fred johnson 1.0 +fred ichabod 0.0 +fred ichabod 0.0 +fred hernandez 0.0 +fred falkner 1.0 +fred falkner 0.0 +fred falkner 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred ellison 0.0 +fred davidson 0.0 +fred davidson 0.0 +fred davidson 0.0 +ethan zipper 0.0 +ethan zipper 0.0 +ethan xylophone 0.0 +ethan white 0.0 +ethan white 0.0 +ethan van buren 0.0 +ethan underhill 0.5 +ethan robinson 0.0 +ethan robinson 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan quirinius 0.0 +ethan polk 1.0 +ethan polk 0.0 +ethan polk 0.0 +ethan polk 0.0 +ethan ovid 0.0 +ethan nixon 0.0 +ethan miller 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan laertes 0.0 +ethan king 0.0 +ethan johnson 0.0 +ethan hernandez 0.0 +ethan garcia 0.0 +ethan falkner 0.0 +ethan falkner 0.0 +ethan ellison 0.0 +ethan ellison 0.0 +ethan carson 0.0 +ethan brown 1.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan brown 0.0 +ethan allen 0.0 +david young 0.0 +david young 0.0 +david xylophone 1.0 +david xylophone 0.0 +david xylophone 0.0 +david white 0.0 +david van buren 0.0 +david van buren 0.0 +david underhill 0.0 +david underhill 0.0 +david underhill 0.0 +david thompson 1.0 +david robinson 0.0 +david robinson 0.0 +david quirinius 0.0 +david quirinius 0.0 +david quirinius 0.0 +david ovid 0.0 +david ovid 0.0 +david nixon 0.0 +david laertes 0.0 +david ichabod 1.0 +david ichabod 0.0 +david hernandez 1.0 +david ellison 0.0 +david ellison 0.0 +david ellison 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david davidson 0.0 +david brown 0.0 +david brown 0.0 +david allen 0.0 +david allen 0.0 +calvin zipper 0.0 +calvin zipper 0.0 +calvin young 0.0 +calvin young 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin xylophone 0.0 +calvin white 0.0 +calvin white 0.0 +calvin van buren 1.0 +calvin van buren 0.0 +calvin underhill 0.0 +calvin thompson 0.0 +calvin thompson 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin steinbeck 0.0 +calvin robinson 0.0 +calvin quirinius 0.0 +calvin quirinius 0.0 +calvin polk 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin ovid 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin nixon 0.0 +calvin laertes 0.0 +calvin laertes 0.0 +calvin johnson 0.0 +calvin hernandez 0.0 +calvin garcia 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin falkner 0.0 +calvin ellison 0.0 +calvin davidson 0.0 +calvin davidson 0.0 +calvin carson 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin brown 0.0 +calvin allen 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob zipper 0.0 +bob young 0.0 +bob xylophone 0.0 +bob xylophone 0.0 +bob white 0.0 +bob white 0.0 +bob van buren 0.0 +bob steinbeck 0.0 +bob quirinius 0.0 +bob polk 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob ovid 0.0 +bob miller 0.0 +bob laertes 0.0 +bob laertes 0.0 +bob king 1.0 +bob king 0.0 +bob king 0.0 +bob ichabod 0.0 +bob hernandez 1.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob garcia 0.0 +bob falkner 0.0 +bob ellison 1.0 +bob ellison 0.0 +bob ellison 0.0 +bob ellison 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob davidson 0.0 +bob carson 0.0 +bob brown 0.0 +bob brown 0.0 +bob brown 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice zipper 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice xylophone 0.0 +alice van buren 0.0 +alice underhill 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice steinbeck 0.0 +alice robinson 0.0 +alice robinson 0.0 +alice quirinius 0.0 +alice quirinius 0.0 +alice polk 1.0 +alice ovid 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice nixon 0.0 +alice miller 0.0 +alice laertes 0.0 +alice laertes 0.0 +alice king 0.0 +alice king 0.0 +alice king 0.0 +alice johnson 0.0 +alice hernandez 0.0 +alice hernandez 0.0 +alice garcia 0.0 +alice falkner 0.0 +alice davidson 0.0 +alice carson 0.0 +alice brown 0.0 +alice allen 0.0 +alice allen 0.0 +alice allen 0.0 + 0.0 + 0.0 + 0.0 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 new file mode 100644 index 0000000000000..9091a9156134c --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -0,0 +1,1294 @@ +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.70307 52.64 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703071 71.07 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703072 2.96 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703073 10.07 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703075 5.64 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703077 10.16 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703078 61.52 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.703079 27.32 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.70308 1.76 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703082 37.25 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703083 20.52 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703084 1.76 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703085 1.01 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703086 9.96 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703087 10.63 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.703089 8.76 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.70309 50.99 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703091 15.85 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703092 36.84 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703093 14.85 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703094 57.11 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703095 9.77 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703096 11.64 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703098 1.35 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703099 11.69 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703101 8.72 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703102 63.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703103 8.65 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703104 2.04 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703105 28.47 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703106 11.81 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703107 16.26 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703108 28.47 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.703109 38.98 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.70311 8.16 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703114 73.94 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703115 27.52 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703116 33.45 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703117 21.81 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.703119 58.02 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703122 53.56 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703123 94.35 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703124 8.93 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703125 14.94 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703126 5.49 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703127 3.98 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.703128 11.45 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.70313 5.83 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703131 1.75 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703133 27.34 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703135 29.14 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703136 11.87 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703137 18.11 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703138 55.68 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.703139 12.67 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.70314 2.83 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703141 76.06 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703142 24.25 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703143 26.86 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703144 3.43 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703145 8.46 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703146 89.12 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703147 54.94 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703148 26.97 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.703149 58.05 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.70315 33.01 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703151 95.69 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703152 6.85 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703153 4.11 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703155 6.93 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703156 21.79 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703157 1.29 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703158 71.89 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.703159 64.06 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.70316 11.93 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703161 15.82 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703162 3.51 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703164 30.27 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703165 8.38 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703167 17.66 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703168 32.03 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.703169 39.96 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.70317 11.44 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703171 24.94 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703172 3.44 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703173 8.77 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703175 33.37 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703177 11.43 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703178 9.12 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.703179 10.82 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.70318 10.28 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703182 1.23 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703183 36.74 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703184 8.95 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703186 13.81 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703187 64.89 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.703189 36.96 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.70319 90.09 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703192 2.63 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703193 28.42 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703194 15.07 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703195 3.81 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703196 0.08 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703197 16.01 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.703199 45.69 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.7032 12.72 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703201 35.15 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703202 31.41 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703203 11.63 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703206 6.61 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703207 21.14 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703208 1.23 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703211 5.24 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703212 10.52 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703213 38.71 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703214 31.35 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703215 18.78 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703216 11.23 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703217 23.57 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703218 9.67 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.703219 1.42 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703222 38.23 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703223 3.43 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703224 17.92 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703225 35.51 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703226 34.31 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703227 17.65 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703228 4.19 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.703229 88.52 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.70323 28.02 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703231 11.99 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703232 61.96 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703235 6.35 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703237 0.24 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703241 19.33 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703243 6.15 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703244 4.47 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703245 0.72 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703246 45.94 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703247 1.29 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703248 14.36 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.703249 19.42 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.70325 25.89 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703251 68.98 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703252 49.85 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703253 55.75 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703255 5.42 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703256 23.78 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703257 14.96 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703258 19.65 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.703259 11.37 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.70326 3.81 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703261 8.66 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703262 1.81 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703265 6.88 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703266 47.71 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703267 12.22 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703268 94.73 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.703269 43.84 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.70327 5.01 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703271 61.16 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 10.94 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703274 10.79 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703275 20.57 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703276 63.54 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703277 3.37 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703278 9.74 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.703279 20.85 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.70328 40.68 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703281 19.95 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703284 99.88 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703285 58.66 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703286 9.53 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703287 0.89 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703288 60.57 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.70329 16.89 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703291 1.15 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703292 4.24 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703295 8.58 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703296 30.43 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703297 25.67 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.7033 7.51 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703301 39.32 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703302 23.55 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703303 88.64 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703304 9.04 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703305 18.68 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703306 3.95 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703307 31.28 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703308 16.95 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.703309 11.16 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.70331 9.24 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703311 7.38 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703313 9.35 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703314 39.12 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703315 4.93 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703316 16.86 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703317 1.45 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703318 21.81 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.703319 83.21 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.70332 77.09 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703321 3.91 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703322 2.48 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703323 36.22 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703324 14.08 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 +2013-03-01 09:11:58.703325 9.24 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 3-0-58a982694ba2b1e34de82b1de54936a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 new file mode 100644 index 0000000000000..d02ca48857b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -0,0 +1,474 @@ +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703074 58.47 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703076 18.51 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703077 66.68 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703087 25.19 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703088 1.97 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703089 41.57 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703091 68.85 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703092 54.02 1 +2013-03-01 09:11:58.703096 87.84 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703098 21.29 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.703104 75.85 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.70311 65.88 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703111 85.94 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703112 13.29 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703113 58.65 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.703118 8.69 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703125 78.52 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703131 63.81 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703132 1.86 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703135 88.38 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703136 27.89 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703139 43.53 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703144 21.59 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703154 16.93 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703156 62.42 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703157 8.99 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703174 36.33 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703178 93.29 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703179 60.94 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703184 73.93 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703185 8.91 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703186 91.46 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703188 32.85 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703189 37.74 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703198 97.18 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703206 80.94 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703207 55.06 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.703209 25.92 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.70321 37.12 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.703219 32.73 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.70322 7.37 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703221 26.64 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703224 42.93 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703233 40.81 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703242 31.23 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703244 25.67 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703246 72.87 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703248 81.28 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.70325 93.79 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703254 0.32 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703258 21.21 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703259 52.28 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703262 78.56 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703264 52.49 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703265 11.46 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703266 83.67 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703269 61.06 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703272 7.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703273 30.49 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703275 90.69 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703276 88.46 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.703278 69.42 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.70328 45.81 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703281 62.11 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703283 17.62 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703289 0.56 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703293 42.86 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703294 29.74 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703296 43.66 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703299 23.19 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.70331 69.26 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703313 20.69 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703315 53.04 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703318 85.62 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703323 65.61 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703324 98.36 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 +2013-03-01 09:11:58.703325 65.81 1 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c new file mode 100644 index 0000000000000..119dd71df1426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-0-6642a21d87e0401ba1a668ea8b244f0c @@ -0,0 +1,1049 @@ + 65560 + 65560 + 65560 +alice allen 65662 +alice allen 65662 +alice allen 65662 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65737 +alice johnson 65739 +alice king 65660 +alice king 65660 +alice king 65660 +alice laertes 65669 +alice laertes 65669 +alice miller 65590 +alice nixon 65586 +alice nixon 65586 +alice nixon 65586 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65636 +alice robinson 65606 +alice robinson 65606 +alice steinbeck 65578 +alice steinbeck 65578 +alice steinbeck 65578 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65578 +alice xylophone 65578 +alice xylophone 65578 +alice zipper 65553 +alice zipper 65553 +alice zipper 65553 +bob brown 65584 +bob brown 65584 +bob brown 65584 +bob carson 65713 +bob davidson 65664 +bob davidson 65664 +bob davidson 65664 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob ellison 65591 +bob falkner 65789 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob garcia 65585 +bob hernandez 65557 +bob ichabod 65549 +bob king 65715 +bob king 65715 +bob king 65715 +bob laertes 65602 +bob laertes 65602 +bob miller 65608 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob ovid 65564 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65543 +bob xylophone 65574 +bob xylophone 65574 +bob young 65556 +bob zipper 65559 +bob zipper 65559 +bob zipper 65559 +calvin allen 65669 +calvin brown 65537 +calvin brown 65537 +calvin brown 65537 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65541 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin falkner 65573 +calvin garcia 65664 +calvin hernandez 65578 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65570 +calvin nixon 65654 +calvin nixon 65654 +calvin nixon 65654 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin ovid 65554 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65741 +calvin robinson 65581 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin steinbeck 65680 +calvin thompson 65560 +calvin thompson 65560 +calvin underhill 65732 +calvin van buren 65552 +calvin van buren 65552 +calvin white 65553 +calvin white 65553 +calvin xylophone 65575 +calvin xylophone 65575 +calvin xylophone 65575 +calvin young 65574 +calvin young 65574 +calvin zipper 65669 +calvin zipper 65669 +david allen 65588 +david allen 65588 +david brown 65637 +david brown 65637 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david davidson 65559 +david ellison 65634 +david ellison 65634 +david ellison 65634 +david hernandez 65763 +david ichabod 65699 +david ichabod 65699 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 65623 +david quirinius 65697 +david quirinius 65697 +david quirinius 65697 +david robinson 65762 +david robinson 65762 +david thompson 65550 +david underhill 65602 +david underhill 65602 +david underhill 65602 +david van buren 65625 +david van buren 65625 +david white 65678 +david xylophone 65537 +david xylophone 65537 +david xylophone 65537 +david young 65551 +david young 65551 +ethan allen 65747 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan brown 65539 +ethan carson 65742 +ethan ellison 65714 +ethan ellison 65714 +ethan falkner 65577 +ethan falkner 65577 +ethan garcia 65736 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 65614 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan laertes 65562 +ethan miller 65712 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan polk 65589 +ethan quirinius 65542 +ethan quirinius 65542 +ethan quirinius 65542 +ethan robinson 65547 +ethan robinson 65547 +ethan underhill 65570 +ethan van buren 65572 +ethan white 65677 +ethan white 65677 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 65593 +fred davidson 65552 +fred davidson 65552 +fred davidson 65552 +fred ellison 65548 +fred ellison 65548 +fred ellison 65548 +fred falkner 65637 +fred falkner 65637 +fred falkner 65637 +fred hernandez 65541 +fred ichabod 65572 +fred ichabod 65572 +fred johnson 65758 +fred king 65694 +fred king 65694 +fred laertes 65769 +fred miller 65536 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred nixon 65560 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred polk 65603 +fred quirinius 65697 +fred quirinius 65697 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65544 +fred steinbeck 65544 +fred underhill 65629 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred van buren 65537 +fred white 65589 +fred young 65594 +fred young 65594 +fred zipper 65553 +gabriella allen 65646 +gabriella allen 65646 +gabriella brown 65704 +gabriella brown 65704 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 65706 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella falkner 65623 +gabriella garcia 65571 +gabriella hernandez 65587 +gabriella hernandez 65587 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella ichabod 65559 +gabriella king 65651 +gabriella king 65651 +gabriella laertes 65781 +gabriella miller 65646 +gabriella ovid 65556 +gabriella ovid 65556 +gabriella polk 65701 +gabriella polk 65701 +gabriella steinbeck 65582 +gabriella steinbeck 65582 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella thompson 65682 +gabriella van buren 65581 +gabriella van buren 65581 +gabriella white 65638 +gabriella young 65699 +gabriella young 65699 +gabriella zipper 65540 +gabriella zipper 65540 +holly allen 65596 +holly brown 65599 +holly brown 65599 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly hernandez 65602 +holly ichabod 65711 +holly ichabod 65711 +holly ichabod 65711 +holly johnson 65655 +holly johnson 65655 +holly johnson 65655 +holly king 65549 +holly king 65549 +holly laertes 65664 +holly miller 65653 +holly nixon 65539 +holly nixon 65539 +holly polk 65743 +holly polk 65743 +holly robinson 65564 +holly thompson 65538 +holly thompson 65538 +holly thompson 65538 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly underhill 65634 +holly van buren 65727 +holly white 65536 +holly white 65536 +holly xylophone 65544 +holly young 65606 +holly young 65606 +holly zipper 65607 +holly zipper 65607 +irene allen 65556 +irene brown 65633 +irene brown 65633 +irene brown 65633 +irene carson 65590 +irene ellison 65659 +irene ellison 65659 +irene falkner 65620 +irene falkner 65620 +irene garcia 65660 +irene garcia 65660 +irene garcia 65660 +irene ichabod 65645 +irene ichabod 65645 +irene johnson 65583 +irene laertes 65664 +irene laertes 65664 +irene laertes 65664 +irene miller 65730 +irene nixon 65631 +irene nixon 65631 +irene nixon 65631 +irene ovid 65691 +irene ovid 65691 +irene ovid 65691 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene polk 65551 +irene quirinius 65724 +irene quirinius 65724 +irene quirinius 65724 +irene robinson 65554 +irene steinbeck 65683 +irene thompson 65688 +irene underhill 65591 +irene underhill 65591 +irene van buren 65579 +irene van buren 65579 +irene xylophone 65775 +jessica brown 65588 +jessica carson 65553 +jessica carson 65553 +jessica carson 65553 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica davidson 65549 +jessica ellison 65567 +jessica ellison 65567 +jessica falkner 65584 +jessica garcia 65676 +jessica garcia 65676 +jessica ichabod 65704 +jessica johnson 65607 +jessica johnson 65607 +jessica miller 65733 +jessica nixon 65590 +jessica nixon 65590 +jessica ovid 65582 +jessica ovid 65582 +jessica polk 65637 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica quirinius 65562 +jessica robinson 65576 +jessica thompson 65581 +jessica thompson 65581 +jessica underhill 65656 +jessica underhill 65656 +jessica underhill 65656 +jessica van buren 65615 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica white 65544 +jessica xylophone 65562 +jessica young 65623 +jessica young 65623 +jessica zipper 65600 +jessica zipper 65600 +jessica zipper 65600 +katie allen 65542 +katie brown 65590 +katie davidson 65619 +katie ellison 65675 +katie ellison 65675 +katie falkner 65728 +katie garcia 65625 +katie garcia 65625 +katie hernandez 65550 +katie ichabod 65658 +katie ichabod 65658 +katie ichabod 65658 +katie king 65629 +katie king 65629 +katie king 65629 +katie miller 65541 +katie miller 65541 +katie nixon 65669 +katie ovid 65681 +katie polk 65746 +katie polk 65746 +katie robinson 65697 +katie van buren 65643 +katie van buren 65643 +katie white 65620 +katie white 65620 +katie xylophone 65585 +katie young 65644 +katie young 65644 +katie young 65644 +katie zipper 65568 +katie zipper 65568 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke allen 65547 +luke brown 65719 +luke davidson 65656 +luke davidson 65656 +luke ellison 65582 +luke ellison 65582 +luke ellison 65582 +luke falkner 65589 +luke falkner 65589 +luke garcia 65687 +luke garcia 65687 +luke ichabod 65629 +luke ichabod 65629 +luke johnson 65545 +luke johnson 65545 +luke johnson 65545 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke laertes 65608 +luke miller 65752 +luke ovid 65569 +luke ovid 65569 +luke polk 65645 +luke polk 65645 +luke quirinius 65655 +luke robinson 65634 +luke robinson 65634 +luke thompson 65626 +luke underhill 65553 +luke underhill 65553 +luke underhill 65553 +luke van buren 65678 +luke white 65693 +luke xylophone 65597 +luke zipper 65641 +mike allen 65706 +mike brown 65654 +mike carson 65698 +mike carson 65698 +mike carson 65698 +mike davidson 65658 +mike davidson 65658 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike ellison 65598 +mike falkner 65609 +mike garcia 65571 +mike garcia 65571 +mike garcia 65571 +mike hernandez 65548 +mike hernandez 65548 +mike ichabod 65621 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike king 65563 +mike miller 65549 +mike nixon 65619 +mike nixon 65619 +mike polk 65619 +mike polk 65619 +mike polk 65619 +mike quirinius 65717 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike steinbeck 65550 +mike van buren 65620 +mike van buren 65620 +mike white 65648 +mike white 65648 +mike white 65648 +mike white 65648 +mike young 65545 +mike young 65545 +mike young 65545 +mike zipper 65552 +mike zipper 65552 +mike zipper 65552 +nick allen 65641 +nick allen 65641 +nick brown 65724 +nick davidson 65601 +nick ellison 65691 +nick ellison 65691 +nick falkner 65583 +nick falkner 65583 +nick garcia 65695 +nick garcia 65695 +nick garcia 65695 +nick ichabod 65572 +nick ichabod 65572 +nick ichabod 65572 +nick johnson 65585 +nick johnson 65585 +nick laertes 65624 +nick miller 65757 +nick nixon 65650 +nick ovid 65719 +nick polk 65716 +nick quirinius 65588 +nick quirinius 65588 +nick robinson 65547 +nick robinson 65547 +nick steinbeck 65689 +nick thompson 65610 +nick underhill 65619 +nick van buren 65603 +nick xylophone 65644 +nick young 65654 +nick young 65654 +nick zipper 65757 +nick zipper 65757 +oscar allen 65644 +oscar brown 65614 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar carson 65537 +oscar davidson 65556 +oscar ellison 65630 +oscar ellison 65630 +oscar falkner 65692 +oscar garcia 65751 +oscar hernandez 65683 +oscar hernandez 65683 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar ichabod 65536 +oscar johnson 65645 +oscar johnson 65645 +oscar king 65541 +oscar king 65541 +oscar king 65541 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar laertes 65625 +oscar nixon 65596 +oscar ovid 65536 +oscar ovid 65536 +oscar ovid 65536 +oscar polk 65541 +oscar polk 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar quirinius 65541 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar robinson 65537 +oscar steinbeck 65709 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar thompson 65542 +oscar underhill 65626 +oscar van buren 65581 +oscar van buren 65581 +oscar van buren 65581 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar white 65552 +oscar xylophone 65773 +oscar xylophone 65773 +oscar xylophone 65773 +oscar zipper 65568 +oscar zipper 65568 +oscar zipper 65568 +priscilla brown 65670 +priscilla brown 65670 +priscilla brown 65670 +priscilla carson 65658 +priscilla carson 65658 +priscilla carson 65658 +priscilla ichabod 65627 +priscilla ichabod 65627 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla johnson 65543 +priscilla king 65646 +priscilla nixon 65564 +priscilla nixon 65564 +priscilla ovid 65541 +priscilla ovid 65541 +priscilla polk 65747 +priscilla quirinius 65672 +priscilla thompson 65654 +priscilla underhill 65715 +priscilla underhill 65715 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla van buren 65607 +priscilla white 65652 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla xylophone 65538 +priscilla young 65585 +priscilla young 65585 +priscilla zipper 65622 +priscilla zipper 65622 +quinn allen 65657 +quinn allen 65657 +quinn brown 65691 +quinn brown 65691 +quinn brown 65691 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn davidson 65549 +quinn ellison 65705 +quinn ellison 65705 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn garcia 65568 +quinn ichabod 65564 +quinn king 65558 +quinn king 65558 +quinn laertes 65542 +quinn laertes 65542 +quinn laertes 65542 +quinn nixon 65659 +quinn ovid 65699 +quinn quirinius 65747 +quinn robinson 65627 +quinn steinbeck 65578 +quinn steinbeck 65578 +quinn thompson 65643 +quinn thompson 65643 +quinn underhill 65549 +quinn underhill 65549 +quinn underhill 65549 +quinn van buren 65725 +quinn young 65647 +quinn zipper 65579 +quinn zipper 65579 +rachel allen 65661 +rachel allen 65661 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel brown 65586 +rachel carson 65677 +rachel carson 65677 +rachel davidson 65755 +rachel ellison 65761 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel falkner 65616 +rachel johnson 65658 +rachel king 65604 +rachel king 65604 +rachel laertes 65562 +rachel laertes 65562 +rachel ovid 65721 +rachel ovid 65721 +rachel polk 65686 +rachel quirinius 65787 +rachel robinson 65544 +rachel robinson 65544 +rachel robinson 65544 +rachel thompson 65648 +rachel thompson 65648 +rachel thompson 65648 +rachel underhill 65667 +rachel white 65615 +rachel white 65615 +rachel young 65727 +rachel zipper 65757 +rachel zipper 65757 +sarah carson 65679 +sarah carson 65679 +sarah carson 65679 +sarah ellison 65611 +sarah falkner 65606 +sarah falkner 65606 +sarah garcia 65563 +sarah garcia 65563 +sarah garcia 65563 +sarah ichabod 65667 +sarah ichabod 65667 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah johnson 65659 +sarah king 65650 +sarah king 65650 +sarah miller 65557 +sarah ovid 65550 +sarah robinson 65677 +sarah robinson 65677 +sarah steinbeck 65721 +sarah white 65622 +sarah white 65622 +sarah xylophone 65678 +sarah young 65595 +sarah zipper 65550 +tom brown 65593 +tom brown 65593 +tom carson 65539 +tom carson 65539 +tom carson 65539 +tom davidson 65780 +tom ellison 65578 +tom ellison 65578 +tom ellison 65578 +tom falkner 65574 +tom falkner 65574 +tom hernandez 65575 +tom hernandez 65575 +tom ichabod 65588 +tom johnson 65536 +tom johnson 65536 +tom king 65576 +tom laertes 65617 +tom laertes 65617 +tom miller 65594 +tom miller 65594 +tom miller 65594 +tom nixon 65672 +tom ovid 65628 +tom polk 65652 +tom polk 65652 +tom quirinius 65563 +tom quirinius 65563 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom robinson 65626 +tom steinbeck 65666 +tom van buren 65621 +tom van buren 65621 +tom van buren 65621 +tom white 65548 +tom young 65544 +tom young 65544 +tom zipper 65789 +ulysses brown 65735 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses carson 65602 +ulysses davidson 65750 +ulysses ellison 65575 +ulysses garcia 65666 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses hernandez 65651 +ulysses ichabod 65551 +ulysses ichabod 65551 +ulysses johnson 65776 +ulysses king 65649 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses laertes 65691 +ulysses miller 65610 +ulysses miller 65610 +ulysses nixon 65603 +ulysses ovid 65656 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses polk 65563 +ulysses quirinius 65786 +ulysses robinson 65744 +ulysses steinbeck 65611 +ulysses steinbeck 65611 +ulysses thompson 65788 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses underhill 65570 +ulysses van buren 65684 +ulysses white 65654 +ulysses white 65654 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses xylophone 65623 +ulysses young 65675 +ulysses young 65675 +ulysses young 65675 +victor allen 65684 +victor allen 65684 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor brown 65550 +victor davidson 65579 +victor davidson 65579 +victor davidson 65579 +victor ellison 65641 +victor ellison 65641 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor hernandez 65571 +victor johnson 65606 +victor johnson 65606 +victor johnson 65606 +victor king 65721 +victor king 65721 +victor laertes 65638 +victor laertes 65638 +victor miller 65570 +victor nixon 65709 +victor nixon 65709 +victor ovid 65649 +victor polk 65625 +victor quirinius 65620 +victor quirinius 65620 +victor robinson 65596 +victor robinson 65596 +victor steinbeck 65618 +victor steinbeck 65618 +victor steinbeck 65618 +victor thompson 65548 +victor van buren 65664 +victor van buren 65664 +victor white 65548 +victor white 65548 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor xylophone 65549 +victor young 65628 +victor zipper 65743 +wendy allen 65628 +wendy allen 65628 +wendy allen 65628 +wendy brown 65580 +wendy brown 65580 +wendy ellison 65545 +wendy ellison 65545 +wendy falkner 65595 +wendy falkner 65595 +wendy falkner 65595 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy garcia 65659 +wendy hernandez 65650 +wendy ichabod 65730 +wendy king 65586 +wendy king 65586 +wendy king 65586 +wendy laertes 65566 +wendy laertes 65566 +wendy laertes 65566 +wendy miller 65582 +wendy miller 65582 +wendy nixon 65611 +wendy nixon 65611 +wendy ovid 65589 +wendy ovid 65589 +wendy polk 65656 +wendy polk 65656 +wendy quirinius 65766 +wendy quirinius 65766 +wendy robinson 65622 +wendy robinson 65622 +wendy robinson 65622 +wendy steinbeck 65612 +wendy thompson 65650 +wendy thompson 65650 +wendy underhill 65662 +wendy underhill 65662 +wendy underhill 65662 +wendy van buren 65680 +wendy van buren 65680 +wendy white 65705 +wendy xylophone 65687 +wendy xylophone 65687 +wendy young 65674 +wendy young 65674 +xavier allen 65611 +xavier allen 65611 +xavier allen 65611 +xavier brown 65600 +xavier brown 65600 +xavier brown 65600 +xavier carson 65731 +xavier carson 65731 +xavier davidson 65644 +xavier davidson 65644 +xavier davidson 65644 +xavier ellison 65541 +xavier ellison 65541 +xavier garcia 65672 +xavier hernandez 65541 +xavier hernandez 65541 +xavier hernandez 65541 +xavier ichabod 65597 +xavier ichabod 65597 +xavier johnson 65654 +xavier johnson 65654 +xavier king 65590 +xavier king 65590 +xavier laertes 65743 +xavier ovid 65788 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier polk 65587 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier quirinius 65599 +xavier thompson 65608 +xavier underhill 65710 +xavier white 65703 +xavier white 65703 +xavier xylophone 65572 +xavier zipper 65561 +yuri allen 65565 +yuri allen 65565 +yuri brown 65538 +yuri brown 65538 +yuri carson 65670 +yuri carson 65670 +yuri ellison 65570 +yuri ellison 65570 +yuri falkner 65658 +yuri falkner 65658 +yuri garcia 65639 +yuri hernandez 65706 +yuri johnson 65587 +yuri johnson 65587 +yuri johnson 65587 +yuri king 65721 +yuri laertes 65637 +yuri laertes 65637 +yuri nixon 65635 +yuri nixon 65635 +yuri polk 65607 +yuri polk 65607 +yuri polk 65607 +yuri quirinius 65544 +yuri quirinius 65544 +yuri quirinius 65544 +yuri steinbeck 65592 +yuri steinbeck 65592 +yuri thompson 65676 +yuri underhill 65718 +yuri underhill 65718 +yuri white 65659 +yuri xylophone 65714 +zach allen 65667 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach brown 65559 +zach carson 65572 +zach ellison 65748 +zach falkner 65620 +zach falkner 65620 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach garcia 65544 +zach ichabod 65599 +zach ichabod 65599 +zach king 65556 +zach king 65556 +zach king 65556 +zach miller 65584 +zach miller 65584 +zach miller 65584 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach ovid 65578 +zach quirinius 65691 +zach robinson 65599 +zach steinbeck 65602 +zach steinbeck 65602 +zach thompson 65636 +zach thompson 65636 +zach underhill 65573 +zach white 65733 +zach xylophone 65542 +zach xylophone 65542 +zach young 65576 +zach zipper 65579 +zach zipper 65579 +zach zipper 65579 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e new file mode 100644 index 0000000000000..657e81a94f4c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-1-2bf20f39e6ffef258858f7943a974e7e @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 47.359999895095825 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 3.934999942779541 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 63.40999794006348 +alice steinbeck 92.37000274658203 +alice underhill 93.17499923706055 +alice van buren 38.939998626708984 +alice xylophone 13.816667238871256 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 71.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 56.584999084472656 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 43.46500015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 55.51333363850912 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 48.09499931335449 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 27.6299991607666 +bob xylophone 29.359999656677246 +bob young 35.16999816894531 +bob zipper 3.819999933242798 +bob zipper 32.07500076293945 +bob zipper 52.63999938964844 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 59.42500114440918 +calvin davidson 20.40500020980835 +calvin davidson 86.54000091552734 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 62.670000076293945 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 60.98499870300293 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 52.06666819254557 +calvin thompson 8.90999984741211 +calvin thompson 89.22500228881836 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 35.864999771118164 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 39.810001373291016 +calvin young 70.27999941507976 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.005001068115234 +david allen 51.25 +david brown 32.56499910354614 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 54.17499876022339 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david ellison 47.689998626708984 +david ellison 62.587501525878906 +david ellison 85.2300033569336 +david hernandez 75.0833346048991 +david ichabod 35.763334115346275 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 34.72999954223633 +david ovid 43.915000915527344 +david ovid 58.89999961853027 +david quirinius 23.5649995803833 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 37.703334172566734 +david robinson 70.55999755859375 +david thompson 43.619998931884766 +david underhill 49.719998359680176 +david underhill 60.36499881744385 +david underhill 60.46666717529297 +david van buren 25.500000476837158 +david van buren 50.27999973297119 +david white 45.189998626708984 +david xylophone 30.465000867843628 +david xylophone 30.465000867843628 +david xylophone 54.34000015258789 +david young 10.25 +david young 19.310000777244568 +ethan allen 23.790000518163044 +ethan brown 15.630000114440918 +ethan brown 22.93666648864746 +ethan brown 27.78000044822693 +ethan brown 52.19333346684774 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 57.635000228881836 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 26.44499921798706 +ethan hernandez 32.30333264668783 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 54.75 +ethan laertes 54.87999868392944 +ethan laertes 59.209999084472656 +ethan laertes 75.31500053405762 +ethan laertes 76.94499969482422 +ethan laertes 80.4749984741211 +ethan laertes 95.06999969482422 +ethan miller 25.3700008392334 +ethan nixon 50.88999938964844 +ethan ovid 58.4950008392334 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 59.869998931884766 +ethan quirinius 13.650000214576721 +ethan quirinius 70.94500160217285 +ethan quirinius 88.18000030517578 +ethan robinson 67.94000244140625 +ethan robinson 80.58500289916992 +ethan underhill 55.630001068115234 +ethan van buren 37.85499954223633 +ethan white 58.70666631062826 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 5.914999961853027 +ethan zipper 97.51000213623047 +fred davidson 26.435000896453857 +fred davidson 28.144000816345216 +fred davidson 78.30999755859375 +fred ellison 46.65999984741211 +fred ellison 65.40666516621907 +fred ellison 71.98499870300293 +fred falkner 25.144999980926514 +fred falkner 37.62000068028768 +fred falkner 75.125 +fred hernandez 55.9900016784668 +fred ichabod 37.06999969482422 +fred ichabod 75.83499908447266 +fred johnson 96.08999633789062 +fred king 10.220000267028809 +fred king 68.40500068664551 +fred laertes 42.68750047683716 +fred miller 70.02999877929688 +fred nixon 30.589999198913574 +fred nixon 32.71666653951009 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 47.31999969482422 +fred polk 63.97999954223633 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 27.40999984741211 +fred robinson 64.42000007629395 +fred steinbeck 21.239999771118164 +fred steinbeck 21.30000066757202 +fred steinbeck 65.44333394368489 +fred underhill 85.36999893188477 +fred van buren 22.37499976158142 +fred van buren 44.49000096321106 +fred van buren 45.94999980926514 +fred van buren 56.88999938964844 +fred white 42.329999923706055 +fred young 46.79999923706055 +fred young 72.69999885559082 +fred zipper 50.14000129699707 +gabriella allen 28.040000438690186 +gabriella allen 79.64500045776367 +gabriella brown 26.164999961853027 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 18.946666717529297 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 31.609999974568684 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 62.22666517893473 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 83.76666514078777 +gabriella king 20.670000076293945 +gabriella king 62.89999961853027 +gabriella laertes 62.62499809265137 +gabriella miller 26.043334086736042 +gabriella ovid 44.78000068664551 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 90.22500228881836 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 66.86999893188477 +gabriella thompson 72.34500122070312 +gabriella thompson 75.73666636149089 +gabriella thompson 94.25 +gabriella van buren 38.935001373291016 +gabriella van buren 48.349998474121094 +gabriella white 55.18000030517578 +gabriella young 32.16333262125651 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 63.435001373291016 +holly brown 68.51666641235352 +holly brown 86.08666737874348 +holly falkner 83.82666524251302 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 28.649999300638836 +holly hernandez 50.22999954223633 +holly ichabod 48.86333433787028 +holly ichabod 73.77000045776367 +holly ichabod 82.91499710083008 +holly johnson 23.447500228881836 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 35.34499979019165 +holly king 42.310001373291016 +holly laertes 70.59666697184245 +holly miller 35.86000061035156 +holly nixon 43.82499885559082 +holly nixon 78.80749893188477 +holly polk 30.365000247955322 +holly polk 75.96499919891357 +holly robinson 82.70499801635742 +holly thompson 0.07999999821186066 +holly thompson 65.52499842643738 +holly thompson 86.69000244140625 +holly underhill 42.54999923706055 +holly underhill 53.02000045776367 +holly underhill 56.13333384195963 +holly underhill 65.84000015258789 +holly van buren 48.893333435058594 +holly white 26.5633331934611 +holly white 40.5 +holly xylophone 49.55666716893514 +holly young 41.698571750095915 +holly young 75.20999908447266 +holly zipper 79.72999827067058 +holly zipper 81.08666610717773 +irene allen 29.75999927520752 +irene brown 28.596666653951008 +irene brown 47.189998626708984 +irene brown 49.46666622161865 +irene carson 86.64999898274739 +irene ellison 38.255001068115234 +irene ellison 45.71333376566569 +irene falkner 22.079999923706055 +irene falkner 83.44666544596355 +irene garcia 38.93499994277954 +irene garcia 42.84666601816813 +irene garcia 58.43000030517578 +irene ichabod 60.7079984664917 +irene ichabod 64.58000183105469 +irene johnson 26.165000438690186 +irene laertes 21.02999997138977 +irene laertes 40.04499816894531 +irene laertes 47.04333241780599 +irene miller 65.44000244140625 +irene nixon 46.03999996185303 +irene nixon 46.96666653951009 +irene nixon 67.09499931335449 +irene ovid 35.130001068115234 +irene ovid 42.535000801086426 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 35.17500114440918 +irene polk 48.94666576385498 +irene polk 49.6 +irene polk 51.885000228881836 +irene quirinius 33.78000005086263 +irene quirinius 42.610000133514404 +irene quirinius 53.4800001780192 +irene robinson 92.19499969482422 +irene steinbeck 73.28000068664551 +irene thompson 46.27375066280365 +irene underhill 24.75 +irene underhill 57.349998474121094 +irene van buren 50.8799991607666 +irene van buren 74.5625 +irene xylophone 83.53499984741211 +jessica brown 26.185000479221344 +jessica carson 56.22999954223633 +jessica carson 62.20000076293945 +jessica carson 62.2400016784668 +jessica davidson 50.01666768391927 +jessica davidson 63.59499931335449 +jessica davidson 69.26666514078777 +jessica davidson 94.53333282470703 +jessica ellison 11.180000305175781 +jessica ellison 64.2060001373291 +jessica falkner 61.57333437601725 +jessica garcia 38.55250036716461 +jessica garcia 57.00999975204468 +jessica ichabod 32.63250035047531 +jessica johnson 9.5600004196167 +jessica johnson 51.959999084472656 +jessica miller 77.83999633789062 +jessica nixon 40.72249960899353 +jessica nixon 90.06999969482422 +jessica ovid 37.45250064134598 +jessica ovid 59.68000030517578 +jessica polk 49.68000030517578 +jessica quirinius 25.65750002861023 +jessica quirinius 37.64200019836426 +jessica quirinius 54.25500011444092 +jessica quirinius 58.019999186197914 +jessica robinson 42.66333484649658 +jessica thompson 30.40666675567627 +jessica thompson 43.87500023841858 +jessica underhill 43.33333269755045 +jessica underhill 45.639999866485596 +jessica underhill 57.584999084472656 +jessica van buren 67.00000047683716 +jessica white 6.170000106096268 +jessica white 63.32500076293945 +jessica white 65.1450023651123 +jessica white 73.93000030517578 +jessica white 96.62000274658203 +jessica xylophone 69.87500190734863 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 42.43833335240682 +jessica zipper 46.7450008392334 +jessica zipper 56.97999954223633 +katie allen 55.47666549682617 +katie brown 31.699999809265137 +katie davidson 93.22000122070312 +katie ellison 48.31999933719635 +katie ellison 64.08499892552693 +katie falkner 51.665000915527344 +katie garcia 57.71000099182129 +katie garcia 61.21000051498413 +katie hernandez 41.150000381469724 +katie ichabod 44.243333180745445 +katie ichabod 51.800000508626304 +katie ichabod 69.18799896240235 +katie king 39.83000183105469 +katie king 46.80333296457926 +katie king 51.85000038146973 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 23.190000534057617 +katie ovid 67.94500160217285 +katie polk 26.62750005722046 +katie polk 33.9350004196167 +katie robinson 13.890000343322754 +katie van buren 44.434998512268066 +katie van buren 65.41999816894531 +katie white 37.96500015258789 +katie white 59.223333517710365 +katie xylophone 39.30000019073486 +katie young 36.660000801086426 +katie young 67.78333282470703 +katie young 72.76666577657063 +katie zipper 23.766667087872822 +katie zipper 58.75 +luke allen 50.959999084472656 +luke allen 53.36666742960612 +luke allen 54.63249969482422 +luke allen 57.670000076293945 +luke allen 70.39500045776367 +luke brown 49.595000982284546 +luke davidson 7.050000190734863 +luke davidson 18.87000036239624 +luke ellison 16.25 +luke ellison 32.9519996881485 +luke ellison 71.93500137329102 +luke falkner 21.71999979019165 +luke falkner 31.81250023841858 +luke garcia 18.65499973297119 +luke garcia 41.2300001780192 +luke ichabod 41.25750005245209 +luke ichabod 73.55000114440918 +luke johnson 31.670000076293945 +luke johnson 32.84499979019165 +luke johnson 39.54500102996826 +luke laertes 11.819999694824219 +luke laertes 21.184999227523804 +luke laertes 21.993332862854004 +luke laertes 26.696666717529297 +luke laertes 45.9900016784668 +luke miller 52.350000858306885 +luke ovid 23.804999828338623 +luke ovid 64.30000305175781 +luke polk 41.02499961853027 +luke polk 58.4566650390625 +luke quirinius 40.41999816894531 +luke robinson 48.559998750686646 +luke robinson 56.76499938964844 +luke thompson 78.04333368937175 +luke underhill 34.0166662534078 +luke underhill 47.28999996185303 +luke underhill 59.32000160217285 +luke van buren 59.91999944051107 +luke white 74.19599990844726 +luke xylophone 64.77999925613403 +luke zipper 30.434999465942383 +mike allen 30.539999961853027 +mike brown 69.86833318074544 +mike carson 30.25333309173584 +mike carson 61.33799934387207 +mike carson 89.375 +mike davidson 32.55333391825358 +mike davidson 66.74333318074544 +mike ellison 35.905999755859376 +mike ellison 39.82499885559082 +mike ellison 58.56399993896484 +mike ellison 64.52999877929688 +mike ellison 66.93749856948853 +mike falkner 48.53750002384186 +mike garcia 51.02999973297119 +mike garcia 67.93000030517578 +mike garcia 70.8499984741211 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 36.17800045013428 +mike king 41.69500136375427 +mike king 49.57000017166138 +mike king 59.654998779296875 +mike king 71.57000122070312 +mike king 78.50999927520752 +mike miller 29.570000171661377 +mike nixon 45.029999828338624 +mike nixon 48.429999669392906 +mike polk 30.864000129699708 +mike polk 46.95499873161316 +mike polk 79.55500030517578 +mike quirinius 85.0699971516927 +mike steinbeck 24.267500042915344 +mike steinbeck 43.52500021457672 +mike steinbeck 61.426666577657066 +mike steinbeck 68.46000022888184 +mike van buren 27.639999389648438 +mike van buren 56.16333134969076 +mike white 34.8924994468689 +mike white 43.5566660563151 +mike white 53.689998626708984 +mike white 77.54499864578247 +mike young 34.3319993019104 +mike young 52.8100004196167 +mike young 55.64333359400431 +mike zipper 56.86666742960612 +mike zipper 63.3149995803833 +mike zipper 83.91999816894531 +nick allen 57.086001586914065 +nick allen 60.15400094985962 +nick brown 42.939998626708984 +nick davidson 63.07499885559082 +nick ellison 45.34000015258789 +nick ellison 65.88500022888184 +nick falkner 41.87999868392944 +nick falkner 64.05666732788086 +nick garcia 34.34499979019165 +nick garcia 51.08666737874349 +nick garcia 62.88600044250488 +nick ichabod 20.253333409627277 +nick ichabod 53.635000228881836 +nick ichabod 77.36000061035156 +nick johnson 20.114999175071716 +nick johnson 81.91666666666667 +nick laertes 91.56666819254558 +nick miller 71.5500005086263 +nick nixon 77.04249954223633 +nick ovid 74.62666702270508 +nick polk 39.27500009536743 +nick quirinius 60.79499816894531 +nick quirinius 67.44999694824219 +nick robinson 31.672499418258667 +nick robinson 57.66999816894531 +nick steinbeck 59.15999984741211 +nick thompson 18.88666645685832 +nick underhill 43.009998893737794 +nick van buren 34.720000902811684 +nick xylophone 75.3499984741211 +nick young 0.27000001072883606 +nick young 47.813334147135414 +nick zipper 46.22333272298177 +nick zipper 52.54333209991455 +oscar allen 37.396666844685875 +oscar brown 13.100000381469727 +oscar carson 31.91333230336507 +oscar carson 41.77333331108093 +oscar carson 57.3149995803833 +oscar carson 73.59500122070312 +oscar carson 95.44000244140625 +oscar davidson 75.18000030517578 +oscar ellison 34.04499959945679 +oscar ellison 34.04499959945679 +oscar falkner 61.72000050544739 +oscar garcia 67.4800033569336 +oscar hernandez 41.63333400090536 +oscar hernandez 47.93999986648559 +oscar ichabod 45.839999516805015 +oscar ichabod 68.62000274658203 +oscar ichabod 72.18249797821045 +oscar ichabod 76.69000244140625 +oscar johnson 23.880000114440918 +oscar johnson 65.04000091552734 +oscar king 36.69500017166138 +oscar king 49.7049994468689 +oscar king 67.98399925231934 +oscar laertes 43.616665522257485 +oscar laertes 44.755000591278076 +oscar laertes 45.26666831970215 +oscar laertes 53.710001945495605 +oscar nixon 36.56999937693278 +oscar ovid 45.89999961853027 +oscar ovid 46.93999934196472 +oscar ovid 55.277999591827395 +oscar polk 42.31999969482422 +oscar polk 63.900001525878906 +oscar quirinius 63.81500053405762 +oscar quirinius 66.28428527287075 +oscar quirinius 70.24000295003255 +oscar quirinius 81.26249980926514 +oscar robinson 11.34000015258789 +oscar robinson 47.845001220703125 +oscar robinson 59.74333349863688 +oscar robinson 63.346666971842446 +oscar steinbeck 42.49999976158142 +oscar thompson 38.23500061035156 +oscar thompson 51.469999154408775 +oscar thompson 60.029999542236325 +oscar thompson 63.079999923706055 +oscar underhill 66.97666676839192 +oscar van buren 24.085000872612 +oscar van buren 61.880001068115234 +oscar van buren 72.9533322652181 +oscar white 44.72333272298177 +oscar white 46.60999870300293 +oscar white 54.7599983215332 +oscar white 60.85500144958496 +oscar xylophone 34.946666399637856 +oscar xylophone 39.8299994468689 +oscar xylophone 57.119998931884766 +oscar zipper 28.499999046325684 +oscar zipper 47.46750068664551 +oscar zipper 59.1933339436849 +priscilla brown 47.40400066375732 +priscilla brown 77.1479995727539 +priscilla brown 80.5199966430664 +priscilla carson 7.960000038146973 +priscilla carson 28.480000153183937 +priscilla carson 45.92750144004822 +priscilla ichabod 38.95666758219401 +priscilla ichabod 62.32999928792318 +priscilla johnson 44.04499912261963 +priscilla johnson 50.53750038146973 +priscilla johnson 55.98333485921224 +priscilla johnson 59.64499855041504 +priscilla johnson 89.1500015258789 +priscilla king 50.44666735331217 +priscilla nixon 44.32222270965576 +priscilla nixon 45.267999792099 +priscilla ovid 44.78333361943563 +priscilla ovid 52.72999954223633 +priscilla polk 34.89399948120117 +priscilla quirinius 35.609999895095825 +priscilla thompson 35.16249918937683 +priscilla underhill 68.22000122070312 +priscilla underhill 73.97200012207031 +priscilla van buren 50.47000026702881 +priscilla van buren 51.39500045776367 +priscilla van buren 53.541999435424806 +priscilla white 50.47599992752075 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 41.106666485468544 +priscilla xylophone 63.9574990272522 +priscilla young 0.2900000065565109 +priscilla young 19.866666316986084 +priscilla zipper 32.084999322891235 +priscilla zipper 43.90333366394043 +quinn allen 47.90333382288615 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 53.98666508992513 +quinn brown 66.82500171661377 +quinn davidson 40.666666666666664 +quinn davidson 54.095001220703125 +quinn davidson 79.78333282470703 +quinn davidson 92.13000106811523 +quinn ellison 52.714999198913574 +quinn ellison 63.352500438690186 +quinn garcia 20.19000056385994 +quinn garcia 54.60000038146973 +quinn garcia 59.010000824928284 +quinn garcia 68.98999881744385 +quinn ichabod 48.60499930381775 +quinn king 61.27333450317383 +quinn king 81.46000289916992 +quinn laertes 32.08000040054321 +quinn laertes 44.45666694641113 +quinn laertes 49.85499858856201 +quinn nixon 72.2471422467913 +quinn ovid 34.423333168029785 +quinn quirinius 53.165000915527344 +quinn robinson 32.624999046325684 +quinn steinbeck 24.802499771118164 +quinn steinbeck 55.477500915527344 +quinn thompson 50.500000381469725 +quinn thompson 55.68600006103516 +quinn underhill 39.66600060462952 +quinn underhill 41.47666676839193 +quinn underhill 56.580000162124634 +quinn van buren 49.40333207448324 +quinn young 55.59000142415365 +quinn zipper 11.359999974568685 +quinn zipper 48.45000123977661 +rachel allen 45.940001249313354 +rachel allen 85.97999954223633 +rachel brown 33.01999984184901 +rachel brown 34.08250021934509 +rachel brown 37.999999046325684 +rachel brown 41.75000019868215 +rachel brown 53.679999669392906 +rachel carson 43.32400016784668 +rachel carson 66.2233320871989 +rachel davidson 14.220000267028809 +rachel ellison 17.549999833106995 +rachel falkner 56.883334159851074 +rachel falkner 57.5199998219808 +rachel falkner 58.80666637420654 +rachel falkner 70.69428443908691 +rachel johnson 36.22499990463257 +rachel king 50.970001220703125 +rachel king 83.53750133514404 +rachel laertes 42.29857151848929 +rachel laertes 71.65999984741211 +rachel ovid 42.25333329041799 +rachel ovid 47.01749947667122 +rachel polk 64.90333239237468 +rachel quirinius 53.2624990940094 +rachel robinson 40.712857246398926 +rachel robinson 53.092498898506165 +rachel robinson 64.94999694824219 +rachel thompson 24.555000439286232 +rachel thompson 31.460000038146973 +rachel thompson 46.804000282287596 +rachel underhill 47.22333272298177 +rachel white 39.87999979654948 +rachel white 41.83428575311388 +rachel young 75.7966677347819 +rachel zipper 45.794999519983925 +rachel zipper 56.909999179840085 +sarah carson 24.576666196187336 +sarah carson 36.33750060200691 +sarah carson 43.65749907493591 +sarah ellison 37.054999351501465 +sarah falkner 48.58285754067557 +sarah falkner 62.36500072479248 +sarah garcia 33.38000011444092 +sarah garcia 35.513333002726235 +sarah garcia 64.31333287556966 +sarah ichabod 36.10599975585937 +sarah ichabod 45.830000162124634 +sarah johnson 26.464999675750732 +sarah johnson 40.9300012588501 +sarah johnson 43.44000196456909 +sarah johnson 64.24333318074544 +sarah king 49.06999909877777 +sarah king 63.01333363850912 +sarah miller 41.709999084472656 +sarah ovid 63.682499408721924 +sarah robinson 39.196666399637856 +sarah robinson 66.88999938964844 +sarah steinbeck 66.89000034332275 +sarah white 41.42599945068359 +sarah white 52.95249938964844 +sarah xylophone 68.31999969482422 +sarah young 35.92750024795532 +sarah zipper 53.697500705718994 +tom brown 38.37000020345052 +tom brown 44.68000049591065 +tom carson 27.994999766349792 +tom carson 54.25250005722046 +tom carson 62.790000915527344 +tom davidson 38.679999113082886 +tom ellison 33.68600053787232 +tom ellison 46.00666618347168 +tom ellison 67.79666646321614 +tom falkner 55.61800079345703 +tom falkner 58.82500012715658 +tom hernandez 50.52250051498413 +tom hernandez 50.52250051498413 +tom ichabod 24.98399963378906 +tom johnson 34.83750009536743 +tom johnson 73.72399978637695 +tom king 69.98000106811523 +tom laertes 41.97285750934056 +tom laertes 70.40333429972331 +tom miller 43.885000586509705 +tom miller 57.10500144958496 +tom miller 76.20499992370605 +tom nixon 62.43000030517578 +tom ovid 38.096666971842446 +tom polk 51.26750087738037 +tom polk 68.22666676839192 +tom quirinius 37.720001220703125 +tom quirinius 53.20399913787842 +tom robinson 43.44333299001058 +tom robinson 54.637142998831614 +tom robinson 59.34250068664551 +tom robinson 99.1500015258789 +tom steinbeck 51.883334477742515 +tom van buren 28.380000829696655 +tom van buren 35.64999930063883 +tom van buren 54.59000015258789 +tom white 51.970001220703125 +tom young 44.7319995880127 +tom young 53.894999980926514 +tom zipper 55.44000116984049 +ulysses brown 48.72666708628336 +ulysses carson 38.742000579833984 +ulysses carson 45.513333002726235 +ulysses carson 48.75249934196472 +ulysses carson 74.64600067138672 +ulysses davidson 63.20857129778181 +ulysses ellison 68.52666759490967 +ulysses garcia 58.77250051498413 +ulysses hernandez 32.371999168395995 +ulysses hernandez 50.57000102996826 +ulysses hernandez 61.39999961853027 +ulysses ichabod 19.1299991607666 +ulysses ichabod 83.06666692097981 +ulysses johnson 51.485000451405845 +ulysses king 46.98333422342936 +ulysses laertes 29.046666741371155 +ulysses laertes 32.88599967956543 +ulysses laertes 60.12399845123291 +ulysses miller 44.552857535226 +ulysses miller 71.39249873161316 +ulysses nixon 51.300000286102296 +ulysses ovid 29.360000610351562 +ulysses polk 40.74399948120117 +ulysses polk 48.9800017674764 +ulysses polk 57.86249828338623 +ulysses polk 81.21333312988281 +ulysses quirinius 68.41500091552734 +ulysses robinson 69.53999853134155 +ulysses steinbeck 44.61833381652832 +ulysses steinbeck 48.362499713897705 +ulysses thompson 45.063334465026855 +ulysses underhill 30.829999764760334 +ulysses underhill 41.43857192993164 +ulysses underhill 44.08333269755045 +ulysses underhill 55.470001220703125 +ulysses underhill 58.9471435546875 +ulysses underhill 68.1900007724762 +ulysses underhill 78.83333333333333 +ulysses van buren 72.38428633553642 +ulysses white 36.17250043153763 +ulysses white 39.084000778198245 +ulysses xylophone 27.519999504089355 +ulysses xylophone 47.65999937057495 +ulysses xylophone 50.29999923706055 +ulysses young 23.308333079020183 +ulysses young 34.6339994430542 +ulysses young 88.06999969482422 +victor allen 49.43800010681152 +victor allen 56.7299998147147 +victor brown 40.80600037574768 +victor brown 63.5024995803833 +victor brown 71.03500080108643 +victor brown 81.71999931335449 +victor davidson 44.70333290100098 +victor davidson 59.070000076293944 +victor davidson 67.27199935913086 +victor ellison 31.28999964396159 +victor ellison 42.54999923706055 +victor hernandez 44.41333452860514 +victor hernandez 47.20249938964844 +victor hernandez 47.73333215713501 +victor hernandez 51.04999923706055 +victor hernandez 59.2399995803833 +victor johnson 54.868000626564026 +victor johnson 55.22999954223633 +victor johnson 57.41000175476074 +victor king 38.27999997138977 +victor king 49.993333180745445 +victor laertes 40.63500006993612 +victor laertes 80.5999984741211 +victor miller 71.00000190734863 +victor nixon 38.393332640329994 +victor nixon 52.920000076293945 +victor ovid 53.260000228881836 +victor polk 3.0 +victor quirinius 57.81666644414266 +victor quirinius 59.39999923706055 +victor robinson 24.614999771118164 +victor robinson 74.5049991607666 +victor steinbeck 28.862000381946565 +victor steinbeck 36.61000006539481 +victor steinbeck 43.09000015258789 +victor thompson 42.67599925994873 +victor van buren 44.669999877611794 +victor van buren 45.121999740600586 +victor white 53.67999887466431 +victor white 54.45000012715658 +victor xylophone 12.160000324249268 +victor xylophone 26.0 +victor xylophone 31.769999821980793 +victor xylophone 52.31499926249186 +victor xylophone 69.2899996439616 +victor young 64.25833320617676 +victor zipper 65.24999904632568 +wendy allen 34.04999955495199 +wendy allen 36.88199939727783 +wendy allen 44.96000012755394 +wendy brown 45.97833283742269 +wendy brown 52.73857225690569 +wendy ellison 42.91333246231079 +wendy ellison 53.56000073750814 +wendy falkner 47.602500438690186 +wendy falkner 64.9099988937378 +wendy falkner 77.5999984741211 +wendy garcia 37.38571425846645 +wendy garcia 48.76666768391927 +wendy garcia 53.225000858306885 +wendy garcia 63.93999926249186 +wendy hernandez 36.195000648498535 +wendy ichabod 10.56499981880188 +wendy king 37.57500076293945 +wendy king 53.44333457946777 +wendy king 56.319997787475586 +wendy laertes 38.39249920845032 +wendy laertes 60.19999885559082 +wendy laertes 65.30624961853027 +wendy miller 44.273332595825195 +wendy miller 53.5675014257431 +wendy nixon 54.995998764038085 +wendy nixon 64.28250026702881 +wendy ovid 43.80499863624573 +wendy ovid 61.64600105285645 +wendy polk 26.784999758005142 +wendy polk 35.21599998474121 +wendy quirinius 28.75666618347168 +wendy quirinius 60.70000092188517 +wendy robinson 42.5799994468689 +wendy robinson 42.90799944400787 +wendy robinson 43.426000237464905 +wendy steinbeck 43.42333388328552 +wendy thompson 40.352857317243306 +wendy thompson 75.93666712443034 +wendy underhill 24.459999561309814 +wendy underhill 33.440000693003334 +wendy underhill 45.51625019311905 +wendy van buren 43.30333296457926 +wendy van buren 65.58666737874348 +wendy white 39.015000104904175 +wendy xylophone 42.42500034968058 +wendy xylophone 53.981666247049965 +wendy young 27.929999828338623 +wendy young 59.609999656677246 +xavier allen 49.24500061571598 +xavier allen 67.39000034332275 +xavier allen 70.29800033569336 +xavier brown 19.772500306367874 +xavier brown 58.87000111171177 +xavier brown 74.6200008392334 +xavier carson 47.3199987411499 +xavier carson 61.52250027656555 +xavier davidson 48.41999936103821 +xavier davidson 52.70666758219401 +xavier davidson 52.78166747093201 +xavier ellison 41.84999942779541 +xavier ellison 62.80200090408325 +xavier garcia 35.0600004568696 +xavier hernandez 47.45200023651123 +xavier hernandez 49.676666259765625 +xavier hernandez 53.446667432785034 +xavier ichabod 56.70625042915344 +xavier ichabod 60.54799928665161 +xavier johnson 41.16333262125651 +xavier johnson 53.85333331425985 +xavier king 42.72800064086914 +xavier king 66.05333455403645 +xavier laertes 38.47999954223633 +xavier ovid 48.89250057935715 +xavier polk 37.05500018596649 +xavier polk 46.82666703065237 +xavier polk 55.385000705718994 +xavier polk 55.65000057220459 +xavier quirinius 59.62499964237213 +xavier quirinius 60.055998992919925 +xavier quirinius 62.52000045776367 +xavier quirinius 65.2933349609375 +xavier thompson 40.244998931884766 +xavier underhill 26.27800006866455 +xavier white 47.8671429497855 +xavier white 63.38428551810129 +xavier xylophone 49.072500228881836 +xavier zipper 8.204999923706055 +yuri allen 53.61250066757202 +yuri allen 64.86833254496257 +yuri brown 46.57500044504801 +yuri brown 66.75250005722046 +yuri carson 39.40750050544739 +yuri carson 49.01600036621094 +yuri ellison 27.49000017642975 +yuri ellison 70.5933329264323 +yuri falkner 47.23285675048828 +yuri falkner 62.807999801635745 +yuri garcia 43.967499017715454 +yuri hernandez 31.94000039100647 +yuri johnson 21.40666739145915 +yuri johnson 34.02333414554596 +yuri johnson 65.7750015258789 +yuri king 49.47333272298177 +yuri laertes 42.070000648498535 +yuri laertes 60.7549991607666 +yuri nixon 49.87142838750567 +yuri nixon 59.945000330607094 +yuri polk 37.56249952316284 +yuri polk 47.583333333333336 +yuri polk 72.60888735453288 +yuri quirinius 18.62000060081482 +yuri quirinius 51.217501401901245 +yuri quirinius 67.24000072479248 +yuri steinbeck 55.757999420166016 +yuri steinbeck 75.87999725341797 +yuri thompson 36.93499946594238 +yuri underhill 51.533334732055664 +yuri underhill 62.31888887617323 +yuri white 44.34999983651297 +yuri xylophone 25.117499828338623 +zach allen 25.92333350578944 +zach brown 38.3799991607666 +zach brown 47.404998779296875 +zach brown 54.30600090026856 +zach brown 58.970001220703125 +zach brown 65.22499942779541 +zach carson 60.783999633789065 +zach ellison 36.211428437914165 +zach falkner 41.225714683532715 +zach falkner 65.99499940872192 +zach garcia 42.8885714326586 +zach garcia 46.8870005607605 +zach garcia 47.5049991607666 +zach garcia 66.09399967193603 +zach ichabod 40.10166613260905 +zach ichabod 53.16749954223633 +zach king 39.137500405311584 +zach king 48.2825003862381 +zach king 61.18999965985616 +zach miller 44.82800054550171 +zach miller 48.52428477151053 +zach miller 53.593332608540855 +zach ovid 35.19399921447039 +zach ovid 38.35833342870077 +zach ovid 43.87200012207031 +zach ovid 83.01999918619792 +zach quirinius 42.638333320617676 +zach robinson 82.04999923706055 +zach steinbeck 55.86599960327148 +zach steinbeck 67.81428473336356 +zach thompson 29.303333282470703 +zach thompson 46.48999913533529 +zach underhill 48.681429045540945 +zach white 66.60250091552734 +zach xylophone 41.875 +zach xylophone 57.2416664759318 +zach young 73.5999984741211 +zach zipper 58.1480016708374 +zach zipper 60.1825008392334 +zach zipper 62.794999837875366 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f new file mode 100644 index 0000000000000..6cfa5ad413fa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-2-16239d2b069789ba99fbac50c4f0724f @@ -0,0 +1,1049 @@ + 65560.0 + 65718.0 + 65740.0 +alice allen 65662.0 +alice allen 65720.0 +alice allen 65758.0 +alice brown 65696.0 +alice carson 65559.0 +alice davidson 65547.0 +alice falkner 65669.0 +alice garcia 65613.0 +alice hernandez 65737.0 +alice hernandez 65784.0 +alice johnson 65739.0 +alice king 65660.0 +alice king 65734.0 +alice king 65738.0 +alice laertes 65669.0 +alice laertes 65671.0 +alice miller 65590.0 +alice nixon 65586.0 +alice nixon 65595.0 +alice nixon 65604.0 +alice ovid 65737.0 +alice polk 65548.0 +alice quirinius 65636.0 +alice quirinius 65728.0 +alice robinson 65606.0 +alice robinson 65789.0 +alice steinbeck 65578.0 +alice steinbeck 65673.0 +alice steinbeck 65786.0 +alice underhill 65750.0 +alice van buren 65562.0 +alice xylophone 65578.0 +alice xylophone 65585.0 +alice xylophone 65599.0 +alice zipper 65553.0 +alice zipper 65662.0 +alice zipper 65766.0 +bob brown 65584.0 +bob brown 65777.0 +bob brown 65783.0 +bob carson 65713.0 +bob davidson 65664.0 +bob davidson 65693.0 +bob davidson 65768.0 +bob ellison 65591.0 +bob ellison 65624.0 +bob ellison 65721.0 +bob ellison 65760.0 +bob falkner 65789.0 +bob garcia 65585.0 +bob garcia 65598.0 +bob garcia 65673.0 +bob garcia 65754.0 +bob garcia 65782.0 +bob hernandez 65557.0 +bob ichabod 65549.0 +bob king 65715.0 +bob king 65757.0 +bob king 65783.0 +bob laertes 65602.0 +bob laertes 65663.0 +bob miller 65608.0 +bob ovid 65564.0 +bob ovid 65619.0 +bob ovid 65686.0 +bob ovid 65726.0 +bob polk 65594.0 +bob quirinius 65700.0 +bob steinbeck 65637.0 +bob van buren 65778.0 +bob white 65543.0 +bob white 65605.0 +bob xylophone 65574.0 +bob xylophone 65666.0 +bob young 65556.0 +bob zipper 65559.0 +bob zipper 65633.0 +bob zipper 65739.0 +calvin allen 65669.0 +calvin brown 65537.0 +calvin brown 65580.0 +calvin brown 65677.0 +calvin carson 65637.0 +calvin davidson 65541.0 +calvin davidson 65564.0 +calvin ellison 65667.0 +calvin falkner 65573.0 +calvin falkner 65596.0 +calvin falkner 65738.0 +calvin falkner 65762.0 +calvin falkner 65778.0 +calvin falkner 65784.0 +calvin garcia 65664.0 +calvin hernandez 65578.0 +calvin johnson 65731.0 +calvin laertes 65570.0 +calvin laertes 65684.0 +calvin nixon 65654.0 +calvin nixon 65724.0 +calvin nixon 65749.0 +calvin ovid 65554.0 +calvin ovid 65643.0 +calvin ovid 65663.0 +calvin ovid 65715.0 +calvin polk 65731.0 +calvin quirinius 65741.0 +calvin quirinius 65769.0 +calvin robinson 65581.0 +calvin steinbeck 65680.0 +calvin steinbeck 65762.0 +calvin steinbeck 65779.0 +calvin thompson 65560.0 +calvin thompson 65640.0 +calvin underhill 65732.0 +calvin van buren 65552.0 +calvin van buren 65771.0 +calvin white 65553.0 +calvin white 65561.0 +calvin xylophone 65575.0 +calvin xylophone 65614.5 +calvin xylophone 65713.0 +calvin young 65574.0 +calvin young 65746.0 +calvin zipper 65669.0 +calvin zipper 65739.0 +david allen 65588.0 +david allen 65617.0 +david brown 65637.0 +david brown 65760.0 +david davidson 65559.0 +david davidson 65756.0 +david davidson 65778.0 +david davidson 65779.0 +david ellison 65634.0 +david ellison 65724.0 +david ellison 65724.0 +david hernandez 65763.0 +david ichabod 65699.0 +david ichabod 65715.0 +david laertes 65762.0 +david nixon 65536.0 +david ovid 65623.0 +david ovid 65628.0 +david quirinius 65697.0 +david quirinius 65759.0 +david quirinius 65779.0 +david robinson 65762.0 +david robinson 65775.0 +david thompson 65550.0 +david underhill 65602.0 +david underhill 65662.0 +david underhill 65751.0 +david van buren 65625.0 +david van buren 65634.0 +david white 65678.0 +david xylophone 65537.0 +david xylophone 65670.0 +david xylophone 65764.0 +david young 65551.0 +david young 65694.0 +ethan allen 65747.0 +ethan brown 65539.0 +ethan brown 65617.0 +ethan brown 65685.0 +ethan brown 65685.0 +ethan brown 65722.0 +ethan brown 65733.0 +ethan carson 65742.0 +ethan ellison 65714.0 +ethan ellison 65732.0 +ethan falkner 65577.0 +ethan falkner 65614.0 +ethan garcia 65736.0 +ethan hernandez 65630.5 +ethan johnson 65536.0 +ethan king 65614.0 +ethan laertes 65562.0 +ethan laertes 65597.0 +ethan laertes 65628.0 +ethan laertes 65643.0 +ethan laertes 65680.0 +ethan laertes 65745.0 +ethan laertes 65760.0 +ethan miller 65712.0 +ethan nixon 65766.0 +ethan ovid 65697.0 +ethan polk 65589.0 +ethan polk 65615.0 +ethan polk 65622.0 +ethan polk 65622.0 +ethan quirinius 65542.0 +ethan quirinius 65591.0 +ethan quirinius 65706.0 +ethan robinson 65547.0 +ethan robinson 65659.0 +ethan underhill 65570.0 +ethan van buren 65572.0 +ethan white 65677.0 +ethan white 65788.0 +ethan xylophone 65595.0 +ethan zipper 65593.0 +ethan zipper 65680.0 +fred davidson 65552.0 +fred davidson 65595.0 +fred davidson 65721.0 +fred ellison 65548.0 +fred ellison 65691.0 +fred ellison 65771.0 +fred falkner 65637.0 +fred falkner 65648.0 +fred falkner 65783.0 +fred hernandez 65541.0 +fred ichabod 65572.0 +fred ichabod 65789.0 +fred johnson 65758.0 +fred king 65694.0 +fred king 65745.0 +fred laertes 65769.0 +fred miller 65536.0 +fred nixon 65560.0 +fred nixon 65612.0 +fred nixon 65703.0 +fred nixon 65705.0 +fred polk 65603.0 +fred polk 65656.0 +fred polk 65701.0 +fred polk 65706.0 +fred quirinius 65697.0 +fred quirinius 65782.0 +fred robinson 65623.0 +fred steinbeck 65544.0 +fred steinbeck 65651.0 +fred steinbeck 65755.0 +fred underhill 65629.0 +fred van buren 65537.0 +fred van buren 65561.0 +fred van buren 65745.0 +fred van buren 65789.0 +fred white 65589.0 +fred young 65594.0 +fred young 65773.0 +fred zipper 65553.0 +gabriella allen 65646.0 +gabriella allen 65677.0 +gabriella brown 65704.0 +gabriella brown 65753.0 +gabriella carson 65586.0 +gabriella davidson 65565.0 +gabriella ellison 65706.0 +gabriella ellison 65716.0 +gabriella falkner 65623.0 +gabriella falkner 65711.0 +gabriella falkner 65767.0 +gabriella garcia 65571.0 +gabriella hernandez 65587.0 +gabriella hernandez 65717.0 +gabriella ichabod 65559.0 +gabriella ichabod 65633.0 +gabriella ichabod 65702.0 +gabriella ichabod 65712.0 +gabriella ichabod 65717.0 +gabriella king 65651.0 +gabriella king 65657.0 +gabriella laertes 65781.0 +gabriella miller 65646.0 +gabriella ovid 65556.0 +gabriella ovid 65583.0 +gabriella polk 65701.0 +gabriella polk 65790.0 +gabriella steinbeck 65582.0 +gabriella steinbeck 65653.0 +gabriella thompson 65682.0 +gabriella thompson 65755.0 +gabriella thompson 65766.0 +gabriella van buren 65581.0 +gabriella van buren 65644.0 +gabriella white 65638.0 +gabriella young 65699.0 +gabriella young 65774.0 +gabriella zipper 65540.0 +gabriella zipper 65754.0 +holly allen 65596.0 +holly brown 65599.0 +holly brown 65619.0 +holly falkner 65720.0 +holly hernandez 65602.0 +holly hernandez 65686.0 +holly hernandez 65750.0 +holly hernandez 65788.0 +holly ichabod 65711.0 +holly ichabod 65749.0 +holly ichabod 65752.0 +holly johnson 65655.0 +holly johnson 65662.0 +holly johnson 65755.0 +holly king 65549.0 +holly king 65648.0 +holly laertes 65664.0 +holly miller 65653.0 +holly nixon 65539.0 +holly nixon 65658.0 +holly polk 65743.0 +holly polk 65751.0 +holly robinson 65564.0 +holly thompson 65538.0 +holly thompson 65578.0 +holly thompson 65713.0 +holly underhill 65634.0 +holly underhill 65654.0 +holly underhill 65721.0 +holly underhill 65759.0 +holly van buren 65727.0 +holly white 65536.0 +holly white 65602.0 +holly xylophone 65544.0 +holly young 65606.0 +holly young 65765.0 +holly zipper 65607.0 +holly zipper 65755.0 +irene allen 65556.0 +irene brown 65633.0 +irene brown 65664.0 +irene brown 65765.0 +irene carson 65590.0 +irene ellison 65659.0 +irene ellison 65696.0 +irene falkner 65620.0 +irene falkner 65661.0 +irene garcia 65660.0 +irene garcia 65711.0 +irene garcia 65787.0 +irene ichabod 65645.0 +irene ichabod 65722.0 +irene johnson 65583.0 +irene laertes 65664.0 +irene laertes 65710.0 +irene laertes 65722.0 +irene miller 65730.0 +irene nixon 65631.0 +irene nixon 65643.0 +irene nixon 65653.0 +irene ovid 65691.0 +irene ovid 65734.0 +irene ovid 65753.0 +irene polk 65551.0 +irene polk 65575.0 +irene polk 65579.0 +irene polk 65595.0 +irene polk 65610.0 +irene quirinius 65724.0 +irene quirinius 65769.0 +irene quirinius 65773.0 +irene robinson 65554.0 +irene steinbeck 65683.0 +irene thompson 65688.0 +irene underhill 65591.0 +irene underhill 65707.5 +irene van buren 65579.0 +irene van buren 65589.0 +irene xylophone 65775.0 +jessica brown 65588.0 +jessica carson 65553.0 +jessica carson 65672.0 +jessica carson 65747.0 +jessica davidson 65549.0 +jessica davidson 65606.0 +jessica davidson 65675.0 +jessica davidson 65727.0 +jessica ellison 65567.0 +jessica ellison 65663.0 +jessica falkner 65584.0 +jessica garcia 65676.0 +jessica garcia 65789.0 +jessica ichabod 65704.0 +jessica johnson 65607.0 +jessica johnson 65720.0 +jessica miller 65733.0 +jessica nixon 65590.0 +jessica nixon 65774.0 +jessica ovid 65582.0 +jessica ovid 65751.0 +jessica polk 65637.0 +jessica quirinius 65562.0 +jessica quirinius 65608.0 +jessica quirinius 65712.0 +jessica quirinius 65716.0 +jessica robinson 65576.0 +jessica thompson 65581.0 +jessica thompson 65675.0 +jessica underhill 65656.0 +jessica underhill 65702.0 +jessica underhill 65783.0 +jessica van buren 65615.0 +jessica white 65544.0 +jessica white 65570.0 +jessica white 65594.0 +jessica white 65673.0 +jessica white 65779.0 +jessica xylophone 65562.0 +jessica young 65623.0 +jessica young 65711.0 +jessica zipper 65600.0 +jessica zipper 65657.0 +jessica zipper 65778.0 +katie allen 65542.0 +katie brown 65590.0 +katie davidson 65619.0 +katie ellison 65675.0 +katie ellison 65699.0 +katie falkner 65728.0 +katie garcia 65625.0 +katie garcia 65747.0 +katie hernandez 65550.0 +katie ichabod 65658.0 +katie ichabod 65726.0 +katie ichabod 65757.0 +katie king 65629.0 +katie king 65647.0 +katie king 65776.0 +katie miller 65541.0 +katie miller 65661.0 +katie nixon 65669.0 +katie ovid 65681.0 +katie polk 65746.0 +katie polk 65784.0 +katie robinson 65697.0 +katie van buren 65643.0 +katie van buren 65730.0 +katie white 65620.0 +katie white 65719.0 +katie xylophone 65585.0 +katie young 65644.0 +katie young 65746.0 +katie young 65764.0 +katie zipper 65568.0 +katie zipper 65733.0 +luke allen 65547.0 +luke allen 65552.0 +luke allen 65576.0 +luke allen 65681.0 +luke allen 65776.0 +luke brown 65719.0 +luke davidson 65656.0 +luke davidson 65791.0 +luke ellison 65582.0 +luke ellison 65664.0 +luke ellison 65779.0 +luke falkner 65589.0 +luke falkner 65618.0 +luke garcia 65687.0 +luke garcia 65778.0 +luke ichabod 65629.0 +luke ichabod 65654.0 +luke johnson 65545.0 +luke johnson 65716.0 +luke johnson 65718.0 +luke laertes 65608.0 +luke laertes 65657.0 +luke laertes 65685.0 +luke laertes 65730.0 +luke laertes 65756.0 +luke miller 65752.0 +luke ovid 65569.0 +luke ovid 65693.0 +luke polk 65645.0 +luke polk 65658.0 +luke quirinius 65655.0 +luke robinson 65634.0 +luke robinson 65772.0 +luke thompson 65626.0 +luke underhill 65553.0 +luke underhill 65571.0 +luke underhill 65651.0 +luke van buren 65678.0 +luke white 65693.0 +luke xylophone 65597.0 +luke zipper 65641.0 +mike allen 65706.0 +mike brown 65654.0 +mike carson 65698.0 +mike carson 65700.0 +mike carson 65751.0 +mike davidson 65658.0 +mike davidson 65759.0 +mike ellison 65598.0 +mike ellison 65606.0 +mike ellison 65718.0 +mike ellison 65738.0 +mike ellison 65760.0 +mike falkner 65609.0 +mike garcia 65571.0 +mike garcia 65600.0 +mike garcia 65770.0 +mike hernandez 65548.0 +mike hernandez 65672.0 +mike ichabod 65621.0 +mike king 65563.0 +mike king 65586.0 +mike king 65591.0 +mike king 65642.0 +mike king 65769.0 +mike king 65776.0 +mike miller 65549.0 +mike nixon 65619.0 +mike nixon 65704.0 +mike polk 65619.0 +mike polk 65658.0 +mike polk 65704.0 +mike quirinius 65717.0 +mike steinbeck 65550.0 +mike steinbeck 65564.0 +mike steinbeck 65573.0 +mike steinbeck 65749.0 +mike van buren 65620.0 +mike van buren 65770.0 +mike white 65648.0 +mike white 65685.0 +mike white 65769.0 +mike white 65778.0 +mike young 65545.0 +mike young 65581.0 +mike young 65736.0 +mike zipper 65552.0 +mike zipper 65695.0 +mike zipper 65779.0 +nick allen 65641.0 +nick allen 65786.0 +nick brown 65724.0 +nick davidson 65601.0 +nick ellison 65691.0 +nick ellison 65745.0 +nick falkner 65583.0 +nick falkner 65676.0 +nick garcia 65712.0 +nick garcia 65720.0 +nick garcia 65723.0 +nick ichabod 65572.0 +nick ichabod 65681.0 +nick ichabod 65737.0 +nick johnson 65585.0 +nick johnson 65784.0 +nick laertes 65624.0 +nick miller 65757.0 +nick nixon 65650.0 +nick ovid 65719.0 +nick polk 65716.0 +nick quirinius 65588.0 +nick quirinius 65723.0 +nick robinson 65547.0 +nick robinson 65675.0 +nick steinbeck 65689.0 +nick thompson 65610.0 +nick underhill 65619.0 +nick van buren 65603.0 +nick xylophone 65644.0 +nick young 65654.0 +nick young 65660.0 +nick zipper 65757.0 +nick zipper 65765.0 +oscar allen 65644.0 +oscar brown 65614.0 +oscar carson 65537.0 +oscar carson 65548.0 +oscar carson 65549.0 +oscar carson 65624.0 +oscar carson 65697.0 +oscar davidson 65556.0 +oscar ellison 65630.0 +oscar ellison 65630.0 +oscar falkner 65692.0 +oscar garcia 65751.0 +oscar hernandez 65683.0 +oscar hernandez 65707.0 +oscar ichabod 65536.0 +oscar ichabod 65562.0 +oscar ichabod 65637.0 +oscar ichabod 65763.0 +oscar johnson 65645.0 +oscar johnson 65778.0 +oscar king 65541.0 +oscar king 65550.0 +oscar king 65787.0 +oscar laertes 65625.0 +oscar laertes 65690.0 +oscar laertes 65756.0 +oscar laertes 65790.0 +oscar nixon 65596.0 +oscar ovid 65536.0 +oscar ovid 65615.0 +oscar ovid 65665.5 +oscar polk 65541.0 +oscar polk 65643.0 +oscar quirinius 65541.0 +oscar quirinius 65560.0 +oscar quirinius 65689.0 +oscar quirinius 65720.0 +oscar robinson 65537.0 +oscar robinson 65658.0 +oscar robinson 65687.0 +oscar robinson 65782.0 +oscar steinbeck 65709.0 +oscar thompson 65542.0 +oscar thompson 65681.0 +oscar thompson 65727.0 +oscar thompson 65738.0 +oscar underhill 65626.0 +oscar van buren 65581.0 +oscar van buren 65635.0 +oscar van buren 65705.0 +oscar white 65552.0 +oscar white 65564.0 +oscar white 65671.0 +oscar white 65735.0 +oscar xylophone 65773.0 +oscar xylophone 65773.0 +oscar xylophone 65775.0 +oscar zipper 65568.0 +oscar zipper 65740.0 +oscar zipper 65777.0 +priscilla brown 65670.0 +priscilla brown 65690.0 +priscilla brown 65749.0 +priscilla carson 65658.0 +priscilla carson 65687.0 +priscilla carson 65755.0 +priscilla ichabod 65627.0 +priscilla ichabod 65759.0 +priscilla johnson 65543.0 +priscilla johnson 65668.0 +priscilla johnson 65674.5 +priscilla johnson 65681.0 +priscilla johnson 65755.0 +priscilla king 65646.0 +priscilla nixon 65564.0 +priscilla nixon 65600.0 +priscilla ovid 65541.0 +priscilla ovid 65790.0 +priscilla polk 65747.0 +priscilla quirinius 65672.0 +priscilla thompson 65654.0 +priscilla underhill 65715.0 +priscilla underhill 65729.0 +priscilla van buren 65607.0 +priscilla van buren 65685.0 +priscilla van buren 65749.0 +priscilla white 65652.0 +priscilla xylophone 65538.0 +priscilla xylophone 65763.0 +priscilla xylophone 65774.0 +priscilla young 65585.0 +priscilla young 65658.0 +priscilla zipper 65622.0 +priscilla zipper 65726.0 +quinn allen 65657.0 +quinn allen 65708.0 +quinn brown 65691.0 +quinn brown 65700.0 +quinn brown 65733.0 +quinn davidson 65549.0 +quinn davidson 65714.0 +quinn davidson 65776.0 +quinn davidson 65779.0 +quinn ellison 65705.0 +quinn ellison 65778.0 +quinn garcia 65568.0 +quinn garcia 65604.0 +quinn garcia 65610.0 +quinn garcia 65773.0 +quinn ichabod 65609.0 +quinn king 65558.0 +quinn king 65649.0 +quinn laertes 65542.0 +quinn laertes 65560.0 +quinn laertes 65627.0 +quinn nixon 65659.0 +quinn ovid 65699.0 +quinn quirinius 65747.0 +quinn robinson 65627.0 +quinn steinbeck 65578.0 +quinn steinbeck 65763.0 +quinn thompson 65643.0 +quinn thompson 65774.0 +quinn underhill 65549.0 +quinn underhill 65694.0 +quinn underhill 65767.0 +quinn van buren 65725.0 +quinn young 65647.0 +quinn zipper 65579.0 +quinn zipper 65693.0 +rachel allen 65661.0 +rachel allen 65709.0 +rachel brown 65586.0 +rachel brown 65587.0 +rachel brown 65587.0 +rachel brown 65610.0 +rachel brown 65693.0 +rachel carson 65677.0 +rachel carson 65682.0 +rachel davidson 65755.0 +rachel ellison 65761.0 +rachel falkner 65616.0 +rachel falkner 65681.0 +rachel falkner 65693.0 +rachel falkner 65764.0 +rachel johnson 65658.0 +rachel king 65604.0 +rachel king 65643.0 +rachel laertes 65562.0 +rachel laertes 65624.0 +rachel ovid 65721.0 +rachel ovid 65736.0 +rachel polk 65686.0 +rachel quirinius 65787.0 +rachel robinson 65544.0 +rachel robinson 65717.0 +rachel robinson 65724.0 +rachel thompson 65648.0 +rachel thompson 65662.0 +rachel thompson 65733.0 +rachel underhill 65667.0 +rachel white 65615.0 +rachel white 65717.0 +rachel young 65727.0 +rachel zipper 65757.0 +rachel zipper 65785.0 +sarah carson 65616.0 +sarah carson 65693.0 +sarah carson 65694.0 +sarah ellison 65611.0 +sarah falkner 65606.0 +sarah falkner 65680.0 +sarah garcia 65563.0 +sarah garcia 65638.0 +sarah garcia 65661.0 +sarah ichabod 65667.0 +sarah ichabod 65671.0 +sarah johnson 65659.0 +sarah johnson 65716.0 +sarah johnson 65731.0 +sarah johnson 65751.0 +sarah king 65650.0 +sarah king 65699.0 +sarah miller 65557.0 +sarah ovid 65550.0 +sarah robinson 65677.0 +sarah robinson 65763.0 +sarah steinbeck 65721.0 +sarah white 65622.0 +sarah white 65747.0 +sarah xylophone 65678.0 +sarah young 65595.0 +sarah zipper 65550.0 +tom brown 65593.0 +tom brown 65675.0 +tom carson 65539.0 +tom carson 65624.0 +tom carson 65780.0 +tom davidson 65780.0 +tom ellison 65578.0 +tom ellison 65670.0 +tom ellison 65756.0 +tom falkner 65574.0 +tom falkner 65625.0 +tom hernandez 65575.0 +tom hernandez 65632.0 +tom ichabod 65588.0 +tom johnson 65536.0 +tom johnson 65789.0 +tom king 65576.0 +tom laertes 65617.0 +tom laertes 65701.0 +tom miller 65594.0 +tom miller 65603.0 +tom miller 65704.0 +tom nixon 65672.0 +tom ovid 65628.0 +tom polk 65652.0 +tom polk 65742.0 +tom quirinius 65563.0 +tom quirinius 65783.0 +tom robinson 65626.0 +tom robinson 65632.0 +tom robinson 65691.0 +tom robinson 65758.0 +tom steinbeck 65666.0 +tom van buren 65621.0 +tom van buren 65652.0 +tom van buren 65669.0 +tom white 65548.0 +tom young 65544.0 +tom young 65546.0 +tom zipper 65789.0 +ulysses brown 65735.0 +ulysses carson 65602.0 +ulysses carson 65643.0 +ulysses carson 65703.0 +ulysses carson 65716.0 +ulysses davidson 65750.0 +ulysses ellison 65575.0 +ulysses garcia 65666.0 +ulysses hernandez 65651.0 +ulysses hernandez 65702.0 +ulysses hernandez 65786.0 +ulysses ichabod 65551.0 +ulysses ichabod 65566.0 +ulysses johnson 65776.0 +ulysses king 65649.0 +ulysses laertes 65691.0 +ulysses laertes 65711.0 +ulysses laertes 65781.0 +ulysses miller 65610.0 +ulysses miller 65637.0 +ulysses nixon 65603.0 +ulysses ovid 65656.0 +ulysses polk 65563.0 +ulysses polk 65580.0 +ulysses polk 65612.0 +ulysses polk 65777.0 +ulysses quirinius 65786.0 +ulysses robinson 65744.0 +ulysses steinbeck 65611.0 +ulysses steinbeck 65680.0 +ulysses thompson 65788.0 +ulysses underhill 65570.0 +ulysses underhill 65616.0 +ulysses underhill 65620.0 +ulysses underhill 65623.0 +ulysses underhill 65641.0 +ulysses underhill 65713.0 +ulysses underhill 65785.0 +ulysses van buren 65684.0 +ulysses white 65654.0 +ulysses white 65675.0 +ulysses xylophone 65623.0 +ulysses xylophone 65636.0 +ulysses xylophone 65781.0 +ulysses young 65675.0 +ulysses young 65736.0 +ulysses young 65748.0 +victor allen 65684.0 +victor allen 65707.0 +victor brown 65550.0 +victor brown 65555.0 +victor brown 65622.0 +victor brown 65673.0 +victor davidson 65579.0 +victor davidson 65628.0 +victor davidson 65783.0 +victor ellison 65641.0 +victor ellison 65782.0 +victor hernandez 65571.0 +victor hernandez 65659.0 +victor hernandez 65708.0 +victor hernandez 65735.0 +victor hernandez 65775.0 +victor johnson 65606.0 +victor johnson 65607.0 +victor johnson 65607.0 +victor king 65721.0 +victor king 65743.0 +victor laertes 65638.0 +victor laertes 65644.0 +victor miller 65570.0 +victor nixon 65709.0 +victor nixon 65791.0 +victor ovid 65649.0 +victor polk 65625.0 +victor quirinius 65620.0 +victor quirinius 65651.0 +victor robinson 65596.0 +victor robinson 65673.0 +victor steinbeck 65618.0 +victor steinbeck 65661.0 +victor steinbeck 65686.0 +victor thompson 65548.0 +victor van buren 65664.0 +victor van buren 65774.0 +victor white 65548.0 +victor white 65601.0 +victor xylophone 65549.0 +victor xylophone 65618.0 +victor xylophone 65644.0 +victor xylophone 65677.0 +victor xylophone 65755.0 +victor young 65628.0 +victor zipper 65743.0 +wendy allen 65628.0 +wendy allen 65711.0 +wendy allen 65782.0 +wendy brown 65580.0 +wendy brown 65657.0 +wendy ellison 65545.0 +wendy ellison 65603.0 +wendy falkner 65595.0 +wendy falkner 65604.0 +wendy falkner 65635.0 +wendy garcia 65659.0 +wendy garcia 65746.0 +wendy garcia 65747.0 +wendy garcia 65777.0 +wendy hernandez 65650.0 +wendy ichabod 65730.0 +wendy king 65586.0 +wendy king 65664.0 +wendy king 65670.0 +wendy laertes 65566.0 +wendy laertes 65683.0 +wendy laertes 65727.0 +wendy miller 65582.0 +wendy miller 65626.0 +wendy nixon 65611.0 +wendy nixon 65746.0 +wendy ovid 65589.0 +wendy ovid 65643.0 +wendy polk 65656.0 +wendy polk 65692.0 +wendy quirinius 65766.0 +wendy quirinius 65767.0 +wendy robinson 65622.0 +wendy robinson 65715.0 +wendy robinson 65774.0 +wendy steinbeck 65612.0 +wendy thompson 65650.0 +wendy thompson 65737.0 +wendy underhill 65662.0 +wendy underhill 65758.0 +wendy underhill 65775.0 +wendy van buren 65680.0 +wendy van buren 65699.0 +wendy white 65705.0 +wendy xylophone 65687.0 +wendy xylophone 65773.0 +wendy young 65674.0 +wendy young 65685.0 +xavier allen 65611.0 +xavier allen 65618.0 +xavier allen 65771.0 +xavier brown 65600.0 +xavier brown 65704.0 +xavier brown 65723.0 +xavier carson 65731.0 +xavier carson 65758.0 +xavier davidson 65644.0 +xavier davidson 65664.0 +xavier davidson 65755.0 +xavier ellison 65541.0 +xavier ellison 65622.0 +xavier garcia 65672.0 +xavier hernandez 65541.0 +xavier hernandez 65544.0 +xavier hernandez 65766.0 +xavier ichabod 65597.0 +xavier ichabod 65663.0 +xavier johnson 65655.0 +xavier johnson 65744.0 +xavier king 65590.0 +xavier king 65601.0 +xavier laertes 65743.0 +xavier ovid 65788.0 +xavier polk 65587.0 +xavier polk 65653.0 +xavier polk 65675.0 +xavier polk 65696.0 +xavier quirinius 65599.0 +xavier quirinius 65650.0 +xavier quirinius 65656.0 +xavier quirinius 65737.0 +xavier thompson 65608.0 +xavier underhill 65710.0 +xavier white 65703.0 +xavier white 65732.0 +xavier xylophone 65572.0 +xavier zipper 65561.0 +yuri allen 65565.0 +yuri allen 65682.0 +yuri brown 65538.0 +yuri brown 65688.0 +yuri carson 65670.0 +yuri carson 65769.0 +yuri ellison 65570.0 +yuri ellison 65581.0 +yuri falkner 65658.0 +yuri falkner 65681.0 +yuri garcia 65639.0 +yuri hernandez 65706.0 +yuri johnson 65587.0 +yuri johnson 65697.0 +yuri johnson 65712.0 +yuri king 65721.0 +yuri laertes 65637.0 +yuri laertes 65773.0 +yuri nixon 65635.0 +yuri nixon 65740.0 +yuri polk 65607.0 +yuri polk 65713.0 +yuri polk 65742.0 +yuri quirinius 65544.0 +yuri quirinius 65617.0 +yuri quirinius 65695.0 +yuri steinbeck 65592.0 +yuri steinbeck 65679.0 +yuri thompson 65676.0 +yuri underhill 65718.0 +yuri underhill 65750.0 +yuri white 65659.0 +yuri xylophone 65714.0 +zach allen 65667.0 +zach brown 65559.0 +zach brown 65588.0 +zach brown 65691.0 +zach brown 65759.0 +zach brown 65762.0 +zach carson 65572.0 +zach ellison 65748.0 +zach falkner 65620.0 +zach falkner 65627.0 +zach garcia 65544.0 +zach garcia 65623.0 +zach garcia 65629.0 +zach garcia 65764.5 +zach ichabod 65599.0 +zach ichabod 65612.0 +zach king 65556.0 +zach king 65702.0 +zach king 65773.0 +zach miller 65583.0 +zach miller 65665.0 +zach miller 65719.0 +zach ovid 65578.0 +zach ovid 65669.0 +zach ovid 65703.0 +zach ovid 65784.0 +zach quirinius 65691.0 +zach robinson 65599.0 +zach steinbeck 65602.0 +zach steinbeck 65695.0 +zach thompson 65636.0 +zach thompson 65696.0 +zach underhill 65573.0 +zach white 65733.0 +zach xylophone 65542.0 +zach xylophone 65780.0 +zach young 65576.0 +zach zipper 65579.0 +zach zipper 65649.0 +zach zipper 65676.0 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 new file mode 100644 index 0000000000000..072a8a891a836 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-3-d90b27fca067b0b3c48d873b3ef32af7 @@ -0,0 +1,1049 @@ +65536 +65536 +65536 +65536 +65536 +65536 +65537 +65537 +65537 +65537 +65538 +65538 +65538 +65539 +65539 +65539 +65540 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65541 +65542 +65542 +65542 +65542 +65543 +65543 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65544 +65545 +65545 +65545 +65547 +65547 +65547 +65547 +65548 +65548 +65548 +65548 +65548 +65548 +65548 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65549 +65550 +65550 +65550 +65550 +65550 +65550 +65550 +65551 +65551 +65551 +65552 +65552 +65552 +65552 +65552 +65553 +65553 +65553 +65553 +65554 +65554 +65555 +65556 +65556 +65556 +65557 +65557 +65558 +65559 +65559 +65559 +65559 +65560 +65560 +65560 +65560 +65561 +65562 +65562 +65562 +65562 +65562 +65562 +65563 +65563 +65563 +65563 +65563 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65564 +65565 +65565 +65566 +65566 +65567 +65568 +65568 +65568 +65569 +65570 +65570 +65570 +65570 +65570 +65571 +65571 +65571 +65572 +65572 +65572 +65572 +65572 +65573 +65573 +65573 +65574 +65574 +65574 +65575 +65575 +65575 +65575 +65576 +65576 +65576 +65576 +65577 +65578 +65578 +65578 +65578 +65578 +65578 +65579 +65579 +65579 +65579 +65579 +65580 +65580 +65580 +65581 +65581 +65581 +65581 +65581 +65582 +65582 +65582 +65582 +65582 +65583 +65583 +65583 +65583 +65584 +65584 +65584 +65584 +65585 +65585 +65585 +65585 +65585 +65586 +65586 +65586 +65586 +65586 +65587 +65587 +65587 +65587 +65588 +65588 +65588 +65588 +65588 +65589 +65589 +65589 +65589 +65589 +65590 +65590 +65590 +65590 +65590 +65591 +65591 +65591 +65591 +65592 +65593 +65593 +65594 +65594 +65594 +65594 +65595 +65595 +65595 +65595 +65595 +65595 +65596 +65596 +65596 +65596 +65596 +65597 +65597 +65597 +65598 +65599 +65599 +65599 +65600 +65600 +65600 +65600 +65601 +65601 +65601 +65602 +65602 +65602 +65602 +65603 +65603 +65603 +65603 +65603 +65604 +65604 +65604 +65605 +65606 +65606 +65606 +65606 +65606 +65607 +65607 +65607 +65607 +65607 +65607 +65608 +65608 +65608 +65608 +65609 +65610 +65610 +65610 +65610 +65610 +65610 +65611 +65611 +65611 +65612 +65612 +65612 +65614 +65614 +65614 +65615 +65615 +65615 +65616 +65616 +65617 +65617 +65617 +65617 +65618 +65618 +65618 +65618 +65618 +65619 +65619 +65619 +65619 +65619 +65619 +65620 +65620 +65620 +65620 +65620 +65620 +65621 +65621 +65622 +65622 +65622 +65622 +65622 +65622 +65623 +65623 +65623 +65623 +65623 +65623 +65623 +65624 +65624 +65624 +65624 +65625 +65625 +65625 +65625 +65626 +65626 +65626 +65626 +65627 +65627 +65627 +65627 +65628 +65628 +65628 +65628 +65628 +65628 +65629 +65629 +65629 +65630 +65630 +65631 +65632 +65632 +65632 +65633 +65633 +65633 +65633 +65634 +65634 +65634 +65634 +65635 +65635 +65635 +65636 +65636 +65636 +65636 +65637 +65637 +65637 +65637 +65637 +65637 +65637 +65638 +65638 +65638 +65639 +65640 +65641 +65641 +65641 +65641 +65642 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65643 +65644 +65644 +65644 +65644 +65644 +65645 +65645 +65645 +65646 +65646 +65646 +65647 +65647 +65648 +65648 +65648 +65648 +65649 +65649 +65649 +65650 +65650 +65650 +65650 +65650 +65650 +65650 +65651 +65651 +65651 +65651 +65651 +65651 +65652 +65652 +65652 +65653 +65653 +65653 +65653 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65654 +65655 +65655 +65656 +65656 +65656 +65656 +65656 +65656 +65657 +65657 +65657 +65657 +65657 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65658 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65659 +65660 +65660 +65660 +65661 +65661 +65661 +65661 +65661 +65662 +65662 +65662 +65662 +65662 +65662 +65663 +65663 +65663 +65663 +65664 +65664 +65664 +65664 +65664 +65664 +65664 +65665 +65666 +65666 +65667 +65667 +65667 +65667 +65667 +65667 +65668 +65669 +65669 +65669 +65669 +65669 +65669 +65670 +65670 +65670 +65670 +65670 +65671 +65671 +65671 +65672 +65672 +65672 +65672 +65672 +65672 +65673 +65673 +65673 +65673 +65673 +65674 +65674 +65674 +65674 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65675 +65676 +65676 +65676 +65677 +65677 +65677 +65677 +65677 +65677 +65678 +65678 +65678 +65678 +65679 +65679 +65680 +65680 +65680 +65680 +65680 +65680 +65680 +65681 +65681 +65681 +65681 +65681 +65682 +65682 +65682 +65683 +65683 +65683 +65683 +65684 +65684 +65684 +65684 +65685 +65685 +65685 +65685 +65685 +65685 +65686 +65686 +65686 +65687 +65687 +65687 +65687 +65688 +65688 +65689 +65689 +65690 +65690 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65691 +65692 +65692 +65693 +65693 +65693 +65693 +65693 +65693 +65694 +65694 +65694 +65695 +65695 +65695 +65695 +65695 +65696 +65696 +65696 +65696 +65697 +65697 +65697 +65697 +65697 +65697 +65698 +65698 +65698 +65699 +65699 +65699 +65699 +65699 +65699 +65700 +65700 +65700 +65701 +65701 +65701 +65702 +65702 +65702 +65702 +65702 +65703 +65703 +65703 +65703 +65703 +65704 +65704 +65704 +65704 +65704 +65704 +65705 +65705 +65705 +65705 +65706 +65706 +65706 +65706 +65706 +65706 +65707 +65707 +65708 +65708 +65709 +65709 +65709 +65710 +65711 +65711 +65711 +65711 +65711 +65711 +65712 +65712 +65712 +65712 +65712 +65713 +65713 +65713 +65713 +65713 +65714 +65714 +65714 +65715 +65715 +65715 +65715 +65715 +65716 +65716 +65716 +65716 +65716 +65716 +65717 +65717 +65717 +65717 +65717 +65718 +65718 +65718 +65718 +65719 +65719 +65719 +65719 +65720 +65720 +65720 +65720 +65720 +65720 +65721 +65721 +65721 +65721 +65721 +65721 +65721 +65722 +65722 +65722 +65722 +65723 +65723 +65724 +65724 +65724 +65724 +65724 +65724 +65725 +65726 +65726 +65726 +65726 +65727 +65727 +65727 +65727 +65727 +65728 +65728 +65729 +65730 +65730 +65730 +65730 +65731 +65731 +65731 +65731 +65732 +65732 +65732 +65733 +65733 +65733 +65733 +65733 +65733 +65734 +65734 +65735 +65735 +65735 +65736 +65736 +65736 +65736 +65737 +65737 +65737 +65737 +65737 +65738 +65738 +65738 +65738 +65739 +65739 +65739 +65740 +65740 +65740 +65741 +65742 +65742 +65742 +65743 +65743 +65743 +65743 +65744 +65744 +65745 +65745 +65745 +65745 +65746 +65746 +65746 +65746 +65747 +65747 +65747 +65747 +65747 +65747 +65747 +65748 +65748 +65749 +65749 +65749 +65749 +65749 +65750 +65750 +65750 +65750 +65750 +65751 +65751 +65751 +65751 +65751 +65752 +65752 +65753 +65753 +65754 +65754 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65755 +65756 +65756 +65756 +65756 +65756 +65757 +65757 +65757 +65757 +65757 +65758 +65758 +65758 +65758 +65758 +65758 +65759 +65759 +65759 +65759 +65759 +65760 +65760 +65760 +65760 +65760 +65761 +65762 +65762 +65762 +65762 +65762 +65763 +65763 +65763 +65763 +65763 +65764 +65764 +65764 +65765 +65765 +65765 +65766 +65766 +65766 +65766 +65766 +65767 +65767 +65767 +65768 +65769 +65769 +65769 +65769 +65769 +65769 +65769 +65770 +65770 +65771 +65771 +65771 +65772 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65773 +65774 +65774 +65774 +65774 +65774 +65774 +65775 +65775 +65775 +65775 +65775 +65775 +65776 +65776 +65776 +65776 +65776 +65776 +65776 +65777 +65777 +65777 +65777 +65777 +65777 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65778 +65779 +65779 +65779 +65779 +65779 +65779 +65779 +65780 +65780 +65780 +65781 +65781 +65781 +65782 +65782 +65782 +65782 +65782 +65783 +65783 +65783 +65783 +65783 +65783 +65783 +65784 +65784 +65784 +65784 +65784 +65785 +65785 +65786 +65786 +65786 +65786 +65786 +65787 +65787 +65787 +65787 +65787 +65788 +65788 +65788 +65788 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65789 +65790 +65790 +65790 +65791 +65791 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 new file mode 100644 index 0000000000000..9cc7e7ea6c2b2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf.q (deterministic)-4-f2e4d659b65a833e9281b6786d3d55c1 @@ -0,0 +1,1049 @@ + 24.37875 + 27.900000000000002 + 43.64 +alice allen 16.919999999999998 +alice allen 20.39 +alice allen 23.59 +alice brown 6.91 +alice carson 41.74 +alice davidson 26.346000000000004 +alice falkner 32.166666666666664 +alice garcia 15.412 +alice hernandez 19.958181818181817 +alice hernandez 30.482857142857142 +alice johnson 25.51 +alice king 13.085 +alice king 25.616666666666664 +alice king 42.335 +alice laertes 20.549999999999997 +alice laertes 30.436 +alice miller 29.563333333333333 +alice nixon 19.28666666666667 +alice nixon 24.5625 +alice nixon 32.99 +alice ovid 31.35 +alice polk 17.863999999999997 +alice quirinius 19.032857142857143 +alice quirinius 23.9425 +alice robinson 23.338750000000005 +alice robinson 39.85 +alice steinbeck 22.862000000000002 +alice steinbeck 26.328000000000003 +alice steinbeck 27.08777777777778 +alice underhill 24.032222222222224 +alice van buren 19.642000000000003 +alice xylophone 24.438000000000002 +alice xylophone 28.739999999999995 +alice xylophone 30.0825 +alice zipper 26.3 +alice zipper 28.735000000000003 +alice zipper 31.05545454545455 +bob brown 12.902222222222223 +bob brown 13.945 +bob brown 33.843333333333334 +bob carson 28.627999999999997 +bob davidson 19.8525 +bob davidson 23.482 +bob davidson 24.67 +bob ellison 16.315714285714286 +bob ellison 18.4 +bob ellison 26.913999999999998 +bob ellison 27.59 +bob falkner 9.27 +bob garcia 11.63 +bob garcia 22.221249999999998 +bob garcia 23.59636363636364 +bob garcia 26.88857142857143 +bob garcia 28.715000000000003 +bob hernandez 37.23 +bob ichabod 28.33875 +bob king 8.615 +bob king 19.77 +bob king 26.7325 +bob laertes 21.33 +bob laertes 37.88 +bob miller 25.495 +bob ovid 25.675 +bob ovid 25.83 +bob ovid 28.37875 +bob ovid 32.5025 +bob polk 9.74 +bob quirinius 34.57 +bob steinbeck 9.725 +bob van buren 29.552857142857142 +bob white 17.685 +bob white 29.46285714285715 +bob xylophone 17.03 +bob xylophone 33.24 +bob young 19.824 +bob zipper 24.095 +bob zipper 33.36 +bob zipper 34.99 +calvin allen 21.3 +calvin brown 20.808 +calvin brown 24.16 +calvin brown 24.636666666666667 +calvin carson 22.815 +calvin davidson 22.116666666666664 +calvin davidson 22.364 +calvin ellison 24.92 +calvin falkner 18.343999999999998 +calvin falkner 19.56 +calvin falkner 22.946000000000005 +calvin falkner 23.327777777777776 +calvin falkner 23.974999999999998 +calvin falkner 33.382 +calvin garcia 17.285 +calvin hernandez 12.663333333333334 +calvin johnson 24.898571428571433 +calvin laertes 28.105 +calvin laertes 28.362000000000002 +calvin nixon 26.784285714285716 +calvin nixon 27.36 +calvin nixon 32.282 +calvin ovid 22.063333333333336 +calvin ovid 22.81500000000001 +calvin ovid 25.495714285714286 +calvin ovid 30.926666666666666 +calvin polk 27.820000000000004 +calvin quirinius 16.28 +calvin quirinius 25.552500000000002 +calvin robinson 31.814999999999998 +calvin steinbeck 12.85 +calvin steinbeck 14.939999999999998 +calvin steinbeck 17.535 +calvin thompson 28.592857142857145 +calvin thompson 40.79 +calvin underhill 24.062 +calvin van buren 26.525 +calvin van buren 28.865 +calvin white 28.256249999999998 +calvin white 43.275 +calvin xylophone 24.13111111111111 +calvin xylophone 25.27 +calvin xylophone 36.455 +calvin young 19.06 +calvin young 21.455999999999996 +calvin zipper 10.674999999999999 +calvin zipper 26.012857142857143 +david allen 25.134285714285713 +david allen 41.72333333333333 +david brown 8.52 +david brown 28.968181818181815 +david davidson 17.63 +david davidson 26.563333333333336 +david davidson 30.7325 +david davidson 33.33 +david ellison 23.79909090909091 +david ellison 24.74888888888889 +david ellison 26.198571428571427 +david hernandez 27.766 +david ichabod 16.66 +david ichabod 19.538 +david laertes 24.587500000000002 +david nixon 26.01375 +david ovid 24.131428571428575 +david ovid 32.72 +david quirinius 16.5 +david quirinius 25.08 +david quirinius 29.415 +david robinson 22.2175 +david robinson 30.99 +david thompson 25.38 +david underhill 1.17 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 26.45833333333334 +david van buren 35.7825 +david white 15.833333333333334 +david xylophone 10.71 +david xylophone 26.341428571428565 +david xylophone 33.224000000000004 +david young 9.64 +david young 21.22 +ethan allen 22.68 +ethan brown 19.37 +ethan brown 21.58666666666667 +ethan brown 21.799999999999997 +ethan brown 29.099999999999998 +ethan brown 32.43666666666667 +ethan brown 39.84 +ethan carson 24.15666666666667 +ethan ellison 27.80777777777778 +ethan ellison 48.71 +ethan falkner 17.993333333333336 +ethan falkner 26.775000000000002 +ethan garcia 19.15 +ethan hernandez 25.081111111111113 +ethan johnson 32.81875 +ethan king 19.51 +ethan laertes 16.463 +ethan laertes 17.625999999999998 +ethan laertes 25.020714285714288 +ethan laertes 26.697142857142858 +ethan laertes 28.14 +ethan laertes 29.668571428571425 +ethan laertes 36.589999999999996 +ethan miller 24.326666666666664 +ethan nixon 34.78666666666667 +ethan ovid 20.642857142857142 +ethan polk 6.98 +ethan polk 12.756666666666666 +ethan polk 30.324 +ethan polk 40.46 +ethan quirinius 23.419999999999998 +ethan quirinius 24.36 +ethan quirinius 29.068 +ethan robinson 24.463750000000005 +ethan robinson 31.630000000000003 +ethan underhill 19.86 +ethan van buren 22.241999999999997 +ethan white 31.3175 +ethan white 32.87 +ethan xylophone 30.996000000000002 +ethan zipper 22.728333333333335 +ethan zipper 29.66 +fred davidson 30.116666666666667 +fred davidson 33.55200000000001 +fred davidson 39.37 +fred ellison 16.72 +fred ellison 17.462 +fred ellison 35.1 +fred falkner 14.51 +fred falkner 27.207000000000004 +fred falkner 27.887500000000003 +fred hernandez 36.045 +fred ichabod 29.017000000000003 +fred ichabod 30.405000000000005 +fred johnson 16.9925 +fred king 20.024 +fred king 32.54666666666667 +fred laertes 25.610000000000003 +fred miller 25.92 +fred nixon 14.915 +fred nixon 21.830000000000002 +fred nixon 24.4125 +fred nixon 31.360000000000003 +fred polk 18.698 +fred polk 19.743000000000002 +fred polk 20.96 +fred polk 31.11 +fred quirinius 20.085 +fred quirinius 33.9 +fred robinson 22.502 +fred steinbeck 21.123749999999998 +fred steinbeck 25.572 +fred steinbeck 30.81 +fred underhill 29.198888888888884 +fred van buren 21.34 +fred van buren 23.285 +fred van buren 26.520000000000003 +fred van buren 33.6 +fred white 21.41 +fred young 16.876250000000002 +fred young 20.996666666666666 +fred zipper 23.627499999999998 +gabriella allen 24.113333333333333 +gabriella allen 28.4725 +gabriella brown 29.963333333333335 +gabriella brown 30.65222222222222 +gabriella carson 16.6325 +gabriella davidson 34.52 +gabriella ellison 20.18 +gabriella ellison 29.62 +gabriella falkner 14.37 +gabriella falkner 17.738333333333333 +gabriella falkner 28.61 +gabriella garcia 39.025 +gabriella hernandez 20.818333333333335 +gabriella hernandez 24.601666666666663 +gabriella ichabod 10.4925 +gabriella ichabod 20.686666666666667 +gabriella ichabod 23.185 +gabriella ichabod 23.43 +gabriella ichabod 27.44636363636364 +gabriella king 13.645 +gabriella king 22.23 +gabriella laertes 23.735 +gabriella miller 17.165 +gabriella ovid 22.884545454545453 +gabriella ovid 25.29 +gabriella polk 20.38714285714286 +gabriella polk 25.832000000000004 +gabriella steinbeck 6.226666666666667 +gabriella steinbeck 29.683333333333337 +gabriella thompson 25.565454545454546 +gabriella thompson 29.031 +gabriella thompson 29.122500000000006 +gabriella van buren 24.353 +gabriella van buren 34.21666666666667 +gabriella white 36.5175 +gabriella young 21.28142857142857 +gabriella young 21.32 +gabriella zipper 21.798461538461545 +gabriella zipper 28.676666666666666 +holly allen 27.18 +holly brown 22.76 +holly brown 30.950000000000003 +holly falkner 29.666666666666668 +holly hernandez 19.875 +holly hernandez 23.7075 +holly hernandez 24.5 +holly hernandez 26.50333333333333 +holly ichabod 23.262857142857143 +holly ichabod 25.85090909090909 +holly ichabod 29.521666666666665 +holly johnson 18.939999999999998 +holly johnson 23.2625 +holly johnson 26.49285714285714 +holly king 20.61333333333333 +holly king 30.95888888888889 +holly laertes 17.509999999999998 +holly miller 40.8975 +holly nixon 27.775714285714287 +holly nixon 30.642500000000002 +holly polk 21.02 +holly polk 24.446666666666665 +holly robinson 26.083750000000006 +holly thompson 18.801428571428573 +holly thompson 23.91 +holly thompson 29.97125 +holly underhill 18.19 +holly underhill 22.22888888888889 +holly underhill 22.813333333333336 +holly underhill 30.613999999999997 +holly van buren 20.113333333333333 +holly white 25.284999999999997 +holly white 41.0125 +holly xylophone 26.88571428571429 +holly young 30.8425 +holly young 33.24333333333334 +holly zipper 27.784000000000002 +holly zipper 28.384285714285713 +irene allen 34.605000000000004 +irene brown 18.740000000000002 +irene brown 28.974999999999998 +irene brown 32.230000000000004 +irene carson 25.665833333333335 +irene ellison 10.225000000000001 +irene ellison 26.119999999999997 +irene falkner 9.94 +irene falkner 19.41 +irene garcia 9.790000000000001 +irene garcia 19.666666666666668 +irene garcia 21.22666666666667 +irene ichabod 20.956666666666667 +irene ichabod 24.488333333333333 +irene johnson 25.34 +irene laertes 15.85 +irene laertes 21.573333333333334 +irene laertes 22.041999999999998 +irene miller 34.994285714285716 +irene nixon 22.52 +irene nixon 32.485 +irene nixon 33.165 +irene ovid 17.73 +irene ovid 22.96 +irene ovid 30.92 +irene polk 5.35 +irene polk 25.535 +irene polk 33.76 +irene polk 35.05 +irene polk 45.14 +irene quirinius 38.36 +irene quirinius 41.864999999999995 +irene quirinius 42.0 +irene robinson 30.86 +irene steinbeck 15.08 +irene thompson 28.419999999999998 +irene underhill 27.977999999999998 +irene underhill 28.438 +irene van buren 26.93625 +irene van buren 27.797999999999995 +irene xylophone 29.10454545454546 +jessica brown 38.325 +jessica carson 16.038 +jessica carson 29.668333333333337 +jessica carson 33.06 +jessica davidson 18.926 +jessica davidson 26.2975 +jessica davidson 27.611428571428572 +jessica davidson 29.86 +jessica ellison 26.873333333333335 +jessica ellison 27.123333333333335 +jessica falkner 21.75142857142858 +jessica garcia 16.939090909090908 +jessica garcia 26.48 +jessica ichabod 28.971666666666664 +jessica johnson 21.601428571428574 +jessica johnson 24.42 +jessica miller 26.90571428571429 +jessica nixon 19.15 +jessica nixon 27.025000000000002 +jessica ovid 30.72285714285714 +jessica ovid 30.895 +jessica polk 27.912857142857145 +jessica quirinius 17.05 +jessica quirinius 21.529999999999998 +jessica quirinius 25.16 +jessica quirinius 26.347999999999995 +jessica robinson 24.322857142857142 +jessica thompson 28.658000000000005 +jessica thompson 30.873636363636365 +jessica underhill 14.6725 +jessica underhill 25.831666666666667 +jessica underhill 31.345000000000002 +jessica van buren 19.575 +jessica white 18.35 +jessica white 19.175 +jessica white 20.812 +jessica white 26.0 +jessica white 29.307142857142857 +jessica xylophone 22.26 +jessica young 27.9525 +jessica young 37.61333333333334 +jessica zipper 7.03 +jessica zipper 15.794999999999998 +jessica zipper 19.95 +katie allen 27.283846153846152 +katie brown 24.156666666666666 +katie davidson 13.498000000000001 +katie ellison 19.2 +katie ellison 24.888571428571428 +katie falkner 28.959999999999997 +katie garcia 28.287142857142857 +katie garcia 36.196666666666665 +katie hernandez 25.14428571428572 +katie ichabod 19.363333333333333 +katie ichabod 20.458571428571428 +katie ichabod 28.924999999999997 +katie king 21.64125 +katie king 21.855 +katie king 22.895 +katie miller 16.263333333333335 +katie miller 30.274285714285718 +katie nixon 25.022499999999997 +katie ovid 24.055000000000003 +katie polk 21.296666666666667 +katie polk 32.03 +katie robinson 36.26 +katie van buren 28.332 +katie van buren 31.408000000000005 +katie white 23.48 +katie white 26.236666666666665 +katie xylophone 32.415 +katie young 18.209999999999997 +katie young 22.88125 +katie young 28.39888888888889 +katie zipper 10.285 +katie zipper 27.495 +luke allen 9.42 +luke allen 21.374615384615385 +luke allen 25.32 +luke allen 27.174999999999997 +luke allen 35.434 +luke brown 25.08 +luke davidson 28.205 +luke davidson 28.790000000000003 +luke ellison 7.8 +luke ellison 16.04 +luke ellison 23.426666666666666 +luke falkner 18.0 +luke falkner 22.19 +luke garcia 29.619999999999997 +luke garcia 32.722 +luke ichabod 21.150000000000002 +luke ichabod 32.78142857142857 +luke johnson 21.58666666666667 +luke johnson 23.03 +luke johnson 23.054 +luke laertes 20.264 +luke laertes 33.72 +luke laertes 39.8 +luke laertes 41.36 +luke laertes 42.254999999999995 +luke miller 20.054444444444446 +luke ovid 19.819999999999997 +luke ovid 30.832857142857147 +luke polk 24.348750000000003 +luke polk 26.57625 +luke quirinius 38.07 +luke robinson 30.119999999999994 +luke robinson 30.31375 +luke thompson 29.026874999999997 +luke underhill 21.735714285714288 +luke underhill 22.175 +luke underhill 26.785714285714285 +luke van buren 17.072222222222223 +luke white 29.063333333333333 +luke xylophone 28.994 +luke zipper 33.995 +mike allen 32.78 +mike brown 27.592222222222222 +mike carson 28.8675 +mike carson 29.88 +mike carson 32.07142857142857 +mike davidson 21.240000000000002 +mike davidson 46.31 +mike ellison 20.5275 +mike ellison 21.99 +mike ellison 24.36 +mike ellison 24.511111111111113 +mike ellison 27.703333333333337 +mike falkner 40.335 +mike garcia 24.3525 +mike garcia 24.582 +mike garcia 35.12 +mike hernandez 8.783333333333333 +mike hernandez 19.40666666666667 +mike ichabod 29.120000000000005 +mike king 14.256666666666668 +mike king 17.889999999999997 +mike king 20.493333333333336 +mike king 23.86 +mike king 26.081 +mike king 30.974 +mike miller 29.275 +mike nixon 17.306 +mike nixon 25.572 +mike polk 18.96 +mike polk 23.75142857142857 +mike polk 33.42 +mike quirinius 19.37375 +mike steinbeck 14.155 +mike steinbeck 19.305833333333332 +mike steinbeck 20.721249999999998 +mike steinbeck 31.75 +mike van buren 15.520000000000001 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 22.4025 +mike white 24.7725 +mike white 35.235 +mike young 1.5 +mike young 24.679 +mike young 34.02833333333333 +mike zipper 17.97 +mike zipper 26.247333333333337 +mike zipper 44.169999999999995 +nick allen 23.744999999999997 +nick allen 36.93 +nick brown 27.669999999999998 +nick davidson 31.97285714285714 +nick ellison 23.061666666666667 +nick ellison 27.676666666666666 +nick falkner 22.555714285714284 +nick falkner 27.46 +nick garcia 17.465 +nick garcia 18.854 +nick garcia 33.60333333333333 +nick ichabod 19.231428571428573 +nick ichabod 27.645000000000003 +nick ichabod 35.836666666666666 +nick johnson 5.58 +nick johnson 25.274 +nick laertes 26.57857142857143 +nick miller 22.208333333333332 +nick nixon 16.107499999999998 +nick ovid 31.350000000000005 +nick polk 35.70333333333334 +nick quirinius 20.753333333333334 +nick quirinius 30.573333333333334 +nick robinson 21.48 +nick robinson 23.185 +nick steinbeck 19.56555555555556 +nick thompson 31.474999999999998 +nick underhill 38.24 +nick van buren 20.77375 +nick xylophone 30.909999999999997 +nick young 10.725000000000001 +nick young 24.95 +nick zipper 16.185000000000002 +nick zipper 34.72 +oscar allen 24.645 +oscar brown 39.55 +oscar carson 21.893333333333334 +oscar carson 22.868 +oscar carson 27.4875 +oscar carson 28.09428571428571 +oscar carson 30.373333333333335 +oscar davidson 9.046666666666667 +oscar ellison 24.185000000000002 +oscar ellison 30.1675 +oscar falkner 19.295 +oscar garcia 22.495833333333334 +oscar hernandez 16.6825 +oscar hernandez 25.736 +oscar ichabod 17.64 +oscar ichabod 21.11 +oscar ichabod 23.508000000000003 +oscar ichabod 30.392222222222227 +oscar johnson 19.9375 +oscar johnson 21.114444444444445 +oscar king 24.590000000000003 +oscar king 26.675 +oscar king 39.6 +oscar laertes 14.975 +oscar laertes 15.525 +oscar laertes 22.6 +oscar laertes 41.6 +oscar nixon 25.4025 +oscar ovid 24.854285714285712 +oscar ovid 25.309 +oscar ovid 29.63 +oscar polk 21.235999999999997 +oscar polk 21.27 +oscar quirinius 24.200000000000003 +oscar quirinius 24.391428571428573 +oscar quirinius 27.83285714285714 +oscar quirinius 27.853333333333328 +oscar robinson 12.3625 +oscar robinson 12.545 +oscar robinson 20.234 +oscar robinson 28.071666666666673 +oscar steinbeck 31.101111111111113 +oscar thompson 19.4875 +oscar thompson 19.975714285714286 +oscar thompson 21.1425 +oscar thompson 21.166363636363638 +oscar underhill 27.644 +oscar van buren 25.843333333333334 +oscar van buren 29.073333333333334 +oscar van buren 29.682727272727274 +oscar white 19.0775 +oscar white 23.483333333333334 +oscar white 24.705000000000002 +oscar white 28.0075 +oscar xylophone 30.020000000000003 +oscar xylophone 30.46833333333333 +oscar xylophone 33.64 +oscar zipper 21.69 +oscar zipper 23.478 +oscar zipper 31.36 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 31.14769230769231 +priscilla carson 14.33 +priscilla carson 18.951428571428572 +priscilla carson 27.084999999999997 +priscilla ichabod 28.160999999999994 +priscilla ichabod 49.46 +priscilla johnson 8.365 +priscilla johnson 18.176666666666666 +priscilla johnson 25.02666666666667 +priscilla johnson 26.918333333333337 +priscilla johnson 30.695999999999998 +priscilla king 19.747142857142855 +priscilla nixon 29.035555555555554 +priscilla nixon 30.27333333333333 +priscilla ovid 13.591999999999999 +priscilla ovid 35.879999999999995 +priscilla polk 23.12 +priscilla quirinius 21.826666666666668 +priscilla thompson 20.44 +priscilla underhill 28.23 +priscilla underhill 34.33200000000001 +priscilla van buren 18.122857142857143 +priscilla van buren 20.16 +priscilla van buren 26.447999999999997 +priscilla white 26.37769230769231 +priscilla xylophone 13.95 +priscilla xylophone 20.596666666666668 +priscilla xylophone 27.22 +priscilla young 29.19 +priscilla young 46.28 +priscilla zipper 11.64 +priscilla zipper 31.159999999999997 +quinn allen 26.347272727272728 +quinn allen 26.85833333333333 +quinn brown 26.822857142857146 +quinn brown 30.406000000000006 +quinn brown 41.53 +quinn davidson 17.375714285714288 +quinn davidson 20.22666666666667 +quinn davidson 25.6375 +quinn davidson 30.173333333333332 +quinn ellison 23.052 +quinn ellison 40.565 +quinn garcia 20.544 +quinn garcia 24.104999999999997 +quinn garcia 25.174 +quinn garcia 28.446000000000005 +quinn ichabod 15.12 +quinn king 12.73 +quinn king 15.12125 +quinn laertes 17.29 +quinn laertes 28.221666666666668 +quinn laertes 32.96 +quinn nixon 26.034000000000002 +quinn ovid 28.71 +quinn quirinius 8.61 +quinn robinson 16.852 +quinn steinbeck 30.093333333333334 +quinn steinbeck 49.21 +quinn thompson 7.365 +quinn thompson 33.43125 +quinn underhill 24.045 +quinn underhill 27.905454545454543 +quinn underhill 31.21 +quinn van buren 27.807692307692314 +quinn young 30.56 +quinn zipper 18.31 +quinn zipper 21.380000000000003 +rachel allen 32.501666666666665 +rachel allen 46.57 +rachel brown 23.08 +rachel brown 23.880000000000003 +rachel brown 24.43 +rachel brown 34.11 +rachel brown 35.345 +rachel carson 27.468125 +rachel carson 37.446666666666665 +rachel davidson 22.75 +rachel ellison 22.848333333333333 +rachel falkner 18.78125 +rachel falkner 28.876250000000002 +rachel falkner 29.577777777777776 +rachel falkner 31.831249999999997 +rachel johnson 31.108000000000004 +rachel king 17.4175 +rachel king 30.873749999999998 +rachel laertes 17.470000000000002 +rachel laertes 33.51 +rachel ovid 3.03 +rachel ovid 15.38 +rachel polk 18.564285714285713 +rachel quirinius 31.692500000000003 +rachel robinson 0.6 +rachel robinson 23.953333333333333 +rachel robinson 37.645 +rachel thompson 11.96 +rachel thompson 29.484 +rachel thompson 38.43 +rachel underhill 27.55333333333333 +rachel white 23.511428571428574 +rachel white 33.7 +rachel young 24.85166666666667 +rachel zipper 22.85 +rachel zipper 37.382 +sarah carson 10.38 +sarah carson 22.639 +sarah carson 44.92 +sarah ellison 16.36 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 11.296666666666667 +sarah garcia 20.723333333333333 +sarah garcia 24.115 +sarah ichabod 26.948333333333327 +sarah ichabod 33.80428571428571 +sarah johnson 18.3925 +sarah johnson 23.087500000000002 +sarah johnson 26.57857142857143 +sarah johnson 37.01 +sarah king 9.556666666666667 +sarah king 25.6125 +sarah miller 19.14875 +sarah ovid 29.205 +sarah robinson 11.326666666666668 +sarah robinson 35.809999999999995 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 26.850000000000005 +sarah xylophone 33.40571428571429 +sarah young 30.66 +sarah zipper 29.521666666666672 +tom brown 16.38 +tom brown 23.645 +tom carson 23.630000000000003 +tom carson 31.935 +tom carson 41.83 +tom davidson 30.404285714285717 +tom ellison 27.056 +tom ellison 27.401999999999997 +tom ellison 29.812 +tom falkner 15.901999999999997 +tom falkner 25.49857142857143 +tom hernandez 11.418000000000001 +tom hernandez 30.705000000000002 +tom ichabod 14.83 +tom johnson 30.748571428571434 +tom johnson 37.086666666666666 +tom king 17.923333333333332 +tom laertes 19.201666666666668 +tom laertes 22.276666666666667 +tom miller 17.9925 +tom miller 19.791666666666668 +tom miller 19.9225 +tom nixon 25.70625 +tom ovid 29.66 +tom polk 27.0975 +tom polk 28.646666666666672 +tom quirinius 37.68333333333333 +tom quirinius 38.28 +tom robinson 18.07 +tom robinson 19.094 +tom robinson 27.34125 +tom robinson 31.135714285714283 +tom steinbeck 32.70333333333333 +tom van buren 20.723333333333333 +tom van buren 24.8525 +tom van buren 31.631666666666664 +tom white 25.646000000000004 +tom young 3.12 +tom young 19.588333333333335 +tom zipper 23.317272727272726 +ulysses brown 16.196666666666665 +ulysses carson 16.3475 +ulysses carson 22.448181818181823 +ulysses carson 28.258 +ulysses carson 32.10833333333333 +ulysses davidson 37.775 +ulysses ellison 30.517000000000003 +ulysses garcia 32.92 +ulysses hernandez 13.877500000000001 +ulysses hernandez 20.856666666666666 +ulysses hernandez 21.32625 +ulysses ichabod 3.29 +ulysses ichabod 24.629999999999995 +ulysses johnson 32.208333333333336 +ulysses king 25.29111111111111 +ulysses laertes 14.936666666666667 +ulysses laertes 25.89 +ulysses laertes 26.63 +ulysses miller 2.36 +ulysses miller 26.403333333333336 +ulysses nixon 34.4575 +ulysses ovid 23.810000000000002 +ulysses polk 22.4075 +ulysses polk 26.778000000000002 +ulysses polk 38.73166666666667 +ulysses polk 47.68 +ulysses quirinius 33.07833333333333 +ulysses robinson 17.386666666666667 +ulysses steinbeck 22.2675 +ulysses steinbeck 24.904000000000003 +ulysses thompson 22.687142857142856 +ulysses underhill 6.66 +ulysses underhill 22.539 +ulysses underhill 24.853333333333335 +ulysses underhill 27.314 +ulysses underhill 29.424999999999997 +ulysses underhill 32.905 +ulysses underhill 41.653333333333336 +ulysses van buren 21.868181818181817 +ulysses white 15.296666666666667 +ulysses white 28.343333333333334 +ulysses xylophone 24.718 +ulysses xylophone 30.205 +ulysses xylophone 35.61 +ulysses young 21.56 +ulysses young 32.28125 +ulysses young 37.275 +victor allen 23.548000000000002 +victor allen 24.759999999999998 +victor brown 22.10181818181818 +victor brown 23.73 +victor brown 25.427272727272726 +victor brown 26.218571428571433 +victor davidson 20.55 +victor davidson 22.21666666666667 +victor davidson 29.778 +victor ellison 13.0775 +victor ellison 33.666 +victor hernandez 10.896 +victor hernandez 18.922 +victor hernandez 24.908888888888892 +victor hernandez 27.426666666666666 +victor hernandez 35.6675 +victor johnson 20.02 +victor johnson 27.070000000000004 +victor johnson 29.0775 +victor king 18.066666666666666 +victor king 21.488 +victor laertes 26.77777777777778 +victor laertes 28.095000000000002 +victor miller 5.3100000000000005 +victor nixon 21.395714285714288 +victor nixon 28.33 +victor ovid 35.225 +victor polk 21.990000000000002 +victor quirinius 24.62833333333333 +victor quirinius 29.742500000000003 +victor robinson 14.575 +victor robinson 25.92 +victor steinbeck 26.136666666666667 +victor steinbeck 26.485 +victor steinbeck 34.745999999999995 +victor thompson 18.735 +victor van buren 27.758333333333336 +victor van buren 37.38333333333333 +victor white 24.607999999999997 +victor white 30.66 +victor xylophone 2.775 +victor xylophone 8.356666666666667 +victor xylophone 24.259999999999998 +victor xylophone 25.636666666666667 +victor xylophone 31.610000000000003 +victor young 22.264444444444443 +victor zipper 39.84 +wendy allen 3.4 +wendy allen 24.695000000000004 +wendy allen 29.912 +wendy brown 28.22 +wendy brown 36.74 +wendy ellison 17.549999999999997 +wendy ellison 22.720000000000002 +wendy falkner 13.765 +wendy falkner 24.424444444444443 +wendy falkner 27.86733333333333 +wendy garcia 12.3 +wendy garcia 22.396666666666665 +wendy garcia 26.8325 +wendy garcia 28.596666666666664 +wendy hernandez 21.111428571428572 +wendy ichabod 4.44 +wendy king 23.654285714285713 +wendy king 29.325714285714287 +wendy king 34.21666666666667 +wendy laertes 31.160714285714285 +wendy laertes 31.46666666666667 +wendy laertes 39.22 +wendy miller 12.73 +wendy miller 30.343333333333334 +wendy nixon 19.92714285714286 +wendy nixon 29.675714285714285 +wendy ovid 21.193749999999998 +wendy ovid 28.49846153846154 +wendy polk 20.94 +wendy polk 22.999999999999996 +wendy quirinius 21.05 +wendy quirinius 26.8425 +wendy robinson 8.39 +wendy robinson 24.05 +wendy robinson 26.974285714285713 +wendy steinbeck 26.765 +wendy thompson 24.14 +wendy thompson 28.995384615384616 +wendy underhill 23.118333333333336 +wendy underhill 25.581666666666667 +wendy underhill 32.985 +wendy van buren 25.151666666666667 +wendy van buren 27.077142857142857 +wendy white 24.4025 +wendy xylophone 22.85181818181818 +wendy xylophone 26.96 +wendy young 4.83 +wendy young 21.325 +xavier allen 19.133333333333333 +xavier allen 26.11466666666667 +xavier allen 34.58 +xavier brown 2.63 +xavier brown 24.764285714285712 +xavier brown 30.166666666666668 +xavier carson 29.006666666666664 +xavier carson 32.106 +xavier davidson 14.094999999999999 +xavier davidson 15.906666666666666 +xavier davidson 27.353333333333335 +xavier ellison 22.174166666666668 +xavier ellison 35.01 +xavier garcia 30.357500000000005 +xavier hernandez 19.87 +xavier hernandez 20.805 +xavier hernandez 33.497499999999995 +xavier ichabod 12.34 +xavier ichabod 26.166249999999998 +xavier johnson 20.33222222222222 +xavier johnson 22.503333333333334 +xavier king 1.3 +xavier king 31.348571428571425 +xavier laertes 7.420000000000001 +xavier ovid 25.576 +xavier polk 11.094285714285714 +xavier polk 19.93 +xavier polk 23.63125 +xavier polk 30.194 +xavier quirinius 13.776666666666666 +xavier quirinius 22.27 +xavier quirinius 24.977692307692312 +xavier quirinius 34.95 +xavier thompson 16.47 +xavier underhill 1.31 +xavier white 19.331666666666667 +xavier white 34.68 +xavier xylophone 21.09625 +xavier zipper 14.89 +yuri allen 18.490000000000002 +yuri allen 22.689999999999998 +yuri brown 15.502857142857142 +yuri brown 22.934285714285714 +yuri carson 27.139999999999997 +yuri carson 35.27 +yuri ellison 10.52 +yuri ellison 25.2025 +yuri falkner 24.633076923076924 +yuri falkner 28.52 +yuri garcia 25.545 +yuri hernandez 16.35 +yuri johnson 19.9525 +yuri johnson 27.636000000000003 +yuri johnson 39.92 +yuri king 15.450000000000001 +yuri laertes 0.41000000000000003 +yuri laertes 33.15 +yuri nixon 27.795 +yuri nixon 39.145 +yuri polk 0.8 +yuri polk 9.705 +yuri polk 25.513333333333332 +yuri quirinius 16.29 +yuri quirinius 19.254999999999995 +yuri quirinius 37.878 +yuri steinbeck 27.6275 +yuri steinbeck 48.89 +yuri thompson 23.330000000000002 +yuri underhill 20.504444444444445 +yuri underhill 21.66 +yuri white 31.205 +yuri xylophone 18.790000000000003 +zach allen 13.06 +zach brown 19.985 +zach brown 26.52333333333333 +zach brown 34.66 +zach brown 34.972857142857144 +zach brown 37.45399999999999 +zach carson 26.195999999999998 +zach ellison 17.55 +zach falkner 3.42 +zach falkner 16.18 +zach garcia 20.062 +zach garcia 25.935 +zach garcia 28.974285714285717 +zach garcia 35.449999999999996 +zach ichabod 10.59 +zach ichabod 31.691999999999997 +zach king 6.81 +zach king 20.817 +zach king 32.542500000000004 +zach miller 13.23 +zach miller 26.30666666666667 +zach miller 26.73 +zach ovid 21.122500000000002 +zach ovid 26.983999999999998 +zach ovid 33.15 +zach ovid 40.59 +zach quirinius 13.38 +zach robinson 20.451999999999998 +zach steinbeck 20.358333333333334 +zach steinbeck 29.65 +zach thompson 16.45 +zach thompson 21.430000000000003 +zach underhill 31.438333333333333 +zach white 23.111428571428572 +zach xylophone 21.221428571428568 +zach xylophone 23.156666666666666 +zach young 24.72666666666667 +zach zipper 19.878888888888884 +zach zipper 34.84571428571429 +zach zipper 35.36 diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 b/sql/hive/src/test/resources/golden/windowing_udaf2-0-96659fde37d7a38ea15b367b47f59ce2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 new file mode 100644 index 0000000000000..17c31c0f04592 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_udaf2-1-b4bdee4908b1cb8e240c549ae5cfe4c0 @@ -0,0 +1 @@ +130091 130091 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e new file mode 100644 index 0000000000000..31b1f85a5eb5a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-0-f498cccf82480be03022d2a36f87651e @@ -0,0 +1,1049 @@ + 4294967354 + 4294967416 + 4294967457 +alice allen 4294967487 +alice allen 4294967525 +alice allen 4294967531 +alice brown 4294967355 +alice carson 4294967370 +alice davidson 4294967517 +alice falkner 4294967316 +alice garcia 4294967369 +alice hernandez 4294967299 +alice hernandez 4294967314 +alice johnson 4294967424 +alice king 4294967387 +alice king 4294967516 +alice king 4294967546 +alice laertes 4294967519 +alice laertes 8589934835 +alice miller 4294967324 +alice nixon 4294967410 +alice nixon 4294967413 +alice nixon 4294967484 +alice ovid 8589934726 +alice polk 4294967366 +alice quirinius 4294967505 +alice quirinius 4294967549 +alice robinson 4294967445 +alice robinson 4294967502 +alice steinbeck 4294967364 +alice steinbeck 4294967474 +alice steinbeck 4294967549 +alice underhill 4294967441 +alice van buren 4294967428 +alice xylophone 4294967363 +alice xylophone 4294967519 +alice xylophone 8589934832 +alice zipper 4294967380 +alice zipper 4294967520 +alice zipper 8589935026 +bob brown 4294967422 +bob brown 4294967427 +bob brown 4294967431 +bob carson 4294967408 +bob davidson 4294967354 +bob davidson 4294967435 +bob davidson 4294967504 +bob ellison 4294967344 +bob ellison 4294967362 +bob ellison 4294967436 +bob ellison 4294967530 +bob falkner 8589934966 +bob garcia 4294967369 +bob garcia 4294967435 +bob garcia 4294967439 +bob garcia 8589934707 +bob garcia 8589934867 +bob hernandez 4294967500 +bob ichabod 4294967424 +bob king 4294967297 +bob king 4294967539 +bob king 8589934870 +bob laertes 4294967380 +bob laertes 4294967472 +bob miller 4294967349 +bob ovid 4294967395 +bob ovid 4294967400 +bob ovid 4294967401 +bob ovid 4294967512 +bob polk 4294967337 +bob quirinius 4294967346 +bob steinbeck 4294967342 +bob van buren 4294967422 +bob white 4294967362 +bob white 4294967493 +bob xylophone 4294967407 +bob xylophone 4294967465 +bob young 4294967413 +bob zipper 4294967299 +bob zipper 8589934723 +bob zipper 8589934840 +calvin allen 12884902208 +calvin brown 4294967411 +calvin brown 4294967437 +calvin brown 4294967530 +calvin carson 8589934876 +calvin davidson 4294967468 +calvin davidson 8589934837 +calvin ellison 4294967480 +calvin falkner 4294967300 +calvin falkner 4294967305 +calvin falkner 4294967345 +calvin falkner 8589934749 +calvin falkner 8589934840 +calvin falkner 8589934978 +calvin garcia 8589934927 +calvin hernandez 12884902173 +calvin johnson 4294967546 +calvin laertes 4294967431 +calvin laertes 4294967499 +calvin nixon 4294967300 +calvin nixon 4294967412 +calvin nixon 4294967488 +calvin ovid 4294967329 +calvin ovid 4294967349 +calvin ovid 8589934723 +calvin ovid 8589934835 +calvin polk 8589934962 +calvin quirinius 4294967521 +calvin quirinius 4294967532 +calvin robinson 4294967326 +calvin steinbeck 4294967474 +calvin steinbeck 4294967505 +calvin steinbeck 8589934722 +calvin thompson 4294967513 +calvin thompson 8589934700 +calvin underhill 4294967478 +calvin van buren 4294967300 +calvin van buren 4294967508 +calvin white 4294967304 +calvin white 8589934924 +calvin xylophone 4294967376 +calvin xylophone 8589934599 +calvin xylophone 8589934866 +calvin young 4294967342 +calvin young 8589934817 +calvin zipper 12884902359 +calvin zipper 17179869649 +david allen 4294967371 +david allen 4294967381 +david brown 8589934762 +david brown 12884902420 +david davidson 4294967522 +david davidson 8589934819 +david davidson 12884902188 +david davidson 12884902327 +david ellison 4294967463 +david ellison 8589934777 +david ellison 12884902263 +david hernandez 4294967324 +david ichabod 4294967487 +david ichabod 12884902220 +david laertes 12884902107 +david nixon 4294967381 +david ovid 4294967396 +david ovid 4294967443 +david quirinius 4294967457 +david quirinius 4294967530 +david quirinius 12884902194 +david robinson 4294967465 +david robinson 17179869575 +david thompson 4294967361 +david underhill 4294967384 +david underhill 8589934942 +david underhill 12884902357 +david van buren 4294967309 +david van buren 8589934901 +david white 4294967428 +david xylophone 4294967479 +david xylophone 4294967480 +david xylophone 8589934856 +david young 4294967296 +david young 4294967305 +ethan allen 4294967351 +ethan brown 4294967320 +ethan brown 4294967331 +ethan brown 4294967403 +ethan brown 4294967420 +ethan brown 8589934797 +ethan brown 8589934805 +ethan carson 4294967352 +ethan ellison 4294967514 +ethan ellison 8589934887 +ethan falkner 4294967318 +ethan falkner 4294967461 +ethan garcia 4294967310 +ethan hernandez 4294967349 +ethan johnson 8589934738 +ethan king 8589934731 +ethan laertes 4294967422 +ethan laertes 4294967531 +ethan laertes 8589934767 +ethan laertes 8589934806 +ethan laertes 8589934830 +ethan laertes 8589934995 +ethan laertes 12884902063 +ethan miller 4294967352 +ethan nixon 8589935019 +ethan ovid 8589934909 +ethan polk 4294967329 +ethan polk 4294967382 +ethan polk 4294967479 +ethan polk 8589935021 +ethan quirinius 4294967348 +ethan quirinius 4294967501 +ethan quirinius 8589934695 +ethan robinson 4294967353 +ethan robinson 8589935019 +ethan underhill 8589934897 +ethan van buren 4294967511 +ethan white 4294967427 +ethan white 8589934975 +ethan xylophone 8589934956 +ethan zipper 4294967462 +ethan zipper 12884902348 +fred davidson 8589934724 +fred davidson 8589934850 +fred davidson 12884902468 +fred ellison 4294967393 +fred ellison 8589934797 +fred ellison 8589934978 +fred falkner 4294967547 +fred falkner 12884902170 +fred falkner 17179869760 +fred hernandez 8589934833 +fred ichabod 8589934853 +fred ichabod 12884902455 +fred johnson 8589934904 +fred king 8589934651 +fred king 8589934951 +fred laertes 8589934883 +fred miller 12884902228 +fred nixon 4294967297 +fred nixon 4294967375 +fred nixon 4294967514 +fred nixon 12884902182 +fred polk 4294967332 +fred polk 4294967458 +fred polk 4294967507 +fred polk 8589934944 +fred quirinius 8589934894 +fred quirinius 12884902335 +fred robinson 8589934904 +fred steinbeck 4294967329 +fred steinbeck 4294967411 +fred steinbeck 4294967472 +fred underhill 4294967387 +fred van buren 8589934830 +fred van buren 12884902319 +fred van buren 12884902382 +fred van buren 17179869836 +fred white 8589934763 +fred young 4294967485 +fred young 8589934832 +fred zipper 12884902371 +gabriella allen 4294967405 +gabriella allen 12884902509 +gabriella brown 4294967403 +gabriella brown 4294967543 +gabriella carson 8589934950 +gabriella davidson 4294967507 +gabriella ellison 4294967393 +gabriella ellison 12884902284 +gabriella falkner 4294967378 +gabriella falkner 4294967523 +gabriella falkner 12884902338 +gabriella garcia 4294967419 +gabriella hernandez 4294967462 +gabriella hernandez 4294967481 +gabriella ichabod 4294967337 +gabriella ichabod 8589934740 +gabriella ichabod 8589934797 +gabriella ichabod 8589934818 +gabriella ichabod 17179869508 +gabriella king 4294967393 +gabriella king 8589934906 +gabriella laertes 4294967410 +gabriella miller 8589934768 +gabriella ovid 4294967522 +gabriella ovid 8589934895 +gabriella polk 4294967302 +gabriella polk 8589934868 +gabriella steinbeck 4294967435 +gabriella steinbeck 4294967500 +gabriella thompson 4294967412 +gabriella thompson 8589934814 +gabriella thompson 12884902318 +gabriella van buren 4294967470 +gabriella van buren 8589934783 +gabriella white 4294967335 +gabriella young 4294967431 +gabriella young 8589934980 +gabriella zipper 4294967510 +gabriella zipper 8589934792 +holly allen 12884901926 +holly brown 8589934722 +holly brown 8589934857 +holly falkner 8589934849 +holly hernandez 8589934749 +holly hernandez 8589934805 +holly hernandez 8589935056 +holly hernandez 12884902485 +holly ichabod 4294967329 +holly ichabod 8589934754 +holly ichabod 8589934981 +holly johnson 4294967535 +holly johnson 12884902194 +holly johnson 17179869874 +holly king 8589934785 +holly king 8589934939 +holly laertes 12884902333 +holly miller 8589934823 +holly nixon 4294967383 +holly nixon 8589934744 +holly polk 4294967434 +holly polk 8589934782 +holly robinson 12884902369 +holly thompson 4294967339 +holly thompson 12884902395 +holly thompson 17179869547 +holly underhill 8589934913 +holly underhill 8589934924 +holly underhill 12884902376 +holly underhill 12884902412 +holly van buren 4294967539 +holly white 17179869548 +holly white 17179869900 +holly xylophone 8589934846 +holly young 4294967500 +holly young 8589934932 +holly zipper 4294967509 +holly zipper 17179869531 +irene allen 12884902413 +irene brown 4294967428 +irene brown 8589934934 +irene brown 12884902207 +irene carson 8589934797 +irene ellison 8589934732 +irene ellison 8589934773 +irene falkner 4294967404 +irene falkner 4294967548 +irene garcia 4294967323 +irene garcia 8589934887 +irene garcia 12884902479 +irene ichabod 4294967509 +irene ichabod 8589934860 +irene johnson 8589934990 +irene laertes 4294967481 +irene laertes 12884902196 +irene laertes 17179869632 +irene miller 4294967387 +irene nixon 4294967538 +irene nixon 12884902129 +irene nixon 12884902324 +irene ovid 8589934764 +irene ovid 8589934886 +irene ovid 8589934903 +irene polk 4294967465 +irene polk 4294967521 +irene polk 8589934672 +irene polk 8589934842 +irene polk 17179869877 +irene quirinius 8589934875 +irene quirinius 12884902269 +irene quirinius 17179869628 +irene robinson 8589934676 +irene steinbeck 4294967549 +irene thompson 4294967479 +irene underhill 8589934694 +irene underhill 12884902077 +irene van buren 8589934932 +irene van buren 12884902202 +irene xylophone 8589934901 +jessica brown 8589934867 +jessica carson 4294967508 +jessica carson 8589934740 +jessica carson 17179869819 +jessica davidson 4294967384 +jessica davidson 8589934864 +jessica davidson 12884902256 +jessica davidson 12884902321 +jessica ellison 4294967316 +jessica ellison 12884902128 +jessica falkner 8589934980 +jessica garcia 4294967540 +jessica garcia 21474837337 +jessica ichabod 8589934816 +jessica johnson 8589935006 +jessica johnson 12884902222 +jessica miller 8589934898 +jessica nixon 8589934742 +jessica nixon 12884902240 +jessica ovid 8589934830 +jessica ovid 12884902307 +jessica polk 21474837163 +jessica quirinius 8589934701 +jessica quirinius 8589934872 +jessica quirinius 12884902159 +jessica quirinius 12884902276 +jessica robinson 4294967542 +jessica thompson 8589934698 +jessica thompson 12884902232 +jessica underhill 8589934810 +jessica underhill 8589934878 +jessica underhill 17179869479 +jessica van buren 8589934726 +jessica white 12884902155 +jessica white 12884902281 +jessica white 12884902296 +jessica white 12884902314 +jessica white 17179869676 +jessica xylophone 17179869697 +jessica young 17179869859 +jessica young 17179869861 +jessica zipper 4294967372 +jessica zipper 8589934727 +jessica zipper 17179869778 +katie allen 8589934791 +katie brown 17179869660 +katie davidson 12884902181 +katie ellison 12884902184 +katie ellison 12884902355 +katie falkner 8589934911 +katie garcia 8589934683 +katie garcia 12884902046 +katie hernandez 8589934812 +katie ichabod 8589934795 +katie ichabod 8589934862 +katie ichabod 8589934869 +katie king 4294967339 +katie king 4294967421 +katie king 8589934826 +katie miller 8589934829 +katie miller 12884902267 +katie nixon 21474837149 +katie ovid 4294967519 +katie polk 8589934726 +katie polk 12884902291 +katie robinson 17179869645 +katie van buren 8589934722 +katie van buren 17179869441 +katie white 4294967306 +katie white 8589934885 +katie xylophone 12884902193 +katie young 8589934819 +katie young 8589935024 +katie young 12884902058 +katie zipper 4294967354 +katie zipper 12884902310 +luke allen 8589934864 +luke allen 8589934931 +luke allen 8589935059 +luke allen 12884902257 +luke allen 12884902322 +luke brown 8589934779 +luke davidson 4294967354 +luke davidson 12884902360 +luke ellison 12884902183 +luke ellison 21474836998 +luke ellison 21474837060 +luke falkner 8589934772 +luke falkner 17179869561 +luke garcia 4294967304 +luke garcia 21474837157 +luke ichabod 12884902150 +luke ichabod 12884902366 +luke johnson 4294967527 +luke johnson 8589934812 +luke johnson 12884902161 +luke laertes 8589935027 +luke laertes 12884902031 +luke laertes 12884902184 +luke laertes 12884902213 +luke laertes 12884902378 +luke miller 8589934826 +luke ovid 4294967492 +luke ovid 8589934913 +luke polk 8589934837 +luke polk 12884902340 +luke quirinius 8589934855 +luke robinson 4294967307 +luke robinson 17179869711 +luke thompson 4294967521 +luke underhill 8589934829 +luke underhill 12884902299 +luke underhill 21474837138 +luke van buren 8589934852 +luke white 12884902418 +luke xylophone 8589934804 +luke zipper 4294967353 +mike allen 17179869750 +mike brown 17179869735 +mike carson 4294967477 +mike carson 8589934803 +mike carson 17179869855 +mike davidson 12884902377 +mike davidson 17179869841 +mike ellison 8589934833 +mike ellison 12884902165 +mike ellison 12884902513 +mike ellison 17179869587 +mike ellison 17179869824 +mike falkner 4294967301 +mike garcia 4294967398 +mike garcia 8589934800 +mike garcia 12884902292 +mike hernandez 8589934824 +mike hernandez 12884902281 +mike ichabod 4294967494 +mike king 4294967347 +mike king 4294967400 +mike king 12884902363 +mike king 12884902475 +mike king 17179869528 +mike king 17179869592 +mike miller 17179869705 +mike nixon 12884902293 +mike nixon 17179869708 +mike polk 17179869752 +mike polk 21474837097 +mike polk 21474837344 +mike quirinius 12884902240 +mike steinbeck 8589934653 +mike steinbeck 12884902273 +mike steinbeck 12884902301 +mike steinbeck 17179869903 +mike van buren 8589934942 +mike van buren 12884902402 +mike white 12884902485 +mike white 17179869676 +mike white 21474836928 +mike white 25769804626 +mike young 8589934704 +mike young 8589934878 +mike young 17179869685 +mike zipper 4294967501 +mike zipper 17179869582 +mike zipper 25769804400 +nick allen 8589934664 +nick allen 8589934860 +nick brown 21474836962 +nick davidson 4294967357 +nick ellison 12884902066 +nick ellison 17179869779 +nick falkner 8589935020 +nick falkner 12884902433 +nick garcia 8589934885 +nick garcia 17179869635 +nick garcia 17179869681 +nick ichabod 12884902193 +nick ichabod 12884902223 +nick ichabod 12884902252 +nick johnson 17179869591 +nick johnson 17179869702 +nick laertes 8589934919 +nick miller 12884902419 +nick nixon 8589934910 +nick ovid 12884902267 +nick polk 17179869712 +nick quirinius 4294967296 +nick quirinius 12884902183 +nick robinson 17179869506 +nick robinson 17179869731 +nick steinbeck 4294967355 +nick thompson 8589934922 +nick underhill 25769804624 +nick van buren 8589934635 +nick xylophone 12884902279 +nick young 12884902399 +nick young 21474837140 +nick zipper 12884902300 +nick zipper 17179869849 +oscar allen 17179869779 +oscar brown 12884902062 +oscar carson 12884902232 +oscar carson 17179869663 +oscar carson 17179869779 +oscar carson 21474837066 +oscar carson 21474837089 +oscar davidson 17179869895 +oscar ellison 4294967304 +oscar ellison 8589934740 +oscar falkner 4294967526 +oscar garcia 21474837156 +oscar hernandez 4294967343 +oscar hernandez 8589935049 +oscar ichabod 8589934837 +oscar ichabod 21474836952 +oscar ichabod 21474837021 +oscar ichabod 25769804491 +oscar johnson 12884902182 +oscar johnson 30064772044 +oscar king 12884902159 +oscar king 17179869738 +oscar king 17179869834 +oscar laertes 4294967550 +oscar laertes 8589934727 +oscar laertes 12884902043 +oscar laertes 12884902478 +oscar nixon 17179869458 +oscar ovid 12884902128 +oscar ovid 12884902240 +oscar ovid 25769804460 +oscar polk 21474836829 +oscar polk 21474837063 +oscar quirinius 8589934728 +oscar quirinius 17179869698 +oscar quirinius 21474837051 +oscar quirinius 25769804521 +oscar robinson 8589934656 +oscar robinson 12884902249 +oscar robinson 21474837105 +oscar robinson 25769804694 +oscar steinbeck 4294967548 +oscar thompson 8589934776 +oscar thompson 12884902164 +oscar thompson 12884902317 +oscar thompson 17179869884 +oscar underhill 8589934895 +oscar van buren 4294967500 +oscar van buren 8589934984 +oscar van buren 21474837205 +oscar white 4294967454 +oscar white 8589934826 +oscar white 21474836931 +oscar white 21474837305 +oscar xylophone 12884902193 +oscar xylophone 12884902307 +oscar xylophone 17179869593 +oscar zipper 8589934865 +oscar zipper 8589934874 +oscar zipper 8589934911 +priscilla brown 8589934848 +priscilla brown 8589935013 +priscilla brown 17179869801 +priscilla carson 12884902145 +priscilla carson 21474836880 +priscilla carson 30064772126 +priscilla ichabod 4294967547 +priscilla ichabod 17179869756 +priscilla johnson 4294967468 +priscilla johnson 8589934667 +priscilla johnson 17179869667 +priscilla johnson 17179869787 +priscilla johnson 25769804279 +priscilla king 12884902153 +priscilla nixon 12884902188 +priscilla nixon 25769804766 +priscilla ovid 12884902234 +priscilla ovid 30064772049 +priscilla polk 17179869480 +priscilla quirinius 12884902171 +priscilla thompson 25769804637 +priscilla underhill 4294967333 +priscilla underhill 17179869740 +priscilla van buren 12884902324 +priscilla van buren 21474837167 +priscilla van buren 21474837343 +priscilla white 4294967419 +priscilla xylophone 8589934792 +priscilla xylophone 12884902245 +priscilla xylophone 12884902287 +priscilla young 21474836992 +priscilla young 34359739656 +priscilla zipper 12884902296 +priscilla zipper 12884902537 +quinn allen 4294967542 +quinn allen 17179869552 +quinn brown 12884902251 +quinn brown 17179869401 +quinn brown 17179869626 +quinn davidson 8589934992 +quinn davidson 17179869690 +quinn davidson 25769804455 +quinn davidson 30064771771 +quinn ellison 12884902376 +quinn ellison 34359739559 +quinn garcia 8589934828 +quinn garcia 12884902387 +quinn garcia 12884902460 +quinn garcia 21474837066 +quinn ichabod 30064772171 +quinn king 4294967458 +quinn king 4294967538 +quinn laertes 8589935080 +quinn laertes 17179869711 +quinn laertes 21474837142 +quinn nixon 17179869672 +quinn ovid 17179869695 +quinn quirinius 21474836827 +quinn robinson 12884902445 +quinn steinbeck 17179869739 +quinn steinbeck 21474836905 +quinn thompson 17179869645 +quinn thompson 25769804317 +quinn underhill 8589934815 +quinn underhill 12884902185 +quinn underhill 30064771762 +quinn van buren 4294967362 +quinn young 8589934731 +quinn zipper 12884902453 +quinn zipper 17179869841 +rachel allen 8589934882 +rachel allen 12884902208 +rachel brown 8589934768 +rachel brown 12884902075 +rachel brown 17179869910 +rachel brown 17179869911 +rachel brown 21474837280 +rachel carson 8589934728 +rachel carson 17179869970 +rachel davidson 30064771666 +rachel ellison 4294967423 +rachel falkner 4294967348 +rachel falkner 12884902482 +rachel falkner 21474837331 +rachel falkner 25769804739 +rachel johnson 38654707197 +rachel king 12884902157 +rachel king 30064771759 +rachel laertes 17179869678 +rachel laertes 25769804379 +rachel ovid 12884902055 +rachel ovid 17179869857 +rachel polk 12884902391 +rachel quirinius 17179869456 +rachel robinson 17179869499 +rachel robinson 17179869703 +rachel robinson 25769804290 +rachel thompson 17179869910 +rachel thompson 21474836989 +rachel thompson 21474837392 +rachel underhill 8589934862 +rachel white 17179869585 +rachel white 21474837039 +rachel young 17179869708 +rachel zipper 4294967434 +rachel zipper 21474837228 +sarah carson 4294967319 +sarah carson 17179869688 +sarah carson 30064772084 +sarah ellison 4294967542 +sarah falkner 17179869797 +sarah falkner 21474837349 +sarah garcia 8589934733 +sarah garcia 8589934858 +sarah garcia 17179869599 +sarah ichabod 12884902196 +sarah ichabod 12884902401 +sarah johnson 12884902455 +sarah johnson 21474836981 +sarah johnson 21474837145 +sarah johnson 25769804480 +sarah king 12884902453 +sarah king 21474837191 +sarah miller 8589934958 +sarah ovid 21474837184 +sarah robinson 21474837237 +sarah robinson 21474837389 +sarah steinbeck 21474837313 +sarah white 17179869905 +sarah white 25769804341 +sarah xylophone 12884902207 +sarah young 21474837319 +sarah zipper 25769804616 +tom brown 8589934894 +tom brown 21474837024 +tom carson 4294967388 +tom carson 12884902278 +tom carson 21474836983 +tom davidson 8589934895 +tom ellison 12884902192 +tom ellison 17179869965 +tom ellison 25769804262 +tom falkner 12884902272 +tom falkner 17179869815 +tom hernandez 4294967296 +tom hernandez 12884902109 +tom ichabod 17179869628 +tom johnson 25769804829 +tom johnson 30064771891 +tom king 12884902390 +tom laertes 12884902181 +tom laertes 12884902236 +tom miller 12884901992 +tom miller 17179869647 +tom miller 21474837107 +tom nixon 17179869677 +tom ovid 12884902279 +tom polk 8589934748 +tom polk 8589934892 +tom quirinius 12884902174 +tom quirinius 21474836986 +tom robinson 8589934753 +tom robinson 12884902203 +tom robinson 12884902358 +tom robinson 21474836952 +tom steinbeck 8589934912 +tom van buren 8589934823 +tom van buren 12884902122 +tom van buren 25769804641 +tom white 21474837076 +tom young 4294967535 +tom young 21474837038 +tom zipper 30064772355 +ulysses brown 8589934991 +ulysses carson 8589934789 +ulysses carson 21474837258 +ulysses carson 25769804457 +ulysses carson 34359739082 +ulysses davidson 12884902216 +ulysses ellison 17179869551 +ulysses garcia 12884902382 +ulysses hernandez 12884902210 +ulysses hernandez 12884902276 +ulysses hernandez 17179869748 +ulysses ichabod 4294967353 +ulysses ichabod 12884902217 +ulysses johnson 21474837122 +ulysses king 8589934995 +ulysses laertes 8589934801 +ulysses laertes 21474837354 +ulysses laertes 25769804499 +ulysses miller 21474837284 +ulysses miller 30064771926 +ulysses nixon 17179869288 +ulysses ovid 17179869754 +ulysses polk 8589934855 +ulysses polk 8589934862 +ulysses polk 12884902420 +ulysses polk 17179869479 +ulysses quirinius 17179869659 +ulysses robinson 4294967531 +ulysses steinbeck 8589935027 +ulysses steinbeck 21474837100 +ulysses thompson 12884902194 +ulysses underhill 8589934760 +ulysses underhill 8589934799 +ulysses underhill 12884902240 +ulysses underhill 17179869759 +ulysses underhill 17179869760 +ulysses underhill 17179869939 +ulysses underhill 21474837264 +ulysses van buren 8589934938 +ulysses white 25769804453 +ulysses white 30064772086 +ulysses xylophone 8589935029 +ulysses xylophone 12884902249 +ulysses xylophone 25769804765 +ulysses young 4294967427 +ulysses young 17179869391 +ulysses young 30064771844 +victor allen 8589934793 +victor allen 12884902264 +victor brown 4294967455 +victor brown 17179869657 +victor brown 21474837426 +victor brown 30064771922 +victor davidson 17179869715 +victor davidson 17179869872 +victor davidson 25769804287 +victor ellison 17179869611 +victor ellison 17179869709 +victor hernandez 8589934847 +victor hernandez 12884902463 +victor hernandez 17179869647 +victor hernandez 17179869720 +victor hernandez 25769804310 +victor johnson 17179869652 +victor johnson 21474837148 +victor johnson 25769804771 +victor king 8589934917 +victor king 25769804714 +victor laertes 12884902188 +victor laertes 21474837186 +victor miller 21474837170 +victor nixon 8589934778 +victor nixon 12884902261 +victor ovid 12884902350 +victor polk 17179869376 +victor quirinius 21474837074 +victor quirinius 21474837279 +victor robinson 21474836948 +victor robinson 21474837097 +victor steinbeck 12884902162 +victor steinbeck 17179869721 +victor steinbeck 21474836916 +victor thompson 25769804395 +victor van buren 21474837010 +victor van buren 25769804601 +victor white 8589934816 +victor white 30064771798 +victor xylophone 17179869560 +victor xylophone 25769804719 +victor xylophone 25769804760 +victor xylophone 34359739093 +victor xylophone 34359739095 +victor young 21474837052 +victor zipper 12884902345 +wendy allen 21474837127 +wendy allen 25769804525 +wendy allen 25769804732 +wendy brown 12884902342 +wendy brown 21474836889 +wendy ellison 12884902392 +wendy ellison 21474836763 +wendy falkner 8589934926 +wendy falkner 17179869470 +wendy falkner 25769804816 +wendy garcia 17179869439 +wendy garcia 17179869732 +wendy garcia 30064771654 +wendy garcia 30064771704 +wendy hernandez 17179869752 +wendy ichabod 17179869547 +wendy king 17179869612 +wendy king 21474837301 +wendy king 30064772042 +wendy laertes 8589934872 +wendy laertes 12884902469 +wendy laertes 21474837084 +wendy miller 17179869661 +wendy miller 17179869682 +wendy nixon 12884902521 +wendy nixon 21474836846 +wendy ovid 21474837025 +wendy ovid 38654706512 +wendy polk 8589934960 +wendy polk 21474837144 +wendy quirinius 12884902263 +wendy quirinius 17179869652 +wendy robinson 21474837104 +wendy robinson 25769804321 +wendy robinson 25769804728 +wendy steinbeck 12884902299 +wendy thompson 17179869494 +wendy thompson 21474837072 +wendy underhill 17179869898 +wendy underhill 21474837064 +wendy underhill 25769804845 +wendy van buren 25769804447 +wendy van buren 25769804679 +wendy white 17179869866 +wendy xylophone 17179869596 +wendy xylophone 25769804554 +wendy young 4294967313 +wendy young 25769804562 +xavier allen 12884902364 +xavier allen 17179869960 +xavier allen 21474836864 +xavier brown 8589934824 +xavier brown 17179869646 +xavier brown 25769804653 +xavier carson 17179869770 +xavier carson 21474837445 +xavier davidson 30064772118 +xavier davidson 34359739403 +xavier davidson 38654706539 +xavier ellison 34359739490 +xavier ellison 34359739559 +xavier garcia 21474837142 +xavier hernandez 21474837012 +xavier hernandez 25769804421 +xavier hernandez 38654707021 +xavier ichabod 12884902315 +xavier ichabod 17179869567 +xavier johnson 8589934922 +xavier johnson 38654707066 +xavier king 12884902272 +xavier king 21474836962 +xavier laertes 17179869795 +xavier ovid 17179869597 +xavier polk 12884902254 +xavier polk 17179869581 +xavier polk 17179869743 +xavier polk 34359739344 +xavier quirinius 12884902240 +xavier quirinius 21474836996 +xavier quirinius 25769804437 +xavier quirinius 25769804456 +xavier thompson 17179869822 +xavier underhill 8589934813 +xavier white 12884902262 +xavier white 12884902366 +xavier xylophone 17179869722 +xavier zipper 12884902377 +yuri allen 8589935035 +yuri allen 12884902279 +yuri brown 8589934912 +yuri brown 12884902319 +yuri carson 21474837146 +yuri carson 25769804245 +yuri ellison 25769804504 +yuri ellison 25769804568 +yuri falkner 25769804699 +yuri falkner 42949674720 +yuri garcia 4294967362 +yuri hernandez 21474837117 +yuri johnson 21474837002 +yuri johnson 21474837165 +yuri johnson 25769804545 +yuri king 30064772090 +yuri laertes 30064772076 +yuri laertes 34359739328 +yuri nixon 12884902232 +yuri nixon 12884902265 +yuri polk 12884902362 +yuri polk 21474837245 +yuri polk 25769804539 +yuri quirinius 12884902198 +yuri quirinius 17179869606 +yuri quirinius 30064771819 +yuri steinbeck 4294967535 +yuri steinbeck 8589934657 +yuri thompson 12884902467 +yuri underhill 17179869566 +yuri underhill 17179869715 +yuri white 34359739045 +yuri xylophone 12884902412 +zach allen 17179869908 +zach brown 21474836879 +zach brown 21474836891 +zach brown 21474837040 +zach brown 21474837073 +zach brown 30064771852 +zach carson 21474837185 +zach ellison 8589934898 +zach falkner 17179869807 +zach falkner 25769804634 +zach garcia 17179869536 +zach garcia 21474837142 +zach garcia 30064772246 +zach garcia 34359739192 +zach ichabod 17179869613 +zach ichabod 17179869838 +zach king 17179869700 +zach king 21474837427 +zach king 34359739578 +zach miller 4294967391 +zach miller 12884902310 +zach miller 17179869709 +zach ovid 17179869731 +zach ovid 21474837032 +zach ovid 21474837127 +zach ovid 30064771625 +zach quirinius 34359739151 +zach robinson 21474836938 +zach steinbeck 17179869667 +zach steinbeck 25769804623 +zach thompson 12884902354 +zach thompson 17179869659 +zach underhill 12884902149 +zach white 25769804490 +zach xylophone 12884902198 +zach xylophone 21474837163 +zach young 17179869687 +zach zipper 17179869708 +zach zipper 17179869834 +zach zipper 21474837369 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 new file mode 100644 index 0000000000000..1436509e4ec17 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-1-6378faf36ffd3f61e61cee6c0cb70e6 @@ -0,0 +1,1049 @@ + 9.220000267028809 + 43.72999954223633 + 89.52999877929688 +alice allen 2.7899999618530273 +alice allen 21.450000762939453 +alice allen 73.62999725341797 +alice brown 71.30999755859375 +alice carson 39.029998779296875 +alice davidson 70.3499984741211 +alice falkner 90.25 +alice garcia 48.45000076293945 +alice hernandez 88.16999816894531 +alice hernandez 90.55999755859375 +alice johnson 4.46999979019165 +alice king 19.139999389648438 +alice king 23.170000076293945 +alice king 52.22999954223633 +alice laertes 68.94999694824219 +alice laertes 69.52999877929688 +alice miller 68.95999908447266 +alice nixon 40.0 +alice nixon 48.150001525878906 +alice nixon 79.83000183105469 +alice ovid 9.039999961853027 +alice polk 62.900001525878906 +alice quirinius 37.13999938964844 +alice quirinius 62.29999923706055 +alice robinson 5.079999923706055 +alice robinson 56.099998474121094 +alice steinbeck 38.619998931884766 +alice steinbeck 55.5099983215332 +alice steinbeck 92.37000274658203 +alice underhill 98.18000030517578 +alice van buren 38.939998626708984 +alice xylophone 33.58000183105469 +alice xylophone 43.15999984741211 +alice xylophone 78.20999908447266 +alice zipper 26.43000030517578 +alice zipper 42.47999954223633 +alice zipper 89.93000030517578 +bob brown 8.069999694824219 +bob brown 70.93000030517578 +bob brown 93.08999633789062 +bob carson 50.09000015258789 +bob davidson 1.2899999618530273 +bob davidson 53.93000030517578 +bob davidson 74.72000122070312 +bob ellison 41.34000015258789 +bob ellison 65.0199966430664 +bob ellison 75.02999877929688 +bob ellison 80.30000305175781 +bob falkner 16.989999771118164 +bob garcia 4.460000038146973 +bob garcia 5.400000095367432 +bob garcia 45.59000015258789 +bob garcia 80.30000305175781 +bob garcia 87.56999969482422 +bob hernandez 22.68000030517578 +bob ichabod 82.55999755859375 +bob king 8.789999961853027 +bob king 12.539999961853027 +bob king 39.0099983215332 +bob laertes 0.7900000214576721 +bob laertes 10.670000076293945 +bob miller 61.91999816894531 +bob ovid 46.86000061035156 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob polk 7.980000019073486 +bob quirinius 46.099998474121094 +bob steinbeck 9.699999809265137 +bob van buren 33.66999816894531 +bob white 45.34000015258789 +bob white 45.349998474121094 +bob xylophone 19.690000534057617 +bob xylophone 107.93000221252441 +bob young 35.16999816894531 +bob zipper 1.25 +bob zipper 3.819999933242798 +bob zipper 34.349998474121094 +calvin allen 63.119998931884766 +calvin brown 28.110000610351562 +calvin brown 85.9000015258789 +calvin brown 90.19999694824219 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 85.51000213623047 +calvin ellison 26.489999771118164 +calvin falkner 2.9700000286102295 +calvin falkner 56.040000915527344 +calvin falkner 56.33000183105469 +calvin falkner 80.5999984741211 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin garcia 41.849998474121094 +calvin hernandez 33.869998931884766 +calvin johnson 66.61000061035156 +calvin laertes 23.1299991607666 +calvin laertes 50.310001373291016 +calvin nixon 9.8100004196167 +calvin nixon 41.20000076293945 +calvin nixon 69.73999786376953 +calvin ovid 69.95999908447266 +calvin ovid 71.26000213623047 +calvin ovid 79.12000274658203 +calvin ovid 84.72000122070312 +calvin polk 65.72000122070312 +calvin quirinius 29.540000915527344 +calvin quirinius 53.02000045776367 +calvin robinson 40.439998626708984 +calvin steinbeck 15.220000267028809 +calvin steinbeck 22.850000381469727 +calvin steinbeck 93.30000305175781 +calvin thompson 8.90999984741211 +calvin thompson 93.7300033569336 +calvin underhill 59.70000076293945 +calvin van buren 34.209999084472656 +calvin van buren 64.0 +calvin white 50.279998779296875 +calvin white 90.69000244140625 +calvin xylophone 21.700000762939453 +calvin xylophone 25.420000076293945 +calvin xylophone 56.810001373291016 +calvin young 24.489999771118164 +calvin young 39.810001373291016 +calvin zipper 9.1899995803833 +calvin zipper 95.37999725341797 +david allen 51.25 +david allen 64.87000274658203 +david brown 3.2100000381469727 +david brown 93.63999938964844 +david davidson 1.0800000429153442 +david davidson 62.720001220703125 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 47.689998626708984 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david hernandez 99.91000366210938 +david ichabod 5.28000020980835 +david ichabod 82.55000305175781 +david laertes 76.70999908447266 +david nixon 50.31999969482422 +david ovid 25.110000610351562 +david ovid 61.70000076293945 +david quirinius 20.639999389648438 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david robinson 25.280000686645508 +david robinson 161.1199951171875 +david thompson 80.89999771118164 +david underhill 8.319999694824219 +david underhill 88.7699966430664 +david underhill 97.55999755859375 +david van buren 83.56999969482422 +david van buren 115.43999862670898 +david white 45.189998626708984 +david xylophone 8.069999694824219 +david xylophone 54.34000015258789 +david xylophone 72.9800033569336 +david young 10.25 +david young 35.650001525878906 +ethan allen 32.75 +ethan brown 7.110000133514404 +ethan brown 10.09000015258789 +ethan brown 15.630000114440918 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 82.30000305175781 +ethan carson 76.33000183105469 +ethan ellison 0.2800000011920929 +ethan ellison 81.47000122070312 +ethan falkner 50.02000045776367 +ethan falkner 59.43000030517578 +ethan garcia 43.189998626708984 +ethan hernandez 49.779998779296875 +ethan johnson 90.05000305175781 +ethan king 4.349999904632568 +ethan laertes 15.449999809265137 +ethan laertes 54.75 +ethan laertes 59.209999084472656 +ethan laertes 70.38999938964844 +ethan laertes 80.70999908447266 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan miller 25.3700008392334 +ethan nixon 37.779998779296875 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 21.31999969482422 +ethan polk 23.440000534057617 +ethan polk 122.71999740600586 +ethan quirinius 3.859999895095825 +ethan quirinius 51.84000015258789 +ethan quirinius 97.23999786376953 +ethan robinson 67.94000244140625 +ethan robinson 78.62000274658203 +ethan underhill 55.630001068115234 +ethan van buren 36.70000076293945 +ethan white 60.849998474121094 +ethan white 63.41999816894531 +ethan xylophone 57.11000061035156 +ethan zipper 2.9200000762939453 +ethan zipper 97.51000213623047 +fred davidson 18.860000610351562 +fred davidson 37.2400016784668 +fred davidson 78.30999755859375 +fred ellison 31.179998874664307 +fred ellison 48.59000015258789 +fred ellison 96.77999877929688 +fred falkner 10.289999961853027 +fred falkner 72.04000091552734 +fred falkner 85.0 +fred hernandez 55.9900016784668 +fred ichabod 47.359999656677246 +fred ichabod 81.31999969482422 +fred johnson 96.08999633789062 +fred king 48.369998931884766 +fred king 72.13999843597412 +fred laertes 57.63999938964844 +fred miller 46.970001220703125 +fred nixon 28.690000534057617 +fred nixon 38.04999923706055 +fred nixon 70.5199966430664 +fred nixon 93.02999877929688 +fred polk 23.959999084472656 +fred polk 39.18000030517578 +fred polk 47.31999969482422 +fred polk 90.12000274658203 +fred quirinius 15.300000190734863 +fred quirinius 29.399999618530273 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 41.310001373291016 +fred steinbeck 91.05000305175781 +fred underhill 90.7699966430664 +fred van buren 1.0199999809265137 +fred van buren 21.940000534057617 +fred van buren 52.869998931884766 +fred van buren 83.58000183105469 +fred white 37.79999923706055 +fred young 46.79999923706055 +fred young 97.70999908447266 +fred zipper 29.020000457763672 +gabriella allen 46.27000045776367 +gabriella allen 64.22000122070312 +gabriella brown 15.260000228881836 +gabriella brown 84.83000183105469 +gabriella carson 42.7599983215332 +gabriella davidson 6.550000190734863 +gabriella ellison 48.08000183105469 +gabriella ellison 71.54000091552734 +gabriella falkner 10.170000076293945 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella garcia 43.0099983215332 +gabriella hernandez 76.91999816894531 +gabriella hernandez 92.9800033569336 +gabriella ichabod 10.729999542236328 +gabriella ichabod 26.639999389648438 +gabriella ichabod 66.36000061035156 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella king 20.670000076293945 +gabriella king 80.45999908447266 +gabriella laertes 65.37999725341797 +gabriella miller 50.83000183105469 +gabriella ovid 77.7400016784668 +gabriella ovid 92.4000015258789 +gabriella polk 35.68000030517578 +gabriella polk 88.05000305175781 +gabriella steinbeck 46.45000076293945 +gabriella steinbeck 78.63999938964844 +gabriella thompson 73.31999969482422 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella van buren 69.80000305175781 +gabriella van buren 70.05999755859375 +gabriella white 55.18000030517578 +gabriella young 9.25 +gabriella young 59.709999084472656 +gabriella zipper 36.2599983215332 +gabriella zipper 91.62999725341797 +holly allen 44.56999969482422 +holly brown 77.80999755859375 +holly brown 78.7300033569336 +holly falkner 80.73999786376953 +holly hernandez 20.81999969482422 +holly hernandez 21.190000534057617 +holly hernandez 24.790000915527344 +holly hernandez 30.25 +holly ichabod 83.2699966430664 +holly ichabod 84.69000244140625 +holly ichabod 90.51000213623047 +holly johnson 36.95000076293945 +holly johnson 64.36000061035156 +holly johnson 65.62000274658203 +holly king 42.310001373291016 +holly king 55.38999938964844 +holly laertes 52.5 +holly miller 50.400001525878906 +holly nixon 53.779998779296875 +holly nixon 88.0199966430664 +holly polk 3.619999885559082 +holly polk 98.30999755859375 +holly robinson 69.31999969482422 +holly thompson 0.07999999821186066 +holly thompson 86.69000244140625 +holly thompson 145.93999481201172 +holly underhill 42.54999923706055 +holly underhill 50.40999984741211 +holly underhill 79.95999908447266 +holly underhill 96.68000030517578 +holly van buren 68.80999755859375 +holly white 7.960000038146973 +holly white 32.91999816894531 +holly xylophone 93.11000061035156 +holly young 60.220001220703125 +holly young 66.16999816894531 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 38.849998474121094 +irene brown 4.789999961853027 +irene brown 53.939998626708984 +irene brown 87.66999816894531 +irene carson 94.54000091552734 +irene ellison 45.2400016784668 +irene ellison 50.08000183105469 +irene falkner 22.079999923706055 +irene falkner 99.91999816894531 +irene garcia 15.369999885559082 +irene garcia 58.43000030517578 +irene garcia 86.93000030517578 +irene ichabod 41.439998626708984 +irene ichabod 99.62000274658203 +irene johnson 5.880000114440918 +irene laertes 9.569999694824219 +irene laertes 42.66999816894531 +irene laertes 44.43000030517578 +irene miller 65.44000244140625 +irene nixon 15.100000381469727 +irene nixon 29.780000686645508 +irene nixon 42.560001373291016 +irene ovid 5.239999771118164 +irene ovid 35.130001068115234 +irene ovid 79.75 +irene polk 0.9800000190734863 +irene polk 24.020000457763672 +irene polk 42.2400016784668 +irene polk 47.08000183105469 +irene polk 95.83999633789062 +irene quirinius 12.899999618530273 +irene quirinius 58.86000061035156 +irene quirinius 70.0 +irene robinson 94.2699966430664 +irene steinbeck 94.33000183105469 +irene thompson 78.30000305175781 +irene underhill 28.309999465942383 +irene underhill 57.349998474121094 +irene van buren 54.439998626708984 +irene van buren 54.9900016784668 +irene xylophone 74.19000244140625 +jessica brown 51.290000915527344 +jessica carson 25.549999237060547 +jessica carson 31.860000610351562 +jessica carson 62.20000076293945 +jessica davidson 33.54999923706055 +jessica davidson 49.77000045776367 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica ellison 11.180000305175781 +jessica ellison 22.780000686645508 +jessica falkner 99.6500015258789 +jessica garcia 5.539999961853027 +jessica garcia 87.92999941110611 +jessica ichabod 59.15999984741211 +jessica johnson 9.5600004196167 +jessica johnson 40.79999923706055 +jessica miller 151.0199966430664 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 119.9000015258789 +jessica polk 49.68000030517578 +jessica quirinius 22.940000534057617 +jessica quirinius 32.470001220703125 +jessica quirinius 35.619998931884766 +jessica quirinius 46.869998931884766 +jessica robinson 112.36000442504883 +jessica thompson 38.33000183105469 +jessica thompson 89.55000305175781 +jessica underhill 26.079999923706055 +jessica underhill 45.41999816894531 +jessica underhill 46.209999084472656 +jessica van buren 9.739999771118164 +jessica white 11.550000190734863 +jessica white 36.58000183105469 +jessica white 73.93000030517578 +jessica white 74.30000305175781 +jessica white 96.62000274658203 +jessica xylophone 53.060001373291016 +jessica young 11.1899995803833 +jessica young 43.369998931884766 +jessica zipper 6.630000114440918 +jessica zipper 12.020000457763672 +jessica zipper 92.43999862670898 +katie allen 64.66999816894531 +katie brown 27.719999313354492 +katie davidson 170.84000396728516 +katie ellison 3.609999895095825 +katie ellison 80.97000122070312 +katie falkner 18.5 +katie garcia 24.729999542236328 +katie garcia 84.4000015258789 +katie hernandez 38.61999988555908 +katie ichabod 30.709999084472656 +katie ichabod 39.97999954223633 +katie ichabod 43.16999816894531 +katie king 39.34000015258789 +katie king 39.83000183105469 +katie king 97.80999755859375 +katie miller 31.399999618530273 +katie miller 74.77999877929688 +katie nixon 121.3700008392334 +katie ovid 50.65999984741211 +katie polk 11.680000305175781 +katie polk 40.2400016784668 +katie robinson 13.890000343322754 +katie van buren 17.739999771118164 +katie van buren 52.529998779296875 +katie white 1.309999942779541 +katie white 34.72999954223633 +katie xylophone 14.130000114440918 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 18.93000030517578 +katie zipper 58.75 +luke allen 15.180000305175781 +luke allen 50.959999084472656 +luke allen 66.61000061035156 +luke allen 89.55000305175781 +luke allen 99.38999938964844 +luke brown 51.790000915527344 +luke davidson 7.050000190734863 +luke davidson 28.950000762939453 +luke ellison 1.8700000047683716 +luke ellison 16.25 +luke ellison 87.83000183105469 +luke falkner 32.25 +luke falkner 39.60000038146973 +luke garcia 13.350000381469727 +luke garcia 30.3700008392334 +luke ichabod 8.449999809265137 +luke ichabod 97.87000274658203 +luke johnson 11.149999618530273 +luke johnson 14.4399995803833 +luke johnson 31.670000076293945 +luke laertes 0.5199999809265137 +luke laertes 4.800000190734863 +luke laertes 11.819999694824219 +luke laertes 16.690000534057617 +luke laertes 45.9900016784668 +luke miller 97.6500015258789 +luke ovid 38.04999923706055 +luke ovid 159.68000030517578 +luke polk 46.880001068115234 +luke polk 95.27999877929688 +luke quirinius 40.41999816894531 +luke robinson 55.099998474121094 +luke robinson 65.69999694824219 +luke thompson 94.37999725341797 +luke underhill 59.68000030517578 +luke underhill 95.52999877929688 +luke underhill 96.94000244140625 +luke van buren 148.62999725341797 +luke white 67.12000274658203 +luke xylophone 48.279998779296875 +luke zipper 24.829999923706055 +mike allen 48.53999900817871 +mike brown 48.22999954223633 +mike carson 20.06999969482422 +mike carson 47.56999969482422 +mike carson 81.66000366210938 +mike davidson 27.309999465942383 +mike davidson 54.83000183105469 +mike ellison 28.559999465942383 +mike ellison 37.099998474121094 +mike ellison 62.13999938964844 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike falkner 16.479999542236328 +mike garcia 70.8499984741211 +mike garcia 75.83000183105469 +mike garcia 79.20999908447266 +mike hernandez 37.900001525878906 +mike hernandez 59.45000076293945 +mike ichabod 64.7699966430664 +mike king 38.790000915527344 +mike king 62.7400016784668 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 85.0999984741211 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 60.119998931884766 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 27.06999969482422 +mike polk 99.68000030517578 +mike quirinius 89.37999725341797 +mike steinbeck 5.849999904632568 +mike steinbeck 85.13999938964844 +mike steinbeck 93.07000207901001 +mike steinbeck 97.45999908447266 +mike van buren 80.83999633789062 +mike van buren 114.56999969482422 +mike white 9.569999694824219 +mike white 28.889999389648438 +mike white 32.0099983215332 +mike white 91.87999725341797 +mike young 7.820000171661377 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike zipper 26.729999542236328 +mike zipper 83.91999816894531 +mike zipper 97.38999938964844 +nick allen 21.830000400543213 +nick allen 35.08000183105469 +nick brown 42.5099983215332 +nick davidson 49.439998626708984 +nick ellison 9.680000305175781 +nick ellison 89.01000213623047 +nick falkner 10.130000114440918 +nick falkner 88.47000122070312 +nick garcia 13.9399995803833 +nick garcia 26.389999389648438 +nick garcia 46.43000030517578 +nick ichabod 23.450000762939453 +nick ichabod 47.59000015258789 +nick ichabod 74.41999816894531 +nick johnson 3.9700000286102295 +nick johnson 94.08000183105469 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 87.98999786376953 +nick polk 59.27000141143799 +nick quirinius 67.44999694824219 +nick quirinius 81.16999816894531 +nick robinson 57.66999816894531 +nick robinson 60.709999084472656 +nick steinbeck 97.83000183105469 +nick thompson 11.90999984741211 +nick underhill 20.809999465942383 +nick van buren 51.290000915527344 +nick xylophone 103.45999908447266 +nick young 0.27000001072883606 +nick young 24.799999237060547 +nick zipper 56.619998931884766 +nick zipper 119.0199966430664 +oscar allen 18.6299991607666 +oscar brown 13.100000381469727 +oscar carson 6.869999885559082 +oscar carson 55.20000076293945 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar davidson 64.45999908447266 +oscar ellison 57.88999938964844 +oscar ellison 107.7100019454956 +oscar falkner 98.4800033569336 +oscar garcia 67.4800033569336 +oscar hernandez 95.4800033569336 +oscar hernandez 125.92999649047852 +oscar ichabod 3.3299999237060547 +oscar ichabod 33.52000045776367 +oscar ichabod 71.80000305175781 +oscar ichabod 76.69000244140625 +oscar johnson 16.09000015258789 +oscar johnson 139.69000244140625 +oscar king 19.059999465942383 +oscar king 25.8799991607666 +oscar king 59.5 +oscar laertes 5.510000228881836 +oscar laertes 8.420000076293945 +oscar laertes 9.260000228881836 +oscar laertes 27.1200008392334 +oscar nixon 41.619998931884766 +oscar ovid 37.13999938964844 +oscar ovid 82.23999786376953 +oscar ovid 91.52999877929688 +oscar polk 30.610000610351562 +oscar polk 63.900001525878906 +oscar quirinius 41.45000076293945 +oscar quirinius 65.43000030517578 +oscar quirinius 113.35000228881836 +oscar quirinius 139.10000610351562 +oscar robinson 11.34000015258789 +oscar robinson 42.849998474121094 +oscar robinson 74.52999877929688 +oscar robinson 131.31999969482422 +oscar steinbeck 29.59000015258789 +oscar thompson 31.90999984741211 +oscar thompson 41.34000015258789 +oscar thompson 60.529998779296875 +oscar thompson 70.88999938964844 +oscar underhill 87.4000015258789 +oscar van buren 2.180000066757202 +oscar van buren 61.880001068115234 +oscar van buren 91.77999877929688 +oscar white 19.0 +oscar white 28.450000762939453 +oscar white 51.849998474121094 +oscar white 59.83000183105469 +oscar xylophone 21.799999237060547 +oscar xylophone 57.119998931884766 +oscar xylophone 57.22999954223633 +oscar zipper 13.989999771118164 +oscar zipper 32.88999938964844 +oscar zipper 39.81999969482422 +priscilla brown 70.23999786376953 +priscilla brown 80.5199966430664 +priscilla brown 104.63999938964844 +priscilla carson 7.960000038146973 +priscilla carson 79.80999946594238 +priscilla carson 85.43000316619873 +priscilla ichabod 80.04000091552734 +priscilla ichabod 92.61000061035156 +priscilla johnson 61.939998626708984 +priscilla johnson 67.9800033569336 +priscilla johnson 68.32999992370605 +priscilla johnson 91.4800033569336 +priscilla johnson 92.48000144958496 +priscilla king 43.91999816894531 +priscilla nixon 95.80999755859375 +priscilla nixon 107.69000244140625 +priscilla ovid 52.72999954223633 +priscilla ovid 125.73999643325806 +priscilla polk 15.149999618530273 +priscilla quirinius 9.710000038146973 +priscilla thompson 9.800000190734863 +priscilla underhill 35.720001220703125 +priscilla underhill 68.22000122070312 +priscilla van buren 68.88999938964844 +priscilla van buren 91.61000061035156 +priscilla van buren 170.5500030517578 +priscilla white 78.27999877929688 +priscilla xylophone 0.15000000596046448 +priscilla xylophone 21.489999771118164 +priscilla xylophone 59.61000061035156 +priscilla young 0.4300000071525574 +priscilla young 4.320000171661377 +priscilla zipper 18.6299991607666 +priscilla zipper 25.670000076293945 +quinn allen 54.72999954223633 +quinn allen 83.33000183105469 +quinn brown 24.280000686645508 +quinn brown 52.439998626708984 +quinn brown 80.58000183105469 +quinn davidson 61.57999849319458 +quinn davidson 67.18000030517578 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn ellison 19.280000686645508 +quinn ellison 30.649999618530273 +quinn garcia 40.97999954223633 +quinn garcia 59.9900016784668 +quinn garcia 74.0199966430664 +quinn garcia 172.8499984741211 +quinn ichabod 36.790000915527344 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 4.710000038146973 +quinn laertes 41.290000915527344 +quinn laertes 76.5199966430664 +quinn nixon 86.64000129699707 +quinn ovid 52.500000953674316 +quinn quirinius 32.18000030517578 +quinn robinson 38.64999866485596 +quinn steinbeck 8.449999809265137 +quinn steinbeck 66.51000213623047 +quinn thompson 74.9399995803833 +quinn thompson 76.27999877929688 +quinn underhill 17.15999984741211 +quinn underhill 79.4800033569336 +quinn underhill 140.92000198364258 +quinn van buren 82.5199966430664 +quinn young 45.060001373291016 +quinn zipper 22.25 +quinn zipper 58.0 +rachel allen 15.8100004196167 +rachel allen 74.44999694824219 +rachel brown 2.9600000381469727 +rachel brown 30.809999465942383 +rachel brown 33.36000061035156 +rachel brown 34.40999984741211 +rachel brown 52.16999816894531 +rachel carson 37.599998474121094 +rachel carson 98.95999908447266 +rachel davidson 4.920000076293945 +rachel ellison 10.600000381469727 +rachel falkner 46.150001525878906 +rachel falkner 80.91999816894531 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel johnson 62.22999954223633 +rachel king 36.220001220703125 +rachel king 59.45000076293945 +rachel laertes 44.220001220703125 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 89.27999877929688 +rachel quirinius 12.4399995803833 +rachel robinson 4.570000171661377 +rachel robinson 30.360000610351562 +rachel robinson 64.94999694824219 +rachel thompson 0.5600000023841858 +rachel thompson 4.170000076293945 +rachel thompson 58.52000045776367 +rachel underhill 48.45000076293945 +rachel white 43.709999084472656 +rachel white 94.72000122070312 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 72.18000030517578 +sarah carson 1.909999966621399 +sarah carson 14.210000038146973 +sarah carson 78.88999938964844 +sarah ellison 16.989999771118164 +sarah falkner 90.27999877929688 +sarah falkner 99.36000061035156 +sarah garcia 41.290000915527344 +sarah garcia 58.010000228881836 +sarah garcia 153.8800048828125 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 16.239999771118164 +sarah johnson 45.099998474121094 +sarah johnson 73.87999725341797 +sarah johnson 77.66000366210938 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.709999084472656 +sarah ovid 60.02000045776367 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 40.16999816894531 +sarah white 37.849998474121094 +sarah white 89.80999755859375 +sarah xylophone 68.31999969482422 +sarah young 45.560001373291016 +sarah zipper 83.08000183105469 +tom brown 8.609999656677246 +tom brown 12.319999694824219 +tom carson 5.440000057220459 +tom carson 16.079999923706055 +tom carson 18.889999389648438 +tom davidson 170.0 +tom ellison 76.73999786376953 +tom ellison 98.2300033569336 +tom ellison 155.99999618530273 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 103.29000282287598 +tom johnson 14.920000076293945 +tom johnson 43.56999969482422 +tom king 15.75 +tom laertes 43.310001373291016 +tom laertes 64.6500015258789 +tom miller 21.229999542236328 +tom miller 68.25 +tom miller 139.04000091552734 +tom nixon 153.83999633789062 +tom ovid 8.670000076293945 +tom polk 38.29999923706055 +tom polk 54.43000030517578 +tom quirinius 10.1899995803833 +tom quirinius 75.31999969482422 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 123.5199966430664 +tom steinbeck 26.489999771118164 +tom van buren 3.2799999713897705 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 40.040000915527344 +tom young 22.850000381469727 +tom young 84.30999755859375 +tom zipper 122.78000259399414 +ulysses brown 72.79000091552734 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 146.7100067138672 +ulysses carson 220.18000030517578 +ulysses davidson 55.16999816894531 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 35.16999816894531 +ulysses hernandez 54.470001220703125 +ulysses hernandez 68.25 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 102.5999984741211 +ulysses king 74.19000244140625 +ulysses laertes 1.9199999570846558 +ulysses laertes 24.860000610351562 +ulysses laertes 50.1899995803833 +ulysses miller 2.9600000381469727 +ulysses miller 76.27999877929688 +ulysses nixon 80.95999908447266 +ulysses ovid 29.360000610351562 +ulysses polk 8.710000038146973 +ulysses polk 60.060001373291016 +ulysses polk 65.0199966430664 +ulysses polk 97.10000038146973 +ulysses quirinius 112.56999969482422 +ulysses robinson 104.85999870300293 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 74.0 +ulysses thompson 198.83000564575195 +ulysses underhill 14.119999885559082 +ulysses underhill 22.360000610351562 +ulysses underhill 35.88999938964844 +ulysses underhill 57.369998931884766 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses van buren 95.52999877929688 +ulysses white 59.54999923706055 +ulysses white 170.0800018310547 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses xylophone 57.3100004196167 +ulysses young 14.930000305175781 +ulysses young 32.52000045776367 +ulysses young 114.55999946594238 +victor allen 44.27000045776367 +victor allen 89.5 +victor brown 59.34000015258789 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 60.2599983215332 +victor davidson 66.5999984741211 +victor davidson 98.54999923706055 +victor ellison 17.8700008392334 +victor ellison 68.8499984741211 +victor hernandez 19.030000686645508 +victor hernandez 59.619998931884766 +victor hernandez 69.87999725341797 +victor hernandez 71.3499984741211 +victor hernandez 74.5199966430664 +victor johnson 18.200000762939453 +victor johnson 42.89000141620636 +victor johnson 72.55999755859375 +victor king 47.880001068115234 +victor king 66.66999816894531 +victor laertes 62.91999816894531 +victor laertes 67.58999633789062 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 68.5 +victor ovid 125.84000015258789 +victor polk 17.210000038146973 +victor quirinius 50.70000076293945 +victor quirinius 134.4000015258789 +victor robinson 51.560001373291016 +victor robinson 58.66999816894531 +victor steinbeck 12.460000038146973 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 58.65999984741211 +victor van buren 34.970001220703125 +victor van buren 41.68000030517578 +victor white 5.670000076293945 +victor white 135.02999687194824 +victor xylophone 10.09000015258789 +victor xylophone 11.220000267028809 +victor xylophone 28.5 +victor xylophone 62.38999938964844 +victor xylophone 76.0999984741211 +victor young 88.55000305175781 +victor zipper 26.289999961853027 +wendy allen 56.06999969482422 +wendy allen 93.96999740600586 +wendy allen 220.7900003194809 +wendy brown 27.8700008392334 +wendy brown 50.2599983215332 +wendy ellison 94.66000366210938 +wendy ellison 124.93999481201172 +wendy falkner 22.010000228881836 +wendy falkner 97.68000030517578 +wendy falkner 141.36000061035156 +wendy garcia 30.6200008392334 +wendy garcia 57.25 +wendy garcia 82.1500015258789 +wendy garcia 133.3400001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 45.189998626708984 +wendy king 63.33000183105469 +wendy king 183.75 +wendy laertes 46.619998931884766 +wendy laertes 70.37999725341797 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 12.420000076293945 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 86.62999725341797 +wendy ovid 95.33000183105469 +wendy polk 32.369998931884766 +wendy polk 42.04000073671341 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 26.469999313354492 +wendy robinson 71.06999969482422 +wendy robinson 117.02000045776367 +wendy steinbeck 120.67000007629395 +wendy thompson 67.34000015258789 +wendy thompson 85.76000213623047 +wendy underhill 68.04000091552734 +wendy underhill 79.19000244140625 +wendy underhill 89.77999877929688 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 73.68000030517578 +wendy xylophone 76.69999694824219 +wendy xylophone 90.60000038146973 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 83.93000030517578 +xavier allen 98.22000122070312 +xavier brown 7.789999961853027 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 20.790000915527344 +xavier carson 94.68000030517578 +xavier davidson 15.920000076293945 +xavier davidson 82.41000366210938 +xavier davidson 106.5199966430664 +xavier ellison 12.850000381469727 +xavier ellison 77.97000122070312 +xavier garcia 70.04000091552734 +xavier hernandez 6.670000076293945 +xavier hernandez 38.56999969482422 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 203.65999794006348 +xavier king 8.569999694824219 +xavier king 87.22000122070312 +xavier laertes 15.899999618530273 +xavier ovid 112.91000366210938 +xavier polk 13.869999885559082 +xavier polk 61.209999084472656 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 62.52000045776367 +xavier quirinius 83.01000022888184 +xavier quirinius 89.55000305175781 +xavier quirinius 97.14999961853027 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 59.20000171661377 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 52.849998474121094 +yuri allen 94.98999977111816 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 98.82999801635742 +yuri falkner 39.6299991607666 +yuri falkner 86.0 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 69.59000015258789 +yuri laertes 37.59000015258789 +yuri laertes 61.95000076293945 +yuri nixon 2.200000047683716 +yuri nixon 82.81000328063965 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 105.11999702453613 +yuri quirinius 10.260000228881836 +yuri quirinius 54.310001373291016 +yuri quirinius 57.93000030517578 +yuri steinbeck 17.790000915527344 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 49.119998931884766 +zach brown 57.08000183105469 +zach brown 67.37999725341797 +zach brown 100.46000289916992 +zach carson 95.86999893188477 +zach ellison 6.840000152587891 +zach falkner 9.130000114440918 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 84.37999725341797 +zach garcia 106.86999893188477 +zach garcia 167.62000274658203 +zach ichabod 64.25 +zach ichabod 106.69000244140625 +zach king 46.18000030517578 +zach king 70.51000213623047 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 122.81000137329102 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 71.5 +zach thompson 91.63999938964844 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 43.84999942779541 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 52.60000133514404 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 new file mode 100644 index 0000000000000..e55bede9242e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-2-5f0eab306ea3c22b11ace9b542a7ee56 @@ -0,0 +1,1049 @@ + 257.04000091552734 + 261.16000175476074 + 284.2699966430664 +alice allen 73.62999725341797 +alice allen 195.0099983215332 +alice allen 196.729998588562 +alice brown 245.52000427246094 +alice carson 424.63000106811523 +alice davidson 319.00999450683594 +alice falkner 90.25 +alice garcia 174.36999893188477 +alice hernandez 185.6699981689453 +alice hernandez 380.1699981689453 +alice johnson 367.2900071144104 +alice king 58.78999900817871 +alice king 294.1199951171875 +alice king 371.23999404907227 +alice laertes 68.94999694824219 +alice laertes 258.3999938964844 +alice miller 154.19000244140625 +alice nixon 209.91000366210938 +alice nixon 246.36000442504883 +alice nixon 260.19000244140625 +alice ovid 49.8199987411499 +alice polk 148.63999938964844 +alice quirinius 239.81999588012695 +alice quirinius 301.4100036621094 +alice robinson 140.47999572753906 +alice robinson 266.4900016784668 +alice steinbeck 169.76000213623047 +alice steinbeck 186.70999908447266 +alice steinbeck 446.8099937438965 +alice underhill 98.18000030517578 +alice van buren 112.42000198364258 +alice xylophone 78.20999908447266 +alice xylophone 91.22000122070312 +alice xylophone 413.1199951171875 +alice zipper 89.93000030517578 +alice zipper 279.54000091552734 +alice zipper 293.25000381469727 +bob brown 188.89999389648438 +bob brown 228.80999946594238 +bob brown 247.37999725341797 +bob carson 207.67000198364258 +bob davidson 53.93000030517578 +bob davidson 113.83999919891357 +bob davidson 259.0899963378906 +bob ellison 65.0199966430664 +bob ellison 80.30000305175781 +bob ellison 243.86000061035156 +bob ellison 245.02999877929688 +bob falkner 208.82000160217285 +bob garcia 33.410000801086426 +bob garcia 87.56999969482422 +bob garcia 120.17999649047852 +bob garcia 148.65999841690063 +bob garcia 178.87000274658203 +bob hernandez 337.23999977111816 +bob ichabod 82.55999755859375 +bob king 114.11000156402588 +bob king 134.81999588012695 +bob king 152.7699956893921 +bob laertes 42.89999961853027 +bob laertes 393.99999433755875 +bob miller 146.1500015258789 +bob ovid 62.849998474121094 +bob ovid 88.77999877929688 +bob ovid 97.08999633789062 +bob ovid 102.93000030517578 +bob polk 261.4599976539612 +bob quirinius 298.7199897766113 +bob steinbeck 103.01999950408936 +bob van buren 174.89999771118164 +bob white 194.25 +bob white 347.7799949645996 +bob xylophone 19.690000534057617 +bob xylophone 191.52999687194824 +bob young 78.17999649047852 +bob zipper 132.86000061035156 +bob zipper 139.6900042295456 +bob zipper 295.59000039100647 +calvin allen 255.68000411987305 +calvin brown 85.9000015258789 +calvin brown 238.02000427246094 +calvin brown 275.8699951171875 +calvin carson 80.2300033569336 +calvin davidson 31.770000457763672 +calvin davidson 181.76000213623047 +calvin ellison 188.0300006866455 +calvin falkner 93.61000061035156 +calvin falkner 94.30999755859375 +calvin falkner 125.91999816894531 +calvin falkner 137.1699981689453 +calvin falkner 140.99999594688416 +calvin falkner 168.81999969482422 +calvin garcia 307.439998626709 +calvin hernandez 303.4599952697754 +calvin johnson 152.8300018310547 +calvin laertes 150.69999885559082 +calvin laertes 216.81000137329102 +calvin nixon 131.57999801635742 +calvin nixon 143.3699951171875 +calvin nixon 196.34000301361084 +calvin ovid 69.95999908447266 +calvin ovid 176.13999938964844 +calvin ovid 176.3800048828125 +calvin ovid 248.65999603271484 +calvin polk 147.04000091552734 +calvin quirinius 226.66999435424805 +calvin quirinius 266.7100009918213 +calvin robinson 289.7900047302246 +calvin steinbeck 92.05000305175781 +calvin steinbeck 118.15000057220459 +calvin steinbeck 333.6000003814697 +calvin thompson 93.7300033569336 +calvin thompson 249.56000137329102 +calvin underhill 208.3400001525879 +calvin van buren 136.51000213623047 +calvin van buren 347.0999946594238 +calvin white 90.69000244140625 +calvin white 112.15999984741211 +calvin xylophone 25.420000076293945 +calvin xylophone 237.71999740600586 +calvin xylophone 315.2099952697754 +calvin young 222.96000289916992 +calvin young 243.3199977874756 +calvin zipper 95.37999725341797 +calvin zipper 531.3600015640259 +david allen 202.43000030517578 +david allen 302.4399948120117 +david brown 93.63999938964844 +david brown 258.05999851226807 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david davidson 106.50000202655792 +david davidson 149.94000244140625 +david ellison 85.2300033569336 +david ellison 94.1500015258789 +david ellison 208.3900032043457 +david hernandez 99.91000366210938 +david ichabod 82.55000305175781 +david ichabod 320.47999143600464 +david laertes 250.1699981689453 +david nixon 174.58999633789062 +david ovid 198.21000289916992 +david ovid 230.47999954223633 +david quirinius 29.239999771118164 +david quirinius 79.97000122070312 +david quirinius 180.92999649047852 +david robinson 147.65999603271484 +david robinson 168.7100009918213 +david thompson 41.88999938964844 +david underhill 97.55999755859375 +david underhill 277.5999984741211 +david underhill 369.4600009918213 +david van buren 83.56999969482422 +david van buren 289.189998626709 +david white 124.6099967956543 +david xylophone 135.70000076293945 +david xylophone 237.06000137329102 +david xylophone 338.20999908447266 +david young 172.49000549316406 +david young 184.9800033569336 +ethan allen 240.42000198364258 +ethan brown 61.86000061035156 +ethan brown 73.18000030517578 +ethan brown 105.29000043869019 +ethan brown 177.8300018310547 +ethan brown 185.98999691009521 +ethan brown 284.729998588562 +ethan carson 265.22999572753906 +ethan ellison 166.5 +ethan ellison 244.99000671505928 +ethan falkner 59.43000030517578 +ethan falkner 196.17000198364258 +ethan garcia 271.5999946594238 +ethan hernandez 264.50999450683594 +ethan johnson 90.05000305175781 +ethan king 36.49000024795532 +ethan laertes 95.06999969482422 +ethan laertes 96.29000091552734 +ethan laertes 189.66000270843506 +ethan laertes 192.79999923706055 +ethan laertes 249.04000091552734 +ethan laertes 249.76000213623047 +ethan laertes 369.9599952697754 +ethan miller 314.5599994659424 +ethan nixon 493.03000259399414 +ethan ovid 57.290000915527344 +ethan polk 2.3499999046325684 +ethan polk 59.869998931884766 +ethan polk 219.6599998474121 +ethan polk 263.8600025177002 +ethan quirinius 97.23999786376953 +ethan quirinius 111.70999908447266 +ethan quirinius 317.69000363349915 +ethan robinson 78.62000274658203 +ethan robinson 149.5800018310547 +ethan underhill 231.25000381469727 +ethan van buren 152.60000228881836 +ethan white 155.81999969482422 +ethan white 235.55999755859375 +ethan xylophone 414.61000061035156 +ethan zipper 97.51000213623047 +ethan zipper 297.20999908447266 +fred davidson 78.30999755859375 +fred davidson 105.8499984741211 +fred davidson 220.56000137329102 +fred ellison 56.489999771118164 +fred ellison 96.77999877929688 +fred ellison 199.52000045776367 +fred falkner 66.77999973297119 +fred falkner 85.0 +fred falkner 169.91000366210938 +fred hernandez 117.85000228881836 +fred ichabod 81.31999969482422 +fred ichabod 202.45000457763672 +fred johnson 96.08999633789062 +fred king 140.24999618530273 +fred king 343.82000064849854 +fred laertes 57.63999938964844 +fred miller 176.18000030517578 +fred nixon 28.690000534057617 +fred nixon 187.40999603271484 +fred nixon 246.77000045776367 +fred nixon 338.34999084472656 +fred polk 90.12000274658203 +fred polk 323.1899948120117 +fred polk 357.19000244140625 +fred polk 496.16999435424805 +fred quirinius 218.82999801635742 +fred quirinius 224.12000179290771 +fred robinson 89.02999877929688 +fred steinbeck 32.22999954223633 +fred steinbeck 91.05000305175781 +fred steinbeck 231.92000198364258 +fred underhill 183.31999969482422 +fred van buren 83.58000183105469 +fred van buren 318.38000106811523 +fred van buren 346.1400008201599 +fred van buren 391.9999942779541 +fred white 187.38000106811523 +fred young 97.70999908447266 +fred young 141.22999954223633 +fred zipper 163.89999771118164 +gabriella allen 274.8599967956543 +gabriella allen 283.5 +gabriella brown 163.89999961853027 +gabriella brown 465.0 +gabriella carson 147.78999710083008 +gabriella davidson 263.64000415802 +gabriella ellison 71.54000091552734 +gabriella ellison 188.55999755859375 +gabriella falkner 51.720001220703125 +gabriella falkner 87.61000061035156 +gabriella falkner 162.21999835968018 +gabriella garcia 43.0099983215332 +gabriella hernandez 190.5500030517578 +gabriella hernandez 267.4700012207031 +gabriella ichabod 71.12999725341797 +gabriella ichabod 90.3499984741211 +gabriella ichabod 148.6999969482422 +gabriella ichabod 175.70000076293945 +gabriella ichabod 285.72999572753906 +gabriella king 166.75000190734863 +gabriella king 177.6999969482422 +gabriella laertes 65.37999725341797 +gabriella miller 148.4800033569336 +gabriella ovid 92.4000015258789 +gabriella ovid 137.82999801635742 +gabriella polk 244.07000350952148 +gabriella polk 282.00000762939453 +gabriella steinbeck 272.2799987792969 +gabriella steinbeck 461.060001373291 +gabriella thompson 88.36000061035156 +gabriella thompson 94.25 +gabriella thompson 158.80999755859375 +gabriella van buren 146.0800018310547 +gabriella van buren 151.63999938964844 +gabriella white 138.72000122070312 +gabriella young 30.739999771118164 +gabriella young 146.62999725341797 +gabriella zipper 91.62999725341797 +gabriella zipper 357.5099983215332 +holly allen 44.56999969482422 +holly brown 173.64999389648438 +holly brown 174.2100067138672 +holly falkner 166.22999572753906 +holly hernandez 180.0800018310547 +holly hernandez 248.71000480651855 +holly hernandez 336.87000274658203 +holly hernandez 523.2800025939941 +holly ichabod 179.37000274658203 +holly ichabod 180.18000030517578 +holly ichabod 184.66000366210938 +holly johnson 64.36000061035156 +holly johnson 145.61000061035156 +holly johnson 157.12999725341797 +holly king 275.2299995422363 +holly king 288.52000427246094 +holly laertes 246.21000289916992 +holly miller 290.21999740600586 +holly nixon 177.39999389648438 +holly nixon 228.58999633789062 +holly polk 98.30999755859375 +holly polk 307.0799951553345 +holly robinson 219.27999877929688 +holly thompson 75.41999816894531 +holly thompson 86.69000244140625 +holly thompson 523.360002592206 +holly underhill 96.68000030517578 +holly underhill 163.54000091552734 +holly underhill 187.47000122070312 +holly underhill 328.0099983215332 +holly van buren 161.7699966430664 +holly white 122.98999786376953 +holly white 335.93999576568604 +holly xylophone 191.34000396728516 +holly young 60.220001220703125 +holly young 297.20999908447266 +holly zipper 99.12999725341797 +holly zipper 99.29000091552734 +irene allen 234.6400032043457 +irene brown 4.789999961853027 +irene brown 176.4499969482422 +irene brown 338.2099952697754 +irene carson 292.0 +irene ellison 201.06000137329102 +irene ellison 230.79000091552734 +irene falkner 99.91999816894531 +irene falkner 210.11000061035156 +irene garcia 40.78999996185303 +irene garcia 86.93000030517578 +irene garcia 183.02000045776367 +irene ichabod 99.62000274658203 +irene ichabod 281.96999740600586 +irene johnson 243.59999752044678 +irene laertes 112.54000091552734 +irene laertes 227.45000076293945 +irene laertes 246.53000259399414 +irene miller 395.9100036621094 +irene nixon 29.780000686645508 +irene nixon 199.45999908447266 +irene nixon 261.46000480651855 +irene ovid 158.97000122070312 +irene ovid 339.94000244140625 +irene ovid 362.82000732421875 +irene polk 95.83999633789062 +irene polk 183.43000411987305 +irene polk 258.7100033760071 +irene polk 284.6300048828125 +irene polk 507.2400016784668 +irene quirinius 157.5800018310547 +irene quirinius 250.61000061035156 +irene quirinius 431.6499996185303 +irene robinson 191.72999572753906 +irene steinbeck 94.33000183105469 +irene thompson 256.0 +irene underhill 135.55999755859375 +irene underhill 327.0299892425537 +irene van buren 54.439998626708984 +irene van buren 193.71000289916992 +irene xylophone 168.5 +jessica brown 422.5299949645996 +jessica carson 103.66000366210938 +jessica carson 144.92000198364258 +jessica carson 259.1099967956543 +jessica davidson 95.33999633789062 +jessica davidson 99.20999908447266 +jessica davidson 137.17000198364258 +jessica davidson 227.79999923706055 +jessica ellison 207.35000228881836 +jessica ellison 237.4300022125244 +jessica falkner 99.6500015258789 +jessica garcia 174.70999908447266 +jessica garcia 185.62000179290771 +jessica ichabod 124.59000015258789 +jessica johnson 272.0500030517578 +jessica johnson 294.2899990081787 +jessica miller 77.83999633789062 +jessica nixon 77.0999984741211 +jessica nixon 90.06999969482422 +jessica ovid 71.68000030517578 +jessica ovid 309.44000244140625 +jessica polk 472.2099952697754 +jessica quirinius 35.619998931884766 +jessica quirinius 192.7000026702881 +jessica quirinius 208.6500015258789 +jessica quirinius 370.0599937438965 +jessica robinson 254.5300064086914 +jessica thompson 115.9000015258789 +jessica thompson 180.60000610351562 +jessica underhill 199.10999870300293 +jessica underhill 234.29000091552734 +jessica underhill 257.09000396728516 +jessica van buren 9.739999771118164 +jessica white 96.62000274658203 +jessica white 166.54000091552734 +jessica white 240.52999877929688 +jessica white 432.17999362945557 +jessica white 497.6400032043457 +jessica xylophone 385.4799995422363 +jessica young 47.410000801086426 +jessica young 240.6500015258789 +jessica zipper 323.4199962615967 +jessica zipper 344.8399953842163 +jessica zipper 480.06999588012695 +katie allen 312.9700012207031 +katie brown 573.4599933624268 +katie davidson 96.91000366210938 +katie ellison 163.52999877929688 +katie ellison 384.4699947834015 +katie falkner 125.57000160217285 +katie garcia 84.4000015258789 +katie garcia 160.28999710083008 +katie hernandez 257.9600028991699 +katie ichabod 187.63999557495117 +katie ichabod 274.97999572753906 +katie ichabod 362.9200019836426 +katie king 97.80999755859375 +katie king 169.56999969482422 +katie king 314.1999969482422 +katie miller 31.399999618530273 +katie miller 228.40999603271484 +katie nixon 23.190000534057617 +katie ovid 207.1200065612793 +katie polk 143.2599983215332 +katie polk 247.02000045776367 +katie robinson 83.84999942779541 +katie van buren 297.0300064086914 +katie van buren 464.54999351501465 +katie white 344.1700019836426 +katie white 465.8599934577942 +katie xylophone 175.89999675750732 +katie young 31.010000228881836 +katie young 72.51000213623047 +katie young 97.56999969482422 +katie zipper 101.9000015258789 +katie zipper 314.75 +luke allen 89.55000305175781 +luke allen 133.4800033569336 +luke allen 210.8800048828125 +luke allen 392.0300064086914 +luke allen 420.6299934387207 +luke brown 129.20999908447266 +luke davidson 28.950000762939453 +luke davidson 106.41000080108643 +luke ellison 42.09000027179718 +luke ellison 136.52000427246094 +luke ellison 187.51000213623047 +luke falkner 172.8799991607666 +luke falkner 216.0199966430664 +luke garcia 50.94000053405762 +luke garcia 345.1200008392334 +luke ichabod 67.90000057220459 +luke ichabod 97.87000274658203 +luke johnson 59.00999927520752 +luke johnson 105.32000160217285 +luke johnson 187.2899990081787 +luke laertes 105.42000198364258 +luke laertes 147.14999723434448 +luke laertes 158.86000061035156 +luke laertes 167.01999855041504 +luke laertes 281.19999504089355 +luke miller 97.6500015258789 +luke ovid 186.53000259399414 +luke ovid 340.1300048828125 +luke polk 95.27999877929688 +luke polk 277.6700019836426 +luke quirinius 115.83999633789062 +luke robinson 137.33999633789062 +luke robinson 145.23999786376953 +luke thompson 94.37999725341797 +luke underhill 96.94000244140625 +luke underhill 194.73999786376953 +luke underhill 372.6899948120117 +luke van buren 193.93999862670898 +luke white 67.12000274658203 +luke xylophone 102.37999725341797 +luke zipper 223.54000282287598 +mike allen 79.60999870300293 +mike brown 202.81999588012695 +mike carson 81.66000366210938 +mike carson 105.02999877929688 +mike carson 405.4499931335449 +mike davidson 137.74999809265137 +mike davidson 393.17999267578125 +mike ellison 79.37999725341797 +mike ellison 85.73999786376953 +mike ellison 127.15999603271484 +mike ellison 228.07999992370605 +mike ellison 263.8899955749512 +mike falkner 254.50000381469727 +mike garcia 70.8499984741211 +mike garcia 173.63999938964844 +mike garcia 177.5199966430664 +mike hernandez 59.45000076293945 +mike hernandez 327.6900062561035 +mike ichabod 64.7699966430664 +mike king 78.26000213623047 +mike king 84.2300033569336 +mike king 94.68000030517578 +mike king 133.5900001525879 +mike king 134.87999725341797 +mike king 173.45999908447266 +mike miller 57.890000343322754 +mike nixon 92.95999908447266 +mike nixon 203.68999862670898 +mike polk 32.140000343322754 +mike polk 99.68000030517578 +mike polk 306.61000061035156 +mike quirinius 89.37999725341797 +mike steinbeck 85.13999938964844 +mike steinbeck 97.45999908447266 +mike steinbeck 153.86000204086304 +mike steinbeck 221.21999502182007 +mike van buren 80.83999633789062 +mike van buren 174.21000289916992 +mike white 91.87999725341797 +mike white 341.80999755859375 +mike white 341.86000061035156 +mike white 389.20000076293945 +mike young 74.58999633789062 +mike young 83.54000091552734 +mike young 112.19000101089478 +mike zipper 86.98999786376953 +mike zipper 97.38999938964844 +mike zipper 174.61000061035156 +nick allen 173.32000207901 +nick allen 257.7300033569336 +nick brown 192.45000076293945 +nick davidson 258.9799919128418 +nick ellison 183.34000396728516 +nick ellison 193.02000427246094 +nick falkner 10.130000114440918 +nick falkner 182.72000122070312 +nick garcia 142.65999603271484 +nick garcia 183.7699966430664 +nick garcia 277.8299951553345 +nick ichabod 110.43999862670898 +nick ichabod 112.54999923706055 +nick ichabod 241.68999481201172 +nick johnson 192.56000518798828 +nick johnson 325.9499976634979 +nick laertes 96.25 +nick miller 82.97000122070312 +nick nixon 96.37999725341797 +nick ovid 184.3699951171875 +nick polk 199.57000064849854 +nick quirinius 174.80999755859375 +nick quirinius 243.8300018310547 +nick robinson 129.65999603271484 +nick robinson 216.54999923706055 +nick steinbeck 97.83000183105469 +nick thompson 205.4900016784668 +nick underhill 166.42000007629395 +nick van buren 222.6500015258789 +nick xylophone 75.3499984741211 +nick young 332.23999786376953 +nick young 346.41000083088875 +nick zipper 222.9199981689453 +nick zipper 529.7199974060059 +oscar allen 246.42999839782715 +oscar brown 274.01000022888184 +oscar carson 78.9800033569336 +oscar carson 87.4800033569336 +oscar carson 98.51000213623047 +oscar carson 203.86000442504883 +oscar carson 321.82000064849854 +oscar davidson 361.6699981689453 +oscar ellison 146.44000244140625 +oscar ellison 234.32000160217285 +oscar falkner 98.4800033569336 +oscar garcia 231.04000091552734 +oscar hernandez 85.48999786376953 +oscar hernandez 95.4800033569336 +oscar ichabod 71.80000305175781 +oscar ichabod 123.78000068664551 +oscar ichabod 173.31000518798828 +oscar ichabod 251.22000122070312 +oscar johnson 146.27000427246094 +oscar johnson 260.1600036621094 +oscar king 124.2699966430664 +oscar king 249.5399990081787 +oscar king 284.8599910736084 +oscar laertes 15.640000343322754 +oscar laertes 254.8499984741211 +oscar laertes 261.41000175476074 +oscar laertes 261.8400020599365 +oscar nixon 41.619998931884766 +oscar ovid 82.23999786376953 +oscar ovid 187.76000213623047 +oscar ovid 260.6100044250488 +oscar polk 63.900001525878906 +oscar polk 252.71000289916992 +oscar quirinius 73.4800033569336 +oscar quirinius 165.3800048828125 +oscar quirinius 244.2699966430664 +oscar quirinius 248.75 +oscar robinson 93.31999969482422 +oscar robinson 163.55999755859375 +oscar robinson 191.8300018310547 +oscar robinson 315.1999912261963 +oscar steinbeck 376.6899948120117 +oscar thompson 131.1400032043457 +oscar thompson 148.01000213623047 +oscar thompson 325.42000579833984 +oscar thompson 545.7399940490723 +oscar underhill 87.4000015258789 +oscar van buren 61.880001068115234 +oscar van buren 188.8699951171875 +oscar van buren 209.53000235557556 +oscar white 129.73999786376953 +oscar white 148.9800033569336 +oscar white 275.1500015258789 +oscar white 303.8599910736084 +oscar xylophone 115.22999954223633 +oscar xylophone 319.75000381469727 +oscar xylophone 475.3300018310547 +oscar zipper 109.53999710083008 +oscar zipper 214.40999603271484 +oscar zipper 214.6500015258789 +priscilla brown 77.56999969482422 +priscilla brown 165.5199966430664 +priscilla brown 408.4499969482422 +priscilla carson 168.8300018310547 +priscilla carson 195.7900047302246 +priscilla carson 207.5300006866455 +priscilla ichabod 92.61000061035156 +priscilla ichabod 206.16000366210938 +priscilla johnson 89.1500015258789 +priscilla johnson 156.4600067138672 +priscilla johnson 158.88000106811523 +priscilla johnson 190.61000061035156 +priscilla johnson 211.01000022888184 +priscilla king 371.9299964904785 +priscilla nixon 95.80999755859375 +priscilla nixon 278.87999725341797 +priscilla ovid 96.27000284194946 +priscilla ovid 198.3400001525879 +priscilla polk 252.5800018310547 +priscilla quirinius 131.8499994277954 +priscilla thompson 230.36000156402588 +priscilla underhill 143.56999969482422 +priscilla underhill 354.37000274658203 +priscilla van buren 82.72000122070312 +priscilla van buren 145.61000061035156 +priscilla van buren 183.72000122070312 +priscilla white 78.27999877929688 +priscilla xylophone 21.489999771118164 +priscilla xylophone 159.26000213623047 +priscilla xylophone 406.1000007688999 +priscilla young 163.2900013923645 +priscilla young 260.59000366926193 +priscilla zipper 311.399995803833 +priscilla zipper 327.97999572753906 +quinn allen 257.94000244140625 +quinn allen 365.2299995422363 +quinn brown 80.58000183105469 +quinn brown 80.81999969482422 +quinn brown 198.71000289916992 +quinn davidson 83.4000015258789 +quinn davidson 95.11000061035156 +quinn davidson 154.79000091552734 +quinn davidson 227.13999938964844 +quinn ellison 237.17000007629395 +quinn ellison 361.14000129699707 +quinn garcia 92.33000183105469 +quinn garcia 148.63999938964844 +quinn garcia 226.78999710083008 +quinn garcia 246.70000076293945 +quinn ichabod 89.63999938964844 +quinn king 74.62000274658203 +quinn king 86.2300033569336 +quinn laertes 112.36000061035156 +quinn laertes 243.6900019645691 +quinn laertes 265.51000213623047 +quinn nixon 149.3300018310547 +quinn ovid 393.2099943161011 +quinn quirinius 266.8200035095215 +quinn robinson 247.6400032043457 +quinn steinbeck 144.81999969482422 +quinn steinbeck 213.65999507904053 +quinn thompson 156.51000213623047 +quinn thompson 274.1599931716919 +quinn underhill 248.3000030517578 +quinn underhill 252.61999130249023 +quinn underhill 321.9799976348877 +quinn van buren 82.5199966430664 +quinn young 90.97999954223633 +quinn zipper 58.0 +quinn zipper 249.38999938964844 +rachel allen 15.8100004196167 +rachel allen 151.80999755859375 +rachel brown 193.5800018310547 +rachel brown 312.0800018310547 +rachel brown 347.7999954223633 +rachel brown 423.98999214172363 +rachel brown 437.64999771118164 +rachel carson 98.95999908447266 +rachel carson 385.3799934387207 +rachel davidson 396.38999938964844 +rachel ellison 299.12000465393066 +rachel falkner 88.80000305175781 +rachel falkner 99.23999786376953 +rachel falkner 172.54999542236328 +rachel falkner 233.55999755859375 +rachel johnson 197.92999649047852 +rachel king 36.220001220703125 +rachel king 219.8400001525879 +rachel laertes 97.17000198364258 +rachel laertes 109.5999984741211 +rachel ovid 80.20999872684479 +rachel ovid 260.18999683856964 +rachel polk 89.27999877929688 +rachel quirinius 205.1400022506714 +rachel robinson 254.1300015449524 +rachel robinson 286.0400047302246 +rachel robinson 332.4199981689453 +rachel thompson 137.73000198602676 +rachel thompson 213.31000137329102 +rachel thompson 380.85999488830566 +rachel underhill 175.6099967956543 +rachel white 94.72000122070312 +rachel white 196.18000030517578 +rachel young 230.6400032043457 +rachel zipper 148.9000015258789 +rachel zipper 238.98000192642212 +sarah carson 175.62000274658203 +sarah carson 307.70000088214874 +sarah carson 386.8999948501587 +sarah ellison 161.80999946594238 +sarah falkner 99.36000061035156 +sarah falkner 281.62000274658203 +sarah garcia 73.6500015258789 +sarah garcia 153.73000144958496 +sarah garcia 312.8899955749512 +sarah ichabod 81.31999969482422 +sarah ichabod 97.26000213623047 +sarah johnson 140.37999725341797 +sarah johnson 177.57000732421875 +sarah johnson 248.4499969482422 +sarah johnson 309.1800022125244 +sarah king 216.75 +sarah king 268.5399932861328 +sarah miller 222.31000518798828 +sarah ovid 146.25000381469727 +sarah robinson 143.43000030517578 +sarah robinson 310.75 +sarah steinbeck 208.72000122070312 +sarah white 140.22999572753906 +sarah white 181.86000061035156 +sarah xylophone 68.31999969482422 +sarah young 185.80999755859375 +sarah zipper 168.22000122070312 +tom brown 181.1000051498413 +tom brown 404.3500061035156 +tom carson 142.60999822616577 +tom carson 299.57999992370605 +tom carson 592.3499927520752 +tom davidson 180.61000061035156 +tom ellison 98.2300033569336 +tom ellison 154.58999633789062 +tom ellison 173.02999877929688 +tom falkner 88.22000122070312 +tom falkner 139.11000442504883 +tom hernandez 81.63999938964844 +tom hernandez 263.67000579833984 +tom ichabod 214.0699977874756 +tom johnson 405.95000076293945 +tom johnson 438.9099922180176 +tom king 218.18000030517578 +tom laertes 244.37000274658203 +tom laertes 473.0999984741211 +tom miller 68.25 +tom miller 85.59000015258789 +tom miller 127.56999969482422 +tom nixon 85.02999877929688 +tom ovid 217.32000160217285 +tom polk 188.87000274658203 +tom polk 206.52000045776367 +tom quirinius 120.27000427246094 +tom quirinius 232.63000202178955 +tom robinson 90.69000244140625 +tom robinson 98.72000122070312 +tom robinson 99.1500015258789 +tom robinson 209.5399932861328 +tom steinbeck 277.7100009918213 +tom van buren 40.779998779296875 +tom van buren 217.70000076293945 +tom van buren 375.2099964618683 +tom white 223.4700050354004 +tom young 174.36000061035156 +tom young 304.8199977874756 +tom zipper 213.7900047302246 +ulysses brown 247.1500015258789 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses carson 150.93000030517578 +ulysses carson 162.24000549316406 +ulysses davidson 414.7100009918213 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 106.29999542236328 +ulysses hernandez 134.44000244140625 +ulysses hernandez 160.22000122070312 +ulysses ichabod 98.56999969482422 +ulysses ichabod 309.34999656677246 +ulysses johnson 152.47000122070312 +ulysses king 244.7100067138672 +ulysses laertes 138.4400042295456 +ulysses laertes 173.55999755859375 +ulysses laertes 256.91999912261963 +ulysses miller 76.27999877929688 +ulysses miller 417.67000102996826 +ulysses nixon 174.56999969482422 +ulysses ovid 130.13000106811523 +ulysses polk 123.9399995803833 +ulysses polk 149.95999908447266 +ulysses polk 205.2400016784668 +ulysses polk 237.5699920654297 +ulysses quirinius 330.4700012207031 +ulysses robinson 79.48999786376953 +ulysses steinbeck 144.8300018310547 +ulysses steinbeck 155.66000366210938 +ulysses thompson 159.92000579833984 +ulysses underhill 81.58000183105469 +ulysses underhill 88.4800033569336 +ulysses underhill 99.66999816894531 +ulysses underhill 135.55999755859375 +ulysses underhill 189.1099977493286 +ulysses underhill 289.6800003051758 +ulysses underhill 385.60000228881836 +ulysses van buren 95.52999877929688 +ulysses white 188.8300018310547 +ulysses white 305.79000091552734 +ulysses xylophone 54.099998474121094 +ulysses xylophone 205.2099952697754 +ulysses xylophone 251.94000148773193 +ulysses young 100.77000045776367 +ulysses young 275.8300018310547 +ulysses young 522.1700019836426 +victor allen 220.1699981689453 +victor allen 222.10000228881836 +victor brown 77.88999938964844 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor brown 455.25000381469727 +victor davidson 149.06000137329102 +victor davidson 291.48000717163086 +victor davidson 321.25 +victor ellison 314.37000274658203 +victor ellison 442.50000190734863 +victor hernandez 69.87999725341797 +victor hernandez 99.85000038146973 +victor hernandez 143.02000045776367 +victor hernandez 160.38999938964844 +victor hernandez 391.2999954223633 +victor johnson 145.18000030517578 +victor johnson 190.27000045776367 +victor johnson 308.1900006532669 +victor king 108.10000228881836 +victor king 310.5 +victor laertes 145.42999267578125 +victor laertes 214.72999572753906 +victor miller 173.76000022888184 +victor nixon 68.5 +victor nixon 269.5899963378906 +victor ovid 151.39999771118164 +victor polk 175.8799991607666 +victor quirinius 65.55000305175781 +victor quirinius 168.5500030517578 +victor robinson 177.9100022315979 +victor robinson 204.09999084472656 +victor steinbeck 52.720001220703125 +victor steinbeck 220.45999908447266 +victor steinbeck 309.4900064468384 +victor thompson 58.65999984741211 +victor van buren 206.77999877929688 +victor van buren 222.44000244140625 +victor white 156.36999893188477 +victor white 167.2699966430664 +victor xylophone 158.36999893188477 +victor xylophone 161.54000091552734 +victor xylophone 234.76000308990479 +victor xylophone 267.82999420166016 +victor xylophone 314.95000076293945 +victor young 88.55000305175781 +victor zipper 192.92999649047852 +wendy allen 56.06999969482422 +wendy allen 66.16000306606293 +wendy allen 267.3199996948242 +wendy brown 453.53000259399414 +wendy brown 525.5100040435791 +wendy ellison 193.95000457763672 +wendy ellison 260.9099998474121 +wendy falkner 77.36000061035156 +wendy falkner 97.68000030517578 +wendy falkner 128.30999565124512 +wendy garcia 4.409999847412109 +wendy garcia 76.72000122070312 +wendy garcia 189.42999839782715 +wendy garcia 265.5900001525879 +wendy hernandez 48.11000061035156 +wendy ichabod 104.3700008392334 +wendy king 156.89999771118164 +wendy king 183.31999969482422 +wendy king 403.27000427246094 +wendy laertes 79.98999786376953 +wendy laertes 165.0999984741211 +wendy laertes 365.0 +wendy miller 72.9500002861023 +wendy miller 313.8300037384033 +wendy nixon 45.91999816894531 +wendy nixon 60.2599983215332 +wendy ovid 95.33000183105469 +wendy ovid 180.36000061035156 +wendy polk 386.7400016784668 +wendy polk 443.3400018811226 +wendy quirinius 152.04999828338623 +wendy quirinius 240.23999977111816 +wendy robinson 71.06999969482422 +wendy robinson 249.35000610351562 +wendy robinson 391.4699993133545 +wendy steinbeck 92.11000061035156 +wendy thompson 136.35000228881836 +wendy thompson 183.1500015258789 +wendy underhill 318.6500015258789 +wendy underhill 320.75000190734863 +wendy underhill 328.2300033569336 +wendy van buren 57.459999084472656 +wendy van buren 92.81999969482422 +wendy white 171.36000061035156 +wendy xylophone 153.62999725341797 +wendy xylophone 223.94999885559082 +wendy young 40.22000026702881 +wendy young 513.8299942016602 +xavier allen 102.97000122070312 +xavier allen 168.3300018310547 +xavier allen 197.45999908447266 +xavier brown 55.20000076293945 +xavier brown 90.7300033569336 +xavier brown 96.2300033569336 +xavier carson 193.63999938964844 +xavier carson 265.1600036621094 +xavier davidson 63.349998474121094 +xavier davidson 264.27000427246094 +xavier davidson 288.1999988555908 +xavier ellison 138.42000198364258 +xavier ellison 262.6300048828125 +xavier garcia 148.66000366210938 +xavier hernandez 122.13999938964844 +xavier hernandez 164.97000122070312 +xavier hernandez 306.25 +xavier ichabod 211.84000635147095 +xavier ichabod 244.50000762939453 +xavier johnson 56.53999900817871 +xavier johnson 89.0999984741211 +xavier king 87.22000122070312 +xavier king 151.22999572753906 +xavier laertes 183.65999794006348 +xavier ovid 398.2100067138672 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier polk 261.5100030899048 +xavier polk 318.01000213623047 +xavier quirinius 22.1200008392334 +xavier quirinius 89.55000305175781 +xavier quirinius 246.2400016784668 +xavier quirinius 402.2100009918213 +xavier thompson 283.9400005340576 +xavier underhill 120.45000076293945 +xavier white 138.02999591827393 +xavier white 172.06999969482422 +xavier xylophone 79.41999816894531 +xavier zipper 373.67999935150146 +yuri allen 52.849998474121094 +yuri allen 417.3700008392334 +yuri brown 170.52000427246094 +yuri brown 180.70999908447266 +yuri carson 188.99000549316406 +yuri carson 537.6500015258789 +yuri ellison 86.91999816894531 +yuri ellison 376.32999646663666 +yuri falkner 152.99000358581543 +yuri falkner 181.06999969482422 +yuri garcia 274.6800003051758 +yuri hernandez 153.46999764442444 +yuri johnson 197.28000259399414 +yuri johnson 236.0800018310547 +yuri johnson 258.1899985074997 +yuri king 551.9899978637695 +yuri laertes 37.59000015258789 +yuri laertes 253.4799976348877 +yuri nixon 95.54999732971191 +yuri nixon 248.9700005054474 +yuri polk 82.33999633789062 +yuri polk 275.3200035095215 +yuri polk 305.6399974822998 +yuri quirinius 112.97000122070312 +yuri quirinius 148.27999877929688 +yuri quirinius 449.1699924468994 +yuri steinbeck 292.94000244140625 +yuri steinbeck 357.5 +yuri thompson 428.03999519348145 +yuri underhill 83.87000274658203 +yuri underhill 350.7999897003174 +yuri white 132.09000396728516 +yuri xylophone 107.07000160217285 +zach allen 65.43000030517578 +zach brown 135.6999969482422 +zach brown 247.04999542236328 +zach brown 256.8000030517578 +zach brown 362.38000106811523 +zach brown 418.75 +zach carson 291.7700004577637 +zach ellison 135.149995803833 +zach falkner 91.41999816894531 +zach falkner 196.41999912261963 +zach garcia 84.37999725341797 +zach garcia 160.70000457763672 +zach garcia 167.7599983215332 +zach garcia 205.36999893188477 +zach ichabod 116.2699966430664 +zach ichabod 151.18000030517578 +zach king 127.63000106811523 +zach king 182.2699966430664 +zach king 269.0999984741211 +zach miller 199.71000289916992 +zach miller 220.73999977111816 +zach miller 264.0600047111511 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach ovid 105.94999847561121 +zach ovid 136.04000091552734 +zach quirinius 103.11000061035156 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 182.87000274658203 +zach thompson 116.93999862670898 +zach thompson 319.9499969482422 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 227.52000427246094 +zach xylophone 286.45000076293945 +zach young 313.00999450683594 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 +zach zipper 139.38999938964844 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 new file mode 100644 index 0000000000000..ae2a1e9dd7d3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-3-6f104992e0050576085064815de43194 @@ -0,0 +1,1049 @@ + 43.52666695912679 + 48.735000133514404 + 57.96666590372721 +alice allen 39.21833221117655 +alice allen 42.813999557495116 +alice allen 49.1824996471405 +alice brown 59.68166707456112 +alice carson 46.703333189090095 +alice davidson 59.51833279927572 +alice falkner 64.48333247502644 +alice garcia 55.114999771118164 +alice hernandez 49.61333228151003 +alice hernandez 69.70166667302449 +alice johnson 49.25166805585226 +alice king 20.052499681711197 +alice king 49.019999186197914 +alice king 56.0733331044515 +alice laertes 35.24999898672104 +alice laertes 68.85333251953125 +alice miller 55.542000198364256 +alice nixon 44.010000785191856 +alice nixon 50.900000762939456 +alice nixon 64.78333409627278 +alice ovid 24.90999937057495 +alice polk 47.426666259765625 +alice quirinius 52.80166610081991 +alice quirinius 52.94833393891653 +alice robinson 44.41500027974447 +alice robinson 55.04249954223633 +alice steinbeck 37.735000928243004 +alice steinbeck 41.02800045013428 +alice steinbeck 53.83499916394552 +alice underhill 52.64500021934509 +alice van buren 48.27666727701823 +alice xylophone 28.047500252723694 +alice xylophone 37.82199954986572 +alice xylophone 59.75166606903076 +alice zipper 48.875000635782875 +alice zipper 51.365000108877815 +alice zipper 89.93000030517578 +bob brown 38.134999910990395 +bob brown 56.20666662851969 +bob brown 77.51166598002116 +bob carson 52.77200050354004 +bob davidson 28.945000171661377 +bob davidson 37.946666399637856 +bob davidson 53.90333271026611 +bob ellison 51.383999824523926 +bob ellison 57.30333344141642 +bob ellison 58.53666559855143 +bob ellison 66.7400016784668 +bob falkner 39.053333600362144 +bob garcia 16.705000400543213 +bob garcia 37.16499960422516 +bob garcia 51.17999919255575 +bob garcia 52.37666575113932 +bob garcia 67.39166768391927 +bob hernandez 53.261999893188474 +bob ichabod 43.96999979019165 +bob king 38.03666718800863 +bob king 38.19249892234802 +bob king 67.40999794006348 +bob laertes 21.449999809265137 +bob laertes 50.37166612346967 +bob miller 41.906000471115114 +bob ovid 27.836666425069172 +bob ovid 39.3833335240682 +bob ovid 60.90749979019165 +bob ovid 63.069999313354494 +bob polk 41.88333296775818 +bob quirinius 54.504998207092285 +bob steinbeck 34.33999983469645 +bob van buren 58.29999923706055 +bob white 30.22333288192749 +bob white 41.44000005722046 +bob xylophone 12.163333415985107 +bob xylophone 47.88249921798706 +bob young 27.38499927520752 +bob zipper 27.93800084590912 +bob zipper 59.11800007820129 +bob zipper 66.43000030517578 +calvin allen 71.51000118255615 +calvin brown 50.44166612625122 +calvin brown 53.625000953674316 +calvin brown 67.48333485921223 +calvin carson 62.17750120162964 +calvin davidson 14.03000009059906 +calvin davidson 43.640000343322754 +calvin ellison 52.52750015258789 +calvin falkner 24.016666332880657 +calvin falkner 46.87333329518636 +calvin falkner 47.53666607538859 +calvin falkner 56.47499918937683 +calvin falkner 57.336666107177734 +calvin falkner 72.25 +calvin garcia 46.484999338785805 +calvin hernandez 43.90199909210205 +calvin johnson 76.41500091552734 +calvin laertes 39.09249973297119 +calvin laertes 47.323333422342934 +calvin nixon 30.113332668940227 +calvin nixon 47.7533327738444 +calvin nixon 49.08500075340271 +calvin ovid 41.924999713897705 +calvin ovid 49.10499978065491 +calvin ovid 62.26499938964844 +calvin ovid 62.27999989191691 +calvin polk 52.95333353678385 +calvin quirinius 53.34200019836426 +calvin quirinius 54.831998634338376 +calvin robinson 60.470001220703125 +calvin steinbeck 35.644999980926514 +calvin steinbeck 53.75800037384033 +calvin steinbeck 56.11000084877014 +calvin thompson 42.355000257492065 +calvin thompson 65.91666666666667 +calvin underhill 47.41199951171875 +calvin van buren 31.591666102409363 +calvin van buren 43.18200063705444 +calvin white 45.27500128746033 +calvin white 56.079999923706055 +calvin xylophone 20.394999980926514 +calvin xylophone 40.59999958674113 +calvin xylophone 54.56500005722046 +calvin young 42.246666272481285 +calvin young 64.49000072479248 +calvin zipper 57.49500036239624 +calvin zipper 57.59000039100647 +david allen 40.14999977747599 +david allen 54.545000076293945 +david brown 35.4516666730245 +david brown 62.83999938964844 +david davidson 35.50000067551931 +david davidson 52.75250005722046 +david davidson 74.1500015258789 +david davidson 95.80999755859375 +david ellison 43.43166727821032 +david ellison 52.18750047683716 +david ellison 72.58400039672851 +david hernandez 64.47600173950195 +david ichabod 29.518332719802856 +david ichabod 34.6100010573864 +david laertes 61.127999496459964 +david nixon 53.60249900817871 +david ovid 37.977500796318054 +david ovid 41.58999983469645 +david quirinius 24.96250009536743 +david quirinius 45.23249912261963 +david quirinius 52.282000350952146 +david robinson 42.17750024795532 +david robinson 62.54666519165039 +david thompson 41.88999938964844 +david underhill 48.143333752950035 +david underhill 62.53499941031138 +david underhill 97.55999755859375 +david van buren 34.84749984741211 +david van buren 51.61666679382324 +david white 62.30499839782715 +david xylophone 33.92500019073486 +david xylophone 53.142000222206114 +david xylophone 72.77166684468587 +david young 30.13666756947835 +david young 51.540000915527344 +ethan allen 53.442500591278076 +ethan brown 7.110000133514404 +ethan brown 41.260000228881836 +ethan brown 41.495000084241234 +ethan brown 41.83833312988281 +ethan brown 46.497499227523804 +ethan brown 63.17750072479248 +ethan carson 64.31999969482422 +ethan ellison 46.72599983215332 +ethan ellison 61.24750167876482 +ethan falkner 36.90250104665756 +ethan falkner 52.71000099182129 +ethan garcia 32.771666407585144 +ethan hernandez 56.239999008178714 +ethan johnson 82.38333384195964 +ethan king 8.399999856948853 +ethan laertes 37.93200054168701 +ethan laertes 40.4883329073588 +ethan laertes 53.396667132774986 +ethan laertes 53.81666628519694 +ethan laertes 55.987499713897705 +ethan laertes 66.36999956766765 +ethan laertes 68.83400039672851 +ethan miller 56.519999504089355 +ethan nixon 50.13333353648583 +ethan ovid 37.51333363850912 +ethan polk 2.3499999046325684 +ethan polk 44.147999954223636 +ethan polk 54.91499996185303 +ethan polk 60.340000788370766 +ethan quirinius 40.21750020980835 +ethan quirinius 47.125000298023224 +ethan quirinius 57.69000013669332 +ethan robinson 41.5060001373291 +ethan robinson 55.0640007019043 +ethan underhill 68.01250076293945 +ethan van buren 43.55250072479248 +ethan white 46.40999913215637 +ethan white 53.03200073242188 +ethan xylophone 70.29333432515462 +ethan zipper 46.92999982833862 +ethan zipper 66.04500198364258 +fred davidson 26.487499618902802 +fred davidson 44.99333477020264 +fred davidson 53.93666648864746 +fred ellison 22.2599999109904 +fred ellison 48.047999954223634 +fred ellison 63.423333485921226 +fred falkner 20.035000324249268 +fred falkner 44.243999004364014 +fred falkner 62.130001068115234 +fred hernandez 37.42199997901916 +fred ichabod 46.56999969482422 +fred ichabod 58.275001525878906 +fred johnson 54.63333257039388 +fred king 46.30499919255575 +fred king 61.48500037193298 +fred laertes 26.203333059946697 +fred miller 43.46400032043457 +fred nixon 28.690000534057617 +fred nixon 32.77999955415726 +fred nixon 52.03799936771393 +fred nixon 60.468332290649414 +fred polk 26.81166632970174 +fred polk 42.48999996185303 +fred polk 46.95200061798096 +fred polk 60.465998840332034 +fred quirinius 43.466000366210935 +fred quirinius 45.79799957275391 +fred robinson 62.42833296457926 +fred steinbeck 32.22999954223633 +fred steinbeck 48.73800039291382 +fred steinbeck 65.91750144958496 +fred underhill 72.94250011444092 +fred van buren 41.28250002861023 +fred van buren 41.5283338278532 +fred van buren 50.871665954589844 +fred van buren 81.77000045776367 +fred white 34.897499561309814 +fred young 58.56666692097982 +fred young 70.61499977111816 +fred zipper 33.90500068664551 +gabriella allen 51.041666666666664 +gabriella allen 55.13999938964844 +gabriella brown 54.63333320617676 +gabriella brown 72.33333396911621 +gabriella carson 49.26333236694336 +gabriella davidson 52.72800083160401 +gabriella ellison 54.68999989827474 +gabriella ellison 71.54000091552734 +gabriella falkner 20.834000015258788 +gabriella falkner 46.348333517710365 +gabriella falkner 48.58500099182129 +gabriella garcia 24.78999964396159 +gabriella hernandez 57.61000029246012 +gabriella hernandez 65.11166667938232 +gabriella ichabod 33.78749895095825 +gabriella ichabod 38.9950008392334 +gabriella ichabod 48.15499925613403 +gabriella ichabod 49.739999008178714 +gabriella ichabod 52.789999643961586 +gabriella king 35.58200044631958 +gabriella king 49.63000059723854 +gabriella laertes 47.81000010172526 +gabriella miller 62.17666753133138 +gabriella ovid 45.94333267211914 +gabriella ovid 50.435001373291016 +gabriella polk 42.58000100851059 +gabriella polk 72.87000179290771 +gabriella steinbeck 65.42000102996826 +gabriella steinbeck 72.0499997138977 +gabriella thompson 49.897499561309814 +gabriella thompson 52.84250023961067 +gabriella thompson 57.23249959945679 +gabriella van buren 39.31000053882599 +gabriella van buren 57.920000076293945 +gabriella white 49.85333410898844 +gabriella young 15.369999885559082 +gabriella young 49.04999907811483 +gabriella zipper 42.82499901453654 +gabriella zipper 59.58499972025553 +holly allen 29.50499963760376 +holly brown 55.284998178482056 +holly brown 55.79833386838436 +holly falkner 40.07666663328806 +holly hernandez 46.40500044822693 +holly hernandez 56.14500045776367 +holly hernandez 56.160000483194985 +holly hernandez 58.95166748017073 +holly ichabod 67.19250011444092 +holly ichabod 68.44500064849854 +holly ichabod 90.09000015258789 +holly johnson 42.795000076293945 +holly johnson 55.76999855041504 +holly johnson 66.11333338419597 +holly king 48.788000869750974 +holly king 64.25 +holly laertes 50.40750074386597 +holly miller 45.60666608810425 +holly nixon 58.096665700276695 +holly nixon 69.59249877929688 +holly polk 41.59666601816813 +holly polk 44.64799900054932 +holly robinson 48.405999755859376 +holly thompson 34.605000495910645 +holly thompson 38.36749941110611 +holly thompson 65.99833394338687 +holly underhill 46.52600040435791 +holly underhill 47.92666663726171 +holly underhill 62.41600036621094 +holly underhill 79.95999908447266 +holly van buren 58.63333225250244 +holly white 31.232499361038208 +holly white 61.494998931884766 +holly xylophone 70.32500044504802 +holly young 54.05000114440918 +holly young 57.103333473205566 +holly zipper 50.59500014781952 +holly zipper 67.81000137329102 +irene allen 53.364000701904295 +irene brown 4.789999961853027 +irene brown 51.65399932861328 +irene brown 87.66999816894531 +irene carson 59.2433336575826 +irene ellison 36.446667989095054 +irene ellison 53.85000038146973 +irene falkner 47.429999995231626 +irene falkner 61.20666694641113 +irene garcia 15.369999885559082 +irene garcia 48.28000005086263 +irene garcia 55.643333435058594 +irene ichabod 40.868333299954735 +irene ichabod 64.45666694641113 +irene johnson 44.37999935150147 +irene laertes 27.625 +irene laertes 46.25 +irene laertes 49.17500019073486 +irene miller 75.87500063578288 +irene nixon 29.780000686645508 +irene nixon 30.070000807444256 +irene nixon 36.34333356221517 +irene ovid 27.21500023206075 +irene ovid 55.6175012588501 +irene ovid 60.353999328613284 +irene polk 47.438334147135414 +irene polk 47.77666505177816 +irene polk 51.74200067520142 +irene polk 52.12200088500977 +irene polk 52.184000205993655 +irene quirinius 52.958333015441895 +irene quirinius 55.29000053405762 +irene quirinius 78.79000091552734 +irene robinson 61.398332595825195 +irene steinbeck 64.34000142415364 +irene thompson 41.92250097543001 +irene underhill 27.72499966621399 +irene underhill 50.783331871032715 +irene van buren 46.65999984741211 +irene van buren 49.71000099182129 +irene xylophone 61.220001220703125 +jessica brown 63.449999491373696 +jessica carson 41.78999951481819 +jessica carson 47.52000069618225 +jessica carson 51.83000183105469 +jessica davidson 34.938333332538605 +jessica davidson 45.91000066200892 +jessica davidson 51.082499504089355 +jessica davidson 64.3099988301595 +jessica ellison 32.53000124295553 +jessica ellison 35.265999984741214 +jessica falkner 54.933334032694496 +jessica garcia 32.575000286102295 +jessica garcia 60.584999084472656 +jessica ichabod 46.704999923706055 +jessica johnson 55.73199977874756 +jessica johnson 72.14000066121419 +jessica miller 55.27799835205078 +jessica nixon 58.53999900817871 +jessica nixon 90.06999969482422 +jessica ovid 36.47500014305115 +jessica ovid 63.03000005086263 +jessica polk 69.52999877929688 +jessica quirinius 19.01333288351695 +jessica quirinius 41.630000829696655 +jessica quirinius 42.58750009536743 +jessica quirinius 47.72999986012777 +jessica robinson 64.81000061035157 +jessica thompson 45.83666737874349 +jessica thompson 57.540000915527344 +jessica underhill 45.360000133514404 +jessica underhill 51.40999889373779 +jessica underhill 64.27250099182129 +jessica van buren 9.739999771118164 +jessica white 38.38999891281128 +jessica white 40.75400023460388 +jessica white 61.89800128936768 +jessica white 62.34749984741211 +jessica white 65.7680004119873 +jessica xylophone 50.808332761128746 +jessica young 18.400000254313152 +jessica young 49.04400033950806 +jessica zipper 35.039999643961586 +jessica zipper 52.78999869028727 +jessica zipper 55.51499891281128 +katie allen 56.10800056457519 +katie brown 48.86833222707113 +katie davidson 96.91000366210938 +katie ellison 31.106667200724285 +katie ellison 38.356666127840676 +katie falkner 17.24333318074544 +katie garcia 53.429999033610024 +katie garcia 53.933334002892174 +katie hernandez 72.71666717529297 +katie ichabod 39.97999954223633 +katie ichabod 54.99599914550781 +katie ichabod 59.41999944051107 +katie king 44.88599967956543 +katie king 60.2549991607666 +katie king 67.27499961853027 +katie miller 31.399999618530273 +katie miller 50.57666619618734 +katie nixon 23.190000534057617 +katie ovid 52.96000158786774 +katie polk 51.029998779296875 +katie polk 54.93600006103516 +katie robinson 13.890000343322754 +katie van buren 52.78999908765157 +katie van buren 53.21750092506409 +katie white 59.799998950958255 +katie white 64.70499992370605 +katie xylophone 53.54499959945679 +katie young 31.010000228881836 +katie young 37.55500102043152 +katie young 49.87499872843424 +katie zipper 29.79666694998741 +katie zipper 50.95000076293945 +luke allen 43.757998657226565 +luke allen 53.18000030517578 +luke allen 62.09749984741211 +luke allen 70.2933349609375 +luke allen 89.55000305175781 +luke brown 45.95999972025553 +luke davidson 28.950000762939453 +luke davidson 53.20500040054321 +luke ellison 5.159999907016754 +luke ellison 18.630000392595928 +luke ellison 76.88000106811523 +luke falkner 6.175000190734863 +luke falkner 54.0049991607666 +luke garcia 15.320000424981117 +luke garcia 25.47000026702881 +luke ichabod 33.950000286102295 +luke ichabod 57.17500114440918 +luke johnson 14.4399995803833 +luke johnson 37.23333263397217 +luke johnson 52.660000801086426 +luke laertes 11.819999694824219 +luke laertes 23.5 +luke laertes 43.71999907493591 +luke laertes 45.9900016784668 +luke laertes 57.85499906539917 +luke miller 67.85000038146973 +luke ovid 38.04999923706055 +luke ovid 79.64200134277344 +luke polk 55.106666564941406 +luke polk 70.18999862670898 +luke quirinius 26.016666491826374 +luke robinson 61.256665547688804 +luke robinson 72.61999893188477 +luke thompson 47.203332940737404 +luke underhill 49.244999408721924 +luke underhill 59.32999897003174 +luke underhill 72.18333307902019 +luke van buren 64.646666208903 +luke white 67.12000274658203 +luke xylophone 46.74333190917969 +luke zipper 36.16000008583069 +mike allen 10.709999859333038 +mike brown 61.0674991607666 +mike carson 50.90249824523926 +mike carson 52.30200061798096 +mike carson 52.51499938964844 +mike davidson 29.733333269755047 +mike davidson 47.396666526794434 +mike ellison 37.807498931884766 +mike ellison 44.339999198913574 +mike ellison 45.89199924468994 +mike ellison 47.91249990463257 +mike ellison 55.295000076293945 +mike falkner 16.479999542236328 +mike garcia 38.98999913533529 +mike garcia 54.91333325703939 +mike garcia 57.096666971842446 +mike hernandez 59.45000076293945 +mike hernandez 62.446667989095054 +mike ichabod 54.69666544596354 +mike king 38.790000915527344 +mike king 43.2050017118454 +mike king 46.85666592915853 +mike king 46.945000648498535 +mike king 53.382500648498535 +mike king 94.68000030517578 +mike miller 3.9600000381469727 +mike nixon 53.894999742507935 +mike nixon 92.95999908447266 +mike polk 12.449999809265137 +mike polk 39.040000319480896 +mike polk 71.40500068664551 +mike quirinius 89.37999725341797 +mike steinbeck 34.05499875545502 +mike steinbeck 45.909999465942384 +mike steinbeck 51.286667346954346 +mike steinbeck 59.292500019073486 +mike van buren 43.13999819755554 +mike van buren 49.59000015258789 +mike white 30.53999964396159 +mike white 34.61499961217245 +mike white 48.08999943733215 +mike white 53.967501163482666 +mike young 10.484999895095825 +mike young 47.070000648498535 +mike young 74.58999633789062 +mike zipper 29.029999288419884 +mike zipper 77.96200027465821 +mike zipper 91.57500076293945 +nick allen 29.146666367848713 +nick allen 64.4325008392334 +nick brown 49.429999669392906 +nick davidson 34.36599922180176 +nick ellison 49.345001220703125 +nick ellison 89.01000213623047 +nick falkner 7.820000171661377 +nick falkner 45.025000631809235 +nick garcia 23.9499994913737 +nick garcia 33.49333349863688 +nick garcia 64.33499908447266 +nick ichabod 30.945000171661377 +nick ichabod 56.27499961853027 +nick ichabod 59.24249863624573 +nick johnson 32.3674995303154 +nick johnson 74.30666859944661 +nick laertes 38.38749980926514 +nick miller 82.97000122070312 +nick nixon 70.01333173116048 +nick ovid 56.82999897003174 +nick polk 33.929999669392906 +nick quirinius 58.91199932098389 +nick quirinius 81.16999816894531 +nick robinson 34.53999948501587 +nick robinson 59.80499839782715 +nick steinbeck 57.25333340962728 +nick thompson 51.3725004196167 +nick underhill 55.47333335876465 +nick van buren 74.21666717529297 +nick xylophone 51.82000001271566 +nick young 0.27000001072883606 +nick young 41.02499961853027 +nick zipper 47.72999954223633 +nick zipper 61.917999267578125 +oscar allen 35.21600015163422 +oscar brown 38.03999948501587 +oscar carson 38.09600009918213 +oscar carson 51.34333419799805 +oscar carson 56.9925012588501 +oscar carson 74.00500106811523 +oscar carson 98.51000213623047 +oscar davidson 65.88750076293945 +oscar ellison 50.507500410079956 +oscar ellison 66.57000096638997 +oscar falkner 64.42000198364258 +oscar garcia 66.36333465576172 +oscar hernandez 42.089999516805015 +oscar hernandez 51.21199997067451 +oscar ichabod 25.300000190734863 +oscar ichabod 41.173332850138344 +oscar ichabod 53.44666830698649 +oscar ichabod 71.80000305175781 +oscar johnson 39.9366668065389 +oscar johnson 44.30500118434429 +oscar king 30.59749937057495 +oscar king 46.149999141693115 +oscar king 49.65999984741211 +oscar laertes 5.510000228881836 +oscar laertes 19.099999745686848 +oscar laertes 39.36250039935112 +oscar laertes 45.340000788370766 +oscar nixon 41.619998931884766 +oscar ovid 45.09000015258789 +oscar ovid 82.23999786376953 +oscar ovid 83.47750091552734 +oscar polk 42.54333368937174 +oscar polk 56.8033332824707 +oscar quirinius 52.94500160217285 +oscar quirinius 65.34666570027669 +oscar quirinius 67.00000127156575 +oscar quirinius 79.4800033569336 +oscar robinson 32.16999944051107 +oscar robinson 38.880001068115234 +oscar robinson 63.9433339436849 +oscar robinson 65.80500030517578 +oscar steinbeck 45.73599967956543 +oscar thompson 40.33599853515625 +oscar thompson 47.860000928243004 +oscar thompson 60.529998779296875 +oscar thompson 60.59333292643229 +oscar underhill 43.980000764131546 +oscar van buren 2.180000066757202 +oscar van buren 53.34999942779541 +oscar van buren 61.880001068115234 +oscar white 38.0633331934611 +oscar white 42.584999084472656 +oscar white 55.179999033610024 +oscar white 74.4900016784668 +oscar xylophone 41.3133331934611 +oscar xylophone 55.5049991607666 +oscar xylophone 67.54500007629395 +oscar zipper 15.680000305175781 +oscar zipper 24.019999504089355 +oscar zipper 39.81999969482422 +priscilla brown 42.88666502634684 +priscilla brown 63.242499351501465 +priscilla brown 77.56999969482422 +priscilla carson 44.799999713897705 +priscilla carson 49.663333892822266 +priscilla carson 78.57333374023438 +priscilla ichabod 56.83666737874349 +priscilla ichabod 58.48666636149088 +priscilla johnson 34.416667779286705 +priscilla johnson 52.890000343322754 +priscilla johnson 53.72666883468628 +priscilla johnson 61.939998626708984 +priscilla johnson 89.1500015258789 +priscilla king 34.30750045180321 +priscilla nixon 27.734999656677246 +priscilla nixon 60.90999984741211 +priscilla ovid 48.13500142097473 +priscilla ovid 66.35999870300293 +priscilla polk 15.149999618530273 +priscilla quirinius 18.606666564941406 +priscilla thompson 48.87000131607056 +priscilla underhill 40.05500078201294 +priscilla underhill 49.54999961853027 +priscilla van buren 42.62666734059652 +priscilla van buren 61.54000017642975 +priscilla van buren 72.80500030517578 +priscilla white 43.177499771118164 +priscilla xylophone 21.489999771118164 +priscilla xylophone 40.144999124109745 +priscilla xylophone 59.61000061035156 +priscilla young 31.610000324249267 +priscilla young 53.71000152826309 +priscilla zipper 18.8799991607666 +priscilla zipper 25.670000076293945 +quinn allen 56.77749991416931 +quinn allen 76.47250080108643 +quinn brown 23.536666870117188 +quinn brown 31.829999446868896 +quinn brown 50.388000297546384 +quinn davidson 41.42499923706055 +quinn davidson 45.90250015258789 +quinn davidson 48.16000032424927 +quinn davidson 71.51000022888184 +quinn ellison 50.6766668955485 +quinn ellison 62.32666842142741 +quinn garcia 39.98599967956543 +quinn garcia 43.27000045776367 +quinn garcia 74.31999969482422 +quinn garcia 92.33000183105469 +quinn ichabod 44.81999969482422 +quinn king 50.99666786193848 +quinn king 74.62000274658203 +quinn laertes 5.884999990463257 +quinn laertes 49.1379997253418 +quinn laertes 56.18000030517578 +quinn nixon 74.66500091552734 +quinn ovid 1.2100000381469727 +quinn quirinius 46.15500068664551 +quinn robinson 44.96249985694885 +quinn steinbeck 24.06999921798706 +quinn steinbeck 41.750000953674316 +quinn thompson 23.744999408721924 +quinn thompson 73.64500045776367 +quinn underhill 41.83333269755045 +quinn underhill 49.63749885559082 +quinn underhill 83.85000228881836 +quinn van buren 54.36333338419596 +quinn young 45.489999771118164 +quinn zipper 22.25 +quinn zipper 33.355000019073486 +rachel allen 15.8100004196167 +rachel allen 71.57666524251302 +rachel brown 2.9600000381469727 +rachel brown 33.022499561309814 +rachel brown 42.442498207092285 +rachel brown 59.21999931335449 +rachel brown 64.52666727701823 +rachel carson 61.17999776204427 +rachel carson 69.85333315531413 +rachel davidson 42.292500495910645 +rachel ellison 10.600000381469727 +rachel falkner 23.615000784397125 +rachel falkner 50.46250069141388 +rachel falkner 54.669999837875366 +rachel falkner 72.96999740600586 +rachel johnson 61.76249885559082 +rachel king 22.005000591278076 +rachel king 66.46500015258789 +rachel laertes 39.025001525878906 +rachel laertes 45.45000076293945 +rachel ovid 0.6000000238418579 +rachel ovid 1.0800000429153442 +rachel polk 78.08499908447266 +rachel quirinius 51.12000131607056 +rachel robinson 30.360000610351562 +rachel robinson 37.683333237965904 +rachel robinson 39.199998219807945 +rachel thompson 0.5600000023841858 +rachel thompson 38.81000010172526 +rachel thompson 49.772499561309814 +rachel underhill 48.45000076293945 +rachel white 42.83999991416931 +rachel white 58.609999656677246 +rachel young 43.130001068115234 +rachel zipper 7.059999942779541 +rachel zipper 49.676667173703514 +sarah carson 54.022500067949295 +sarah carson 54.18333212534586 +sarah carson 87.81000137329102 +sarah ellison 16.989999771118164 +sarah falkner 73.06500005722046 +sarah falkner 99.36000061035156 +sarah garcia 38.43250036239624 +sarah garcia 64.97333272298177 +sarah garcia 73.6500015258789 +sarah ichabod 57.46000003814697 +sarah ichabod 81.31999969482422 +sarah johnson 34.98499917984009 +sarah johnson 45.099998474121094 +sarah johnson 74.42249870300293 +sarah johnson 74.72000122070312 +sarah king 41.869998931884766 +sarah king 48.25 +sarah miller 41.53499984741211 +sarah ovid 33.38000011444092 +sarah robinson 33.83000183105469 +sarah robinson 66.88999938964844 +sarah steinbeck 31.023332993189495 +sarah white 45.974998474121094 +sarah white 61.54666709899902 +sarah xylophone 61.60666529337565 +sarah young 45.560001373291016 +sarah zipper 60.69000053405762 +tom brown 40.08000135421753 +tom brown 55.44499969482422 +tom carson 5.440000057220459 +tom carson 26.32499885559082 +tom carson 31.476666768391926 +tom davidson 53.00749921798706 +tom ellison 67.32666714986165 +tom ellison 76.73999786376953 +tom ellison 77.29499816894531 +tom falkner 60.130001068115234 +tom falkner 88.22000122070312 +tom hernandez 41.36000061035156 +tom hernandez 81.63999938964844 +tom ichabod 42.08666547139486 +tom johnson 33.75999959309896 +tom johnson 53.47666549682617 +tom king 40.0 +tom laertes 32.05000114440918 +tom laertes 43.46000099182129 +tom miller 21.229999542236328 +tom miller 43.37666702270508 +tom miller 44.41333325703939 +tom nixon 46.35333251953125 +tom ovid 43.04499912261963 +tom polk 38.29999923706055 +tom polk 54.45000076293945 +tom quirinius 19.82000058889389 +tom quirinius 22.580000400543213 +tom robinson 66.16999816894531 +tom robinson 74.18666712443034 +tom robinson 80.77000045776367 +tom robinson 98.72000122070312 +tom steinbeck 44.999999046325684 +tom van buren 31.103334546089172 +tom van buren 40.779998779296875 +tom van buren 63.5099983215332 +tom white 49.06500053405762 +tom young 54.16999912261963 +tom young 78.54999923706055 +tom zipper 48.666666984558105 +ulysses brown 72.79000091552734 +ulysses carson 40.28500175476074 +ulysses carson 71.55000305175781 +ulysses carson 77.41999816894531 +ulysses carson 79.54000091552734 +ulysses davidson 41.90166711807251 +ulysses ellison 96.7300033569336 +ulysses garcia 89.80000305175781 +ulysses hernandez 21.339999516805012 +ulysses hernandez 54.470001220703125 +ulysses hernandez 80.11000061035156 +ulysses ichabod 19.1299991607666 +ulysses ichabod 98.56999969482422 +ulysses johnson 42.55000019073486 +ulysses king 81.57000223795573 +ulysses laertes 1.9199999570846558 +ulysses laertes 11.890000343322754 +ulysses laertes 53.599998474121094 +ulysses miller 40.43499946594238 +ulysses miller 47.78200054168701 +ulysses nixon 74.60333251953125 +ulysses ovid 30.940000534057617 +ulysses polk 8.710000038146973 +ulysses polk 39.17500019073486 +ulysses polk 65.0199966430664 +ulysses polk 68.41333389282227 +ulysses quirinius 65.93499946594238 +ulysses robinson 79.48999786376953 +ulysses steinbeck 32.40999984741211 +ulysses steinbeck 43.93499994277954 +ulysses thompson 79.96000289916992 +ulysses underhill 17.85000006357829 +ulysses underhill 42.5533332824707 +ulysses underhill 46.60000157356262 +ulysses underhill 47.84000049829483 +ulysses underhill 51.85000133514404 +ulysses underhill 67.77999877929688 +ulysses underhill 99.66999816894531 +ulysses van buren 69.89999961853027 +ulysses white 45.01500183343887 +ulysses white 71.3933334350586 +ulysses xylophone 27.880000591278076 +ulysses xylophone 39.689998626708984 +ulysses xylophone 54.099998474121094 +ulysses young 32.52000045776367 +ulysses young 39.81333382924398 +ulysses young 80.7933349609375 +victor allen 44.27000045776367 +victor allen 57.994998931884766 +victor brown 56.84499931335449 +victor brown 59.34000015258789 +victor brown 90.37999725341797 +victor brown 91.97000122070312 +victor davidson 52.07000160217285 +victor davidson 54.239999771118164 +victor davidson 68.8033332824707 +victor ellison 45.45750088989735 +victor ellison 58.42999839782715 +victor hernandez 23.164999961853027 +victor hernandez 57.98499870300293 +victor hernandez 59.619998931884766 +victor hernandez 71.42499923706055 +victor hernandez 80.19499969482422 +victor johnson 1.5800000429153442 +victor johnson 46.7450008392334 +victor johnson 72.59000015258789 +victor king 37.559998989105225 +victor king 47.880001068115234 +victor laertes 50.0099983215332 +victor laertes 70.13999938964844 +victor miller 22.1200008392334 +victor nixon 34.029998779296875 +victor nixon 37.08500003814697 +victor ovid 18.815000653266907 +victor polk 3.0 +victor quirinius 26.450000405311584 +victor quirinius 33.080001533031464 +victor robinson 4.590000152587891 +victor robinson 41.21999931335449 +victor steinbeck 41.82500123977661 +victor steinbeck 46.09000015258789 +victor steinbeck 52.720001220703125 +victor thompson 45.346666971842446 +victor van buren 34.970001220703125 +victor van buren 46.57333246866862 +victor white 5.670000076293945 +victor white 74.16999816894531 +victor xylophone 11.220000267028809 +victor xylophone 28.954999923706055 +victor xylophone 34.010000228881836 +victor xylophone 43.179999351501465 +victor xylophone 62.38999938964844 +victor young 70.91000175476074 +victor zipper 48.795000076293945 +wendy allen 0.6100000143051147 +wendy allen 52.64999961853027 +wendy allen 56.06999969482422 +wendy brown 51.874999046325684 +wendy brown 66.73999913533528 +wendy ellison 27.014999389648438 +wendy ellison 94.66000366210938 +wendy falkner 14.425000190734863 +wendy falkner 77.36000061035156 +wendy falkner 85.68000030517578 +wendy garcia 4.409999847412109 +wendy garcia 20.390000343322754 +wendy garcia 38.42500060796738 +wendy garcia 57.25 +wendy hernandez 48.11000061035156 +wendy ichabod 13.149999618530273 +wendy king 33.234999656677246 +wendy king 74.97999954223633 +wendy king 87.94000244140625 +wendy laertes 49.01999855041504 +wendy laertes 54.750000635782875 +wendy laertes 79.98999786376953 +wendy miller 1.2699999809265137 +wendy miller 47.41500186920166 +wendy nixon 30.17999916151166 +wendy nixon 45.91999816894531 +wendy ovid 85.26000213623047 +wendy ovid 86.62999725341797 +wendy polk 43.679999669392906 +wendy polk 54.65333424011866 +wendy quirinius 12.15999984741211 +wendy quirinius 14.300000190734863 +wendy robinson 58.815001487731934 +wendy robinson 71.06999969482422 +wendy robinson 71.77999877929688 +wendy steinbeck 77.31500053405762 +wendy thompson 58.09000015258789 +wendy thompson 85.76000213623047 +wendy underhill 41.23333231608073 +wendy underhill 49.33500003814697 +wendy underhill 81.56000137329102 +wendy van buren 57.459999084472656 +wendy van buren 91.43500137329102 +wendy white 73.68000030517578 +wendy xylophone 31.149999618530273 +wendy xylophone 76.81499862670898 +wendy young 8.449999809265137 +wendy young 33.7599983215332 +xavier allen 45.68000030517578 +xavier allen 52.255000591278076 +xavier allen 83.93000030517578 +xavier brown 7.789999961853027 +xavier brown 77.15500259399414 +xavier brown 80.35000228881836 +xavier carson 20.790000915527344 +xavier carson 55.30000019073486 +xavier davidson 15.920000076293945 +xavier davidson 58.46999931335449 +xavier davidson 82.41000366210938 +xavier ellison 16.614999771118164 +xavier ellison 77.97000122070312 +xavier garcia 49.415000915527344 +xavier hernandez 6.670000076293945 +xavier hernandez 23.054999828338623 +xavier hernandez 67.26000213623047 +xavier ichabod 4.71999979019165 +xavier ichabod 71.19000244140625 +xavier johnson 27.299999237060547 +xavier johnson 89.0999984741211 +xavier king 22.729999542236328 +xavier king 87.22000122070312 +xavier laertes 24.050000190734863 +xavier ovid 58.08000183105469 +xavier polk 13.869999885559082 +xavier polk 58.98750066757202 +xavier polk 72.62000274658203 +xavier polk 76.93000030517578 +xavier quirinius 22.1200008392334 +xavier quirinius 58.24666786193848 +xavier quirinius 62.52000045776367 +xavier quirinius 89.55000305175781 +xavier thompson 9.930000305175781 +xavier underhill 47.27000045776367 +xavier white 8.369999885559082 +xavier white 75.29000091552734 +xavier xylophone 79.41999816894531 +xavier zipper 8.449999809265137 +yuri allen 30.6299991607666 +yuri allen 52.849998474121094 +yuri brown 75.19000244140625 +yuri brown 84.02999877929688 +yuri carson 6.289999961853027 +yuri carson 91.16000366210938 +yuri ellison 1.1200000047683716 +yuri ellison 86.91999816894531 +yuri falkner 6.739999771118164 +yuri falkner 80.8650016784668 +yuri garcia 27.65999984741211 +yuri hernandez 2.069999933242798 +yuri johnson 0.12999999523162842 +yuri johnson 39.900001525878906 +yuri johnson 48.220001220703125 +yuri king 22.270000457763672 +yuri laertes 10.15999984741211 +yuri laertes 37.59000015258789 +yuri nixon 2.200000047683716 +yuri nixon 17.3700008392334 +yuri polk 26.760000228881836 +yuri polk 28.790000915527344 +yuri polk 82.33999633789062 +yuri quirinius 10.260000228881836 +yuri quirinius 38.69000053405762 +yuri quirinius 57.93000030517578 +yuri steinbeck 56.064998626708984 +yuri steinbeck 75.87999725341797 +yuri thompson 14.920000076293945 +yuri underhill 23.770000457763672 +yuri underhill 83.87000274658203 +yuri white 34.58000183105469 +yuri xylophone 20.3799991607666 +zach allen 65.43000030517578 +zach brown 48.0099983215332 +zach brown 57.08000183105469 +zach brown 58.24999809265137 +zach brown 67.37999725341797 +zach brown 75.7300033569336 +zach carson 67.78500175476074 +zach ellison 6.840000152587891 +zach falkner 50.274999141693115 +zach falkner 91.41999816894531 +zach garcia 32.20000076293945 +zach garcia 35.79999923706055 +zach garcia 69.97000122070312 +zach garcia 84.37999725341797 +zach ichabod 36.88999938964844 +zach ichabod 64.25 +zach king 31.864999771118164 +zach king 46.18000030517578 +zach king 86.93000030517578 +zach miller 2.5999999046325684 +zach miller 21.280000686645508 +zach miller 53.27000045776367 +zach ovid 0.10000000149011612 +zach ovid 23.06999969482422 +zach ovid 92.55000305175781 +zach ovid 94.33999633789062 +zach quirinius 39.209999084472656 +zach robinson 76.72000122070312 +zach steinbeck 85.48999786376953 +zach steinbeck 90.05000305175781 +zach thompson 53.59000015258789 +zach thompson 71.5 +zach underhill 86.22000122070312 +zach white 70.52999877929688 +zach xylophone 29.40999984741211 +zach xylophone 71.01000213623047 +zach young 71.31999969482422 +zach zipper 7.539999961853027 +zach zipper 85.87000274658203 +zach zipper 94.43000030517578 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e new file mode 100644 index 0000000000000..ee1c26e331a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-4-cd2e3d2344810cb3ba843d4c01c81d7e @@ -0,0 +1,1049 @@ + 17.601666666666667 + 30.72 + 33.07833333333334 +alice allen 23.081666666666667 +alice allen 23.263333333333332 +alice allen 31.38 +alice brown 11.518333333333333 +alice carson 31.99166666666667 +alice davidson 26.986666666666665 +alice falkner 30.513333333333335 +alice garcia 17.13 +alice hernandez 28.126666666666665 +alice hernandez 28.69666666666667 +alice johnson 28.30333333333333 +alice king 23.451666666666668 +alice king 25.20333333333333 +alice king 30.613999999999997 +alice laertes 23.633333333333336 +alice laertes 30.16428571428571 +alice miller 32.50833333333333 +alice nixon 25.278333333333336 +alice nixon 35.345 +alice nixon 36.458333333333336 +alice ovid 26.04714285714286 +alice polk 23.185 +alice quirinius 20.388333333333335 +alice quirinius 23.064999999999998 +alice robinson 30.296666666666667 +alice robinson 33.15 +alice steinbeck 27.894999999999996 +alice steinbeck 31.47833333333334 +alice steinbeck 36.089999999999996 +alice underhill 22.513333333333335 +alice van buren 36.32000000000001 +alice xylophone 27.355000000000004 +alice xylophone 30.505714285714284 +alice xylophone 30.613999999999997 +alice zipper 20.113333333333333 +alice zipper 28.058333333333337 +alice zipper 29.705000000000002 +bob brown 13.546666666666669 +bob brown 16.50333333333333 +bob brown 27.59 +bob carson 25.781666666666666 +bob davidson 18.073333333333334 +bob davidson 23.573333333333334 +bob davidson 31.894999999999996 +bob ellison 16.493333333333336 +bob ellison 17.889999999999997 +bob ellison 25.84142857142857 +bob ellison 33.07833333333334 +bob falkner 21.783333333333335 +bob garcia 16.492857142857144 +bob garcia 23.088333333333335 +bob garcia 26.42833333333333 +bob garcia 32.709999999999994 +bob garcia 33.91833333333334 +bob hernandez 22.303333333333338 +bob ichabod 23.336666666666662 +bob king 20.591666666666665 +bob king 21.244999999999997 +bob king 33.15333333333333 +bob laertes 21.240000000000002 +bob laertes 28.88 +bob miller 32.158750000000005 +bob ovid 21.83666666666667 +bob ovid 23.678571428571427 +bob ovid 25.12333333333333 +bob ovid 31.46 +bob polk 16.3475 +bob quirinius 28.465714285714284 +bob steinbeck 26.99 +bob van buren 26.127142857142854 +bob white 23.021666666666672 +bob white 23.582857142857144 +bob xylophone 26.18166666666667 +bob xylophone 27.995 +bob young 25.495714285714286 +bob zipper 26.435000000000002 +bob zipper 28.384285714285713 +bob zipper 30.65833333333333 +calvin allen 20.812 +calvin brown 20.808 +calvin brown 28.26 +calvin brown 28.37875 +calvin carson 20.38714285714286 +calvin davidson 22.65142857142857 +calvin davidson 23.585714285714285 +calvin ellison 24.6875 +calvin falkner 16.073333333333334 +calvin falkner 17.054285714285715 +calvin falkner 25.2025 +calvin falkner 28.525714285714283 +calvin falkner 33.382 +calvin falkner 37.29666666666667 +calvin garcia 19.924285714285713 +calvin hernandez 21.759999999999998 +calvin johnson 24.03222222222222 +calvin laertes 24.702857142857145 +calvin laertes 28.788333333333338 +calvin nixon 15.631250000000001 +calvin nixon 25.64428571428571 +calvin nixon 29.084999999999997 +calvin ovid 16.243333333333332 +calvin ovid 25.2025 +calvin ovid 25.935555555555553 +calvin ovid 29.299999999999997 +calvin polk 29.615 +calvin quirinius 19.294999999999998 +calvin quirinius 31.64625 +calvin robinson 27.811428571428575 +calvin steinbeck 17.14 +calvin steinbeck 17.535 +calvin steinbeck 21.551666666666666 +calvin thompson 28.592857142857145 +calvin thompson 33.382 +calvin underhill 20.113333333333333 +calvin van buren 28.384285714285713 +calvin van buren 32.106 +calvin white 26.948333333333334 +calvin white 28.256249999999998 +calvin xylophone 25.33666666666667 +calvin xylophone 27.061428571428568 +calvin xylophone 28.384285714285713 +calvin young 19.331666666666667 +calvin young 24.63 +calvin zipper 16.08125 +calvin zipper 28.80555555555556 +david allen 26.21375 +david allen 26.948333333333334 +david brown 14.222 +david brown 27.1 +david davidson 21.062857142857144 +david davidson 26.12 +david davidson 30.951428571428576 +david davidson 31.96142857142857 +david ellison 22.687142857142856 +david ellison 23.435 +david ellison 25.442999999999998 +david hernandez 28.279999999999998 +david ichabod 16.3475 +david ichabod 23.751428571428573 +david laertes 29.07 +david nixon 25.535714285714285 +david ovid 26.75714285714286 +david ovid 29.912857142857142 +david quirinius 17.179999999999996 +david quirinius 20.808 +david quirinius 22.90625 +david robinson 25.274 +david robinson 25.885 +david thompson 22.19125 +david underhill 20.812 +david underhill 21.546666666666667 +david underhill 28.26 +david van buren 24.472857142857148 +david van buren 32.75125 +david white 22.861428571428576 +david xylophone 19.3325 +david xylophone 26.930000000000003 +david xylophone 30.093333333333334 +david young 21.12375 +david young 25.03857142857143 +ethan allen 27.038333333333338 +ethan brown 16.3475 +ethan brown 19.37 +ethan brown 23.73 +ethan brown 25.57166666666667 +ethan brown 26.168333333333333 +ethan brown 31.893333333333334 +ethan carson 25.655714285714286 +ethan ellison 27.807777777777776 +ethan ellison 32.244285714285716 +ethan falkner 28.287142857142857 +ethan falkner 31.411428571428576 +ethan garcia 24.153750000000002 +ethan hernandez 17.986 +ethan johnson 31.54571428571429 +ethan king 22.62 +ethan laertes 15.045555555555556 +ethan laertes 17.889999999999997 +ethan laertes 24.28 +ethan laertes 25.2025 +ethan laertes 26.56888888888889 +ethan laertes 28.37875 +ethan laertes 34.84571428571429 +ethan miller 28.592857142857145 +ethan nixon 29.86833333333333 +ethan ovid 17.85166666666667 +ethan polk 16.463 +ethan polk 18.184545454545457 +ethan polk 25.737142857142857 +ethan polk 29.895000000000003 +ethan quirinius 21.827777777777776 +ethan quirinius 23.42 +ethan quirinius 35.54333333333333 +ethan robinson 24.03222222222222 +ethan robinson 36.35333333333333 +ethan underhill 21.545555555555556 +ethan van buren 15.21142857142857 +ethan white 29.702857142857145 +ethan white 33.33 +ethan xylophone 29.521666666666665 +ethan zipper 23.994 +ethan zipper 26.765 +fred davidson 27.729999999999997 +fred davidson 29.90625 +fred davidson 30.804999999999996 +fred ellison 16.720000000000002 +fred ellison 22.17125 +fred ellison 33.15833333333334 +fred falkner 17.13142857142857 +fred falkner 26.82 +fred falkner 31.925555555555555 +fred hernandez 28.094285714285718 +fred ichabod 23.352500000000003 +fred ichabod 32.906666666666666 +fred johnson 16.9925 +fred king 20.125 +fred king 30.377142857142854 +fred laertes 26.458571428571428 +fred miller 29.66666666666667 +fred nixon 19.565555555555555 +fred nixon 21.830000000000002 +fred nixon 25.828333333333333 +fred nixon 28.094285714285718 +fred polk 21.744999999999997 +fred polk 22.301666666666666 +fred polk 24.156666666666666 +fred polk 28.217142857142857 +fred quirinius 27.495 +fred quirinius 29.615714285714283 +fred robinson 24.243636363636366 +fred steinbeck 17.91333333333333 +fred steinbeck 21.12375 +fred steinbeck 26.47125 +fred underhill 26.43166666666667 +fred van buren 16.165714285714284 +fred van buren 23.285 +fred van buren 24.875714285714288 +fred van buren 27.878333333333334 +fred white 29.498571428571427 +fred young 17.889999999999997 +fred young 19.565555555555555 +fred zipper 21.581428571428575 +gabriella allen 22.03142857142857 +gabriella allen 26.87375 +gabriella brown 29.675714285714285 +gabriella brown 32.44 +gabriella carson 28.75 +gabriella davidson 27.531250000000004 +gabriella ellison 19.82 +gabriella ellison 27.353333333333335 +gabriella falkner 17.535 +gabriella falkner 19.487500000000004 +gabriella falkner 30.964999999999996 +gabriella garcia 20.544 +gabriella hernandez 20.818333333333335 +gabriella hernandez 28.094285714285718 +gabriella ichabod 10.58 +gabriella ichabod 18.64272727272727 +gabriella ichabod 20.504444444444445 +gabriella ichabod 23.185 +gabriella ichabod 23.35375 +gabriella king 16.18 +gabriella king 27.887500000000003 +gabriella laertes 23.799090909090907 +gabriella miller 15.695714285714283 +gabriella ovid 23.515454545454546 +gabriella ovid 33.33 +gabriella polk 20.38714285714286 +gabriella polk 35.77285714285715 +gabriella steinbeck 16.18 +gabriella steinbeck 32.464999999999996 +gabriella thompson 26.477777777777778 +gabriella thompson 27.29777777777778 +gabriella thompson 30.87666666666667 +gabriella van buren 28.513333333333335 +gabriella van buren 32.41111111111111 +gabriella white 26.765 +gabriella young 24.022499999999997 +gabriella young 29.521666666666665 +gabriella zipper 21.43727272727273 +gabriella zipper 32.106 +holly allen 24.271428571428572 +holly brown 22.959 +holly brown 27.498 +holly falkner 29.66666666666667 +holly hernandez 14.179999999999998 +holly hernandez 22.396666666666665 +holly hernandez 27.434000000000005 +holly hernandez 27.887500000000003 +holly ichabod 27.87375 +holly ichabod 32.525 +holly ichabod 34.042857142857144 +holly johnson 20.808 +holly johnson 25.024285714285718 +holly johnson 30.487142857142857 +holly king 23.185 +holly king 29.008888888888894 +holly laertes 19.41 +holly miller 29.89333333333333 +holly nixon 23.162857142857142 +holly nixon 28.876250000000002 +holly polk 22.7825 +holly polk 26.297499999999996 +holly robinson 24.160000000000004 +holly thompson 19.565555555555555 +holly thompson 27.048999999999996 +holly thompson 29.70555555555556 +holly underhill 17.876250000000002 +holly underhill 27.820000000000004 +holly underhill 30.613999999999997 +holly underhill 30.708 +holly van buren 20.113333333333333 +holly white 23.185 +holly white 29.64272727272727 +holly xylophone 26.400909090909092 +holly young 27.807777777777776 +holly young 31.63 +holly zipper 27.401999999999997 +holly zipper 28.384285714285713 +irene allen 35.345 +irene brown 22.527500000000003 +irene brown 28.384285714285713 +irene brown 32.81875 +irene carson 27.10666666666667 +irene ellison 16.720000000000002 +irene ellison 28.592857142857145 +irene falkner 19.41 +irene falkner 30.564999999999998 +irene garcia 16.9925 +irene garcia 24.03222222222222 +irene garcia 26.297499999999996 +irene ichabod 27.540000000000003 +irene ichabod 29.34875 +irene johnson 25.418181818181814 +irene laertes 22.124285714285712 +irene laertes 22.200000000000003 +irene laertes 24.446666666666665 +irene miller 30.166666666666668 +irene nixon 18.922222222222224 +irene nixon 25.2025 +irene nixon 33.382 +irene ovid 28.256249999999998 +irene ovid 31.63 +irene ovid 32.088750000000005 +irene polk 24.63 +irene polk 25.296363636363637 +irene polk 27.137142857142855 +irene polk 30.65222222222222 +irene polk 33.760000000000005 +irene quirinius 27.044999999999998 +irene quirinius 33.07833333333334 +irene quirinius 41.865 +irene robinson 32.18875 +irene steinbeck 16.463 +irene thompson 25.281666666666666 +irene underhill 24.4025 +irene underhill 28.531 +irene van buren 25.532222222222224 +irene van buren 32.50142857142857 +irene xylophone 26.288181818181815 +jessica brown 28.968181818181822 +jessica carson 19.41 +jessica carson 24.854285714285716 +jessica carson 25.406363636363633 +jessica davidson 22.19625 +jessica davidson 23.888 +jessica davidson 26.297499999999996 +jessica davidson 26.825454545454537 +jessica ellison 22.07777777777778 +jessica ellison 33.33 +jessica falkner 22.637272727272727 +jessica garcia 14.749999999999996 +jessica garcia 29.675714285714285 +jessica ichabod 31.831249999999997 +jessica johnson 21.546666666666667 +jessica johnson 29.986363636363638 +jessica miller 28.735000000000003 +jessica nixon 19.13111111111111 +jessica nixon 26.244999999999997 +jessica ovid 25.274 +jessica ovid 33.181666666666665 +jessica polk 26.79222222222222 +jessica quirinius 20.38714285714286 +jessica quirinius 25.776000000000003 +jessica quirinius 28.26 +jessica quirinius 29.605000000000008 +jessica robinson 24.5625 +jessica thompson 25.736 +jessica thompson 30.87363636363636 +jessica underhill 16.400000000000002 +jessica underhill 25.529090909090915 +jessica underhill 31.63 +jessica van buren 24.446666666666665 +jessica white 20.812 +jessica white 23.26 +jessica white 27.807777777777776 +jessica white 29.031000000000006 +jessica white 30.654545454545453 +jessica xylophone 15.296666666666667 +jessica young 26.718333333333334 +jessica young 27.853749999999998 +jessica zipper 20.3575 +jessica zipper 24.446666666666665 +jessica zipper 29.276363636363637 +katie allen 29.029090909090915 +katie brown 24.156666666666666 +katie davidson 15.383749999999997 +katie ellison 20.978333333333335 +katie ellison 26.96 +katie falkner 24.5625 +katie garcia 27.807777777777776 +katie garcia 28.287142857142857 +katie hernandez 23.667272727272724 +katie ichabod 14.222 +katie ichabod 20.553749999999997 +katie ichabod 31.831249999999997 +katie king 20.05444444444445 +katie king 20.242222222222225 +katie king 23.342857142857145 +katie miller 26.21857142857143 +katie miller 27.675000000000004 +katie nixon 14.476999999999999 +katie ovid 28.37875 +katie polk 20.99 +katie polk 25.090000000000003 +katie robinson 30.65222222222222 +katie van buren 27.133636363636366 +katie van buren 29.675714285714285 +katie white 22.555714285714288 +katie white 24.463749999999997 +katie xylophone 25.74142857142857 +katie young 23.011250000000004 +katie young 26.650000000000002 +katie young 29.301428571428573 +katie zipper 28.26 +katie zipper 29.675714285714285 +luke allen 16.9925 +luke allen 20.595000000000002 +luke allen 27.54181818181818 +luke allen 27.887500000000003 +luke allen 33.07833333333334 +luke brown 29.34875 +luke davidson 27.3575 +luke davidson 31.473333333333333 +luke ellison 14.024444444444443 +luke ellison 22.555714285714288 +luke ellison 28.592857142857145 +luke falkner 21.855 +luke falkner 27.044999999999998 +luke garcia 27.887500000000003 +luke garcia 31.237000000000002 +luke ichabod 28.198571428571427 +luke ichabod 34.345000000000006 +luke johnson 21.239999999999995 +luke johnson 25.462727272727275 +luke johnson 30.188888888888894 +luke laertes 18.344 +luke laertes 20.817 +luke laertes 27.401999999999997 +luke laertes 30.72285714285714 +luke laertes 41.865 +luke miller 22.539000000000005 +luke ovid 16.615454545454543 +luke ovid 26.06625 +luke polk 28.163000000000004 +luke polk 28.840909090909097 +luke quirinius 27.077142857142857 +luke robinson 24.816363636363644 +luke robinson 27.110909090909093 +luke thompson 28.44454545454545 +luke underhill 22.175 +luke underhill 25.518888888888892 +luke underhill 27.34125 +luke van buren 16.54 +luke white 20.544 +luke xylophone 24.5625 +luke zipper 24.764285714285712 +mike allen 23.860000000000003 +mike brown 31.016363636363643 +mike carson 26.066363636363636 +mike carson 28.947142857142858 +mike carson 33.382 +mike davidson 20.544 +mike davidson 21.239999999999995 +mike ellison 18.922 +mike ellison 21.4175 +mike ellison 25.45272727272727 +mike ellison 27.26 +mike ellison 28.39888888888889 +mike falkner 29.397777777777776 +mike garcia 20.544 +mike garcia 24.582 +mike garcia 34.84571428571429 +mike hernandez 10.4925 +mike hernandez 17.7 +mike ichabod 26.772727272727273 +mike king 17.889999999999997 +mike king 19.294999999999998 +mike king 20.004285714285714 +mike king 23.197999999999997 +mike king 23.285 +mike king 27.401999999999997 +mike miller 31.587272727272726 +mike nixon 17.775555555555556 +mike nixon 27.044999999999998 +mike polk 22.175 +mike polk 23.751428571428573 +mike polk 28.095 +mike quirinius 19.13111111111111 +mike steinbeck 14.222 +mike steinbeck 18.100909090909088 +mike steinbeck 18.344 +mike steinbeck 33.760000000000005 +mike van buren 23.42 +mike van buren 25.828333333333333 +mike white 19.13111111111111 +mike white 25.755714285714284 +mike white 29.031000000000006 +mike white 30.516999999999996 +mike young 26.765 +mike young 27.766 +mike young 28.409090909090914 +mike zipper 17.306 +mike zipper 33.23 +mike zipper 41.865 +nick allen 19.331666666666667 +nick allen 32.106 +nick brown 27.578181818181818 +nick davidson 29.100000000000005 +nick ellison 24.764285714285712 +nick ellison 29.521666666666665 +nick falkner 22.555714285714288 +nick falkner 23.15888888888888 +nick garcia 21.546666666666667 +nick garcia 26.25090909090909 +nick garcia 30.166666666666668 +nick ichabod 21.855 +nick ichabod 23.479000000000003 +nick ichabod 29.100000000000005 +nick johnson 25.274 +nick johnson 29.994 +nick laertes 25.820909090909094 +nick miller 19.87888888888889 +nick nixon 17.082 +nick ovid 33.597777777777786 +nick polk 25.736 +nick quirinius 20.707500000000003 +nick quirinius 28.094285714285718 +nick robinson 22.396666666666665 +nick robinson 25.298749999999995 +nick steinbeck 21.192857142857143 +nick thompson 30.72285714285714 +nick underhill 29.345 +nick van buren 25.152727272727272 +nick xylophone 26.948333333333334 +nick young 23.751428571428573 +nick young 24.810000000000002 +nick zipper 24.854285714285716 +nick zipper 27.353333333333335 +oscar allen 18.815 +oscar brown 26.948333333333334 +oscar carson 24.764285714285712 +oscar carson 27.766 +oscar carson 28.094285714285718 +oscar carson 28.31555555555556 +oscar carson 35.22818181818182 +oscar davidson 17.535 +oscar ellison 22.121428571428574 +oscar ellison 28.735000000000003 +oscar falkner 19.294999999999998 +oscar garcia 20.62636363636364 +oscar hernandez 22.539000000000005 +oscar hernandez 23.31888888888889 +oscar ichabod 20.818333333333335 +oscar ichabod 21.546666666666667 +oscar ichabod 26.914545454545454 +oscar ichabod 28.811111111111106 +oscar johnson 22.381818181818183 +oscar johnson 24.266363636363643 +oscar king 15.296666666666667 +oscar king 25.580000000000002 +oscar king 28.37875 +oscar laertes 21.51818181818182 +oscar laertes 23.285 +oscar laertes 24.4025 +oscar laertes 25.345454545454547 +oscar nixon 18.88111111111111 +oscar ovid 24.854285714285716 +oscar ovid 25.274 +oscar ovid 33.29636363636364 +oscar polk 19.331666666666667 +oscar polk 29.34875 +oscar quirinius 22.928 +oscar quirinius 25.66727272727273 +oscar quirinius 25.970909090909092 +oscar quirinius 29.66666666666667 +oscar robinson 20.90666666666667 +oscar robinson 21.855 +oscar robinson 23.42 +oscar robinson 32.90500000000001 +oscar steinbeck 32.02818181818182 +oscar thompson 20.817 +oscar thompson 21.477000000000004 +oscar thompson 21.843636363636367 +oscar thompson 23.559000000000005 +oscar underhill 22.555714285714288 +oscar van buren 27.210000000000008 +oscar van buren 28.592857142857145 +oscar van buren 31.375454545454545 +oscar white 20.818333333333335 +oscar white 21.748 +oscar white 24.582 +oscar white 28.287142857142857 +oscar xylophone 25.845 +oscar xylophone 28.735000000000003 +oscar xylophone 30.72285714285714 +oscar zipper 24.511111111111113 +oscar zipper 25.067777777777778 +oscar zipper 26.21857142857143 +priscilla brown 14.222 +priscilla brown 27.044999999999998 +priscilla brown 30.19909090909091 +priscilla carson 18.07 +priscilla carson 20.70875 +priscilla carson 26.297499999999996 +priscilla ichabod 29.451111111111118 +priscilla ichabod 29.878888888888884 +priscilla johnson 16.9925 +priscilla johnson 22.050000000000004 +priscilla johnson 24.093000000000004 +priscilla johnson 29.200000000000003 +priscilla johnson 29.246 +priscilla king 15.536666666666669 +priscilla nixon 18.9 +priscilla nixon 30.516999999999996 +priscilla ovid 16.005000000000003 +priscilla ovid 29.88111111111111 +priscilla polk 28.018888888888892 +priscilla quirinius 23.064999999999998 +priscilla thompson 27.077142857142857 +priscilla underhill 28.160999999999994 +priscilla underhill 28.56571428571429 +priscilla van buren 20.7 +priscilla van buren 21.830000000000002 +priscilla van buren 24.665 +priscilla white 29.23375 +priscilla xylophone 19.331666666666667 +priscilla xylophone 23.185 +priscilla xylophone 27.34125 +priscilla young 26.32777777777778 +priscilla young 30.613999999999997 +priscilla zipper 13.498 +priscilla zipper 31.972727272727266 +quinn allen 28.786666666666672 +quinn allen 29.471818181818175 +quinn brown 26.314285714285713 +quinn brown 27.38 +quinn brown 28.39888888888889 +quinn davidson 20.808 +quinn davidson 22.71285714285714 +quinn davidson 23.306250000000002 +quinn davidson 27.34125 +quinn ellison 25.002857142857142 +quinn ellison 33.760000000000005 +quinn garcia 20.544 +quinn garcia 27.055999999999994 +quinn garcia 29.183333333333334 +quinn garcia 31.831249999999997 +quinn ichabod 22.101818181818185 +quinn king 17.535 +quinn king 18.035714285714285 +quinn laertes 24.511111111111113 +quinn laertes 28.876250000000002 +quinn laertes 29.202857142857145 +quinn nixon 21.75111111111111 +quinn ovid 29.64125 +quinn quirinius 18.922 +quinn robinson 23.985 +quinn steinbeck 27.077142857142857 +quinn steinbeck 28.160999999999994 +quinn thompson 21.99888888888889 +quinn thompson 34.46857142857143 +quinn underhill 20.113333333333333 +quinn underhill 23.107272727272726 +quinn underhill 26.224285714285717 +quinn van buren 23.612222222222222 +quinn young 24.5625 +quinn zipper 17.889999999999997 +quinn zipper 20.163333333333338 +rachel allen 28.446666666666665 +rachel allen 41.865 +rachel brown 20.92875 +rachel brown 26.21857142857143 +rachel brown 27.905 +rachel brown 30.166666666666668 +rachel brown 35.345 +rachel carson 28.735000000000003 +rachel carson 31.715999999999998 +rachel davidson 29.100000000000005 +rachel ellison 27.055454545454552 +rachel falkner 14.812499999999998 +rachel falkner 28.876250000000002 +rachel falkner 29.308888888888887 +rachel falkner 31.831249999999997 +rachel johnson 31.63 +rachel king 24.511111111111113 +rachel king 30.873749999999998 +rachel laertes 17.306 +rachel laertes 26.765 +rachel ovid 24.042727272727276 +rachel ovid 28.01181818181818 +rachel polk 21.12375 +rachel quirinius 29.831249999999997 +rachel robinson 10.4925 +rachel robinson 22.264444444444447 +rachel robinson 33.43125 +rachel thompson 16.720000000000002 +rachel thompson 26.905714285714286 +rachel thompson 28.876250000000002 +rachel underhill 21.75111111111111 +rachel white 22.175 +rachel white 31.580000000000002 +rachel young 28.150000000000002 +rachel zipper 22.187142857142856 +rachel zipper 33.760000000000005 +sarah carson 21.86818181818182 +sarah carson 22.175 +sarah carson 33.43125 +sarah ellison 17.535 +sarah falkner 29.34875 +sarah falkner 29.64125 +sarah garcia 10.4925 +sarah garcia 20.812 +sarah garcia 28.2175 +sarah ichabod 26.948333333333334 +sarah ichabod 33.62375 +sarah johnson 21.546666666666667 +sarah johnson 24.978000000000005 +sarah johnson 29.608000000000004 +sarah johnson 33.760000000000005 +sarah king 19.41 +sarah king 27.055999999999994 +sarah miller 24.815454545454543 +sarah ovid 28.31625 +sarah robinson 13.498 +sarah robinson 28.256249999999998 +sarah steinbeck 23.26 +sarah white 21.75111111111111 +sarah white 31.63 +sarah xylophone 21.964545454545455 +sarah young 29.335555555555555 +sarah zipper 29.521666666666665 +tom brown 22.873333333333335 +tom brown 30.415555555555557 +tom carson 22.4025 +tom carson 28.39888888888889 +tom carson 29.64125 +tom davidson 30.61142857142857 +tom ellison 23.568 +tom ellison 27.884999999999998 +tom ellison 32.02625 +tom falkner 13.72 +tom falkner 19.849999999999998 +tom hernandez 16.720000000000002 +tom hernandez 29.974285714285713 +tom ichabod 20.113333333333333 +tom johnson 27.077142857142857 +tom johnson 32.90500000000001 +tom king 21.855 +tom laertes 17.981666666666666 +tom laertes 21.80857142857143 +tom miller 18.922 +tom miller 21.239999999999995 +tom miller 22.396666666666665 +tom nixon 27.005000000000003 +tom ovid 34.84571428571429 +tom polk 29.521666666666665 +tom polk 29.805 +tom quirinius 24.764285714285712 +tom quirinius 36.46857142857143 +tom robinson 16.18 +tom robinson 18.07 +tom robinson 27.34125 +tom robinson 34.958571428571425 +tom steinbeck 30.613999999999997 +tom van buren 22.6475 +tom van buren 23.13555555555556 +tom van buren 24.4025 +tom white 27.715714285714284 +tom young 19.41 +tom young 24.63 +tom zipper 22.902 +ulysses brown 16.223333333333333 +ulysses carson 16.3475 +ulysses carson 22.057500000000005 +ulysses carson 28.256249999999998 +ulysses carson 28.27285714285714 +ulysses davidson 24.701249999999998 +ulysses ellison 29.52333333333333 +ulysses garcia 33.382 +ulysses hernandez 18.421818181818185 +ulysses hernandez 20.443749999999998 +ulysses hernandez 22.365 +ulysses ichabod 24.63 +ulysses ichabod 33.24333333333333 +ulysses johnson 33.43125 +ulysses king 27.083333333333332 +ulysses laertes 26.915000000000003 +ulysses laertes 27.305 +ulysses laertes 28.501111111111115 +ulysses miller 18.22 +ulysses miller 26.21857142857143 +ulysses nixon 30.65222222222222 +ulysses ovid 21.366666666666667 +ulysses polk 22.555714285714288 +ulysses polk 22.66625 +ulysses polk 25.11777777777778 +ulysses polk 27.141666666666666 +ulysses quirinius 33.07833333333334 +ulysses robinson 21.12375 +ulysses steinbeck 23.751428571428573 +ulysses steinbeck 25.931428571428572 +ulysses thompson 22.264444444444447 +ulysses underhill 20.812 +ulysses underhill 23.751428571428573 +ulysses underhill 25.071666666666662 +ulysses underhill 25.828333333333333 +ulysses underhill 25.865 +ulysses underhill 28.722499999999997 +ulysses underhill 35.268888888888895 +ulysses van buren 22.134999999999998 +ulysses white 15.296666666666667 +ulysses white 32.093333333333334 +ulysses xylophone 20.38714285714286 +ulysses xylophone 25.274 +ulysses xylophone 29.64125 +ulysses young 22.213333333333335 +ulysses young 22.90285714285714 +ulysses young 32.93125 +victor allen 24.82875 +victor allen 27.51 +victor brown 21.621250000000003 +victor brown 23.73 +victor brown 26.21857142857143 +victor brown 27.548571428571428 +victor davidson 22.391666666666666 +victor davidson 33.16428571428572 +victor davidson 35.197500000000005 +victor ellison 11.100000000000001 +victor ellison 30.96857142857143 +victor hernandez 10.4925 +victor hernandez 18.922 +victor hernandez 24.301250000000003 +victor hernandez 26.69857142857143 +victor hernandez 35.358333333333334 +victor johnson 16.580000000000002 +victor johnson 27.516666666666666 +victor johnson 32.106 +victor king 19.962857142857143 +victor king 33.01857142857143 +victor laertes 21.78142857142857 +victor laertes 33.10999999999999 +victor miller 21.93285714285714 +victor nixon 20.419999999999998 +victor nixon 33.69 +victor ovid 28.75857142857143 +victor polk 18.43111111111111 +victor quirinius 17.84777777777778 +victor quirinius 27.53 +victor robinson 19.37 +victor robinson 20.38714285714286 +victor steinbeck 20.818333333333335 +victor steinbeck 25.16 +victor steinbeck 30.503749999999997 +victor thompson 23.987142857142857 +victor van buren 27.009999999999998 +victor van buren 33.43125 +victor white 24.322857142857142 +victor white 28.287142857142857 +victor xylophone 11.807142857142859 +victor xylophone 13.988571428571428 +victor xylophone 16.720000000000002 +victor xylophone 19.686666666666667 +victor xylophone 37.20428571428572 +victor young 22.264444444444447 +victor zipper 24.854285714285716 +wendy allen 28.24142857142857 +wendy allen 29.675714285714285 +wendy allen 34.275 +wendy brown 22.482857142857142 +wendy brown 27.79714285714286 +wendy ellison 16.80888888888889 +wendy ellison 18.135 +wendy falkner 22.628888888888884 +wendy falkner 23.325000000000003 +wendy falkner 24.0375 +wendy garcia 19.307142857142853 +wendy garcia 21.761428571428574 +wendy garcia 24.63 +wendy garcia 24.854285714285716 +wendy hernandez 16.60875 +wendy ichabod 28.26 +wendy king 22.5 +wendy king 24.793333333333333 +wendy king 28.252857142857145 +wendy laertes 25.881428571428568 +wendy laertes 30.338333333333328 +wendy laertes 30.52857142857143 +wendy miller 15.478333333333332 +wendy miller 25.34333333333333 +wendy nixon 19.54714285714286 +wendy nixon 27.003333333333334 +wendy ovid 14.283750000000001 +wendy ovid 30.878333333333334 +wendy polk 21.69375 +wendy polk 24.63 +wendy quirinius 28.731428571428573 +wendy quirinius 29.74333333333333 +wendy robinson 16.720000000000002 +wendy robinson 23.834285714285716 +wendy robinson 29.911666666666672 +wendy steinbeck 29.272857142857145 +wendy thompson 18.17875 +wendy thompson 22.544285714285714 +wendy underhill 21.69625 +wendy underhill 27.077142857142857 +wendy underhill 30.03333333333333 +wendy van buren 28.624285714285715 +wendy van buren 29.28333333333333 +wendy white 24.4025 +wendy xylophone 16.84 +wendy xylophone 23.426666666666666 +wendy young 20.80125 +wendy young 32.693333333333335 +xavier allen 16.535 +xavier allen 17.398333333333337 +xavier allen 35.708333333333336 +xavier brown 20.787142857142857 +xavier brown 24.764285714285712 +xavier brown 31.784999999999997 +xavier carson 20.818333333333335 +xavier carson 32.106 +xavier davidson 16.862857142857145 +xavier davidson 20.53625 +xavier davidson 27.353333333333335 +xavier ellison 17.991666666666667 +xavier ellison 23.976666666666663 +xavier garcia 35.84428571428572 +xavier hernandez 22.654285714285713 +xavier hernandez 26.948333333333334 +xavier hernandez 28.075 +xavier ichabod 20.344285714285714 +xavier ichabod 20.818333333333335 +xavier johnson 15.754285714285714 +xavier johnson 19.490000000000002 +xavier king 29.246666666666666 +xavier king 29.521666666666665 +xavier laertes 19.294999999999998 +xavier ovid 28.51 +xavier polk 12.728333333333333 +xavier polk 19.37 +xavier polk 22.548333333333332 +xavier polk 28.465714285714284 +xavier quirinius 9.991428571428571 +xavier quirinius 24.156666666666666 +xavier quirinius 25.69666666666667 +xavier quirinius 25.828333333333333 +xavier thompson 23.961428571428574 +xavier underhill 21.830000000000002 +xavier white 19.331666666666667 +xavier white 35.345 +xavier xylophone 21.187142857142856 +xavier zipper 17.488333333333333 +yuri allen 15.705714285714285 +yuri allen 20.808 +yuri brown 19.53 +yuri brown 22.457142857142856 +yuri carson 25.699999999999996 +yuri carson 27.216666666666665 +yuri ellison 15.034999999999998 +yuri ellison 28.463333333333335 +yuri falkner 17.81833333333333 +yuri falkner 19.294999999999998 +yuri garcia 28.287142857142857 +yuri hernandez 32.395 +yuri johnson 25.828333333333333 +yuri johnson 27.301666666666666 +yuri johnson 29.578333333333337 +yuri king 19.921666666666663 +yuri laertes 16.18 +yuri laertes 30.519999999999996 +yuri nixon 16.383333333333333 +yuri nixon 25.828333333333333 +yuri polk 16.18 +yuri polk 20.503333333333334 +yuri polk 30.16333333333333 +yuri quirinius 20.311666666666667 +yuri quirinius 23.185 +yuri quirinius 24.828333333333333 +yuri steinbeck 19.331666666666667 +yuri steinbeck 28.50666666666667 +yuri thompson 35.27 +yuri underhill 23.042857142857144 +yuri underhill 28.786666666666665 +yuri white 30.72285714285714 +yuri xylophone 24.173333333333332 +zach allen 8.983333333333333 +zach brown 18.922 +zach brown 23.036666666666665 +zach brown 29.72666666666667 +zach brown 31.58285714285714 +zach brown 33.07833333333334 +zach carson 27.110000000000003 +zach ellison 18.168333333333333 +zach falkner 16.18 +zach falkner 30.83285714285714 +zach garcia 16.586666666666666 +zach garcia 22.53333333333333 +zach garcia 28.13166666666667 +zach garcia 34.84571428571429 +zach ichabod 17.535 +zach ichabod 30.72285714285714 +zach king 19.878333333333334 +zach king 25.643333333333334 +zach king 28.646666666666665 +zach miller 23.285 +zach miller 23.366666666666664 +zach miller 30.46833333333333 +zach ovid 23.94666666666667 +zach ovid 28.75166666666667 +zach ovid 28.763333333333335 +zach ovid 34.84571428571429 +zach quirinius 20.755 +zach robinson 21.546666666666667 +zach steinbeck 27.243333333333336 +zach steinbeck 30.073333333333334 +zach thompson 14.222 +zach thompson 24.755 +zach underhill 31.885 +zach white 20.208333333333332 +zach xylophone 10.485 +zach xylophone 20.113333333333333 +zach young 20.176666666666666 +zach zipper 21.709999999999997 +zach zipper 22.264999999999997 +zach zipper 34.01166666666667 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 new file mode 100644 index 0000000000000..a9ae190825a01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-5-ee44c5cdc80e1c832b702f9fb76d8145 @@ -0,0 +1,1049 @@ + 65560 + 65718 + 65740 +alice allen 65662 +alice allen 65720 +alice allen 65758 +alice brown 65696 +alice carson 65559 +alice davidson 65547 +alice falkner 65669 +alice garcia 65613 +alice hernandez 65737 +alice hernandez 65784 +alice johnson 65739 +alice king 65660 +alice king 65738 +alice king 131281 +alice laertes 65669 +alice laertes 65671 +alice miller 65590 +alice nixon 65586 +alice nixon 65595 +alice nixon 65604 +alice ovid 65737 +alice polk 65548 +alice quirinius 65636 +alice quirinius 65728 +alice robinson 65606 +alice robinson 65789 +alice steinbeck 65578 +alice steinbeck 65673 +alice steinbeck 65786 +alice underhill 65750 +alice van buren 65562 +alice xylophone 65585 +alice xylophone 65599 +alice xylophone 131247 +alice zipper 65553 +alice zipper 65662 +alice zipper 65766 +bob brown 65584 +bob brown 65777 +bob brown 65783 +bob carson 65713 +bob davidson 65664 +bob davidson 65693 +bob davidson 65768 +bob ellison 65591 +bob ellison 65624 +bob ellison 65721 +bob ellison 65760 +bob falkner 65789 +bob garcia 65585 +bob garcia 65598 +bob garcia 65673 +bob garcia 65754 +bob garcia 65782 +bob hernandez 131340 +bob ichabod 65549 +bob king 65715 +bob king 65757 +bob king 65783 +bob laertes 65602 +bob laertes 131447 +bob miller 65608 +bob ovid 65564 +bob ovid 65686 +bob ovid 196959 +bob ovid 196973 +bob polk 65594 +bob quirinius 65700 +bob steinbeck 65637 +bob van buren 65778 +bob white 65543 +bob white 65605 +bob xylophone 65574 +bob xylophone 65666 +bob young 65556 +bob zipper 65559 +bob zipper 65633 +bob zipper 65739 +calvin allen 65669 +calvin brown 65537 +calvin brown 131272 +calvin brown 197027 +calvin carson 65637 +calvin davidson 65541 +calvin davidson 65564 +calvin ellison 65667 +calvin falkner 65573 +calvin falkner 65596 +calvin falkner 65778 +calvin falkner 131397 +calvin falkner 131411 +calvin falkner 131433 +calvin garcia 131212 +calvin hernandez 131251 +calvin johnson 65731 +calvin laertes 65570 +calvin laertes 65684 +calvin nixon 65654 +calvin nixon 131386 +calvin nixon 131503 +calvin ovid 65554 +calvin ovid 65643 +calvin ovid 65715 +calvin ovid 196944 +calvin polk 65731 +calvin quirinius 65741 +calvin quirinius 65769 +calvin robinson 131320 +calvin steinbeck 131271 +calvin steinbeck 131326 +calvin steinbeck 131415 +calvin thompson 65560 +calvin thompson 131244 +calvin underhill 196944 +calvin van buren 65771 +calvin van buren 131138 +calvin white 65553 +calvin white 65561 +calvin xylophone 65575 +calvin xylophone 65596 +calvin xylophone 262686 +calvin young 65746 +calvin young 131168 +calvin zipper 65669 +calvin zipper 131476 +david allen 65588 +david allen 131222 +david brown 65637 +david brown 131303 +david davidson 65756 +david davidson 65778 +david davidson 65779 +david davidson 131342 +david ellison 65724 +david ellison 65724 +david ellison 131224 +david hernandez 197083 +david ichabod 131454 +david ichabod 197085 +david laertes 65762 +david nixon 65536 +david ovid 65623 +david ovid 196766 +david quirinius 65759 +david quirinius 65779 +david quirinius 131303 +david robinson 65762 +david robinson 65775 +david thompson 65550 +david underhill 65662 +david underhill 65751 +david underhill 131198 +david van buren 65634 +david van buren 262584 +david white 65678 +david xylophone 65537 +david xylophone 131426 +david xylophone 131447 +david young 65551 +david young 131255 +ethan allen 131460 +ethan brown 65539 +ethan brown 65617 +ethan brown 65685 +ethan brown 65685 +ethan brown 65722 +ethan brown 131483 +ethan carson 197189 +ethan ellison 65714 +ethan ellison 131302 +ethan falkner 131222 +ethan falkner 131333 +ethan garcia 131507 +ethan hernandez 65618 +ethan johnson 65536 +ethan king 131280 +ethan laertes 65562 +ethan laertes 65597 +ethan laertes 65628 +ethan laertes 65680 +ethan laertes 65760 +ethan laertes 131304 +ethan laertes 328329 +ethan miller 328296 +ethan nixon 65766 +ethan ovid 65697 +ethan polk 65589 +ethan polk 65615 +ethan polk 131206 +ethan polk 197082 +ethan quirinius 65591 +ethan quirinius 196912 +ethan quirinius 196957 +ethan robinson 65547 +ethan robinson 65659 +ethan underhill 65570 +ethan van buren 131252 +ethan white 65677 +ethan white 197039 +ethan xylophone 65595 +ethan zipper 65593 +ethan zipper 131365 +fred davidson 65595 +fred davidson 65721 +fred davidson 131221 +fred ellison 65548 +fred ellison 65691 +fred ellison 65771 +fred falkner 65637 +fred falkner 131474 +fred falkner 196920 +fred hernandez 131226 +fred ichabod 131109 +fred ichabod 131520 +fred johnson 131332 +fred king 65694 +fred king 197016 +fred laertes 131354 +fred miller 65536 +fred nixon 65560 +fred nixon 65612 +fred nixon 65705 +fred nixon 196929 +fred polk 65656 +fred polk 131231 +fred polk 262645 +fred polk 262733 +fred quirinius 131486 +fred quirinius 196950 +fred robinson 65623 +fred steinbeck 65544 +fred steinbeck 65755 +fred steinbeck 131253 +fred underhill 131188 +fred van buren 65561 +fred van buren 65745 +fred van buren 131380 +fred van buren 328270 +fred white 131136 +fred young 65594 +fred young 131551 +fred zipper 196885 +gabriella allen 65677 +gabriella allen 131283 +gabriella brown 65753 +gabriella brown 197180 +gabriella carson 65586 +gabriella davidson 65565 +gabriella ellison 65706 +gabriella ellison 131505 +gabriella falkner 65767 +gabriella falkner 131183 +gabriella falkner 131397 +gabriella garcia 131127 +gabriella hernandez 131304 +gabriella hernandez 131304 +gabriella ichabod 65559 +gabriella ichabod 65712 +gabriella ichabod 131297 +gabriella ichabod 131311 +gabriella ichabod 131460 +gabriella king 65657 +gabriella king 197031 +gabriella laertes 131543 +gabriella miller 131300 +gabriella ovid 65556 +gabriella ovid 131260 +gabriella polk 65790 +gabriella polk 131425 +gabriella steinbeck 65582 +gabriella steinbeck 131248 +gabriella thompson 131528 +gabriella thompson 197181 +gabriella thompson 262632 +gabriella van buren 65644 +gabriella van buren 131238 +gabriella white 65638 +gabriella young 65699 +gabriella young 65774 +gabriella zipper 65754 +gabriella zipper 196762 +holly allen 65596 +holly brown 131315 +holly brown 131368 +holly falkner 65720 +holly hernandez 65602 +holly hernandez 65686 +holly hernandez 131387 +holly hernandez 131554 +holly ichabod 65752 +holly ichabod 131308 +holly ichabod 131473 +holly johnson 65755 +holly johnson 131240 +holly johnson 131277 +holly king 131286 +holly king 131303 +holly laertes 196950 +holly miller 131381 +holly nixon 196941 +holly nixon 328184 +holly polk 197132 +holly polk 262782 +holly robinson 131241 +holly thompson 65578 +holly thompson 65713 +holly thompson 197092 +holly underhill 65654 +holly underhill 131323 +holly underhill 131385 +holly underhill 131504 +holly van buren 131449 +holly white 131092 +holly white 262734 +holly xylophone 196792 +holly young 65765 +holly young 131229 +holly zipper 131151 +holly zipper 131545 +irene allen 131109 +irene brown 65765 +irene brown 131368 +irene brown 393929 +irene carson 262770 +irene ellison 196956 +irene ellison 196982 +irene falkner 131287 +irene falkner 197046 +irene garcia 65660 +irene garcia 131286 +irene garcia 131375 +irene ichabod 65645 +irene ichabod 131442 +irene johnson 131179 +irene laertes 131324 +irene laertes 131381 +irene laertes 131407 +irene miller 262822 +irene nixon 197105 +irene nixon 262409 +irene nixon 262565 +irene ovid 65734 +irene ovid 196935 +irene ovid 262836 +irene polk 65551 +irene polk 131189 +irene polk 131189 +irene polk 196943 +irene polk 328365 +irene quirinius 131369 +irene quirinius 196998 +irene quirinius 262855 +irene robinson 131259 +irene steinbeck 65683 +irene thompson 262719 +irene underhill 131291 +irene underhill 131386 +irene van buren 131216 +irene van buren 262539 +irene xylophone 131348 +jessica brown 393772 +jessica carson 65747 +jessica carson 131207 +jessica carson 131232 +jessica davidson 65606 +jessica davidson 65675 +jessica davidson 196917 +jessica davidson 197030 +jessica ellison 131108 +jessica ellison 196885 +jessica falkner 131270 +jessica garcia 197059 +jessica garcia 328458 +jessica ichabod 197028 +jessica johnson 131177 +jessica johnson 197085 +jessica miller 197024 +jessica nixon 131549 +jessica nixon 196682 +jessica ovid 65751 +jessica ovid 196890 +jessica polk 459409 +jessica quirinius 131222 +jessica quirinius 131248 +jessica quirinius 131294 +jessica quirinius 393878 +jessica robinson 131174 +jessica thompson 131336 +jessica thompson 196927 +jessica underhill 131218 +jessica underhill 131267 +jessica underhill 197086 +jessica van buren 65615 +jessica white 65544 +jessica white 65594 +jessica white 197012 +jessica white 262435 +jessica white 262571 +jessica xylophone 196866 +jessica young 65711 +jessica young 131183 +jessica zipper 196897 +jessica zipper 262523 +jessica zipper 262695 +katie allen 196740 +katie brown 328113 +katie davidson 131371 +katie ellison 131248 +katie ellison 197182 +katie falkner 131441 +katie garcia 131384 +katie garcia 197051 +katie hernandez 131296 +katie ichabod 131495 +katie ichabod 197131 +katie ichabod 197275 +katie king 131252 +katie king 262588 +katie king 262861 +katie miller 65661 +katie miller 262723 +katie nixon 65669 +katie ovid 65681 +katie polk 65784 +katie polk 197249 +katie robinson 131251 +katie van buren 131237 +katie van buren 197141 +katie white 262510 +katie white 262860 +katie xylophone 197034 +katie young 65644 +katie young 328173 +katie young 393859 +katie zipper 65733 +katie zipper 328287 +luke allen 65776 +luke allen 131268 +luke allen 196819 +luke allen 196855 +luke allen 328011 +luke brown 196967 +luke davidson 65656 +luke davidson 131573 +luke ellison 65582 +luke ellison 131343 +luke ellison 197118 +luke falkner 196797 +luke falkner 196837 +luke garcia 65778 +luke garcia 393974 +luke ichabod 65629 +luke ichabod 262574 +luke johnson 131302 +luke johnson 131312 +luke johnson 131361 +luke laertes 131226 +luke laertes 131504 +luke laertes 197018 +luke laertes 197153 +luke laertes 197177 +luke miller 197052 +luke ovid 65569 +luke ovid 262745 +luke polk 65658 +luke polk 262627 +luke quirinius 131233 +luke robinson 65634 +luke robinson 262569 +luke thompson 196858 +luke underhill 65651 +luke underhill 131240 +luke underhill 328248 +luke van buren 131398 +luke white 65693 +luke xylophone 131312 +luke zipper 131297 +mike allen 196928 +mike brown 197149 +mike carson 65751 +mike carson 131284 +mike carson 393711 +mike davidson 196917 +mike davidson 262912 +mike ellison 65598 +mike ellison 131366 +mike ellison 131412 +mike ellison 131509 +mike ellison 262704 +mike falkner 328183 +mike garcia 131530 +mike garcia 328305 +mike garcia 328461 +mike hernandez 131301 +mike hernandez 328384 +mike ichabod 131157 +mike king 196965 +mike king 197091 +mike king 197121 +mike king 262471 +mike king 262527 +mike king 328279 +mike miller 131317 +mike nixon 131328 +mike nixon 262653 +mike polk 131240 +mike polk 196899 +mike polk 262885 +mike quirinius 525126 +mike steinbeck 65550 +mike steinbeck 131201 +mike steinbeck 131490 +mike steinbeck 262490 +mike van buren 131548 +mike van buren 262547 +mike white 197000 +mike white 197060 +mike white 262425 +mike white 328482 +mike young 196935 +mike young 196976 +mike young 328084 +mike zipper 131147 +mike zipper 197075 +mike zipper 328517 +nick allen 131192 +nick allen 197024 +nick brown 131503 +nick davidson 262686 +nick ellison 197119 +nick ellison 197119 +nick falkner 65583 +nick falkner 328561 +nick garcia 131318 +nick garcia 262755 +nick garcia 328281 +nick ichabod 131430 +nick ichabod 196812 +nick ichabod 328593 +nick johnson 131453 +nick johnson 262597 +nick laertes 196732 +nick miller 131490 +nick nixon 262547 +nick ovid 328266 +nick polk 196852 +nick quirinius 131438 +nick quirinius 328176 +nick robinson 131326 +nick robinson 196980 +nick steinbeck 131250 +nick thompson 65610 +nick underhill 65619 +nick van buren 196795 +nick xylophone 196972 +nick young 394136 +nick young 459634 +nick zipper 262954 +nick zipper 394218 +oscar allen 262674 +oscar brown 196916 +oscar carson 131099 +oscar carson 131330 +oscar carson 196731 +oscar carson 196733 +oscar carson 196918 +oscar davidson 262554 +oscar ellison 65630 +oscar ellison 197116 +oscar falkner 197145 +oscar garcia 328305 +oscar hernandez 197022 +oscar hernandez 328315 +oscar ichabod 131302 +oscar ichabod 131309 +oscar ichabod 196760 +oscar ichabod 196874 +oscar johnson 196942 +oscar johnson 197203 +oscar king 196793 +oscar king 196944 +oscar king 328236 +oscar laertes 131208 +oscar laertes 262522 +oscar laertes 262842 +oscar laertes 328364 +oscar nixon 65596 +oscar ovid 131228 +oscar ovid 262580 +oscar ovid 393817 +oscar polk 131078 +oscar polk 131260 +oscar quirinius 131103 +oscar quirinius 196748 +oscar quirinius 196829 +oscar quirinius 262838 +oscar robinson 196874 +oscar robinson 262803 +oscar robinson 393773 +oscar robinson 394087 +oscar steinbeck 328432 +oscar thompson 196826 +oscar thompson 196992 +oscar thompson 262593 +oscar thompson 459401 +oscar underhill 131301 +oscar van buren 131134 +oscar van buren 328162 +oscar van buren 394034 +oscar white 131457 +oscar white 262345 +oscar white 328538 +oscar white 459337 +oscar xylophone 65773 +oscar xylophone 262708 +oscar xylophone 262906 +oscar zipper 196904 +oscar zipper 262512 +oscar zipper 328262 +priscilla brown 196950 +priscilla brown 328237 +priscilla brown 328624 +priscilla carson 262488 +priscilla carson 262510 +priscilla carson 262703 +priscilla ichabod 131178 +priscilla ichabod 131303 +priscilla johnson 131224 +priscilla johnson 196906 +priscilla johnson 196994 +priscilla johnson 197184 +priscilla johnson 394171 +priscilla king 262692 +priscilla nixon 262691 +priscilla nixon 394188 +priscilla ovid 65541 +priscilla ovid 197067 +priscilla polk 394009 +priscilla quirinius 131306 +priscilla thompson 196875 +priscilla underhill 197084 +priscilla underhill 262701 +priscilla van buren 65685 +priscilla van buren 131368 +priscilla van buren 196814 +priscilla white 196893 +priscilla xylophone 131473 +priscilla xylophone 262597 +priscilla xylophone 262785 +priscilla young 131392 +priscilla young 262788 +priscilla zipper 393888 +priscilla zipper 394031 +quinn allen 197095 +quinn allen 394225 +quinn brown 131470 +quinn brown 131473 +quinn brown 262642 +quinn davidson 197079 +quinn davidson 197112 +quinn davidson 262510 +quinn davidson 459427 +quinn ellison 197268 +quinn ellison 328130 +quinn garcia 65604 +quinn garcia 131321 +quinn garcia 197067 +quinn garcia 328528 +quinn ichabod 65564 +quinn king 65649 +quinn king 196879 +quinn laertes 65542 +quinn laertes 196877 +quinn laertes 262466 +quinn nixon 196837 +quinn ovid 525126 +quinn quirinius 328235 +quinn robinson 131378 +quinn steinbeck 131484 +quinn steinbeck 262528 +quinn thompson 197030 +quinn thompson 262717 +quinn underhill 262791 +quinn underhill 328146 +quinn underhill 393824 +quinn van buren 197234 +quinn young 65647 +quinn zipper 131466 +quinn zipper 262658 +rachel allen 65661 +rachel allen 196935 +rachel brown 131220 +rachel brown 328076 +rachel brown 328320 +rachel brown 393835 +rachel brown 524988 +rachel carson 131259 +rachel carson 459393 +rachel davidson 262632 +rachel ellison 393845 +rachel falkner 196947 +rachel falkner 262474 +rachel falkner 394046 +rachel falkner 525086 +rachel johnson 65658 +rachel king 131354 +rachel king 196907 +rachel laertes 131391 +rachel laertes 197105 +rachel ovid 262664 +rachel ovid 328195 +rachel polk 328389 +rachel quirinius 262779 +rachel robinson 262491 +rachel robinson 262862 +rachel robinson 590712 +rachel thompson 197034 +rachel thompson 328158 +rachel thompson 394094 +rachel underhill 197033 +rachel white 131399 +rachel white 197190 +rachel young 196967 +rachel zipper 328223 +rachel zipper 394149 +sarah carson 131379 +sarah carson 196870 +sarah carson 262491 +sarah ellison 197095 +sarah falkner 131262 +sarah falkner 328251 +sarah garcia 196963 +sarah garcia 197030 +sarah garcia 459657 +sarah ichabod 262504 +sarah ichabod 262766 +sarah johnson 131409 +sarah johnson 262783 +sarah johnson 328591 +sarah johnson 394043 +sarah king 196998 +sarah king 328416 +sarah miller 196893 +sarah ovid 131199 +sarah robinson 262868 +sarah robinson 394066 +sarah steinbeck 262650 +sarah white 197059 +sarah white 262579 +sarah xylophone 131336 +sarah young 394123 +sarah zipper 262818 +tom brown 196848 +tom brown 328268 +tom carson 197328 +tom carson 262517 +tom carson 656251 +tom davidson 262864 +tom ellison 196974 +tom ellison 328416 +tom ellison 393921 +tom falkner 393809 +tom falkner 459407 +tom hernandez 262525 +tom hernandez 328085 +tom ichabod 197048 +tom johnson 328321 +tom johnson 393865 +tom king 196951 +tom laertes 262657 +tom laertes 459805 +tom miller 131278 +tom miller 131459 +tom miller 262633 +tom nixon 262588 +tom ovid 262595 +tom polk 328470 +tom polk 328584 +tom quirinius 262597 +tom quirinius 262681 +tom robinson 196978 +tom robinson 328481 +tom robinson 459857 +tom robinson 525095 +tom steinbeck 262426 +tom van buren 131389 +tom van buren 328095 +tom van buren 328313 +tom white 328128 +tom young 131080 +tom young 393692 +tom zipper 197167 +ulysses brown 196815 +ulysses carson 131277 +ulysses carson 262450 +ulysses carson 262937 +ulysses carson 328311 +ulysses davidson 262750 +ulysses ellison 262445 +ulysses garcia 328445 +ulysses hernandez 131414 +ulysses hernandez 196871 +ulysses hernandez 394370 +ulysses ichabod 393834 +ulysses ichabod 459582 +ulysses johnson 262966 +ulysses king 131363 +ulysses laertes 262739 +ulysses laertes 328412 +ulysses laertes 328462 +ulysses miller 262661 +ulysses miller 328360 +ulysses nixon 394194 +ulysses ovid 328289 +ulysses polk 65563 +ulysses polk 197046 +ulysses polk 328294 +ulysses polk 590698 +ulysses quirinius 525643 +ulysses robinson 394160 +ulysses steinbeck 196783 +ulysses steinbeck 262778 +ulysses thompson 262607 +ulysses underhill 131214 +ulysses underhill 196937 +ulysses underhill 197027 +ulysses underhill 262623 +ulysses underhill 262623 +ulysses underhill 262648 +ulysses underhill 262836 +ulysses van buren 196944 +ulysses white 197033 +ulysses white 393988 +ulysses xylophone 262695 +ulysses xylophone 328151 +ulysses xylophone 328747 +ulysses young 196903 +ulysses young 394037 +ulysses young 459782 +victor allen 197189 +victor allen 262651 +victor brown 262544 +victor brown 262799 +victor brown 327900 +victor brown 591265 +victor davidson 197173 +victor davidson 262486 +victor davidson 328274 +victor ellison 328618 +victor ellison 393962 +victor hernandez 197041 +victor hernandez 197132 +victor hernandez 262771 +victor hernandez 328261 +victor hernandez 459902 +victor johnson 131155 +victor johnson 131169 +victor johnson 394168 +victor king 131486 +victor king 328509 +victor laertes 262573 +victor laertes 328435 +victor miller 196784 +victor nixon 196987 +victor nixon 394249 +victor ovid 196882 +victor polk 262462 +victor quirinius 65620 +victor quirinius 328301 +victor robinson 328334 +victor robinson 394031 +victor steinbeck 65661 +victor steinbeck 262560 +victor steinbeck 262750 +victor thompson 65548 +victor van buren 197173 +victor van buren 328261 +victor white 262588 +victor white 328039 +victor xylophone 131203 +victor xylophone 262596 +victor xylophone 328191 +victor xylophone 393913 +victor xylophone 459542 +victor young 131258 +victor zipper 131349 +wendy allen 131402 +wendy allen 196954 +wendy allen 328359 +wendy brown 328365 +wendy brown 459501 +wendy ellison 262718 +wendy ellison 328191 +wendy falkner 197009 +wendy falkner 262430 +wendy falkner 328177 +wendy garcia 65746 +wendy garcia 393974 +wendy garcia 459883 +wendy garcia 459926 +wendy hernandez 65650 +wendy ichabod 262665 +wendy king 262545 +wendy king 328229 +wendy king 393951 +wendy laertes 262739 +wendy laertes 262794 +wendy laertes 328315 +wendy miller 131377 +wendy miller 328161 +wendy nixon 131258 +wendy nixon 196893 +wendy ovid 196952 +wendy ovid 459594 +wendy polk 328520 +wendy polk 394310 +wendy quirinius 328703 +wendy quirinius 394360 +wendy robinson 131316 +wendy robinson 394030 +wendy robinson 459665 +wendy steinbeck 262645 +wendy thompson 262725 +wendy thompson 393865 +wendy underhill 328445 +wendy underhill 394295 +wendy underhill 460068 +wendy van buren 65699 +wendy van buren 196964 +wendy white 328135 +wendy xylophone 262894 +wendy xylophone 525344 +wendy young 197017 +wendy young 721936 +xavier allen 197025 +xavier allen 525393 +xavier allen 525839 +xavier brown 197058 +xavier brown 262626 +xavier brown 328388 +xavier carson 196990 +xavier carson 328415 +xavier davidson 65644 +xavier davidson 262745 +xavier davidson 393825 +xavier ellison 197095 +xavier ellison 328447 +xavier garcia 262590 +xavier hernandez 196847 +xavier hernandez 197077 +xavier hernandez 393838 +xavier ichabod 262600 +xavier ichabod 328157 +xavier johnson 197084 +xavier johnson 262785 +xavier king 196919 +xavier king 262774 +xavier laertes 262770 +xavier ovid 328414 +xavier polk 196844 +xavier polk 328474 +xavier polk 394013 +xavier polk 590931 +xavier quirinius 65650 +xavier quirinius 131140 +xavier quirinius 328382 +xavier quirinius 459669 +xavier thompson 393799 +xavier underhill 197012 +xavier white 196858 +xavier white 262712 +xavier xylophone 131250 +xavier zipper 394070 +yuri allen 131129 +yuri allen 459977 +yuri brown 262640 +yuri brown 393858 +yuri carson 459799 +yuri carson 591063 +yuri ellison 197085 +yuri ellison 459558 +yuri falkner 196857 +yuri falkner 525350 +yuri garcia 328378 +yuri hernandez 262588 +yuri johnson 393861 +yuri johnson 394444 +yuri johnson 525638 +yuri king 525526 +yuri laertes 131551 +yuri laertes 459611 +yuri nixon 262644 +yuri nixon 393936 +yuri polk 328197 +yuri polk 328404 +yuri polk 328481 +yuri quirinius 131092 +yuri quirinius 196898 +yuri quirinius 525159 +yuri steinbeck 394037 +yuri steinbeck 525180 +yuri thompson 459710 +yuri underhill 328325 +yuri underhill 459781 +yuri white 131252 +yuri xylophone 262809 +zach allen 394026 +zach brown 262789 +zach brown 262789 +zach brown 459521 +zach brown 459846 +zach brown 590938 +zach carson 262320 +zach ellison 262757 +zach falkner 262608 +zach falkner 262608 +zach garcia 262818 +zach garcia 328314 +zach garcia 393686 +zach garcia 394011 +zach ichabod 262518 +zach ichabod 262563 +zach king 196780 +zach king 196905 +zach king 459991 +zach miller 196923 +zach miller 393813 +zach miller 393892 +zach ovid 196876 +zach ovid 262643 +zach ovid 328023 +zach ovid 459615 +zach quirinius 262471 +zach robinson 196967 +zach steinbeck 131394 +zach steinbeck 459294 +zach thompson 131340 +zach thompson 525538 +zach underhill 131304 +zach white 65733 +zach xylophone 262810 +zach xylophone 459455 +zach young 393615 +zach zipper 197130 +zach zipper 262496 +zach zipper 393937 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 new file mode 100644 index 0000000000000..b3f08818f491a --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-6-4d78f7b1d172d20c91f5867bc13a42a0 @@ -0,0 +1,1049 @@ +0.08 0.07999999821186066 +0.1 0.10000000149011612 +0.13 0.12999999523162842 +0.15 0.15000000596046448 +0.27 0.27000001072883606 +0.28 0.2800000011920929 +0.43 0.4300000071525574 +0.52 0.5199999809265137 +0.56 0.5600000023841858 +0.6 0.6000000238418579 +0.61 0.6100000143051147 +0.79 0.7900000214576721 +0.84 0.8399999737739563 +0.98 0.9800000190734863 +1.02 1.2899999916553497 +1.08 1.0800000429153442 +1.08 1.0800000429153442 +1.12 1.1200000047683716 +1.21 2.0000000596046448 +1.25 1.25 +1.27 1.2699999809265137 +1.29 1.2899999618530273 +1.31 1.309999942779541 +1.58 1.5800000429153442 +1.87 1.8700000047683716 +1.91 1.909999966621399 +1.92 3.1699999570846558 +2.07 2.069999933242798 +2.18 2.180000066757202 +2.2 2.200000047683716 +2.35 2.3499999046325684 +2.6 2.5999999046325684 +2.79 2.7899999618530273 +2.92 2.9200000762939453 +2.96 2.9600000381469727 +2.96 2.9600000381469727 +2.97 2.9700000286102295 +3.0 3.0 +3.21 3.340000033378601 +3.28 4.399999976158142 +3.33 3.3299999237060547 +3.61 3.609999895095825 +3.62 3.619999885559082 +3.82 3.819999933242798 +3.86 3.859999895095825 +3.96 3.9600000381469727 +3.97 3.9700000286102295 +4.17 7.7799999713897705 +4.32 4.320000171661377 +4.35 4.349999904632568 +4.41 4.409999847412109 +4.46 4.460000038146973 +4.47 4.46999979019165 +4.57 4.570000171661377 +4.59 4.590000152587891 +4.71 4.710000038146973 +4.72 4.71999979019165 +4.79 4.789999961853027 +4.8 4.800000190734863 +4.92 4.920000076293945 +5.08 5.079999923706055 +5.24 9.559999942779541 +5.28 5.28000020980835 +5.4 5.400000095367432 +5.44 5.440000057220459 +5.45 5.449999809265137 +5.51 5.510000228881836 +5.54 5.539999961853027 +5.62 5.619999885559082 +5.67 5.670000076293945 +5.85 5.849999904632568 +5.88 5.880000114440918 +6.29 6.289999961853027 +6.55 6.550000190734863 +6.57 11.160000324249268 +6.63 9.59000015258789 +6.67 6.670000076293945 +6.72 6.71999979019165 +6.74 6.739999771118164 +6.84 6.840000152587891 +6.87 6.869999885559082 +7.05 7.050000190734863 +7.06 11.769999980926514 +7.11 7.110000133514404 +7.54 7.539999961853027 +7.56 7.559999942779541 +7.79 7.789999961853027 +7.82 7.820000171661377 +7.96 7.960000038146973 +7.96 7.960000038146973 +7.98 7.980000019073486 +8.07 8.069999694824219 +8.07 8.069999694824219 +8.32 8.319999694824219 +8.37 11.339999914169312 +8.42 11.760000109672546 +8.45 8.449999809265137 +8.45 8.449999809265137 +8.45 10.319999814033508 +8.45 16.009999752044678 +8.57 8.569999694824219 +8.61 8.609999656677246 +8.67 8.670000076293945 +8.71 8.710000038146973 +8.79 8.789999961853027 +8.91 13.480000019073486 +9.04 9.039999961853027 +9.13 9.130000114440918 +9.19 15.479999542236328 +9.22 9.220000267028809 +9.25 9.25 +9.26 9.260000228881836 +9.35 12.350000381469727 +9.48 9.479999542236328 +9.56 12.480000495910645 +9.57 9.569999694824219 +9.57 9.569999694824219 +9.68 9.680000305175781 +9.7 9.699999809265137 +9.71 17.25 +9.74 9.739999771118164 +9.8 9.800000190734863 +9.81 9.8100004196167 +9.93 9.930000305175781 +10.09 10.09000015258789 +10.09 10.09000015258789 +10.13 15.640000343322754 +10.16 18.139999866485596 +10.17 14.970000267028809 +10.19 10.1899995803833 +10.2 10.199999809265137 +10.22 10.220000267028809 +10.25 18.859999656677246 +10.26 10.260000228881836 +10.29 10.289999961853027 +10.6 10.600000381469727 +10.66 10.65999984741211 +10.67 10.670000076293945 +10.73 10.729999542236328 +11.15 20.27999973297119 +11.18 13.360000371932983 +11.19 18.979999542236328 +11.22 11.220000267028809 +11.34 16.62000036239624 +11.55 11.550000190734863 +11.57 11.569999694824219 +11.68 17.080000400543213 +11.82 11.819999694824219 +11.89 11.890000343322754 +11.91 11.90999984741211 +12.02 12.020000457763672 +12.16 12.15999984741211 +12.19 20.149999618530273 +12.32 12.319999694824219 +12.42 16.27999997138977 +12.44 12.4399995803833 +12.45 16.799999713897705 +12.46 12.460000038146973 +12.5 15.460000038146973 +12.54 12.539999961853027 +12.85 12.850000381469727 +12.9 12.899999618530273 +13.01 28.47000026702881 +13.1 23.030000686645508 +13.15 20.96999979019165 +13.35 13.350000381469727 +13.87 13.869999885559082 +13.89 13.890000343322754 +13.94 13.9399995803833 +13.99 13.989999771118164 +14.13 24.22000026702881 +14.21 14.210000038146973 +14.3 29.270000457763672 +14.44 14.4399995803833 +14.84 14.84000015258789 +14.92 14.920000076293945 +14.92 25.18000030517578 +14.93 30.40999984741211 +15.1 17.700000286102295 +15.15 24.40999984741211 +15.18 26.730000495910645 +15.22 15.220000267028809 +15.26 15.260000228881836 +15.3 25.5 +15.37 15.369999885559082 +15.45 15.449999809265137 +15.63 28.110000610351562 +15.75 15.75 +15.81 15.8100004196167 +15.86 25.079999923706055 +15.9 21.34999942779541 +15.92 15.920000076293945 +16.08 22.75 +16.09 16.520000159740448 +16.24 27.809999465942383 +16.25 19.419999957084656 +16.48 16.479999542236328 +16.69 16.690000534057617 +16.99 16.989999771118164 +16.99 42.489999771118164 +17.16 21.12999987602234 +17.37 31.360000610351562 +17.74 19.049999713897705 +17.79 45.60000038146973 +17.87 18.710000813007355 +18.2 18.200000762939453 +18.5 31.350000381469727 +18.56 18.559999465942383 +18.63 26.589999198913574 +18.63 30.389999270439148 +18.86 18.96000061184168 +18.89 18.889999389648438 +18.93 18.93000030517578 +19.0 35.62000036239624 +19.03 19.030000686645508 +19.06 19.059999465942383 +19.06 19.059999465942383 +19.13 45.719998359680176 +19.14 19.139999389648438 +19.28 27.600000381469727 +19.69 36.49000024795532 +20.07 46.80000019073486 +20.38 51.72999954223633 +20.64 20.639999389648438 +20.67 31.830000400543213 +20.79 20.790000915527344 +20.81 20.809999465942383 +20.82 21.419999718666077 +20.82 26.359999656677246 +21.18 21.18000030517578 +21.19 21.190000534057617 +21.23 21.229999542236328 +21.28 29.350000381469727 +21.32 21.31999969482422 +21.45 40.510000228881836 +21.49 30.739999771118164 +21.61 37.0600004196167 +21.7 27.58000087738037 +21.8 21.799999237060547 +21.94 23.940000593662262 +22.01 28.850000381469727 +22.08 22.079999923706055 +22.12 22.1200008392334 +22.12 22.1200008392334 +22.25 22.25 +22.27 22.270000457763672 +22.36 22.360000610351562 +22.68 22.68000030517578 +22.78 47.19000053405762 +22.85 33.070000648498535 +22.85 43.980000257492065 +22.94 35.38000011444092 +23.07 23.06999969482422 +23.13 28.799999237060547 +23.17 44.58999979496002 +23.19 23.190000534057617 +23.44 23.440000534057617 +23.45 24.74000072479248 +23.6 33.16000032424927 +23.77 23.770000457763672 +23.96 23.959999084472656 +24.02 24.020000457763672 +24.28 43.310001373291016 +24.49 42.62999963760376 +24.52 32.59000015258789 +24.73 45.369998931884766 +24.79 24.790000915527344 +24.8 34.369998931884766 +24.83 36.05000019073486 +24.86 65.3700008392334 +25.11 44.170000076293945 +25.28 25.280000686645508 +25.37 48.05000114440918 +25.42 40.78999996185303 +25.55 26.62999927997589 +25.67 37.69000053405762 +25.88 61.49999952316284 +26.08 26.079999923706055 +26.39 34.959999084472656 +26.43 26.43000030517578 +26.47 31.389999389648438 +26.49 26.489999771118164 +26.49 48.56999969482422 +26.64 64.32999992370605 +26.71 36.999999046325684 +26.73 45.69000015407801 +26.76 26.760000228881836 +27.07 28.649999737739563 +27.12 32.20000076293945 +27.3 70.61000061035156 +27.31 56.579999923706055 +27.63 27.6299991607666 +27.66 27.65999984741211 +27.72 46.60999870300293 +27.87 27.8700008392334 +28.11 44.59000015258789 +28.31 52.079999923706055 +28.45 74.05000114440918 +28.5 35.36999988555908 +28.56 40.71999931335449 +28.69 28.690000534057617 +28.71 55.46999931335449 +28.79 28.790000915527344 +28.89 56.489999771118164 +28.95 33.410000801086426 +29.02 56.64999961853027 +29.24 99.85000038146973 +29.36 62.52000093460083 +29.4 72.02999925613403 +29.41 64.77999973297119 +29.54 29.540000915527344 +29.59 37.37000012397766 +29.78 66.77999973297119 +30.25 30.32999999821186 +30.36 30.360000610351562 +30.37 31.660000830888748 +30.61 30.610000610351562 +30.62 102.65000009536743 +30.63 30.6299991607666 +30.65 60.19000053405762 +30.71 49.849998474121094 +30.81 55.989999771118164 +31.01 31.010000228881836 +31.15 31.149999618530273 +31.4 31.399999618530273 +31.61 31.610000610351562 +31.67 40.46000003814697 +31.77 42.09000027179718 +31.86 31.860000610351562 +31.91 78.51999855041504 +32.01 60.47999858856201 +32.18 58.61000061035156 +32.2 53.55000019073486 +32.23 42.89999961853027 +32.25 59.83000087738037 +32.37 62.99999809265137 +32.41 32.40999984741211 +32.47 41.14000129699707 +32.52 95.0400013923645 +32.75 56.19000053405762 +32.89 80.07999992370605 +32.92 47.7599983215332 +33.36 45.27000045776367 +33.52 60.010000228881836 +33.55 63.939998507499695 +33.58 54.55000162124634 +33.67 33.66999816894531 +33.76 112.27999687194824 +33.83 59.110002517700195 +33.85 47.719998359680176 +33.87 37.48999881744385 +34.03 71.51999759674072 +34.21 71.57999920845032 +34.35 34.349998474121094 +34.41 59.20000076293945 +34.58 34.58000183105469 +34.73 34.72999954223633 +34.97 45.160000801086426 +35.0 35.0 +35.08 36.060001850128174 +35.13 39.600000858306885 +35.17 64.01999855041504 +35.17 66.52999877929688 +35.56 37.63000130653381 +35.62 80.20999872684479 +35.65 56.83000183105469 +35.68 52.20000046491623 +35.72 98.71999931335449 +35.8 79.96999931335449 +35.89 81.2599983215332 +36.22 55.20000076293945 +36.26 36.2599983215332 +36.58 64.45000267028809 +36.7 73.76000118255615 +36.79 36.790000915527344 +36.89 71.8499984741211 +36.95 36.95000076293945 +37.07 37.06999969482422 +37.1 51.039998054504395 +37.14 53.41999936103821 +37.14 61.15999984741211 +37.24 47.04000186920166 +37.59 50.94000053405762 +37.6 84.39999866485596 +37.72 57.14000117778778 +37.78 68.10999877750874 +37.8 57.94999885559082 +37.85 50.38999843597412 +37.9 77.50000238418579 +38.05 40.24999928474426 +38.05 47.859999656677246 +38.3 98.48999977111816 +38.33 112.09000301361084 +38.57 55.81999969482422 +38.62 73.99999904632568 +38.79 95.44000053405762 +38.85 97.45999908447266 +38.88 48.58000087738037 +38.94 71.3499984741211 +39.01 39.0099983215332 +39.03 57.73999959230423 +39.18 99.6599988937378 +39.21 71.79999923706055 +39.34 39.34000015258789 +39.69 55.69999837875366 +39.81 74.81000137329102 +39.82 39.81999969482422 +39.83 119.80000114440918 +39.87 62.22999954223633 +39.9 64.98000144958496 +39.98 39.97999954223633 +40.0 84.59000015258789 +40.04 101.20000076293945 +40.17 80.41999745368958 +40.24 67.9000015258789 +40.42 78.04999947547913 +40.44 117.94000101089478 +40.78 49.8199987411499 +40.8 40.79999923706055 +40.98 92.01999759674072 +41.2 58.280001163482666 +41.29 41.290000915527344 +41.29 112.87000012397766 +41.31 53.08000135421753 +41.34 53.230000495910645 +41.34 115.33999919891357 +41.36 41.36000061035156 +41.44 85.41999888420105 +41.45 91.29999923706055 +41.62 41.619998931884766 +41.68 109.58000183105469 +41.71 83.06999969482422 +41.81 89.8600025177002 +41.85 76.21999740600586 +41.87 47.48999881744385 +41.89 41.88999938964844 +42.24 72.6500015258789 +42.31 52.91000175476074 +42.42 154.69999504089355 +42.48 71.12999927997589 +42.51 61.069997787475586 +42.55 87.71000003814697 +42.56 71.91000175476074 +42.67 71.45999908447266 +42.76 42.7599983215332 +42.85 85.33999824523926 +43.01 109.53999710083008 +43.02 46.84000039100647 +43.13 43.130001068115234 +43.16 58.079999923706055 +43.17 43.16999816894531 +43.19 156.05999875068665 +43.31 64.10000228881836 +43.37 56.84999895095825 +43.57 43.71999970078468 +43.71 108.69000053405762 +43.73 108.50999927520752 +43.92 48.319998145103455 +44.1 67.1299991607666 +44.22 103.33000373840332 +44.27 74.88000106811523 +44.43 106.65999984741211 +44.57 59.00999927520752 +45.06 45.060001373291016 +45.1 45.099998474121094 +45.19 45.189998626708984 +45.19 117.10000038146973 +45.24 109.34000396728516 +45.34 129.73999881744385 +45.35 109.28999698162079 +45.42 77.61999893188477 +45.45 45.45000076293945 +45.56 137.57999897003174 +45.59 82.54000091552734 +45.68 55.25 +45.92 90.97999954223633 +45.99 47.07000172138214 +46.02 82.27999877929688 +46.09 46.09000015258789 +46.1 98.17999839782715 +46.15 72.7800008058548 +46.18 74.29000091552734 +46.21 52.75999927520752 +46.27 85.61000061035156 +46.43 106.26000118255615 +46.45 110.90000343322754 +46.62 78.0099983215332 +46.8 80.46999740600586 +46.86 62.08000087738037 +46.87 70.80999952554703 +46.88 106.08000183105469 +46.97 88.1100025177002 +47.08 148.28000259399414 +47.27 50.60000038146973 +47.32 118.12999922037125 +47.57 90.32999801635742 +47.59 104.17000007629395 +47.69 99.88999909162521 +47.88 47.880001068115234 +48.01 91.72999802231789 +48.08 79.69000244140625 +48.11 48.11000061035156 +48.15 65.8500018119812 +48.22 105.07000017166138 +48.23 139.52999877929688 +48.25 48.25 +48.28 98.669997215271 +48.37 185.9499979019165 +48.45 48.45000076293945 +48.45 94.54000091552734 +48.52 146.69999885559082 +48.59 89.30999946594238 +49.12 49.119998931884766 +49.28 123.56999969482422 +49.44 110.93999814987183 +49.68 73.63999938964844 +49.77 50.33000046014786 +49.78 66.46999931335449 +50.02 63.380000829696655 +50.08 156.16000366210938 +50.09 106.28000068664551 +50.26 72.05999755859375 +50.28 50.279998779296875 +50.31 117.44000053405762 +50.32 90.13999938964844 +50.4 96.11999988555908 +50.41 98.72999799251556 +50.66 55.37999963760376 +50.7 131.11999821662903 +50.83 98.69000148773193 +50.92 53.70999813079834 +50.96 103.71999835968018 +51.25 67.0 +51.29 87.35000276565552 +51.29 124.93000030517578 +51.72 97.17000198364258 +51.79 139.90000343322754 +51.84 168.94000053405762 +51.85 171.64999961853027 +52.17 206.86999320983887 +52.23 177.1599998474121 +52.44 88.48999881744385 +52.5 105.41000175476074 +52.53 64.98999881744385 +52.72 52.720001220703125 +52.73 74.04999923706055 +52.85 89.63999938964844 +52.87 130.87999725341797 +53.02 100.50999927520752 +53.06 259.9299945831299 +53.18 53.18000030517578 +53.27 53.27000045776367 +53.59 53.59000015258789 +53.78 139.38999938964844 +53.93 57.890000343322754 +53.94 63.529998779296875 +54.1 152.7699956893921 +54.31 77.38000106811523 +54.34 125.46999943256378 +54.43 132.04999923706055 +54.44 103.01999950408936 +54.47 186.52000045776367 +54.73 63.179999351501465 +54.75 112.82999992370605 +54.83 110.82000160217285 +54.99 160.40000343322754 +55.1 161.35999965667725 +55.18 215.58000373840332 +55.2 126.65999984741211 +55.39 137.6699981689453 +55.51 74.55999803543091 +55.63 96.43000030517578 +55.99 187.10999989509583 +56.04 150.5800018310547 +56.07 118.15000057220459 +56.1 135.79000091552734 +56.15 144.64000034332275 +56.33 61.77000188827515 +56.62 78.88999938964844 +56.68 154.13999938964844 +56.81 169.64000129699707 +57.08 69.98000144958496 +57.11 168.0100040435791 +57.12 100.28999710083008 +57.23 65.9399995803833 +57.25 133.46999740600586 +57.29 112.54000091552734 +57.35 110.89999866485596 +57.37 115.109998524189 +57.46 147.78999710083008 +57.64 112.19000101089478 +57.67 57.66999816894531 +57.89 111.15999984741211 +57.93 68.02000045776367 +58.0 123.9399995803833 +58.08 58.08000183105469 +58.09 206.37000274658203 +58.13 105.84999942779541 +58.43 165.0900001525879 +58.52 167.0299997329712 +58.66 136.04000091552734 +58.67 205.36999702453613 +58.75 90.41000083088875 +58.86 165.14000129699707 +59.07 87.86999893188477 +59.16 224.25 +59.21 90.35999870300293 +59.34 127.44999893009663 +59.43 106.50000202655792 +59.45 67.90000057220459 +59.45 197.11999893188477 +59.5 149.63999938964844 +59.55 61.459999203681946 +59.61 85.97000026702881 +59.62 113.3299970626831 +59.68 73.89000034332275 +59.68 94.40999984741211 +59.7 193.1699981689453 +59.71 60.22999906539917 +59.83 145.17000007629395 +59.87 228.80999946594238 +59.99 134.04000282287598 +60.02 66.76000022888184 +60.06 60.060001373291016 +60.12 113.34999942779541 +60.13 214.27000045776367 +60.22 108.10000228881836 +60.26 105.94999847561121 +60.26 165.32999849319458 +60.53 66.37999868392944 +60.6 82.8499984741211 +60.71 72.04999899864197 +60.85 132.36999607086182 +61.21 160.86999797821045 +61.7 127.55000257492065 +61.86 248.9700005054474 +61.88 112.15999984741211 +61.92 125.29999899864197 +61.94 119.6099967956543 +62.14 110.59000015258789 +62.2 149.91000080108643 +62.23 111.3499984741211 +62.3 158.41999912261963 +62.39 110.95999908447266 +62.52 123.97999966144562 +62.72 123.78999900817871 +62.74 153.10000038146973 +62.85 167.01999855041504 +62.9 256.0699996948242 +62.92 129.3899974822998 +63.12 93.47999954223633 +63.33 135.38999938964844 +63.35 116.93999862670898 +63.42 172.76000213623047 +63.51 123.51999855041504 +63.9 135.70000076293945 +64.0 191.55000257492065 +64.22 86.97000122070312 +64.25 131.25 +64.3 122.3800048828125 +64.36 85.59000015258789 +64.46 134.44000053405762 +64.65 143.54000091552734 +64.67 121.15999794006348 +64.77 214.40999603271484 +64.87 194.61000156402588 +64.95 324.87999153137207 +65.02 175.6099967956543 +65.02 259.6299982070923 +65.38 168.7100009918213 +65.43 112.27000069618225 +65.43 289.6800003051758 +65.44 192.89000137150288 +65.55 66.16000306606293 +65.62 139.67000198364258 +65.7 65.69999694824219 +65.72 77.54000091552734 +66.17 177.10999631881714 +66.17 200.60999870300293 +66.36 131.73000144958496 +66.51 83.50000190734863 +66.61 66.61000061035156 +66.61 78.93000030517578 +66.67 129.84999752044678 +66.89 99.96000003814697 +67.12 67.12000274658203 +67.18 234.21000003814697 +67.26 77.9900016784668 +67.38 178.72999572753906 +67.45 197.29999446868896 +67.48 268.0900020599365 +67.59 272.95999336242676 +67.94 125.89000129699707 +67.98 123.36000299453735 +68.01 124.84000396728516 +68.04 166.76000022888184 +68.22 181.57000064849854 +68.25 113.52000045776367 +68.25 163.2900013923645 +68.32 247.04999542236328 +68.41 157.72000312805176 +68.5 156.36999893188477 +68.81 93.02999782562256 +68.85 160.57999649643898 +68.89 89.69999885559082 +68.95 140.99999594688416 +68.96 192.4799976348877 +69.32 156.29000091552734 +69.53 239.17000007629395 +69.74 246.84999418258667 +69.8 101.63000345230103 +69.88 220.45999908447266 +69.96 83.84999942779541 +69.97 169.86000031232834 +70.0 236.76000022888184 +70.04 196.70000076293945 +70.06 92.17999839782715 +70.24 213.77999877929688 +70.35 247.5099983215332 +70.38 179.95999908447266 +70.39 164.79999923706055 +70.52 181.33999824523926 +70.53 70.52999877929688 +70.56 110.53999710083008 +70.85 223.94999885559082 +70.89 149.81999969482422 +70.93 160.79000282287598 +71.01 92.20000267028809 +71.07 112.36000061035156 +71.13 135.149995803833 +71.19 136.1800012588501 +71.26 318.7700004577637 +71.31 231.88999405503273 +71.32 145.21000003814697 +71.35 145.909996509552 +71.5 217.409996509552 +71.54 71.54000091552734 +71.55 90.41000270843506 +71.68 72.9500002861023 +71.68 227.97000122070312 +71.78 189.71999979019165 +71.8 103.66000366210938 +71.89 180.57999992370605 +72.04 156.63000106811523 +72.18 181.46999728679657 +72.51 264.0600047111511 +72.53 265.42000015079975 +72.56 132.61999893188477 +72.62 205.2400016784668 +72.79 72.79000091552734 +72.98 286.76000213623047 +73.18 123.78000068664551 +73.32 175.96999979019165 +73.48 144.8300018310547 +73.63 320.47999143600464 +73.65 114.11000156402588 +73.68 161.0300030708313 +73.88 291.28999376296997 +73.93 156.77999877929688 +74.0 179.8499994277954 +74.02 89.27999687194824 +74.15 74.1500015258789 +74.19 74.47000244259834 +74.19 122.44000244140625 +74.3 159.72000193595886 +74.42 219.62999820709229 +74.45 203.839994430542 +74.52 271.6399955749512 +74.53 342.6200008392334 +74.59 157.12999725341797 +74.62 163.89999961853027 +74.72 139.05000114440918 +74.78 230.83999752998352 +75.03 174.98999881744385 +75.1 214.62999725341797 +75.19 149.6600048840046 +75.29 93.4900016784668 +75.35 256.91999912261963 +75.42 153.46999764442444 +75.66 225.4800033569336 +75.73 236.60000133514404 +75.83 199.4000015258789 +75.88 243.89000129699707 +76.05 179.770001411438 +76.1 257.43999671936035 +76.28 168.48000144958496 +76.28 177.9100022315979 +76.33 363.09000396728516 +76.52 207.39999389648438 +76.69 212.87000370025635 +76.7 307.5399944782257 +76.71 132.17999839782715 +76.72 166.42000007629395 +76.72 258.1899985074997 +76.74 102.81999778747559 +76.92 401.7999897003174 +76.93 384.4699947834015 +77.02 296.6499948501587 +77.1 187.63999557495117 +77.36 281.19999504089355 +77.42 217.32000160217285 +77.57 189.66000270843506 +77.66 267.380003452301 +77.81 178.31999683380127 +77.84 350.7999897003174 +77.89 249.5399990081787 +77.97 178.2599983215332 +78.21 189.1099977493286 +78.26 284.6300048828125 +78.28 306.25 +78.3 168.71000388264656 +78.31 161.80999946594238 +78.62 275.3200035095215 +78.64 94.55999946594238 +78.73 310.6199974119663 +78.89 175.31999969482422 +78.98 293.25000381469727 +79.12 276.419997215271 +79.19 194.30000096559525 +79.21 237.62999820709229 +79.38 151.22999572753906 +79.42 124.6099967956543 +79.48 116.55000305175781 +79.48 200.64000129699707 +79.49 294.1199951171875 +79.54 145.23999786376953 +79.55 254.54000186920166 +79.75 215.13999938964844 +79.83 294.9700012207031 +79.96 79.95999908447266 +79.97 266.4900016784668 +79.99 219.6599998474121 +80.23 248.71000480651855 +80.3 133.4800033569336 +80.3 305.7800064086914 +80.46 249.17000296711922 +80.52 136.21999502182007 +80.58 261.16000175476074 +80.6 281.23999977111816 +80.71 275.0100000500679 +80.74 240.4599997997284 +80.84 142.60999822616577 +80.92 340.5499963760376 +80.96 372.2499928474426 +80.97 93.32000160217285 +80.99 317.74999809265137 +81.17 220.55999755859375 +81.32 158.86000061035156 +81.32 206.16000366210938 +81.47 198.91000175476074 +81.58 173.76000022888184 +81.64 207.5300006866455 +81.66 261.5100030899048 +82.24 243.59999752044678 +82.3 157.18000411987305 +82.34 214.0699977874756 +82.41 135.83000302314758 +82.52 240.23999977111816 +82.55 139.6900042295456 +82.56 175.8799991607666 +82.72 232.63000202178955 +82.97 101.9000015258789 +83.08 181.57000160217285 +83.27 83.2699966430664 +83.33 348.75000198185444 +83.4 196.729998588562 +83.54 299.12000465393066 +83.57 139.38999938964844 +83.58 163.54000091552734 +83.87 263.64000415802 +83.92 432.67000015079975 +83.93 358.9400003552437 +84.03 240.19000244140625 +84.23 209.53000235557556 +84.31 157.0999984741211 +84.38 220.1699981689453 +84.4 443.3400018811226 +84.69 249.4900016784668 +84.72 105.00000095367432 +84.83 157.4800033569336 +85.0 221.21999502182007 +85.03 283.9400005340576 +85.1 217.27999687194824 +85.14 266.7100009918213 +85.23 277.7100009918213 +85.49 261.4599976539612 +85.49 325.9499976634979 +85.51 165.59000205993652 +85.74 341.80999755859375 +85.76 160.57000350952148 +85.87 357.5099983215332 +85.9 202.45000457763672 +86.0 322.60000133514404 +86.22 152.8300018310547 +86.23 152.99000358581543 +86.63 381.5999984741211 +86.69 138.42000198364258 +86.92 147.14999723434448 +86.93 97.59000015258789 +86.93 218.18000030517578 +87.14 219.50999546051025 +87.22 211.01000022888184 +87.4 137.73000198602676 +87.48 153.86000204086304 +87.57 307.0799951553345 +87.61 321.82000064849854 +87.67 248.4600009918213 +87.83 130.96000289916992 +87.94 134.9800043106079 +87.99 227.0399990081787 +88.02 406.7899971008301 +88.05 97.53000259399414 +88.07 210.45000457763672 +88.17 245.6500015258789 +88.22 369.4600009918213 +88.36 305.6399974822998 +88.47 213.9400006532669 +88.48 211.84000635147095 +88.55 199.71000289916992 +88.77 187.49999463558197 +88.78 337.23999977111816 +88.8 254.1300015449524 +88.91 276.40999829769135 +89.01 98.69000244140625 +89.03 431.6499996185303 +89.1 113.83999919891357 +89.15 234.32000160217285 +89.28 343.82000064849854 +89.38 496.16999435424805 +89.53 153.05999755859375 +89.55 89.55000305175781 +89.55 96.27000284194946 +89.55 172.62000274658203 +89.8 205.1400022506714 +89.81 225.64000058174133 +89.93 89.93000030517578 +90.05 90.05000305175781 +90.05 247.1500015258789 +90.07 137.82999801635742 +90.12 234.76000308990479 +90.2 208.32999616861343 +90.25 386.8999948501587 +90.28 334.17000007629395 +90.35 158.36999893188477 +90.38 148.65999841690063 +90.51 268.7700004577637 +90.56 298.8899937272072 +90.69 181.1000051498413 +90.69 523.360002592206 +90.73 260.59000366926193 +90.77 203.03999733924866 +91.05 263.67000579833984 +91.16 298.55999755859375 +91.42 196.41999912261963 +91.48 144.56000471115112 +91.53 301.9800033569336 +91.61 215.59000027179718 +91.63 432.17999362945557 +91.78 330.9499988555908 +91.88 277.8299951553345 +91.97 205.4900016784668 +92.05 317.69000363349915 +92.11 307.70000088214874 +92.33 92.33000183105469 +92.37 226.410005569458 +92.4 265.1600036621094 +92.55 295.59000039100647 +92.61 249.38999938964844 +92.82 182.87000274658203 +92.96 185.98999691009521 +92.98 494.779993057251 +93.03 165.80999958515167 +93.09 456.1800003051758 +93.11 427.2800006866455 +93.61 465.8599934577942 +93.64 314.1999969482422 +93.73 475.3300018310547 +94.08 187.56000137329102 +94.15 362.9200019836426 +94.25 308.1900006532669 +94.27 351.70999336242676 +94.31 216.75 +94.33 193.02000427246094 +94.34 320.75000190734863 +94.38 260.18999683856964 +94.43 174.89999771118164 +94.54 340.19000244140625 +94.66 192.19000625610352 +94.68 189.23999977111816 +94.68 344.1700019836426 +94.72 274.6800003051758 +95.07 417.67000102996826 +95.11 393.99999433755875 +95.28 140.37999725341797 +95.33 244.99000671505928 +95.34 192.92999649047852 +95.38 230.36000156402588 +95.48 406.1000007688999 +95.53 248.58999633789062 +95.53 252.71000289916992 +95.81 134.81999588012695 +95.81 551.9899978637695 +95.84 274.1599931716919 +96.09 191.52999687194824 +96.23 398.2100067138672 +96.25 261.8400020599365 +96.29 199.10999870300293 +96.38 323.4199962615967 +96.62 309.4900064468384 +96.68 336.87000274658203 +96.73 272.0500030517578 +96.78 190.27000045776367 +96.91 180.18000030517578 +96.94 216.54999923706055 +97.09 428.03999519348145 +97.24 346.41000083088875 +97.26 373.67999935150146 +97.39 257.9600028991699 +97.46 449.1699924468994 +97.51 132.09000396728516 +97.56 97.55999755859375 +97.57 592.3499927520752 +97.65 196.34000301361084 +97.68 258.7100033760071 +97.71 175.70000076293945 +97.81 297.20999908447266 +97.83 396.38999938964844 +97.87 254.50000381469727 +98.18 105.29000043869019 +98.22 438.4100036621094 +98.23 525.5100040435791 +98.31 335.93999576568604 +98.48 286.0400047302246 +98.51 132.86000061035156 +98.57 404.3500061035156 +98.72 263.8600025177002 +98.96 288.1999988555908 +99.13 243.6900019645691 +99.15 210.11000061035156 +99.21 347.7999954223633 +99.24 537.6500015258789 +99.29 291.48000717163086 +99.36 106.41000080108643 +99.62 417.3700008392334 +99.65 185.62000179290771 +99.67 180.92999649047852 +99.68 230.6400032043457 +99.91 367.2900071144104 +99.92 376.32999646663666 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c new file mode 100644 index 0000000000000..1a4528978b74f --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-7-20fdc99aa046b2c41d9b85ab338c749c @@ -0,0 +1,1049 @@ + 65560 3.1 + 65718 2.38 + 65740 3.04 +alice allen 65662 1.55 +alice allen 65720 1.79 +alice allen 65758 1.98 +alice brown 65696 0.13 +alice carson 65559 4.2 +alice davidson 65547 1.51 +alice falkner 65669 4.19 +alice garcia 65613 0.72 +alice hernandez 65737 0.92 +alice hernandez 65784 2.09 +alice johnson 65739 2.55 +alice king 65660 3.84 +alice king 65734 2.96 +alice king 65738 2.14 +alice laertes 65669 0.28 +alice laertes 65671 1.16 +alice miller 65590 4.95 +alice nixon 65586 2.98 +alice nixon 65595 2.36 +alice nixon 65604 2.45 +alice ovid 65737 3.2 +alice polk 65548 1.23 +alice quirinius 65636 0.23 +alice quirinius 65728 0.82 +alice robinson 65606 3.99 +alice robinson 65789 4.35 +alice steinbeck 65578 4.72 +alice steinbeck 65673 3.97 +alice steinbeck 65786 3.92 +alice underhill 65750 2.06 +alice van buren 65562 2.43 +alice xylophone 65578 2.22 +alice xylophone 65585 2.11 +alice xylophone 65599 2.92 +alice zipper 65553 3.78 +alice zipper 65662 2.61 +alice zipper 65766 3.12 +bob brown 65584 2.09 +bob brown 65777 1.62 +bob brown 65783 2.4 +bob carson 65713 3.87 +bob davidson 65664 4.25 +bob davidson 65693 3.54 +bob davidson 65768 2.91 +bob ellison 65591 2.23 +bob ellison 65624 1.69 +bob ellison 65721 1.69 +bob ellison 65760 1.69 +bob falkner 65789 0.21 +bob garcia 65585 3.91 +bob garcia 65598 2.64 +bob garcia 65673 3.26 +bob garcia 65754 3.29 +bob garcia 65782 2.86 +bob hernandez 65557 3.72 +bob ichabod 65549 1.39 +bob king 65715 2.76 +bob king 65757 1.71 +bob king 65783 2.24 +bob laertes 65602 4.98 +bob laertes 65663 3.56 +bob miller 65608 4.95 +bob ovid 65564 1.23 +bob ovid 65619 1.53 +bob ovid 65686 1.84 +bob ovid 65726 2.38 +bob polk 65594 0.28 +bob quirinius 65700 3.82 +bob steinbeck 65637 0.22 +bob van buren 65778 2.89 +bob white 65543 4.75 +bob white 65605 2.89 +bob xylophone 65574 1.7 +bob xylophone 65666 2.51 +bob young 65556 0.95 +bob zipper 65559 3.18 +bob zipper 65633 3.2 +bob zipper 65739 3.24 +calvin allen 65669 2.3 +calvin brown 65537 1.1 +calvin brown 65580 2.82 +calvin brown 65677 2.57 +calvin carson 65637 1.33 +calvin davidson 65541 1.98 +calvin davidson 65564 1.6 +calvin ellison 65667 1.85 +calvin falkner 65573 3.52 +calvin falkner 65596 3.31 +calvin falkner 65738 2.36 +calvin falkner 65762 2.26 +calvin falkner 65778 2.7 +calvin falkner 65784 2.98 +calvin garcia 65664 2.9 +calvin hernandez 65578 2.08 +calvin johnson 65731 4.69 +calvin laertes 65570 4.7 +calvin laertes 65684 3.68 +calvin nixon 65654 2.74 +calvin nixon 65724 3.27 +calvin nixon 65749 2.88 +calvin ovid 65554 3.34 +calvin ovid 65643 2.38 +calvin ovid 65663 2.74 +calvin ovid 65715 2.47 +calvin polk 65731 4.36 +calvin quirinius 65741 4.0 +calvin quirinius 65769 2.2 +calvin robinson 65581 3.32 +calvin steinbeck 65680 1.29 +calvin steinbeck 65762 1.3 +calvin steinbeck 65779 1.98 +calvin thompson 65560 4.08 +calvin thompson 65640 3.33 +calvin underhill 65732 2.41 +calvin van buren 65552 1.05 +calvin van buren 65771 1.33 +calvin white 65553 4.7 +calvin white 65561 2.68 +calvin xylophone 65575 4.6 +calvin xylophone 65596 4.77 +calvin xylophone 65713 3.23 +calvin young 65574 0.27 +calvin young 65746 0.9 +calvin zipper 65669 4.4 +calvin zipper 65739 2.29 +david allen 65588 3.86 +david allen 65617 3.18 +david brown 65637 1.17 +david brown 65760 1.01 +david davidson 65559 1.37 +david davidson 65756 1.57 +david davidson 65778 1.89 +david davidson 65779 2.43 +david ellison 65634 3.23 +david ellison 65724 2.95 +david ellison 65724 2.95 +david hernandez 65763 1.15 +david ichabod 65699 1.67 +david ichabod 65715 1.26 +david laertes 65762 1.38 +david nixon 65536 1.27 +david ovid 65623 0.23 +david ovid 65628 1.15 +david quirinius 65697 1.14 +david quirinius 65759 1.65 +david quirinius 65779 1.93 +david robinson 65762 3.51 +david robinson 65775 3.38 +david thompson 65550 3.3 +david underhill 65602 0.12 +david underhill 65662 2.29 +david underhill 65751 2.43 +david van buren 65625 1.55 +david van buren 65634 3.25 +david white 65678 0.17 +david xylophone 65537 1.07 +david xylophone 65670 0.72 +david xylophone 65764 0.94 +david young 65551 4.51 +david young 65694 2.74 +ethan allen 65747 3.61 +ethan brown 65539 2.9 +ethan brown 65617 1.59 +ethan brown 65685 2.17 +ethan brown 65685 2.17 +ethan brown 65722 2.64 +ethan brown 65733 2.75 +ethan carson 65742 2.84 +ethan ellison 65714 4.87 +ethan ellison 65732 3.9 +ethan falkner 65577 3.61 +ethan falkner 65614 1.95 +ethan garcia 65736 4.63 +ethan hernandez 65618 0.46 +ethan johnson 65536 1.76 +ethan king 65614 0.92 +ethan laertes 65562 2.89 +ethan laertes 65597 3.45 +ethan laertes 65628 3.15 +ethan laertes 65643 3.53 +ethan laertes 65680 3.18 +ethan laertes 65745 3.36 +ethan laertes 65760 3.01 +ethan miller 65712 1.97 +ethan nixon 65766 4.1 +ethan ovid 65697 3.81 +ethan polk 65589 0.7 +ethan polk 65615 1.92 +ethan polk 65622 2.0 +ethan polk 65622 2.0 +ethan quirinius 65542 4.64 +ethan quirinius 65591 3.97 +ethan quirinius 65706 2.88 +ethan robinson 65547 2.2 +ethan robinson 65659 2.17 +ethan underhill 65570 2.45 +ethan van buren 65572 1.11 +ethan white 65677 3.42 +ethan white 65788 4.09 +ethan xylophone 65595 4.66 +ethan zipper 65593 2.1 +ethan zipper 65680 2.53 +fred davidson 65552 0.37 +fred davidson 65595 2.31 +fred davidson 65721 2.65 +fred ellison 65548 4.32 +fred ellison 65691 3.44 +fred ellison 65771 3.1 +fred falkner 65637 4.59 +fred falkner 65648 2.79 +fred falkner 65783 2.35 +fred hernandez 65541 3.87 +fred ichabod 65572 1.45 +fred ichabod 65789 1.68 +fred johnson 65758 3.86 +fred king 65694 4.05 +fred king 65745 4.5 +fred laertes 65769 3.89 +fred miller 65536 2.59 +fred nixon 65560 2.52 +fred nixon 65612 2.0 +fred nixon 65703 1.35 +fred nixon 65705 1.16 +fred polk 65603 2.9 +fred polk 65656 1.65 +fred polk 65701 1.75 +fred polk 65706 1.4 +fred quirinius 65697 1.91 +fred quirinius 65782 3.24 +fred robinson 65623 2.6 +fred steinbeck 65544 2.0 +fred steinbeck 65651 3.32 +fred steinbeck 65755 3.51 +fred underhill 65629 2.56 +fred van buren 65537 4.97 +fred van buren 65561 3.28 +fred van buren 65745 3.24 +fred van buren 65789 3.27 +fred white 65589 3.16 +fred young 65594 0.34 +fred young 65773 1.56 +fred zipper 65553 0.37 +gabriella allen 65646 1.68 +gabriella allen 65677 1.6 +gabriella brown 65704 0.02 +gabriella brown 65753 1.86 +gabriella carson 65586 0.37 +gabriella davidson 65565 3.45 +gabriella ellison 65706 1.15 +gabriella ellison 65716 2.06 +gabriella falkner 65623 2.09 +gabriella falkner 65711 2.48 +gabriella falkner 65767 1.82 +gabriella garcia 65571 3.17 +gabriella hernandez 65587 0.74 +gabriella hernandez 65717 0.96 +gabriella ichabod 65559 0.63 +gabriella ichabod 65633 2.42 +gabriella ichabod 65702 3.27 +gabriella ichabod 65712 3.6 +gabriella ichabod 65717 3.56 +gabriella king 65651 3.59 +gabriella king 65657 2.17 +gabriella laertes 65781 2.81 +gabriella miller 65646 3.47 +gabriella ovid 65556 1.23 +gabriella ovid 65583 1.95 +gabriella polk 65701 3.58 +gabriella polk 65790 2.08 +gabriella steinbeck 65582 3.6 +gabriella steinbeck 65653 2.7 +gabriella thompson 65682 1.78 +gabriella thompson 65755 3.21 +gabriella thompson 65766 2.71 +gabriella van buren 65581 3.36 +gabriella van buren 65644 2.6 +gabriella white 65638 4.55 +gabriella young 65699 4.13 +gabriella young 65774 3.58 +gabriella zipper 65540 0.96 +gabriella zipper 65754 2.13 +holly allen 65596 0.05 +holly brown 65599 3.23 +holly brown 65619 3.4 +holly falkner 65720 4.06 +holly hernandez 65602 3.67 +holly hernandez 65686 3.39 +holly hernandez 65750 3.2 +holly hernandez 65788 2.95 +holly ichabod 65711 4.73 +holly ichabod 65749 3.54 +holly ichabod 65752 3.27 +holly johnson 65655 4.19 +holly johnson 65662 3.84 +holly johnson 65755 2.65 +holly king 65549 3.61 +holly king 65648 2.37 +holly laertes 65664 4.14 +holly miller 65653 3.84 +holly nixon 65539 4.09 +holly nixon 65658 3.04 +holly polk 65743 2.1 +holly polk 65751 2.78 +holly robinson 65564 0.24 +holly thompson 65538 2.39 +holly thompson 65578 1.49 +holly thompson 65713 1.54 +holly underhill 65634 4.69 +holly underhill 65654 3.08 +holly underhill 65721 3.14 +holly underhill 65759 2.61 +holly van buren 65727 0.04 +holly white 65536 4.59 +holly white 65602 4.13 +holly xylophone 65544 1.49 +holly young 65606 4.39 +holly young 65765 3.81 +holly zipper 65607 4.12 +holly zipper 65755 3.3 +irene allen 65556 3.45 +irene brown 65633 4.8 +irene brown 65650 3.77 +irene brown 65765 3.53 +irene carson 65590 2.83 +irene ellison 65659 3.15 +irene ellison 65696 2.0 +irene falkner 65620 0.99 +irene falkner 65661 1.41 +irene garcia 65660 1.55 +irene garcia 65711 1.72 +irene garcia 65787 1.57 +irene ichabod 65645 0.95 +irene ichabod 65722 1.49 +irene johnson 65583 4.51 +irene laertes 65664 2.5 +irene laertes 65710 1.5 +irene laertes 65722 2.01 +irene miller 65730 4.33 +irene nixon 65631 2.36 +irene nixon 65643 3.43 +irene nixon 65653 2.43 +irene ovid 65691 3.24 +irene ovid 65734 3.17 +irene ovid 65753 3.18 +irene polk 65551 4.51 +irene polk 65575 2.97 +irene polk 65579 3.58 +irene polk 65595 2.82 +irene polk 65610 2.99 +irene quirinius 65724 3.5 +irene quirinius 65769 3.85 +irene quirinius 65773 4.21 +irene robinson 65554 2.67 +irene steinbeck 65683 1.48 +irene thompson 65688 0.06 +irene underhill 65591 3.61 +irene underhill 65787 4.01 +irene van buren 65579 4.26 +irene van buren 65589 4.37 +irene xylophone 65775 4.81 +jessica brown 65588 2.87 +jessica carson 65553 1.02 +jessica carson 65672 1.82 +jessica carson 65747 1.91 +jessica davidson 65549 4.48 +jessica davidson 65606 2.72 +jessica davidson 65675 2.23 +jessica davidson 65727 2.12 +jessica ellison 65567 3.0 +jessica ellison 65663 3.15 +jessica falkner 65584 2.11 +jessica garcia 65676 2.13 +jessica garcia 65789 3.54 +jessica ichabod 65704 3.48 +jessica johnson 65607 3.55 +jessica johnson 65720 3.0 +jessica miller 65733 2.9 +jessica nixon 65590 2.18 +jessica nixon 65774 3.1 +jessica ovid 65582 3.23 +jessica ovid 65751 3.14 +jessica polk 65637 1.12 +jessica quirinius 65562 3.06 +jessica quirinius 65608 1.75 +jessica quirinius 65712 1.51 +jessica quirinius 65716 1.37 +jessica robinson 65576 1.11 +jessica thompson 65581 4.94 +jessica thompson 65675 3.56 +jessica underhill 65656 3.97 +jessica underhill 65702 3.01 +jessica underhill 65783 3.5 +jessica van buren 65615 2.15 +jessica white 65544 1.89 +jessica white 65570 1.92 +jessica white 65594 2.67 +jessica white 65673 2.1 +jessica white 65779 2.04 +jessica xylophone 65562 0.49 +jessica young 65623 0.5 +jessica young 65711 2.37 +jessica zipper 65600 1.71 +jessica zipper 65657 1.0 +jessica zipper 65778 0.9 +katie allen 65542 1.3 +katie brown 65590 3.06 +katie davidson 65619 1.89 +katie ellison 65675 1.92 +katie ellison 65699 2.55 +katie falkner 65728 2.42 +katie garcia 65625 4.18 +katie garcia 65747 4.33 +katie hernandez 65550 2.1 +katie ichabod 65658 1.84 +katie ichabod 65726 2.41 +katie ichabod 65757 3.2 +katie king 65629 0.86 +katie king 65647 2.09 +katie king 65776 2.74 +katie miller 65541 0.66 +katie miller 65661 1.39 +katie nixon 65669 2.06 +katie ovid 65681 1.61 +katie polk 65746 4.74 +katie polk 65784 2.57 +katie robinson 65697 4.74 +katie van buren 65643 0.61 +katie van buren 65730 2.79 +katie white 65620 0.42 +katie white 65719 1.38 +katie xylophone 65585 3.02 +katie young 65644 2.64 +katie young 65746 2.45 +katie young 65764 2.66 +katie zipper 65568 2.33 +katie zipper 65733 1.97 +luke allen 65547 2.04 +luke allen 65552 1.49 +luke allen 65576 2.14 +luke allen 65681 2.2 +luke allen 65776 1.92 +luke brown 65719 4.3 +luke davidson 65656 3.37 +luke davidson 65791 3.9 +luke ellison 65582 0.23 +luke ellison 65664 0.51 +luke ellison 65779 0.87 +luke falkner 65589 2.22 +luke falkner 65618 1.22 +luke garcia 65687 4.73 +luke garcia 65778 3.56 +luke ichabod 65629 3.07 +luke ichabod 65654 3.58 +luke johnson 65545 4.33 +luke johnson 65716 3.01 +luke johnson 65718 3.17 +luke laertes 65608 3.79 +luke laertes 65657 3.89 +luke laertes 65685 2.82 +luke laertes 65730 2.96 +luke laertes 65756 3.19 +luke miller 65752 4.7 +luke ovid 65569 4.4 +luke ovid 65693 2.58 +luke polk 65645 0.57 +luke polk 65658 2.73 +luke quirinius 65655 4.1 +luke robinson 65634 4.9 +luke robinson 65772 4.19 +luke thompson 65626 0.15 +luke underhill 65553 1.28 +luke underhill 65571 0.84 +luke underhill 65651 1.14 +luke van buren 65678 0.34 +luke white 65693 0.91 +luke xylophone 65597 2.27 +luke zipper 65641 4.63 +mike allen 65706 3.06 +mike brown 65654 3.57 +mike carson 65698 4.46 +mike carson 65700 3.89 +mike carson 65751 3.58 +mike davidson 65658 2.06 +mike davidson 65759 3.34 +mike ellison 65598 3.96 +mike ellison 65606 3.28 +mike ellison 65718 3.38 +mike ellison 65738 2.56 +mike ellison 65760 3.03 +mike falkner 65609 4.85 +mike garcia 65571 1.82 +mike garcia 65600 1.42 +mike garcia 65770 1.92 +mike hernandez 65548 1.42 +mike hernandez 65672 1.75 +mike ichabod 65621 3.73 +mike king 65563 4.34 +mike king 65586 3.75 +mike king 65591 3.09 +mike king 65642 2.69 +mike king 65769 2.36 +mike king 65776 2.55 +mike miller 65549 3.96 +mike nixon 65619 0.09 +mike nixon 65704 2.15 +mike polk 65619 4.13 +mike polk 65658 4.27 +mike polk 65704 3.77 +mike quirinius 65717 2.81 +mike steinbeck 65550 3.18 +mike steinbeck 65564 2.58 +mike steinbeck 65573 2.12 +mike steinbeck 65749 1.72 +mike van buren 65620 0.09 +mike van buren 65770 0.88 +mike white 65648 1.72 +mike white 65685 1.12 +mike white 65769 1.74 +mike white 65778 2.05 +mike young 65545 1.69 +mike young 65581 0.92 +mike young 65736 1.84 +mike zipper 65552 4.8 +mike zipper 65695 4.16 +mike zipper 65779 4.22 +nick allen 65641 3.78 +nick allen 65786 3.74 +nick brown 65724 4.61 +nick davidson 65601 0.88 +nick ellison 65691 4.04 +nick ellison 65745 3.22 +nick falkner 65583 4.44 +nick falkner 65676 3.08 +nick garcia 65695 1.05 +nick garcia 65712 2.18 +nick garcia 65720 1.94 +nick ichabod 65572 2.62 +nick ichabod 65681 2.99 +nick ichabod 65737 3.55 +nick johnson 65585 0.56 +nick johnson 65784 0.42 +nick laertes 65624 0.16 +nick miller 65757 4.23 +nick nixon 65650 0.7 +nick ovid 65719 3.93 +nick polk 65716 3.66 +nick quirinius 65588 2.88 +nick quirinius 65723 2.42 +nick robinson 65547 0.21 +nick robinson 65675 0.57 +nick steinbeck 65689 4.11 +nick thompson 65610 2.32 +nick underhill 65619 2.73 +nick van buren 65603 1.84 +nick xylophone 65644 2.06 +nick young 65654 2.06 +nick young 65660 2.28 +nick zipper 65757 3.8 +nick zipper 65765 2.04 +oscar allen 65644 3.01 +oscar brown 65614 3.95 +oscar carson 65537 4.29 +oscar carson 65548 2.89 +oscar carson 65549 1.95 +oscar carson 65624 2.56 +oscar carson 65697 3.03 +oscar davidson 65556 0.6 +oscar ellison 65630 1.47 +oscar ellison 65630 1.47 +oscar falkner 65692 3.44 +oscar garcia 65751 3.71 +oscar hernandez 65683 3.32 +oscar hernandez 65707 2.25 +oscar ichabod 65536 1.8 +oscar ichabod 65562 1.18 +oscar ichabod 65637 1.91 +oscar ichabod 65763 1.96 +oscar johnson 65645 1.69 +oscar johnson 65778 1.59 +oscar king 65541 3.96 +oscar king 65550 4.31 +oscar king 65787 3.54 +oscar laertes 65625 2.26 +oscar laertes 65690 2.55 +oscar laertes 65756 2.85 +oscar laertes 65790 2.39 +oscar nixon 65596 3.12 +oscar ovid 65536 2.94 +oscar ovid 65615 2.95 +oscar ovid 65659 3.41 +oscar polk 65541 1.12 +oscar polk 65643 1.62 +oscar quirinius 65541 3.5 +oscar quirinius 65560 2.87 +oscar quirinius 65689 3.35 +oscar quirinius 65720 3.03 +oscar robinson 65537 0.29 +oscar robinson 65658 0.29 +oscar robinson 65687 1.5 +oscar robinson 65782 1.78 +oscar steinbeck 65709 4.96 +oscar thompson 65542 0.48 +oscar thompson 65698 2.07 +oscar thompson 65727 2.01 +oscar thompson 65738 1.8 +oscar underhill 65626 3.49 +oscar van buren 65581 2.33 +oscar van buren 65635 2.45 +oscar van buren 65705 2.68 +oscar white 65552 3.05 +oscar white 65564 2.58 +oscar white 65671 2.56 +oscar white 65735 2.47 +oscar xylophone 65773 1.51 +oscar xylophone 65773 1.51 +oscar xylophone 65775 1.82 +oscar zipper 65568 4.77 +oscar zipper 65740 3.81 +oscar zipper 65777 3.26 +priscilla brown 65670 2.91 +priscilla brown 65690 2.83 +priscilla brown 65749 2.07 +priscilla carson 65658 1.43 +priscilla carson 65687 2.97 +priscilla carson 65755 2.87 +priscilla ichabod 65627 4.95 +priscilla ichabod 65759 3.41 +priscilla johnson 65543 3.85 +priscilla johnson 65633 2.98 +priscilla johnson 65668 2.23 +priscilla johnson 65681 1.98 +priscilla johnson 65755 1.94 +priscilla king 65646 1.93 +priscilla nixon 65564 0.31 +priscilla nixon 65600 2.35 +priscilla ovid 65541 3.8 +priscilla ovid 65790 2.37 +priscilla polk 65747 3.1 +priscilla quirinius 65672 0.81 +priscilla thompson 65654 2.04 +priscilla underhill 65715 0.82 +priscilla underhill 65729 1.33 +priscilla van buren 65607 0.23 +priscilla van buren 65685 0.81 +priscilla van buren 65749 1.84 +priscilla white 65652 4.79 +priscilla xylophone 65538 3.56 +priscilla xylophone 65763 2.48 +priscilla xylophone 65774 1.84 +priscilla young 65585 2.92 +priscilla young 65658 3.77 +priscilla zipper 65622 4.62 +priscilla zipper 65726 2.67 +quinn allen 65657 3.02 +quinn allen 65708 3.35 +quinn brown 65691 4.37 +quinn brown 65700 3.28 +quinn brown 65733 3.27 +quinn davidson 65549 0.03 +quinn davidson 65714 1.02 +quinn davidson 65776 2.19 +quinn davidson 65779 2.66 +quinn ellison 65705 0.54 +quinn ellison 65778 2.74 +quinn garcia 65568 1.04 +quinn garcia 65604 0.79 +quinn garcia 65610 1.85 +quinn garcia 65773 1.67 +quinn ichabod 65564 0.65 +quinn king 65558 2.38 +quinn king 65649 1.53 +quinn laertes 65542 2.33 +quinn laertes 65560 2.03 +quinn laertes 65627 2.08 +quinn nixon 65659 0.72 +quinn ovid 65699 2.87 +quinn quirinius 65747 0.86 +quinn robinson 65627 4.14 +quinn steinbeck 65578 4.92 +quinn steinbeck 65763 3.18 +quinn thompson 65643 4.62 +quinn thompson 65774 2.43 +quinn underhill 65549 3.89 +quinn underhill 65694 2.39 +quinn underhill 65767 2.48 +quinn van buren 65725 0.53 +quinn young 65647 3.54 +quinn zipper 65579 1.83 +quinn zipper 65693 2.1 +rachel allen 65661 4.66 +rachel allen 65709 4.53 +rachel brown 65586 0.43 +rachel brown 65587 2.69 +rachel brown 65587 2.69 +rachel brown 65610 2.67 +rachel brown 65693 3.07 +rachel carson 65677 4.58 +rachel carson 65682 4.54 +rachel davidson 65755 2.28 +rachel ellison 65761 0.05 +rachel falkner 65616 1.2 +rachel falkner 65681 2.01 +rachel falkner 65693 2.56 +rachel falkner 65764 2.91 +rachel johnson 65658 3.02 +rachel king 65604 2.32 +rachel king 65643 2.4 +rachel laertes 65562 4.43 +rachel laertes 65624 2.8 +rachel ovid 65721 0.3 +rachel ovid 65736 0.92 +rachel polk 65686 2.56 +rachel quirinius 65787 2.95 +rachel robinson 65544 0.06 +rachel robinson 65717 1.55 +rachel robinson 65724 2.27 +rachel thompson 65648 2.49 +rachel thompson 65662 3.16 +rachel thompson 65733 2.51 +rachel underhill 65667 2.29 +rachel white 65615 1.99 +rachel white 65717 3.08 +rachel young 65727 1.75 +rachel zipper 65757 2.82 +rachel zipper 65785 3.62 +sarah carson 65679 1.04 +sarah carson 65693 0.85 +sarah carson 65694 2.06 +sarah ellison 65611 1.64 +sarah falkner 65606 0.77 +sarah falkner 65680 2.85 +sarah garcia 65563 4.89 +sarah garcia 65638 4.78 +sarah garcia 65661 3.63 +sarah ichabod 65667 3.94 +sarah ichabod 65671 2.33 +sarah johnson 65659 3.51 +sarah johnson 65716 4.21 +sarah johnson 65731 3.81 +sarah johnson 65751 3.37 +sarah king 65650 1.05 +sarah king 65699 0.99 +sarah miller 65557 0.2 +sarah ovid 65550 3.21 +sarah robinson 65677 4.9 +sarah robinson 65763 2.99 +sarah steinbeck 65721 2.82 +sarah white 65622 0.07 +sarah white 65747 2.29 +sarah xylophone 65678 0.15 +sarah young 65595 2.15 +sarah zipper 65550 2.22 +tom brown 65593 1.64 +tom brown 65675 2.83 +tom carson 65539 4.38 +tom carson 65624 4.28 +tom carson 65780 4.03 +tom davidson 65780 2.4 +tom ellison 65578 3.3 +tom ellison 65670 4.04 +tom ellison 65756 3.51 +tom falkner 65574 1.09 +tom falkner 65583 2.05 +tom hernandez 65575 2.35 +tom hernandez 65632 2.64 +tom ichabod 65588 1.48 +tom johnson 65536 4.68 +tom johnson 65789 4.6 +tom king 65576 2.87 +tom laertes 65617 1.51 +tom laertes 65701 1.93 +tom miller 65594 1.14 +tom miller 65603 1.25 +tom miller 65704 1.26 +tom nixon 65672 0.04 +tom ovid 65628 1.95 +tom polk 65652 2.54 +tom polk 65742 2.26 +tom quirinius 65563 4.51 +tom quirinius 65783 4.17 +tom robinson 65626 3.12 +tom robinson 65632 3.61 +tom robinson 65691 3.13 +tom robinson 65758 3.45 +tom steinbeck 65666 1.34 +tom van buren 65621 0.66 +tom van buren 65652 2.71 +tom van buren 65669 3.47 +tom white 65548 2.13 +tom young 65544 3.73 +tom young 65546 2.02 +tom zipper 65789 1.51 +ulysses brown 65735 2.92 +ulysses carson 65602 0.34 +ulysses carson 65643 2.11 +ulysses carson 65703 1.43 +ulysses carson 65716 1.41 +ulysses davidson 65750 3.04 +ulysses ellison 65575 4.39 +ulysses garcia 65666 4.2 +ulysses hernandez 65651 1.75 +ulysses hernandez 65702 2.42 +ulysses hernandez 65786 2.53 +ulysses ichabod 65551 0.33 +ulysses ichabod 65566 2.19 +ulysses johnson 65776 4.79 +ulysses king 65649 4.46 +ulysses laertes 65691 4.55 +ulysses laertes 65711 3.54 +ulysses laertes 65781 3.66 +ulysses miller 65610 0.24 +ulysses miller 65637 1.08 +ulysses nixon 65603 1.85 +ulysses ovid 65656 3.17 +ulysses polk 65563 1.32 +ulysses polk 65580 3.05 +ulysses polk 65612 3.46 +ulysses polk 65777 3.75 +ulysses quirinius 65786 2.13 +ulysses robinson 65744 1.97 +ulysses steinbeck 65611 2.74 +ulysses steinbeck 65680 2.64 +ulysses thompson 65788 1.51 +ulysses underhill 65570 0.38 +ulysses underhill 65616 0.8 +ulysses underhill 65620 2.09 +ulysses underhill 65623 2.69 +ulysses underhill 65641 2.54 +ulysses underhill 65713 2.9 +ulysses underhill 65785 2.97 +ulysses van buren 65684 1.42 +ulysses white 65654 0.14 +ulysses white 65675 1.51 +ulysses xylophone 65623 2.3 +ulysses xylophone 65636 2.69 +ulysses xylophone 65781 3.22 +ulysses young 65675 1.34 +ulysses young 65736 2.01 +ulysses young 65748 2.24 +victor allen 65684 0.83 +victor allen 65707 2.31 +victor brown 65550 4.57 +victor brown 65555 3.54 +victor brown 65622 2.61 +victor brown 65673 2.34 +victor davidson 65579 0.61 +victor davidson 65628 1.52 +victor davidson 65783 2.25 +victor ellison 65641 1.32 +victor ellison 65782 2.59 +victor hernandez 65571 3.62 +victor hernandez 65659 3.68 +victor hernandez 65708 3.35 +victor hernandez 65735 2.88 +victor hernandez 65775 2.62 +victor johnson 65606 3.03 +victor johnson 65607 2.3 +victor johnson 65607 2.3 +victor king 65721 4.09 +victor king 65743 2.45 +victor laertes 65638 1.46 +victor laertes 65644 2.38 +victor miller 65570 0.1 +victor nixon 65709 0.74 +victor nixon 65791 1.73 +victor ovid 65649 4.93 +victor polk 65625 1.04 +victor quirinius 65620 1.32 +victor quirinius 65651 3.15 +victor robinson 65596 0.92 +victor robinson 65673 1.76 +victor steinbeck 65618 2.87 +victor steinbeck 65661 2.19 +victor steinbeck 65686 2.81 +victor thompson 65548 1.59 +victor van buren 65664 4.44 +victor van buren 65774 4.06 +victor white 65548 4.67 +victor white 65601 3.87 +victor xylophone 65549 3.8 +victor xylophone 65618 2.13 +victor xylophone 65644 1.59 +victor xylophone 65677 1.89 +victor xylophone 65755 2.27 +victor young 65628 3.16 +victor zipper 65743 3.98 +wendy allen 65628 3.8 +wendy allen 65711 3.44 +wendy allen 65782 2.4 +wendy brown 65580 4.67 +wendy brown 65657 4.68 +wendy ellison 65545 1.51 +wendy ellison 65603 1.6 +wendy falkner 65595 0.58 +wendy falkner 65604 0.82 +wendy falkner 65635 1.59 +wendy garcia 65659 3.47 +wendy garcia 65746 2.35 +wendy garcia 65747 1.67 +wendy garcia 65777 1.32 +wendy hernandez 65650 2.26 +wendy ichabod 65730 0.44 +wendy king 65586 4.46 +wendy king 65664 4.25 +wendy king 65670 2.94 +wendy laertes 65566 3.13 +wendy laertes 65683 3.99 +wendy laertes 65727 3.57 +wendy miller 65582 1.53 +wendy miller 65626 1.4 +wendy nixon 65611 0.26 +wendy nixon 65746 2.27 +wendy ovid 65589 4.75 +wendy ovid 65643 3.42 +wendy polk 65656 0.62 +wendy polk 65692 1.36 +wendy quirinius 65766 1.35 +wendy quirinius 65767 2.76 +wendy robinson 65622 0.85 +wendy robinson 65715 2.13 +wendy robinson 65774 1.8 +wendy steinbeck 65612 0.07 +wendy thompson 65650 2.27 +wendy thompson 65737 3.2 +wendy underhill 65662 4.55 +wendy underhill 65758 2.84 +wendy underhill 65775 2.54 +wendy van buren 65680 1.1 +wendy van buren 65699 1.0 +wendy white 65705 0.5 +wendy xylophone 65687 0.46 +wendy xylophone 65773 1.39 +wendy young 65674 0.48 +wendy young 65685 1.31 +xavier allen 65611 1.53 +xavier allen 65618 2.07 +xavier allen 65771 2.33 +xavier brown 65600 0.89 +xavier brown 65704 0.58 +xavier brown 65723 1.55 +xavier carson 65731 4.42 +xavier carson 65758 3.91 +xavier davidson 65644 1.84 +xavier davidson 65664 3.4 +xavier davidson 65755 2.67 +xavier ellison 65541 1.47 +xavier ellison 65654 2.49 +xavier garcia 65672 2.76 +xavier hernandez 65541 0.96 +xavier hernandez 65544 1.47 +xavier hernandez 65766 1.28 +xavier ichabod 65597 4.76 +xavier ichabod 65663 2.99 +xavier johnson 65654 1.34 +xavier johnson 65744 3.06 +xavier king 65590 2.68 +xavier king 65601 1.4 +xavier laertes 65743 0.75 +xavier ovid 65788 2.06 +xavier polk 65587 0.99 +xavier polk 65653 1.15 +xavier polk 65675 1.9 +xavier polk 65696 1.93 +xavier quirinius 65599 1.66 +xavier quirinius 65650 1.94 +xavier quirinius 65656 2.46 +xavier quirinius 65737 1.92 +xavier thompson 65608 1.65 +xavier underhill 65710 0.13 +xavier white 65703 0.98 +xavier white 65732 2.22 +xavier xylophone 65572 1.0 +xavier zipper 65561 0.94 +yuri allen 65565 2.03 +yuri allen 65682 1.14 +yuri brown 65538 2.73 +yuri brown 65688 2.02 +yuri carson 65670 3.06 +yuri carson 65769 3.43 +yuri ellison 65570 1.05 +yuri ellison 65581 1.68 +yuri falkner 65658 2.85 +yuri falkner 65681 2.14 +yuri garcia 65639 3.41 +yuri hernandez 65706 1.64 +yuri johnson 65587 1.27 +yuri johnson 65697 1.44 +yuri johnson 65712 2.29 +yuri king 65721 0.33 +yuri laertes 65637 4.3 +yuri laertes 65773 2.15 +yuri nixon 65635 4.02 +yuri nixon 65740 4.18 +yuri polk 65607 0.08 +yuri polk 65713 0.37 +yuri polk 65742 1.25 +yuri quirinius 65544 2.58 +yuri quirinius 65617 2.1 +yuri quirinius 65695 1.91 +yuri steinbeck 65592 4.89 +yuri steinbeck 65679 3.24 +yuri thompson 65676 2.67 +yuri underhill 65718 2.86 +yuri underhill 65750 2.51 +yuri white 65659 4.59 +yuri xylophone 65714 2.53 +zach allen 65667 0.88 +zach brown 65559 4.88 +zach brown 65588 4.53 +zach brown 65691 3.49 +zach brown 65759 3.4 +zach brown 65762 3.55 +zach carson 65572 2.03 +zach ellison 65748 1.76 +zach falkner 65620 0.34 +zach falkner 65627 0.25 +zach garcia 65544 0.99 +zach garcia 65623 2.84 +zach garcia 65629 3.01 +zach garcia 65786 2.55 +zach ichabod 65599 3.36 +zach ichabod 65612 1.92 +zach king 65556 2.36 +zach king 65702 1.52 +zach king 65773 2.58 +zach miller 65584 1.6 +zach miller 65665 0.99 +zach miller 65719 1.55 +zach ovid 65578 1.51 +zach ovid 65703 1.92 +zach ovid 65750 2.63 +zach ovid 65784 2.72 +zach quirinius 65691 2.95 +zach robinson 65599 2.87 +zach steinbeck 65602 2.45 +zach steinbeck 65695 1.86 +zach thompson 65636 0.25 +zach thompson 65696 0.51 +zach underhill 65573 3.97 +zach white 65733 2.31 +zach xylophone 65542 1.69 +zach xylophone 65780 0.88 +zach young 65576 1.82 +zach zipper 65579 4.5 +zach zipper 65649 4.02 +zach zipper 65676 3.12 diff --git a/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e new file mode 100644 index 0000000000000..84b934fad85b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing_windowspec.q (deterministic)-8-45a1d7c2aba45d761e19ff4dfdf5463e @@ -0,0 +1,1049 @@ + 65560 20.0 + 65718 20.0 + 65740 20.0 +alice allen 65662 20.0 +alice allen 65720 20.0 +alice allen 65758 20.0 +alice brown 65696 20.0 +alice carson 65559 20.0 +alice davidson 65547 20.0 +alice falkner 65669 20.0 +alice garcia 65613 20.0 +alice hernandez 65737 20.0 +alice hernandez 65784 20.0 +alice johnson 65739 20.0 +alice king 65660 20.0 +alice king 65734 20.0 +alice king 65738 20.0 +alice laertes 65669 20.0 +alice laertes 65671 20.0 +alice miller 65590 20.0 +alice nixon 65586 20.0 +alice nixon 65595 20.0 +alice nixon 65604 20.0 +alice ovid 65737 20.0 +alice polk 65548 20.0 +alice quirinius 65636 20.0 +alice quirinius 65728 20.0 +alice robinson 65606 20.0 +alice robinson 65789 20.0 +alice steinbeck 65578 20.0 +alice steinbeck 65673 20.0 +alice steinbeck 65786 20.0 +alice underhill 65750 20.0 +alice van buren 65562 20.0 +alice xylophone 65578 20.0 +alice xylophone 65585 20.0 +alice xylophone 65599 20.0 +alice zipper 65553 20.0 +alice zipper 65662 20.0 +alice zipper 65766 20.0 +bob brown 65584 20.0 +bob brown 65777 20.0 +bob brown 65783 20.0 +bob carson 65713 20.0 +bob davidson 65664 20.0 +bob davidson 65693 20.0 +bob davidson 65768 20.0 +bob ellison 65591 20.0 +bob ellison 65624 20.0 +bob ellison 65721 20.0 +bob ellison 65760 20.0 +bob falkner 65789 20.0 +bob garcia 65585 20.0 +bob garcia 65598 20.0 +bob garcia 65673 20.0 +bob garcia 65754 20.0 +bob garcia 65782 20.0 +bob hernandez 65557 20.0 +bob ichabod 65549 20.0 +bob king 65715 20.0 +bob king 65757 20.0 +bob king 65783 20.0 +bob laertes 65602 20.0 +bob laertes 65663 20.0 +bob miller 65608 20.0 +bob ovid 65564 20.0 +bob ovid 65619 20.0 +bob ovid 65686 20.0 +bob ovid 65726 20.0 +bob polk 65594 20.0 +bob quirinius 65700 20.0 +bob steinbeck 65637 20.0 +bob van buren 65778 20.0 +bob white 65543 20.0 +bob white 65605 20.0 +bob xylophone 65574 20.0 +bob xylophone 65666 20.0 +bob young 65556 20.0 +bob zipper 65559 20.0 +bob zipper 65633 20.0 +bob zipper 65739 20.0 +calvin allen 65669 20.0 +calvin brown 65537 20.0 +calvin brown 65580 20.0 +calvin brown 65677 20.0 +calvin carson 65637 20.0 +calvin davidson 65541 20.0 +calvin davidson 65564 20.0 +calvin ellison 65667 20.0 +calvin falkner 65573 20.0 +calvin falkner 65596 20.0 +calvin falkner 65738 20.0 +calvin falkner 65762 20.0 +calvin falkner 65778 20.0 +calvin falkner 65784 20.0 +calvin garcia 65664 20.0 +calvin hernandez 65578 20.0 +calvin johnson 65731 20.0 +calvin laertes 65570 20.0 +calvin laertes 65684 20.0 +calvin nixon 65654 20.0 +calvin nixon 65724 20.0 +calvin nixon 65749 20.0 +calvin ovid 65554 20.0 +calvin ovid 65643 20.0 +calvin ovid 65663 20.0 +calvin ovid 65715 20.0 +calvin polk 65731 20.0 +calvin quirinius 65741 20.0 +calvin quirinius 65769 20.0 +calvin robinson 65581 20.0 +calvin steinbeck 65680 20.0 +calvin steinbeck 65762 20.0 +calvin steinbeck 65779 20.0 +calvin thompson 65560 20.0 +calvin thompson 65640 20.0 +calvin underhill 65732 20.0 +calvin van buren 65552 20.0 +calvin van buren 65771 20.0 +calvin white 65553 20.0 +calvin white 65561 20.0 +calvin xylophone 65575 20.0 +calvin xylophone 65596 20.0 +calvin xylophone 65713 20.0 +calvin young 65574 20.0 +calvin young 65746 20.0 +calvin zipper 65669 20.0 +calvin zipper 65739 20.0 +david allen 65588 20.0 +david allen 65617 20.0 +david brown 65637 20.0 +david brown 65760 20.0 +david davidson 65559 20.0 +david davidson 65756 20.0 +david davidson 65778 20.0 +david davidson 65779 20.0 +david ellison 65634 20.0 +david ellison 65724 20.0 +david ellison 65724 20.0 +david hernandez 65763 20.0 +david ichabod 65699 20.0 +david ichabod 65715 20.0 +david laertes 65762 20.0 +david nixon 65536 20.0 +david ovid 65623 20.0 +david ovid 65628 20.0 +david quirinius 65697 20.0 +david quirinius 65759 20.0 +david quirinius 65779 20.0 +david robinson 65762 20.0 +david robinson 65775 20.0 +david thompson 65550 20.0 +david underhill 65602 20.0 +david underhill 65662 20.0 +david underhill 65751 20.0 +david van buren 65625 20.0 +david van buren 65634 20.0 +david white 65678 20.0 +david xylophone 65537 20.0 +david xylophone 65670 20.0 +david xylophone 65764 20.0 +david young 65551 20.0 +david young 65694 20.0 +ethan allen 65747 20.0 +ethan brown 65539 20.0 +ethan brown 65617 20.0 +ethan brown 65685 20.0 +ethan brown 65685 20.0 +ethan brown 65722 20.0 +ethan brown 65733 20.0 +ethan carson 65742 20.0 +ethan ellison 65714 20.0 +ethan ellison 65732 20.0 +ethan falkner 65577 20.0 +ethan falkner 65614 20.0 +ethan garcia 65736 20.0 +ethan hernandez 65618 20.0 +ethan johnson 65536 20.0 +ethan king 65614 20.0 +ethan laertes 65562 20.0 +ethan laertes 65597 20.0 +ethan laertes 65628 20.0 +ethan laertes 65643 20.0 +ethan laertes 65680 20.0 +ethan laertes 65745 20.0 +ethan laertes 65760 20.0 +ethan miller 65712 20.0 +ethan nixon 65766 20.0 +ethan ovid 65697 20.0 +ethan polk 65589 20.0 +ethan polk 65615 20.0 +ethan polk 65622 20.0 +ethan polk 65622 20.0 +ethan quirinius 65542 20.0 +ethan quirinius 65591 20.0 +ethan quirinius 65706 20.0 +ethan robinson 65547 20.0 +ethan robinson 65659 20.0 +ethan underhill 65570 20.0 +ethan van buren 65572 20.0 +ethan white 65677 20.0 +ethan white 65788 20.0 +ethan xylophone 65595 20.0 +ethan zipper 65593 20.0 +ethan zipper 65680 20.0 +fred davidson 65552 20.0 +fred davidson 65595 20.0 +fred davidson 65721 20.0 +fred ellison 65548 20.0 +fred ellison 65691 20.0 +fred ellison 65771 20.0 +fred falkner 65637 20.0 +fred falkner 65648 20.0 +fred falkner 65783 20.0 +fred hernandez 65541 20.0 +fred ichabod 65572 20.0 +fred ichabod 65789 20.0 +fred johnson 65758 20.0 +fred king 65694 20.0 +fred king 65745 20.0 +fred laertes 65769 20.0 +fred miller 65536 20.0 +fred nixon 65560 20.0 +fred nixon 65612 20.0 +fred nixon 65703 20.0 +fred nixon 65705 20.0 +fred polk 65603 20.0 +fred polk 65656 20.0 +fred polk 65701 20.0 +fred polk 65706 20.0 +fred quirinius 65697 20.0 +fred quirinius 65782 20.0 +fred robinson 65623 20.0 +fred steinbeck 65544 20.0 +fred steinbeck 65651 20.0 +fred steinbeck 65755 20.0 +fred underhill 65629 20.0 +fred van buren 65537 20.0 +fred van buren 65561 20.0 +fred van buren 65745 20.0 +fred van buren 65789 20.0 +fred white 65589 20.0 +fred young 65594 20.0 +fred young 65773 20.0 +fred zipper 65553 20.0 +gabriella allen 65646 20.0 +gabriella allen 65677 20.0 +gabriella brown 65704 20.0 +gabriella brown 65753 20.0 +gabriella carson 65586 20.0 +gabriella davidson 65565 20.0 +gabriella ellison 65706 20.0 +gabriella ellison 65716 20.0 +gabriella falkner 65623 20.0 +gabriella falkner 65711 20.0 +gabriella falkner 65767 20.0 +gabriella garcia 65571 20.0 +gabriella hernandez 65587 20.0 +gabriella hernandez 65717 20.0 +gabriella ichabod 65559 20.0 +gabriella ichabod 65633 20.0 +gabriella ichabod 65702 20.0 +gabriella ichabod 65712 20.0 +gabriella ichabod 65717 20.0 +gabriella king 65651 20.0 +gabriella king 65657 20.0 +gabriella laertes 65781 20.0 +gabriella miller 65646 20.0 +gabriella ovid 65556 20.0 +gabriella ovid 65583 20.0 +gabriella polk 65701 20.0 +gabriella polk 65790 20.0 +gabriella steinbeck 65582 20.0 +gabriella steinbeck 65653 20.0 +gabriella thompson 65682 20.0 +gabriella thompson 65755 20.0 +gabriella thompson 65766 20.0 +gabriella van buren 65581 20.0 +gabriella van buren 65644 20.0 +gabriella white 65638 20.0 +gabriella young 65699 20.0 +gabriella young 65774 20.0 +gabriella zipper 65540 20.0 +gabriella zipper 65754 20.0 +holly allen 65596 20.0 +holly brown 65599 20.0 +holly brown 65619 20.0 +holly falkner 65720 20.0 +holly hernandez 65602 20.0 +holly hernandez 65686 20.0 +holly hernandez 65750 20.0 +holly hernandez 65788 20.0 +holly ichabod 65711 20.0 +holly ichabod 65749 20.0 +holly ichabod 65752 20.0 +holly johnson 65655 20.0 +holly johnson 65662 20.0 +holly johnson 65755 20.0 +holly king 65549 20.0 +holly king 65648 20.0 +holly laertes 65664 20.0 +holly miller 65653 20.0 +holly nixon 65539 20.0 +holly nixon 65658 20.0 +holly polk 65743 20.0 +holly polk 65751 20.0 +holly robinson 65564 20.0 +holly thompson 65538 20.0 +holly thompson 65578 20.0 +holly thompson 65713 20.0 +holly underhill 65634 20.0 +holly underhill 65654 20.0 +holly underhill 65721 20.0 +holly underhill 65759 20.0 +holly van buren 65727 20.0 +holly white 65536 20.0 +holly white 65602 20.0 +holly xylophone 65544 20.0 +holly young 65606 20.0 +holly young 65765 20.0 +holly zipper 65607 20.0 +holly zipper 65755 20.0 +irene allen 65556 20.0 +irene brown 65633 20.0 +irene brown 65650 20.0 +irene brown 65765 20.0 +irene carson 65590 20.0 +irene ellison 65659 20.0 +irene ellison 65696 20.0 +irene falkner 65620 20.0 +irene falkner 65661 20.0 +irene garcia 65660 20.0 +irene garcia 65711 20.0 +irene garcia 65787 20.0 +irene ichabod 65645 20.0 +irene ichabod 65722 20.0 +irene johnson 65583 20.0 +irene laertes 65664 20.0 +irene laertes 65710 20.0 +irene laertes 65722 20.0 +irene miller 65730 20.0 +irene nixon 65631 20.0 +irene nixon 65643 20.0 +irene nixon 65653 20.0 +irene ovid 65691 20.0 +irene ovid 65734 20.0 +irene ovid 65753 20.0 +irene polk 65551 20.0 +irene polk 65575 20.0 +irene polk 65579 20.0 +irene polk 65595 20.0 +irene polk 65610 20.0 +irene quirinius 65724 20.0 +irene quirinius 65769 20.0 +irene quirinius 65773 20.0 +irene robinson 65554 20.0 +irene steinbeck 65683 20.0 +irene thompson 65688 20.0 +irene underhill 65591 20.0 +irene underhill 65787 20.0 +irene van buren 65579 20.0 +irene van buren 65589 20.0 +irene xylophone 65775 20.0 +jessica brown 65588 20.0 +jessica carson 65553 20.0 +jessica carson 65672 20.0 +jessica carson 65747 20.0 +jessica davidson 65549 20.0 +jessica davidson 65606 20.0 +jessica davidson 65675 20.0 +jessica davidson 65727 20.0 +jessica ellison 65567 20.0 +jessica ellison 65663 20.0 +jessica falkner 65584 20.0 +jessica garcia 65676 20.0 +jessica garcia 65789 20.0 +jessica ichabod 65704 20.0 +jessica johnson 65607 20.0 +jessica johnson 65720 20.0 +jessica miller 65733 20.0 +jessica nixon 65590 20.0 +jessica nixon 65774 20.0 +jessica ovid 65582 20.0 +jessica ovid 65751 20.0 +jessica polk 65637 20.0 +jessica quirinius 65562 20.0 +jessica quirinius 65608 20.0 +jessica quirinius 65712 20.0 +jessica quirinius 65716 20.0 +jessica robinson 65576 20.0 +jessica thompson 65581 20.0 +jessica thompson 65675 20.0 +jessica underhill 65656 20.0 +jessica underhill 65702 20.0 +jessica underhill 65783 20.0 +jessica van buren 65615 20.0 +jessica white 65544 20.0 +jessica white 65570 20.0 +jessica white 65594 20.0 +jessica white 65673 20.0 +jessica white 65779 20.0 +jessica xylophone 65562 20.0 +jessica young 65623 20.0 +jessica young 65711 20.0 +jessica zipper 65600 20.0 +jessica zipper 65657 20.0 +jessica zipper 65778 20.0 +katie allen 65542 20.0 +katie brown 65590 20.0 +katie davidson 65619 20.0 +katie ellison 65675 20.0 +katie ellison 65699 20.0 +katie falkner 65728 20.0 +katie garcia 65625 20.0 +katie garcia 65747 20.0 +katie hernandez 65550 20.0 +katie ichabod 65658 20.0 +katie ichabod 65726 20.0 +katie ichabod 65757 20.0 +katie king 65629 20.0 +katie king 65647 20.0 +katie king 65776 20.0 +katie miller 65541 20.0 +katie miller 65661 20.0 +katie nixon 65669 20.0 +katie ovid 65681 20.0 +katie polk 65746 20.0 +katie polk 65784 20.0 +katie robinson 65697 20.0 +katie van buren 65643 20.0 +katie van buren 65730 20.0 +katie white 65620 20.0 +katie white 65719 20.0 +katie xylophone 65585 20.0 +katie young 65644 20.0 +katie young 65746 20.0 +katie young 65764 20.0 +katie zipper 65568 20.0 +katie zipper 65733 20.0 +luke allen 65547 20.0 +luke allen 65552 20.0 +luke allen 65576 20.0 +luke allen 65681 20.0 +luke allen 65776 20.0 +luke brown 65719 20.0 +luke davidson 65656 20.0 +luke davidson 65791 20.0 +luke ellison 65582 20.0 +luke ellison 65664 20.0 +luke ellison 65779 20.0 +luke falkner 65589 20.0 +luke falkner 65618 20.0 +luke garcia 65687 20.0 +luke garcia 65778 20.0 +luke ichabod 65629 20.0 +luke ichabod 65654 20.0 +luke johnson 65545 20.0 +luke johnson 65716 20.0 +luke johnson 65718 20.0 +luke laertes 65608 20.0 +luke laertes 65657 20.0 +luke laertes 65685 20.0 +luke laertes 65730 20.0 +luke laertes 65756 20.0 +luke miller 65752 20.0 +luke ovid 65569 20.0 +luke ovid 65693 20.0 +luke polk 65645 20.0 +luke polk 65658 20.0 +luke quirinius 65655 20.0 +luke robinson 65634 20.0 +luke robinson 65772 20.0 +luke thompson 65626 20.0 +luke underhill 65553 20.0 +luke underhill 65571 20.0 +luke underhill 65651 20.0 +luke van buren 65678 20.0 +luke white 65693 20.0 +luke xylophone 65597 20.0 +luke zipper 65641 20.0 +mike allen 65706 20.0 +mike brown 65654 20.0 +mike carson 65698 20.0 +mike carson 65700 20.0 +mike carson 65751 20.0 +mike davidson 65658 20.0 +mike davidson 65759 20.0 +mike ellison 65598 20.0 +mike ellison 65606 20.0 +mike ellison 65718 20.0 +mike ellison 65738 20.0 +mike ellison 65760 20.0 +mike falkner 65609 20.0 +mike garcia 65571 20.0 +mike garcia 65600 20.0 +mike garcia 65770 20.0 +mike hernandez 65548 20.0 +mike hernandez 65672 20.0 +mike ichabod 65621 20.0 +mike king 65563 20.0 +mike king 65586 20.0 +mike king 65591 20.0 +mike king 65642 20.0 +mike king 65769 20.0 +mike king 65776 20.0 +mike miller 65549 20.0 +mike nixon 65619 20.0 +mike nixon 65704 20.0 +mike polk 65619 20.0 +mike polk 65658 20.0 +mike polk 65704 20.0 +mike quirinius 65717 20.0 +mike steinbeck 65550 20.0 +mike steinbeck 65564 20.0 +mike steinbeck 65573 20.0 +mike steinbeck 65749 20.0 +mike van buren 65620 20.0 +mike van buren 65770 20.0 +mike white 65648 20.0 +mike white 65685 20.0 +mike white 65769 20.0 +mike white 65778 20.0 +mike young 65545 20.0 +mike young 65581 20.0 +mike young 65736 20.0 +mike zipper 65552 20.0 +mike zipper 65695 20.0 +mike zipper 65779 20.0 +nick allen 65641 20.0 +nick allen 65786 20.0 +nick brown 65724 20.0 +nick davidson 65601 20.0 +nick ellison 65691 20.0 +nick ellison 65745 20.0 +nick falkner 65583 20.0 +nick falkner 65676 20.0 +nick garcia 65695 20.0 +nick garcia 65712 20.0 +nick garcia 65720 20.0 +nick ichabod 65572 20.0 +nick ichabod 65681 20.0 +nick ichabod 65737 20.0 +nick johnson 65585 20.0 +nick johnson 65784 20.0 +nick laertes 65624 20.0 +nick miller 65757 20.0 +nick nixon 65650 20.0 +nick ovid 65719 20.0 +nick polk 65716 20.0 +nick quirinius 65588 20.0 +nick quirinius 65723 20.0 +nick robinson 65547 20.0 +nick robinson 65675 20.0 +nick steinbeck 65689 20.0 +nick thompson 65610 20.0 +nick underhill 65619 20.0 +nick van buren 65603 20.0 +nick xylophone 65644 20.0 +nick young 65654 20.0 +nick young 65660 20.0 +nick zipper 65757 20.0 +nick zipper 65765 20.0 +oscar allen 65644 20.0 +oscar brown 65614 20.0 +oscar carson 65537 20.0 +oscar carson 65548 20.0 +oscar carson 65549 20.0 +oscar carson 65624 20.0 +oscar carson 65697 20.0 +oscar davidson 65556 20.0 +oscar ellison 65630 20.0 +oscar ellison 65630 20.0 +oscar falkner 65692 20.0 +oscar garcia 65751 20.0 +oscar hernandez 65683 20.0 +oscar hernandez 65707 20.0 +oscar ichabod 65536 20.0 +oscar ichabod 65562 20.0 +oscar ichabod 65637 20.0 +oscar ichabod 65763 20.0 +oscar johnson 65645 20.0 +oscar johnson 65778 20.0 +oscar king 65541 20.0 +oscar king 65550 20.0 +oscar king 65787 20.0 +oscar laertes 65625 20.0 +oscar laertes 65690 20.0 +oscar laertes 65756 20.0 +oscar laertes 65790 20.0 +oscar nixon 65596 20.0 +oscar ovid 65536 20.0 +oscar ovid 65615 20.0 +oscar ovid 65659 20.0 +oscar polk 65541 20.0 +oscar polk 65643 20.0 +oscar quirinius 65541 20.0 +oscar quirinius 65560 20.0 +oscar quirinius 65689 20.0 +oscar quirinius 65720 20.0 +oscar robinson 65537 20.0 +oscar robinson 65658 20.0 +oscar robinson 65687 20.0 +oscar robinson 65782 20.0 +oscar steinbeck 65709 20.0 +oscar thompson 65542 20.0 +oscar thompson 65698 20.0 +oscar thompson 65727 20.0 +oscar thompson 65738 20.0 +oscar underhill 65626 20.0 +oscar van buren 65581 20.0 +oscar van buren 65635 20.0 +oscar van buren 65705 20.0 +oscar white 65552 20.0 +oscar white 65564 20.0 +oscar white 65671 20.0 +oscar white 65735 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65773 20.0 +oscar xylophone 65775 20.0 +oscar zipper 65568 20.0 +oscar zipper 65740 20.0 +oscar zipper 65777 20.0 +priscilla brown 65670 20.0 +priscilla brown 65690 20.0 +priscilla brown 65749 20.0 +priscilla carson 65658 20.0 +priscilla carson 65687 20.0 +priscilla carson 65755 20.0 +priscilla ichabod 65627 20.0 +priscilla ichabod 65759 20.0 +priscilla johnson 65543 20.0 +priscilla johnson 65633 20.0 +priscilla johnson 65668 20.0 +priscilla johnson 65681 20.0 +priscilla johnson 65755 20.0 +priscilla king 65646 20.0 +priscilla nixon 65564 20.0 +priscilla nixon 65600 20.0 +priscilla ovid 65541 20.0 +priscilla ovid 65790 20.0 +priscilla polk 65747 20.0 +priscilla quirinius 65672 20.0 +priscilla thompson 65654 20.0 +priscilla underhill 65715 20.0 +priscilla underhill 65729 20.0 +priscilla van buren 65607 20.0 +priscilla van buren 65685 20.0 +priscilla van buren 65749 20.0 +priscilla white 65652 20.0 +priscilla xylophone 65538 20.0 +priscilla xylophone 65763 20.0 +priscilla xylophone 65774 20.0 +priscilla young 65585 20.0 +priscilla young 65658 20.0 +priscilla zipper 65622 20.0 +priscilla zipper 65726 20.0 +quinn allen 65657 20.0 +quinn allen 65708 20.0 +quinn brown 65691 20.0 +quinn brown 65700 20.0 +quinn brown 65733 20.0 +quinn davidson 65549 20.0 +quinn davidson 65714 20.0 +quinn davidson 65776 20.0 +quinn davidson 65779 20.0 +quinn ellison 65705 20.0 +quinn ellison 65778 20.0 +quinn garcia 65568 20.0 +quinn garcia 65604 20.0 +quinn garcia 65610 20.0 +quinn garcia 65773 20.0 +quinn ichabod 65564 20.0 +quinn king 65558 20.0 +quinn king 65649 20.0 +quinn laertes 65542 20.0 +quinn laertes 65560 20.0 +quinn laertes 65627 20.0 +quinn nixon 65659 20.0 +quinn ovid 65699 20.0 +quinn quirinius 65747 20.0 +quinn robinson 65627 20.0 +quinn steinbeck 65578 20.0 +quinn steinbeck 65763 20.0 +quinn thompson 65643 20.0 +quinn thompson 65774 20.0 +quinn underhill 65549 20.0 +quinn underhill 65694 20.0 +quinn underhill 65767 20.0 +quinn van buren 65725 20.0 +quinn young 65647 20.0 +quinn zipper 65579 20.0 +quinn zipper 65693 20.0 +rachel allen 65661 20.0 +rachel allen 65709 20.0 +rachel brown 65586 20.0 +rachel brown 65587 20.0 +rachel brown 65587 20.0 +rachel brown 65610 20.0 +rachel brown 65693 20.0 +rachel carson 65677 20.0 +rachel carson 65682 20.0 +rachel davidson 65755 20.0 +rachel ellison 65761 20.0 +rachel falkner 65616 20.0 +rachel falkner 65681 20.0 +rachel falkner 65693 20.0 +rachel falkner 65764 20.0 +rachel johnson 65658 20.0 +rachel king 65604 20.0 +rachel king 65643 20.0 +rachel laertes 65562 20.0 +rachel laertes 65624 20.0 +rachel ovid 65721 20.0 +rachel ovid 65736 20.0 +rachel polk 65686 20.0 +rachel quirinius 65787 20.0 +rachel robinson 65544 20.0 +rachel robinson 65717 20.0 +rachel robinson 65724 20.0 +rachel thompson 65648 20.0 +rachel thompson 65662 20.0 +rachel thompson 65733 20.0 +rachel underhill 65667 20.0 +rachel white 65615 20.0 +rachel white 65717 20.0 +rachel young 65727 20.0 +rachel zipper 65757 20.0 +rachel zipper 65785 20.0 +sarah carson 65679 20.0 +sarah carson 65693 20.0 +sarah carson 65694 20.0 +sarah ellison 65611 20.0 +sarah falkner 65606 20.0 +sarah falkner 65680 20.0 +sarah garcia 65563 20.0 +sarah garcia 65638 20.0 +sarah garcia 65661 20.0 +sarah ichabod 65667 20.0 +sarah ichabod 65671 20.0 +sarah johnson 65659 20.0 +sarah johnson 65716 20.0 +sarah johnson 65731 20.0 +sarah johnson 65751 20.0 +sarah king 65650 20.0 +sarah king 65699 20.0 +sarah miller 65557 20.0 +sarah ovid 65550 20.0 +sarah robinson 65677 20.0 +sarah robinson 65763 20.0 +sarah steinbeck 65721 20.0 +sarah white 65622 20.0 +sarah white 65747 20.0 +sarah xylophone 65678 20.0 +sarah young 65595 20.0 +sarah zipper 65550 20.0 +tom brown 65593 20.0 +tom brown 65675 20.0 +tom carson 65539 20.0 +tom carson 65624 20.0 +tom carson 65780 20.0 +tom davidson 65780 20.0 +tom ellison 65578 20.0 +tom ellison 65670 20.0 +tom ellison 65756 20.0 +tom falkner 65574 20.0 +tom falkner 65583 20.0 +tom hernandez 65575 20.0 +tom hernandez 65632 20.0 +tom ichabod 65588 20.0 +tom johnson 65536 20.0 +tom johnson 65789 20.0 +tom king 65576 20.0 +tom laertes 65617 20.0 +tom laertes 65701 20.0 +tom miller 65594 20.0 +tom miller 65603 20.0 +tom miller 65704 20.0 +tom nixon 65672 20.0 +tom ovid 65628 20.0 +tom polk 65652 20.0 +tom polk 65742 20.0 +tom quirinius 65563 20.0 +tom quirinius 65783 20.0 +tom robinson 65626 20.0 +tom robinson 65632 20.0 +tom robinson 65691 20.0 +tom robinson 65758 20.0 +tom steinbeck 65666 20.0 +tom van buren 65621 20.0 +tom van buren 65652 20.0 +tom van buren 65669 20.0 +tom white 65548 20.0 +tom young 65544 20.0 +tom young 65546 20.0 +tom zipper 65789 20.0 +ulysses brown 65735 20.0 +ulysses carson 65602 20.0 +ulysses carson 65643 20.0 +ulysses carson 65703 20.0 +ulysses carson 65716 20.0 +ulysses davidson 65750 20.0 +ulysses ellison 65575 20.0 +ulysses garcia 65666 20.0 +ulysses hernandez 65651 20.0 +ulysses hernandez 65702 20.0 +ulysses hernandez 65786 20.0 +ulysses ichabod 65551 20.0 +ulysses ichabod 65566 20.0 +ulysses johnson 65776 20.0 +ulysses king 65649 20.0 +ulysses laertes 65691 20.0 +ulysses laertes 65711 20.0 +ulysses laertes 65781 20.0 +ulysses miller 65610 20.0 +ulysses miller 65637 20.0 +ulysses nixon 65603 20.0 +ulysses ovid 65656 20.0 +ulysses polk 65563 20.0 +ulysses polk 65580 20.0 +ulysses polk 65612 20.0 +ulysses polk 65777 20.0 +ulysses quirinius 65786 20.0 +ulysses robinson 65744 20.0 +ulysses steinbeck 65611 20.0 +ulysses steinbeck 65680 20.0 +ulysses thompson 65788 20.0 +ulysses underhill 65570 20.0 +ulysses underhill 65616 20.0 +ulysses underhill 65620 20.0 +ulysses underhill 65623 20.0 +ulysses underhill 65641 20.0 +ulysses underhill 65713 20.0 +ulysses underhill 65785 20.0 +ulysses van buren 65684 20.0 +ulysses white 65654 20.0 +ulysses white 65675 20.0 +ulysses xylophone 65623 20.0 +ulysses xylophone 65636 20.0 +ulysses xylophone 65781 20.0 +ulysses young 65675 20.0 +ulysses young 65736 20.0 +ulysses young 65748 20.0 +victor allen 65684 20.0 +victor allen 65707 20.0 +victor brown 65550 20.0 +victor brown 65555 20.0 +victor brown 65622 20.0 +victor brown 65673 20.0 +victor davidson 65579 20.0 +victor davidson 65628 20.0 +victor davidson 65783 20.0 +victor ellison 65641 20.0 +victor ellison 65782 20.0 +victor hernandez 65571 20.0 +victor hernandez 65659 20.0 +victor hernandez 65708 20.0 +victor hernandez 65735 20.0 +victor hernandez 65775 20.0 +victor johnson 65606 20.0 +victor johnson 65607 20.0 +victor johnson 65607 20.0 +victor king 65721 20.0 +victor king 65743 20.0 +victor laertes 65638 20.0 +victor laertes 65644 20.0 +victor miller 65570 20.0 +victor nixon 65709 20.0 +victor nixon 65791 20.0 +victor ovid 65649 20.0 +victor polk 65625 20.0 +victor quirinius 65620 20.0 +victor quirinius 65651 20.0 +victor robinson 65596 20.0 +victor robinson 65673 20.0 +victor steinbeck 65618 20.0 +victor steinbeck 65661 20.0 +victor steinbeck 65686 20.0 +victor thompson 65548 20.0 +victor van buren 65664 20.0 +victor van buren 65774 20.0 +victor white 65548 20.0 +victor white 65601 20.0 +victor xylophone 65549 20.0 +victor xylophone 65618 20.0 +victor xylophone 65644 20.0 +victor xylophone 65677 20.0 +victor xylophone 65755 20.0 +victor young 65628 20.0 +victor zipper 65743 20.0 +wendy allen 65628 20.0 +wendy allen 65711 20.0 +wendy allen 65782 20.0 +wendy brown 65580 20.0 +wendy brown 65657 20.0 +wendy ellison 65545 20.0 +wendy ellison 65603 20.0 +wendy falkner 65595 20.0 +wendy falkner 65604 20.0 +wendy falkner 65635 20.0 +wendy garcia 65659 20.0 +wendy garcia 65746 20.0 +wendy garcia 65747 20.0 +wendy garcia 65777 20.0 +wendy hernandez 65650 20.0 +wendy ichabod 65730 20.0 +wendy king 65586 20.0 +wendy king 65664 20.0 +wendy king 65670 20.0 +wendy laertes 65566 20.0 +wendy laertes 65683 20.0 +wendy laertes 65727 20.0 +wendy miller 65582 20.0 +wendy miller 65626 20.0 +wendy nixon 65611 20.0 +wendy nixon 65746 20.0 +wendy ovid 65589 20.0 +wendy ovid 65643 20.0 +wendy polk 65656 20.0 +wendy polk 65692 20.0 +wendy quirinius 65766 20.0 +wendy quirinius 65767 20.0 +wendy robinson 65622 20.0 +wendy robinson 65715 20.0 +wendy robinson 65774 20.0 +wendy steinbeck 65612 20.0 +wendy thompson 65650 20.0 +wendy thompson 65737 20.0 +wendy underhill 65662 20.0 +wendy underhill 65758 20.0 +wendy underhill 65775 20.0 +wendy van buren 65680 20.0 +wendy van buren 65699 20.0 +wendy white 65705 20.0 +wendy xylophone 65687 20.0 +wendy xylophone 65773 20.0 +wendy young 65674 20.0 +wendy young 65685 20.0 +xavier allen 65611 20.0 +xavier allen 65618 20.0 +xavier allen 65771 20.0 +xavier brown 65600 20.0 +xavier brown 65704 20.0 +xavier brown 65723 20.0 +xavier carson 65731 20.0 +xavier carson 65758 20.0 +xavier davidson 65644 20.0 +xavier davidson 65664 20.0 +xavier davidson 65755 20.0 +xavier ellison 65541 20.0 +xavier ellison 65654 20.0 +xavier garcia 65672 20.0 +xavier hernandez 65541 20.0 +xavier hernandez 65544 20.0 +xavier hernandez 65766 20.0 +xavier ichabod 65597 20.0 +xavier ichabod 65663 20.0 +xavier johnson 65654 20.0 +xavier johnson 65744 20.0 +xavier king 65590 20.0 +xavier king 65601 20.0 +xavier laertes 65743 20.0 +xavier ovid 65788 20.0 +xavier polk 65587 20.0 +xavier polk 65653 20.0 +xavier polk 65675 20.0 +xavier polk 65696 20.0 +xavier quirinius 65599 20.0 +xavier quirinius 65650 20.0 +xavier quirinius 65656 20.0 +xavier quirinius 65737 20.0 +xavier thompson 65608 20.0 +xavier underhill 65710 20.0 +xavier white 65703 20.0 +xavier white 65732 20.0 +xavier xylophone 65572 20.0 +xavier zipper 65561 20.0 +yuri allen 65565 20.0 +yuri allen 65682 20.0 +yuri brown 65538 20.0 +yuri brown 65688 20.0 +yuri carson 65670 20.0 +yuri carson 65769 20.0 +yuri ellison 65570 20.0 +yuri ellison 65581 20.0 +yuri falkner 65658 20.0 +yuri falkner 65681 20.0 +yuri garcia 65639 20.0 +yuri hernandez 65706 20.0 +yuri johnson 65587 20.0 +yuri johnson 65697 20.0 +yuri johnson 65712 20.0 +yuri king 65721 20.0 +yuri laertes 65637 20.0 +yuri laertes 65773 20.0 +yuri nixon 65635 20.0 +yuri nixon 65740 20.0 +yuri polk 65607 20.0 +yuri polk 65713 20.0 +yuri polk 65742 20.0 +yuri quirinius 65544 20.0 +yuri quirinius 65617 20.0 +yuri quirinius 65695 20.0 +yuri steinbeck 65592 20.0 +yuri steinbeck 65679 20.0 +yuri thompson 65676 20.0 +yuri underhill 65718 20.0 +yuri underhill 65750 20.0 +yuri white 65659 20.0 +yuri xylophone 65714 20.0 +zach allen 65667 20.0 +zach brown 65559 20.0 +zach brown 65588 20.0 +zach brown 65691 20.0 +zach brown 65759 20.0 +zach brown 65762 20.0 +zach carson 65572 20.0 +zach ellison 65748 20.0 +zach falkner 65620 20.0 +zach falkner 65627 20.0 +zach garcia 65544 20.0 +zach garcia 65623 20.0 +zach garcia 65629 20.0 +zach garcia 65786 20.0 +zach ichabod 65599 20.0 +zach ichabod 65612 20.0 +zach king 65556 20.0 +zach king 65702 20.0 +zach king 65773 20.0 +zach miller 65584 20.0 +zach miller 65665 20.0 +zach miller 65719 20.0 +zach ovid 65578 20.0 +zach ovid 65703 20.0 +zach ovid 65750 20.0 +zach ovid 65784 20.0 +zach quirinius 65691 20.0 +zach robinson 65599 20.0 +zach steinbeck 65602 20.0 +zach steinbeck 65695 20.0 +zach thompson 65636 20.0 +zach thompson 65696 20.0 +zach underhill 65573 20.0 +zach white 65733 20.0 +zach xylophone 65542 20.0 +zach xylophone 65780 20.0 +zach young 65576 20.0 +zach zipper 65579 20.0 +zach zipper 65649 20.0 +zach zipper 65676 20.0 diff --git a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar new file mode 100644 index 0000000000000..37af9aafad8a4 Binary files /dev/null and b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar differ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala deleted file mode 100644 index 0270e63557963..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ /dev/null @@ -1,140 +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.sql - -import scala.collection.JavaConversions._ - -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.util._ - - -/** - * *** DUPLICATED FROM sql/core. *** - * - * It is hard to have maven allow one subproject depend on another subprojects test code. - * So, we duplicate this code here. - */ -class QueryTest extends PlanTest { - - /** - * Runs the plan and makes sure the answer contains all of the keywords, or the - * none of keywords are listed in the answer - * @param rdd the [[DataFrame]] to be executed - * @param exists true for make sure the keywords are listed in the output, otherwise - * to make sure none of the keyword are not listed in the output - * @param keywords keyword in string array - */ - def checkExistence(rdd: DataFrame, exists: Boolean, keywords: String*) { - val outputs = rdd.collect().map(_.mkString).mkString - for (key <- keywords) { - if (exists) { - assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") - } else { - assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") - } - } - } - - /** - * Runs the plan and makes sure the answer matches the expected result. - * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. - */ - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Unit = { - QueryTest.checkAnswer(rdd, expectedAnswer) match { - case Some(errorMessage) => fail(errorMessage) - case None => - } - } - - protected def checkAnswer(rdd: DataFrame, expectedAnswer: Row): Unit = { - checkAnswer(rdd, Seq(expectedAnswer)) - } - - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { - test(sqlString) { - checkAnswer(sqlContext.sql(sqlString), expectedAnswer) - } - } -} - -object QueryTest { - /** - * Runs the plan and makes sure the answer matches the expected result. - * If there was exception during the execution or the contents of the DataFrame does not - * match the expected result, an error message will be returned. Otherwise, a [[None]] will - * be returned. - * @param rdd the [[DataFrame]] to be executed - * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. - */ - def checkAnswer(rdd: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { - val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty - def prepareAnswer(answer: Seq[Row]): Seq[Row] = { - // Converts data to types that we can do equality comparison using Scala collections. - // For BigDecimal type, the Scala type has a better definition of equality test (similar to - // Java's java.math.BigDecimal.compareTo). - val converted: Seq[Row] = answer.map { s => - Row.fromSeq(s.toSeq.map { - case d: java.math.BigDecimal => BigDecimal(d) - case o => o - }) - } - if (!isSorted) converted.sortBy(_.toString) else converted - } - val sparkAnswer = try rdd.collect().toSeq catch { - case e: Exception => - val errorMessage = - s""" - |Exception thrown while executing query: - |${rdd.queryExecution} - |== Exception == - |$e - |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} - """.stripMargin - return Some(errorMessage) - } - - if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - val errorMessage = - s""" - |Results do not match for query: - |${rdd.logicalPlan} - |== Analyzed Plan == - |${rdd.queryExecution.analyzed} - |== Physical Plan == - |${rdd.queryExecution.executedPlan} - |== Results == - |${sideBySide( - s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} - """.stripMargin - return Some(errorMessage) - } - - return None - } - - def checkAnswer(rdd: DataFrame, expectedAnswer: java.util.List[Row]): String = { - checkAnswer(rdd, expectedAnswer.toSeq) match { - case Some(errorMessage) => errorMessage - case None => null - } - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala deleted file mode 100644 index 98f1c0e69e29d..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ /dev/null @@ -1,57 +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.sql.catalyst.plans - -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util._ -import org.scalatest.FunSuite - -/** - * *** DUPLICATED FROM sql/catalyst/plans. *** - * - * It is hard to have maven allow one subproject depend on another subprojects test code. - * So, we duplicate this code here. - */ -class PlanTest extends FunSuite { - - /** - * Since attribute references are given globally unique ids during analysis, - * we must normalize them to check if two different queries are identical. - */ - protected def normalizeExprIds(plan: LogicalPlan) = { - plan transformAllExpressions { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) - case a: Alias => - Alias(a.child, a.name)(exprId = ExprId(0)) - } - } - - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) - fail( - s""" - |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 44d24273e722a..fc6c3c35037b0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -17,28 +17,16 @@ package org.apache.spark.sql.hive +import java.io.File + import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} +import org.apache.spark.sql.{SaveMode, AnalysisException, DataFrame, QueryTest} import org.apache.spark.storage.RDDBlockId +import org.apache.spark.util.Utils class CachedTableSuite extends QueryTest { - /** - * Throws a test failed exception when the number of cached tables differs from the expected - * number. - */ - def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = { - val planWithCaching = query.queryExecution.withCachedData - val cachedData = planWithCaching collect { - case cached: InMemoryRelation => cached - } - - assert( - cachedData.size == numCachedTables, - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + - planWithCaching) - } def rddIdOf(tableName: String): Int = { val executedPlan = table(tableName).queryExecution.executedPlan @@ -92,12 +80,12 @@ class CachedTableSuite extends QueryTest { } test("Drop cached table") { - sql("CREATE TABLE test(a INT)") - cacheTable("test") - sql("SELECT * FROM test").collect() - sql("DROP TABLE test") + sql("CREATE TABLE cachedTableTest(a INT)") + cacheTable("cachedTableTest") + sql("SELECT * FROM cachedTableTest").collect() + sql("DROP TABLE cachedTableTest") intercept[AnalysisException] { - sql("SELECT * FROM test").collect() + sql("SELECT * FROM cachedTableTest").collect() } } @@ -170,4 +158,49 @@ class CachedTableSuite extends QueryTest { assertCached(table("udfTest")) uncacheTable("udfTest") } + + test("REFRESH TABLE also needs to recache the data (data source tables)") { + val tempPath: File = Utils.createTempDir() + tempPath.delete() + table("src").save(tempPath.toString, "parquet", SaveMode.Overwrite) + sql("DROP TABLE IF EXISTS refreshTable") + createExternalTable("refreshTable", tempPath.toString, "parquet") + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Cache the table. + sql("CACHE TABLE refreshTable") + assertCached(table("refreshTable")) + // Append new data. + table("src").save(tempPath.toString, "parquet", SaveMode.Append) + // We are still using the old data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Refresh the table. + sql("REFRESH TABLE refreshTable") + // We are using the new data. + assertCached(table("refreshTable")) + checkAnswer( + table("refreshTable"), + table("src").unionAll(table("src")).collect()) + + // Drop the table and create it again. + sql("DROP TABLE refreshTable") + createExternalTable("refreshTable", tempPath.toString, "parquet") + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + // Refresh the table. REFRESH TABLE command should not make a uncached + // table cached. + sql("REFRESH TABLE refreshTable") + checkAnswer( + table("refreshTable"), + table("src").unionAll(table("src")).collect()) + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + + sql("DROP TABLE refreshTable") + Utils.deleteRecursively(tempPath) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index f04437c595bf6..d960a30e00738 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -19,12 +19,29 @@ package org.apache.spark.sql.hive import java.io.{OutputStream, PrintStream} +import scala.util.Try + +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.{AnalysisException, QueryTest} -import scala.util.Try -class ErrorPositionSuite extends QueryTest { +class ErrorPositionSuite extends QueryTest with BeforeAndAfter { + + before { + Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes") + } + + positionTest("ambiguous attribute reference 1", + "SELECT a from dupAttributes", "a") + + positionTest("ambiguous attribute reference 2", + "SELECT a, b from dupAttributes", "a") + + positionTest("ambiguous attribute reference 3", + "SELECT b, a from dupAttributes", "a") positionTest("unresolved attribute 1", "SELECT x FROM src", "x") @@ -119,7 +136,7 @@ class ErrorPositionSuite extends QueryTest { * @param query the query to analyze * @param token a unique token in the string that should be indicated by the exception */ - def positionTest(name: String, query: String, token: String) = { + def positionTest(name: String, query: String, token: String): Unit = { def parseTree = Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") @@ -127,6 +144,10 @@ class ErrorPositionSuite extends QueryTest { val error = intercept[AnalysisException] { quietly(sql(query)) } + + assert(!error.getMessage.contains("Seq(")) + assert(!error.getMessage.contains("List(")) + val (line, expectedLineNum) = query.split("\n").zipWithIndex.collect { case (l, i) if l.contains(token) => (l, i + 1) }.headOption.getOrElse(sys.error(s"Invalid test. Token $token not in $query")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index 3181cfe40016c..2a7374cc172b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -79,9 +79,9 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: - Literal(Seq[Int](1,2,3), ArrayType(IntegerType)) :: - Literal(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) :: - Literal(Row(1,2.0d,3.0f), + Literal.create(Seq[Int](1,2,3), ArrayType(IntegerType)) :: + Literal.create(Map[Int, Int](1->2, 2->1), MapType(IntegerType, IntegerType)) :: + Literal.create(Row(1,2.0d,3.0f), StructType(StructField("c1", IntegerType) :: StructField("c2", DoubleType) :: StructField("c3", FloatType) :: Nil)) :: @@ -116,21 +116,20 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { - dt1.zip(dt2).map { - case (dd1, dd2) => - assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info + dt1.zip(dt2).foreach { case (dd1, dd2) => + assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info } } def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { - row1.zip(row2).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2).foreach { case (r1, r2) => + checkValue(r1, r2) } } def checkValues(row1: Seq[Any], row2: Row): Unit = { - row1.zip(row2.toSeq).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2.toSeq).foreach { case (r1, r2) => + checkValue(r1, r2) } } @@ -141,7 +140,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { assert(r1.compare(r2) === 0) case (r1: Array[Byte], r2: Array[Byte]) if r1 != null && r2 != null && r1.length == r2.length => - r1.zip(r2).map { case (b1, b2) => assert(b1 === b2) } + r1.zip(r2).foreach { case (b1, b2) => assert(b1 === b2) } case (r1, r2) => assert(r1 === r2) } } @@ -166,7 +165,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val constantData = constantExprs.map(_.eval()) val constantNullData = constantData.map(_ => null) val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType)) - val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal(null, e.dataType))) + val constantNullWritableOIs = + constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) checkValues(constantData, constantData.zip(constantWritableOIs).map { case (d, oi) => unwrap(wrap(d, oi), oi) @@ -202,7 +202,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { case (t, idx) => StructField(s"c_$idx", t) }) - checkValues(row, unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) + checkValues(row, + unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) } @@ -212,8 +213,10 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = row(0) :: row(0) :: Nil checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } test("wrap / unwrap Map Type") { @@ -222,7 +225,9 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = Map(row(0) -> row(1)) checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index aad48ada52642..fa8e11ffec2b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.hive.test.TestHive import org.scalatest.FunSuite import org.apache.spark.sql.test.ExamplePointUDT @@ -36,4 +37,11 @@ class HiveMetastoreCatalogSuite extends FunSuite { assert(HiveMetastoreTypes.toMetastoreType(udt) === HiveMetastoreTypes.toMetastoreType(udt.sqlType)) } + + test("duplicated metastore relations") { + import TestHive.implicits._ + val df = TestHive.sql("SELECT * FROM src") + println(df.queryExecution) + df.as('a).join(df.as('b), $"a.key" === $"b.key") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 381cd2a29123e..ecb990e8aac91 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -32,9 +32,12 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) +case class ThreeCloumntable(key: Int, value: String, key1: String) + class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { import org.apache.spark.sql.hive.test.TestHive.implicits._ + val testData = TestHive.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() @@ -112,11 +115,36 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { test("SPARK-4203:random partition directory order") { sql("CREATE TABLE tmp_table (key int, value string)") val tmpDir = Utils.createTempDir() - sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='3') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='4') SELECT 'blarr' FROM tmp_table") + sql( + s""" + |CREATE TABLE table_with_partition(c1 string) + |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='1') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='2') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='3') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='4') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) def listFolders(path: File, acc: List[String]): List[List[String]] = { val dir = path.listFiles() val folders = dir.filter(_.isDirectory).toList @@ -186,4 +214,51 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { sql("DROP TABLE hiveTableWithStructValue") } + + test("SPARK-5498:partition schema does not match table schema") { + val testData = TestHive.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") + + val testDatawithNull = TestHive.sparkContext.parallelize( + (1 to 10).map(i => ThreeCloumntable(i, i.toString, null))).toDF() + + val tmpDir = Utils.createTempDir() + sql( + s""" + |CREATE TABLE table_with_partition(key int,value string) + |PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (ds='1') SELECT key,value FROM testData + """.stripMargin) + + // test schema the same between partition and table + sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") + checkAnswer(sql("select key,value from table_with_partition where ds='1' "), + testData.collect().toSeq + ) + + // test difference type of field + sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") + checkAnswer(sql("select key,value from table_with_partition where ds='1' "), + testData.collect().toSeq + ) + + // add column to table + sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)") + checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "), + testDatawithNull.collect().toSeq + ) + + // change column name to table + sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT") + checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "), + testData.collect().toSeq + ) + + sql("DROP TABLE table_with_partition") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index ff2e6ea9ea51d..0538aa203c5a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -23,8 +23,9 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.BeforeAndAfterEach -import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.TableType +import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.mapred.InvalidInputException import org.apache.spark.sql._ @@ -172,7 +173,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("SELECT * FROM jsonTable"), Row("a", "b")) - FileUtils.deleteDirectory(tempDir) + Utils.deleteRecursively(tempDir) sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) @@ -188,7 +189,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { checkAnswer( sql("SELECT * FROM jsonTable"), Row("a1", "b1", "c1")) - FileUtils.deleteDirectory(tempDir) + Utils.deleteRecursively(tempDir) } test("drop, change, recreate") { @@ -210,7 +211,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("SELECT * FROM jsonTable"), Row("a", "b")) - FileUtils.deleteDirectory(tempDir) + Utils.deleteRecursively(tempDir) sparkContext.parallelize(("a", "b", "c") :: Nil).toDF() .toJSON.saveAsTextFile(tempDir.getCanonicalPath) @@ -229,7 +230,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { checkAnswer( sql("SELECT * FROM jsonTable"), Row("a", "b", "c")) - FileUtils.deleteDirectory(tempDir) + Utils.deleteRecursively(tempDir) } test("invalidate cache and reload") { @@ -579,7 +580,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { Row(3) :: Row(4) :: Nil ) - table("test_parquet_ctas").queryExecution.analyzed match { + table("test_parquet_ctas").queryExecution.optimizedPlan match { case LogicalRelation(p: ParquetRelation2) => // OK case _ => fail( @@ -682,6 +683,27 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { assert(schema === actualSchema) } + test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") { + val tableName = "spark6655" + val schema = StructType(StructField("int", IntegerType, true) :: Nil) + // Manually create the metadata in metastore. + val tbl = new Table("default", tableName) + tbl.setProperty("spark.sql.sources.provider", "json") + tbl.setProperty("spark.sql.sources.schema", schema.json) + tbl.setProperty("EXTERNAL", "FALSE") + tbl.setTableType(TableType.MANAGED_TABLE) + tbl.setSerdeParam("path", catalog.hiveDefaultTableFilePath(tableName)) + catalog.synchronized { + catalog.client.createTable(tbl) + } + + invalidateTable(tableName) + val actualSchema = table(tableName).schema + assert(schema === actualSchema) + sql(s"drop table $tableName") + } + + test("insert into a table") { def createDF(from: Int, to: Int): DataFrame = createDataFrame((from to to).map(i => Tuple2(i, s"str$i"))).toDF("c1", "c2") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala new file mode 100644 index 0000000000000..4990092df6a99 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -0,0 +1,67 @@ +/* + * 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.sql.hive + +import com.google.common.io.Files + +import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + + +class QueryPartitionSuite extends QueryTest { + import org.apache.spark.sql.hive.test.TestHive.implicits._ + + test("SPARK-5068: query data when path doesn't exist"){ + val testData = TestHive.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") + + val tmpDir = Files.createTempDir() + // create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) " + + s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') " + + "SELECT key,value FROM testData") + + // test for the exist path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) + + // delete the path of one partition + tmpDir.listFiles + .find { f => f.isDirectory && f.getName().startsWith("ds=") } + .foreach { f => Utils.deleteRecursively(f) } + + // test for after delete the path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect) + + sql("DROP TABLE table_with_partition") + sql("DROP TABLE createAndInsertTest") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 1e05a024b8807..00a69de9e4262 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -120,7 +120,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // Try to analyze a temp table sql("""SELECT * FROM src""").registerTempTable("tempTable") - intercept[NotImplementedError] { + intercept[UnsupportedOperationException] { analyze("tempTable") } catalog.unregisterTable(Seq("tempTable")) @@ -142,7 +142,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { after: () => Unit, query: String, expectedAnswer: Seq[Row], - ct: ClassTag[_]) = { + ct: ClassTag[_]): Unit = { before() var df = sql(query) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala new file mode 100644 index 0000000000000..81e77ba257bf1 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.sql.hive.client + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.util.Utils +import org.scalatest.FunSuite + +class VersionsSuite extends FunSuite with Logging { + val testType = "derby" + + private def buildConf() = { + lazy val warehousePath = Utils.createTempDir() + lazy val metastorePath = Utils.createTempDir() + metastorePath.delete() + Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$metastorePath;create=true", + "hive.metastore.warehouse.dir" -> warehousePath.toString) + } + + test("success sanity check") { + val badClient = IsolatedClientLoader.forVersion("13", buildConf()).client + val db = new HiveDatabase("default", "") + badClient.createDatabase(db) + } + + private def getNestedMessages(e: Throwable): String = { + var causes = "" + var lastException = e + while (lastException != null) { + causes += lastException.toString + "\n" + lastException = lastException.getCause + } + causes + } + + // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally + // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the + // versions right by forcing a known compatibility failure. + // TODO: currently only works on mysql where we manually create the schema... + ignore("failure sanity check") { + val e = intercept[Throwable] { + val badClient = quietly { IsolatedClientLoader.forVersion("13", buildConf()).client } + } + assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") + } + + private val versions = Seq("12", "13") + + private var client: ClientInterface = null + + versions.foreach { version => + test(s"$version: listTables") { + client = null + client = IsolatedClientLoader.forVersion(version, buildConf()).client + client.listTables("default") + } + + test(s"$version: createDatabase") { + val db = HiveDatabase("default", "") + client.createDatabase(db) + } + + test(s"$version: createTable") { + val table = + HiveTable( + specifiedDatabase = Option("default"), + name = "src", + schema = Seq(HiveColumn("key", "int", "")), + partitionColumns = Seq.empty, + properties = Map.empty, + serdeProperties = Map.empty, + tableType = ManagedTable, + location = None, + inputFormat = + Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), + outputFormat = + Some(classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = + Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName())) + + client.createTable(table) + } + + test(s"$version: getTable") { + client.getTable("default", "src") + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 42a82c1fbf5c7..a3f5921a0cb23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.hive.test.TestHive._ class BigDataBenchmarkSuite extends HiveComparisonTest { val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( "rankings", @@ -63,7 +64,7 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { | searchWord STRING, | duration INT) | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," - | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "uservisits").getCanonicalPath}" + | STORED AS TEXTFILE LOCATION "$userVisitPath" """.stripMargin.cmd), TestTable( "documents", @@ -83,7 +84,10 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") createQueryTest("query2", - "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") + """ + |SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits + |GROUP BY SUBSTR(sourceIP, 1, 10) + """.stripMargin) createQueryTest("query3", """ @@ -113,8 +117,8 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { |CREATE TABLE url_counts_total AS | SELECT SUM(count) AS totalCount, destpage | FROM url_counts_partial GROUP BY destpage - |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic - |-- given different input splits. + |-- The following queries run, but generate different results in HIVE + |-- likely because the UDF is not deterministic given different input splits. |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial |-- SELECT COUNT(*) FROM url_counts_partial |-- SELECT * FROM url_counts_partial diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 8f3285242091c..5ead5f3c19908 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -138,7 +138,7 @@ abstract class HiveComparisonTest case _ => plan.children.iterator.exists(isSorted) } - val orderedAnswer = hiveQuery.logical match { + val orderedAnswer = hiveQuery.analyzed match { // Clean out non-deterministic time schema info. // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. @@ -255,8 +255,9 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") .filterNot(_ contains "hive.exec.post.hooks") - if (allQueries != queryList) + if (allQueries != queryList) { logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -305,13 +306,16 @@ abstract class HiveComparisonTest try { // Hooks often break the harness and don't really affect our test anyway, don't // even try running them. - if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) + if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) { sys.error("hive exec hooks not supported for tests.") + } - logWarning(s"Running query ${i+1}/${queryList.size} with hive.") + logWarning(s"Running query ${i + 1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { - case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _: ExplainCommand => + // No need to execute EXPLAIN queries as we don't check the output. + Nil case _ => TestHive.runSqlHive(queryString) } @@ -394,21 +398,24 @@ abstract class HiveComparisonTest case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => if (System.getProperty("spark.hive.canarytest") != null) { - // When we encounter an error we check to see if the environment is still okay by running a simple query. - // If this fails then we halt testing since something must have gone seriously wrong. + // When we encounter an error we check to see if the environment is still + // okay by running a simple query. If this fails then we halt testing since + // something must have gone seriously wrong. try { new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") - // The testing setup traps exits so wait here for a long time so the developer can see when things started - // to go wrong. + logError(s"FATAL ERROR: Canary query threw $e This implies that the " + + "testing environment has likely been corrupted.") + // The testing setup traps exits so wait here for a long time so the developer + // can see when things started to go wrong. Thread.sleep(1000000) } } - // If the canary query didn't fail then the environment is still okay, so just throw the original exception. + // If the canary query didn't fail then the environment is still okay, + // so just throw the original exception. throw originalException } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index c939e6e99d28a..bdb53ddf59c19 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -22,10 +22,12 @@ import org.apache.spark.sql.hive.test.TestHive class HivePlanTest extends QueryTest { import TestHive._ + import TestHive.implicits._ test("udf constant folding") { - val optimized = sql("SELECT cos(null) FROM src").queryExecution.optimizedPlan - val correctAnswer = sql("SELECT cast(null as double) FROM src").queryExecution.optimizedPlan + Seq.empty[Tuple1[Int]].toDF("a").registerTempTable("t") + val optimized = sql("SELECT cos(null) FROM t").queryExecution.optimizedPlan + val correctAnswer = sql("SELECT cast(null as double) FROM t").queryExecution.optimizedPlan comparePlans(optimized, correctAnswer) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 02518d516261b..f7b37dae0a5f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. * - * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles that should be included. - * Additionally, there is support for whitelisting and blacklisting tests as development progresses. + * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles + * that should be included. Additionally, there is support for whitelisting and blacklisting + * tests as development progresses. */ abstract class HiveQueryFileTest extends HiveComparisonTest { /** A list of tests deemed out of scope and thus completely disregarded */ @@ -54,15 +55,17 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { logDebug(s"Blacklisted test skipped $testCaseName") - } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { + } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || + runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString) } else { // Only output warnings for the built in whitelist as this clutters the output when the user // trying to execute a single test from the commandline. - if(System.getProperty(whiteListProperty) == null && !runAll) + if (System.getProperty(whiteListProperty) == null && !runAll) { ignore(testCaseName) {} + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index de140fc72a2c3..ac10b173307d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -27,7 +27,7 @@ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.{SparkFiles, SparkException} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ @@ -37,7 +37,8 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(a: Int, b: String) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault @@ -66,6 +67,40 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } + createQueryTest("insert table with generator with column name", + """ + | CREATE TABLE gen_tmp (key Int); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(array(1,2,3)) AS val FROM src LIMIT 3; + | SELECT key FROM gen_tmp ORDER BY key ASC; + """.stripMargin) + + createQueryTest("insert table with generator with multiple column names", + """ + | CREATE TABLE gen_tmp (key Int, value String); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(map(key, value)) as (k1, k2) FROM src LIMIT 3; + | SELECT key, value FROM gen_tmp ORDER BY key, value ASC; + """.stripMargin) + + createQueryTest("insert table with generator without column name", + """ + | CREATE TABLE gen_tmp (key Int); + | INSERT OVERWRITE TABLE gen_tmp + | SELECT explode(array(1,2,3)) FROM src LIMIT 3; + | SELECT key FROM gen_tmp ORDER BY key ASC; + """.stripMargin) + + test("multiple generator in projection") { + intercept[AnalysisException] { + sql("SELECT explode(map(key, value)), key FROM src").collect() + } + + intercept[AnalysisException] { + sql("SELECT explode(map(key, value)) as k1, k2, key FROM src").collect() + } + } + createQueryTest("! operator", """ |SELECT a FROM ( @@ -237,7 +272,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } createQueryTest("modulus", - "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") + "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), " + + "(101 / 2) % 10 FROM src LIMIT 1") test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") @@ -309,7 +345,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { "SELECT * FROM src a JOIN src b ON a.key = b.key") createQueryTest("small.cartesian", - "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN (SELECT key FROM src WHERE key = 2) b") + "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN " + + "(SELECT key FROM src WHERE key = 2) b") createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") @@ -453,10 +490,10 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view2", "SELECT * FROM src LATERAL VIEW explode(array(1,2)) tbl") - createQueryTest("lateral view3", "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX") + // scalastyle:off createQueryTest("lateral view4", """ |create table src_lv1 (key string, value string); @@ -466,6 +503,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |insert overwrite table src_lv1 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX |insert overwrite table src_lv2 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX """.stripMargin) + // scalastyle:on createQueryTest("lateral view5", "FROM src SELECT explode(array(key+3, key+4))") @@ -473,6 +511,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view6", "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") + createQueryTest("Specify the udtf output", + "SELECT d FROM (SELECT explode(array(1,1)) d FROM src LIMIT 1) t") + test("sampling") { sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") sql("SELECT * FROM src TABLESAMPLE(100 PERCENT) s") @@ -537,6 +578,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") + createQueryTest("CTE feature #1", + "with q1 as (select key from src) select * from q1 where key = 5") + + createQueryTest("CTE feature #2", + """with q1 as (select * from src where key= 5), + |q2 as (select * from src s2 where key = 4) + |select value from q1 union all select value from q2 + """.stripMargin) + + createQueryTest("CTE feature #3", + """with q1 as (select key from src) + |from q1 + |select * where key = 4 + """.stripMargin) + test("predicates contains an empty AttributeSet() references") { sql( """ @@ -584,7 +640,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - def isExplanation(result: DataFrame) = { + def isExplanation(result: DataFrame): Boolean = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } explanation.contains("== Physical Plan ==") } @@ -793,6 +849,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("DROP TABLE alter1") } + test("ADD JAR command 2") { + // this is a test case from mapjoin_addjar.q + val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath + val testData = TestHive.getHiveFile("data/files/sample.json").getCanonicalPath + if (HiveShim.version == "0.13.1") { + sql(s"ADD JAR $testJar") + sql( + """CREATE TABLE t1(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") + sql("select * from src join t1 on src.key = t1.a") + sql("DROP TABLE t1") + } + } + test("ADD FILE command") { val testFile = TestHive.getHiveFile("data/files/v1.txt").getCanonicalFile sql(s"ADD FILE $testFile") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index f4440e5b7846a..8ad3627504229 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -25,7 +25,8 @@ case class Nested(a: Int, B: Int) case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveResolutionSuite extends HiveComparisonTest { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 7486bfa82b00b..5586a793618bd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -25,17 +25,25 @@ import org.apache.spark.sql.hive.test.TestHive * A set of tests that validates support for Hive SerDe. */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - - override def beforeAll() = { + override def beforeAll(): Unit = { + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + super.beforeAll() TestHive.cacheTables = false + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") } + // table sales is not a cache table, and will be clear after reset + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales", false) + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") - createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") - createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index ab0e0443c7faa..f0f04f8c73fb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -35,8 +35,10 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { val nullVal = "null" baseTypes.init.foreach { i => - createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") - createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + createQueryTest(s"case when then $i else $nullVal end ", + s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", + s"SELECT case when true then $nullVal else $i end FROM src limit 1") } test("[SPARK-2210] boolean cast on boolean value should be removed") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index cb405f56bf53d..7f49eac490572 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -22,7 +22,7 @@ import java.util import java.util.Properties import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFAverage, GenericUDF} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} @@ -93,6 +93,15 @@ class HiveUdfSuite extends QueryTest { sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } + test("SPARK-6409 UDAFAverage test") { + sql(s"CREATE TEMPORARY FUNCTION test_avg AS '${classOf[GenericUDAFAverage].getName}'") + checkAnswer( + sql("SELECT test_avg(1), test_avg(substr(value,5)) FROM src"), + Seq(Row(1.0, 260.182))) + sql("DROP TEMPORARY FUNCTION IF EXISTS test_avg") + TestHive.reset() + } + test("SPARK-2693 udaf aggregates test") { checkAnswer(sql("SELECT percentile(key, 1) FROM src LIMIT 1"), sql("SELECT max(key) FROM src").collect().toSeq) @@ -114,9 +123,10 @@ class HiveUdfSuite extends QueryTest { IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() testData.registerTempTable("integerTable") - sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") + val udfName = classOf[UDFIntegerToString].getName + sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") checkAnswer( - sql("SELECT testUDFIntegerToString(i) FROM integerTable"), //.collect(), + sql("SELECT testUDFIntegerToString(i) FROM integerTable"), Seq(Row("1"), Row("2"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFIntegerToString") @@ -132,7 +142,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( - sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), //.collect(), + sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), Seq(Row(0), Row(2), Row(13))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListListInt") @@ -147,7 +157,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( - sql("SELECT testUDFListString(l) FROM listStringTable"), //.collect(), + sql("SELECT testUDFListString(l) FROM listStringTable"), Seq(Row("a,b,c"), Row("d,e"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListString") @@ -161,7 +171,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") checkAnswer( - sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), //.collect(), + sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), Seq(Row("hello world"), Row("hello goodbye"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUdf") @@ -178,7 +188,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( - sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), //.collect(), + sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), Seq(Row("0, 0"), Row("2, 2"), Row("13, 13"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList") @@ -238,7 +248,8 @@ class PairUdf extends GenericUDF { override def initialize(p1: Array[ObjectInspector]): ObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( Seq("id", "value"), - Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, PrimitiveObjectInspectorFactory.javaIntObjectInspector) + Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.javaIntObjectInspector) ) override def evaluate(args: Array[DeferredObject]): AnyRef = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 0000000000000..934452fe579a1 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * 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.sql.hive.execution + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // Create the table used in windowing.q + sql("DROP TABLE IF EXISTS part") + sql( + """ + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) + """.stripMargin) + val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + + sql("DROP TABLE IF EXISTS over1k") + sql( + """ + |create table over1k( + | t tinyint, + | si smallint, + | i int, + | b bigint, + | f float, + | d double, + | bo boolean, + | s string, + | ts timestamp, + | dec decimal(4,2), + | bin binary) + |row format delimited + |fields terminated by '|' + """.stripMargin) + val testData2 = TestHive.getHiveFile("data/files/over1k").getCanonicalPath + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData2' overwrite into table over1k + """.stripMargin) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_multipartitioning.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_multipartitioning.q (deterministic) 1", + s""" + |select s, + |rank() over (partition by s order by si) r, + |sum(b) over (partition by s order by si) sum + |from over1k + |order by s, r, sum; + """.stripMargin, reset = false) + + /* timestamp comparison issue with Hive? + createQueryTest("windowing_multipartitioning.q (deterministic) 2", + s""" + |select s, + |rank() over (partition by s order by dec desc) r, + |sum(b) over (partition by s order by ts desc) as sum + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |order by s, r, sum; + """.stripMargin, reset = false) + */ + + createQueryTest("windowing_multipartitioning.q (deterministic) 3", + s""" + |select s, sum(i) over (partition by s), sum(f) over (partition by si) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 4", + s""" + |select s, rank() over (partition by s order by bo), + |rank() over (partition by si order by bin desc) from over1k + |where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 5", + s""" + |select s, sum(f) over (partition by i), row_number() over (order by f) + |from over1k where s = 'tom allen' or s = 'bob steinbeck'; + """.stripMargin, reset = false) + + createQueryTest("windowing_multipartitioning.q (deterministic) 6", + s""" + |select s, rank() over w1, + |rank() over w2 + |from over1k + |where s = 'tom allen' or s = 'bob steinbeck' + |window + |w1 as (partition by s order by dec), + |w2 as (partition by si order by f) ; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_navfn.q + // Results of the original query file are not deterministic. + // Also, the original query of + // select i, lead(s) over (partition by bin order by d,i desc) from over1k ; + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_navfn.q (deterministic)", + s""" + |select s, row_number() over (partition by d order by dec) rn from over1k + |order by s, rn desc; + |select i, lead(s) over (partition by cast(bin as string) order by d,i desc) as l + |from over1k + |order by i desc, l; + |select i, lag(dec) over (partition by i order by s,i,dec) l from over1k + |order by i, l; + |select s, last_value(t) over (partition by d order by f) l from over1k + |order by s, l; + |select s, first_value(s) over (partition by bo order by s) f from over1k + |order by s, f; + |select t, s, i, last_value(i) over (partition by t order by s) + |from over1k where (s = 'oscar allen' or s = 'oscar carson') and t = 10; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_ntile.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_ntile.q (deterministic)", + s""" + |select i, ntile(10) over (partition by s order by i) n from over1k + |order by i, n; + |select s, ntile(100) over (partition by i order by s) n from over1k + |order by s, n; + |select f, ntile(4) over (partition by d order by f) n from over1k + |order by f, n; + |select d, ntile(1000) over (partition by dec order by d) n from over1k + |order by d, n; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_udaf.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_udaf.q (deterministic)", + s""" + |select s, min(i) over (partition by s) m from over1k + |order by s, m; + |select s, avg(f) over (partition by si order by s) a from over1k + |order by s, a; + |select s, avg(i) over (partition by t, b order by s) a from over1k + |order by s, a; + |select max(i) over w m from over1k + |order by m window w as (partition by f) ; + |select s, avg(d) over (partition by t order by f) a from over1k + |order by s, a; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_windowspec.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_windowspec.q (deterministic)", + s""" + |select s, sum(b) over (partition by i order by s,b rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over (partition by d order by s,f rows unbounded preceding) as sum + |from over1k order by s, sum; + |select s, sum(f) over + |(partition by ts order by f range between current row and unbounded following) as sum + |from over1k order by s, sum; + |select s, avg(f) + |over (partition by ts order by s,f rows between current row and 5 following) avg + |from over1k order by s, avg; + |select s, avg(d) over + |(partition by t order by s,d desc rows between 5 preceding and 5 following) avg + |from over1k order by s, avg; + |select s, sum(i) over(partition by ts order by s) sum from over1k + |order by s, sum; + |select f, sum(f) over + |(partition by ts order by f range between unbounded preceding and current row) sum + |from over1k order by f, sum; + |select s, i, round(avg(d) over (partition by s order by i) / 10.0 , 2) avg + |from over1k order by s, i, avg; + |select s, i, round((avg(d) over w1 + 10.0) - (avg(d) over w1 - 10.0),2) avg + |from over1k + |order by s, i, avg window w1 as (partition by s order by i); + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests based on windowing_rank.q + // Results of the original query file are not deterministic. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing_rank.q (deterministic) 1", + s""" + |select s, rank() over (partition by f order by t) r from over1k order by s, r; + |select s, dense_rank() over (partition by ts order by i,s desc) as r from over1k + |order by s desc, r desc; + |select s, cume_dist() over (partition by bo order by b,s) cd from over1k + |order by s, cd; + |select s, percent_rank() over (partition by dec order by f) r from over1k + |order by s desc, r desc; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 2", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 3", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | ) joined + | ) ranked + |where dec = 89.5 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + createQueryTest("windowing_rank.q (deterministic) 4", + s""" + |select ts, dec, rnk + |from + | (select ts, dec, + | rank() over (partition by ts order by dec) as rnk + | from + | (select other.ts, other.dec + | from over1k other + | join over1k on (other.b = over1k.b) + | where other.t < 10 + | ) joined + | ) ranked + |where rnk = 1 + |order by ts, dec, rnk; + """.stripMargin, reset = false) + + ///////////////////////////////////////////////////////////////////////////// + // Tests from windowing.q + // We port tests in windowing.q to here because this query file contains too + // many tests and the syntax of test "-- 7. testJoinWithWindowingAndPTF" + // is not supported right now. + ///////////////////////////////////////////////////////////////////////////// + createQueryTest("windowing.q -- 1. testWindowing", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over + |(distribute by p_mfgr sort by p_name rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 2. testGroupByWithPartitioning", + s""" + |select p_mfgr, p_name, p_size, + |min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name)as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 3. testGroupByHavingWithSWQ", + s""" + |select p_mfgr, p_name, p_size, min(p_retailprice), + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 4. testCount", + s""" + |select p_mfgr, p_name, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 5. testCountWithWindowingUDAF", + s""" + |select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 6. testCountInSubQ", + s""" + |select sub1.r, sub1.dr, sub1.cd, sub1.s1, sub1.deltaSz + |from (select p_mfgr, p_name, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |count(p_size) over(distribute by p_mfgr sort by p_name) as cd, + |p_retailprice, sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |) sub1 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 8. testMixedCaseAlias", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name, p_size desc) as R + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 9. testHavingWithWindowingNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 10. testHavingWithWindowingCondRankNoGBY", + s""" + |select p_mfgr, p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + | rows between unbounded preceding and current row) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 11. testFirstLast", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 12. testFirstLastWithWhere", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2, + |first_value(p_size) over w1 as f, + |last_value(p_size, false) over w1 as l + |from part + |where p_mfgr = 'Manufacturer#3' + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 13. testSumWindow", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |rows between current row and current row) as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 14. testNoSortClause", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 15. testExpressions", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 16. testMultipleWindows", + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |sum(p_size) over (distribute by p_mfgr sort by p_name + |range between unbounded preceding and current row) as s1, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 5 preceding and current row) as s2, + |first_value(p_size) over w1 as fv1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + + createQueryTest("windowing.q -- 17. testCountStar", + s""" + |select p_mfgr,p_name, p_size, + |count(*) over(distribute by p_mfgr sort by p_name ) as c, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 18. testUDAFs", + s""" + |select p_mfgr,p_name, p_size, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) over w1 as mi, + |max(p_retailprice) over w1 as ma, + |avg(p_retailprice) over w1 as ag + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 19. testUDAFsWithGBY", + """ + |select p_mfgr,p_name, p_size, p_retailprice, + |sum(p_retailprice) over w1 as s, + |min(p_retailprice) as mi , + |max(p_retailprice) as ma , + |avg(p_retailprice) over w1 as ag + |from part + |group by p_mfgr,p_name, p_size, p_retailprice + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following); + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 20. testSTATs", + """ + |select p_mfgr,p_name, p_size, + |stddev(p_retailprice) over w1 as sdev, + |stddev_pop(p_retailprice) over w1 as sdev_pop, + |collect_set(p_size) over w1 as uniq_size, + |variance(p_retailprice) over w1 as var, + |corr(p_size, p_retailprice) over w1 as cor, + |covar_pop(p_size, p_retailprice) over w1 as covarp + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 21. testDISTs", + """ + |select p_mfgr,p_name, p_size, + |histogram_numeric(p_retailprice, 5) over w1 as hist, + |percentile(p_partkey, 0.5) over w1 as per, + |row_number() over(distribute by p_mfgr sort by p_name) as rn + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 24. testLateralViews", + """ + |select p_mfgr, p_name, + |lv_col, p_size, sum(p_size) over w1 as s + |from (select p_mfgr, p_name, p_size, array(1,2,3) arr from part) p + |lateral view explode(arr) part_lv as lv_col + |window w1 as (distribute by p_mfgr sort by p_size, lv_col + | rows between 2 preceding and current row) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 26. testGroupByHavingWithSWQAndAlias", + """ + |select p_mfgr, p_name, p_size, min(p_retailprice) as mi, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |p_size, p_size - lag(p_size,1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz + |from part + |group by p_mfgr, p_name, p_size + |having p_size > 0 + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 27. testMultipleRangeWindows", + """ + |select p_mfgr,p_name, p_size, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between 10 preceding and current row) as s2, + |sum(p_size) over (distribute by p_mfgr sort by p_size + |range between current row and 10 following ) as s1 + |from part + |window w1 as (rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 28. testPartOrderInUDAFInvoke", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over (partition by p_mfgr order by p_name + |rows between 2 preceding and 2 following) as s + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 29. testPartOrderInWdwDef", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (partition by p_mfgr order by p_name + | rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 30. testDefaultPartitioningSpecRules", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following), + | w2 as (partition by p_mfgr order by p_name) + """.stripMargin, reset = false) + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 31. testWindowCrossReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as w1 + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 32. testWindowInheritance", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2 + |from part + |window w1 as (partition by p_mfgr order by p_name + | range between 2 preceding and 2 following), + | w2 as (w1 rows between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* p_name is not a numeric column. What is Hive's semantic? + createQueryTest("windowing.q -- 33. testWindowForwardReference", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over w3 as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + /* + createQueryTest("windowing.q -- 34. testWindowDefinitionPropagation", + """ + |select p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s1, + |sum(p_size) over w2 as s2, + |sum(p_size) over (w3 rows between 2 preceding and 2 following) as s3 + |from part + |window w1 as (distribute by p_mfgr sort by p_name + | range between 2 preceding and 2 following), + | w2 as w3, + | w3 as (distribute by p_mfgr sort by p_name + | range between unbounded preceding and current row) + """.stripMargin, reset = false) + */ + + /* Seems Hive evaluate SELECT DISTINCT before window functions? + createQueryTest("windowing.q -- 35. testDistinctWithWindowing", + """ + |select DISTINCT p_mfgr, p_name, p_size, + |sum(p_size) over w1 as s + |from part + |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) + """.stripMargin, reset = false) + */ + + createQueryTest("windowing.q -- 36. testRankWithPartitioning", + """ + |select p_mfgr, p_name, p_size, + |rank() over (partition by p_mfgr order by p_name ) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 37. testPartitioningVariousForms", + """ + |select p_mfgr, + |round(sum(p_retailprice) over (partition by p_mfgr order by p_mfgr),2) as s1, + |min(p_retailprice) over (partition by p_mfgr) as s2, + |max(p_retailprice) over (distribute by p_mfgr sort by p_mfgr) as s3, + |round(avg(p_retailprice) over (distribute by p_mfgr),2) as s4, + |count(p_retailprice) over (cluster by p_mfgr ) as s5 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 38. testPartitioningVariousForms2", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (partition by p_mfgr, p_name order by p_mfgr, p_name + |rows between unbounded preceding and current row) as s1, + |min(p_retailprice) over (distribute by p_mfgr, p_name sort by p_mfgr, p_name + |rows between unbounded preceding and current row) as s2, + |max(p_retailprice) over (partition by p_mfgr, p_name order by p_name) as s3 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 39. testUDFOnOrderCols", + """ + |select p_mfgr, p_type, substr(p_type, 2) as short_ptype, + |rank() over (partition by p_mfgr order by substr(p_type, 2)) as r + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 40. testNoBetweenForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name rows unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 41. testNoBetweenForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size range unbounded preceding) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 42. testUnboundedFollowingForRows", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_name + |rows between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 43. testUnboundedFollowingForRange", + """ + |select p_mfgr, p_name, p_size, + |sum(p_retailprice) over (distribute by p_mfgr sort by p_size + |range between current row and unbounded following) as s1 + |from part + """.stripMargin, reset = false) + + createQueryTest("windowing.q -- 44. testOverNoPartitionSingleAggregate", + """ + |select p_name, p_retailprice, + |round(avg(p_retailprice) over(),2) + |from part + |order by p_name + """.stripMargin, reset = false) +} + +class HiveWindowFunctionQueryWithoutCodeGenSuite extends HiveWindowFunctionQueryBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} + +abstract class HiveWindowFunctionQueryFileBaseSuite + extends HiveCompatibilitySuite with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + // The following settings are used for generating golden files with Hive. + // We have to use kryo to correctly let Hive serialize plans with window functions. + // This is used to generate golden files. + sql("set hive.plan.serialization.format=kryo") + // Explicitly set fs to local fs. + sql(s"set fs.default.name=file://$testTempDir/") + // Ask Hive to run jobs in-process as a single map and reduce task. + sql("set mapred.job.tracker=local") + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } + + override def blackList: Seq[String] = Seq( + // Partitioned table functions are not supported. + "ptf*", + // tests of windowing.q are in HiveWindowFunctionQueryBaseSuite + "windowing.q", + + // This one failed on the expression of + // sum(lag(p_retailprice,1,0.0)) over w1 + // lag(p_retailprice,1,0.0) is a GenericUDF and the argument inspector of + // p_retailprice created by HiveInspectors is + // PrimitiveObjectInspectorFactory.javaDoubleObjectInspector. + // However, seems Hive assumes it is + // PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, which introduces an error. + "windowing_expressions", + + // Hive's results are not deterministic + "windowing_multipartitioning", + "windowing_navfn", + "windowing_ntile", + "windowing_udaf", + "windowing_windowspec", + "windowing_rank" + ) + + override def whiteList: Seq[String] = Seq( + "windowing_udaf2", + "windowing_columnPruning", + "windowing_adjust_rowcontainer_sz" + ) + + override def testCases: Seq[(String, File)] = super.testCases.filter { + case (name, _) => realWhiteList.contains(name) + } +} + +class HiveWindowFunctionQueryFileWithoutCodeGenSuite extends HiveWindowFunctionQueryFileBaseSuite { + var originalCodegenEnabled: Boolean = _ + override def beforeAll(): Unit = { + super.beforeAll() + originalCodegenEnabled = conf.codegenEnabled + sql("set spark.sql.codegen=false") + } + + override def afterAll(): Unit = { + sql(s"set spark.sql.codegen=$originalCodegenEnabled") + super.afterAll() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8474d850c9c6c..067b577f1560e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -143,7 +143,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { sql: String, expectedOutputColumns: Seq[String], expectedScannedColumns: Seq[String], - expectedPartValues: Seq[Seq[String]]) = { + expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 1187228f4c3db..616352d223c5b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -18,14 +18,17 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries -import org.apache.spark.sql.hive.{MetastoreRelation, HiveShim} +import org.apache.spark.sql.catalyst.errors.DialectException +import org.apache.spark.sql.DefaultDialect +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} +import org.apache.spark.sql.hive.MetastoreRelation import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.sql.hive.{HiveQLDialect, HiveShim} import org.apache.spark.sql.parquet.ParquetRelation2 import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.types._ -import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -34,15 +37,129 @@ case class Nested3(f3: Int) case class NestedArray2(b: Seq[Int]) case class NestedArray1(a: NestedArray2) +case class Order( + id: Int, + make: String, + `type`: String, + price: Int, + pdate: String, + customer: String, + city: String, + state: String, + month: Int) + +case class WindowData( + month: Int, + area: String, + product: Int) +/** A SQL Dialect for testing purpose, and it can not be nested type */ +class MyDialect extends DefaultDialect + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("SPARK-6835: udtf in lateral view") { + val df = Seq((1, 1)).toDF("c1", "c2") + df.registerTempTable("table1") + val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, c1 + 2) d AS v") + checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) + } + + test("SPARK-6851: Self-joined converted parquet tables") { + val orders = Seq( + Order(1, "Atlas", "MTB", 234, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(3, "Swift", "MTB", 285, "2015-01-17", "John S", "Redwood City", "CA", 20151), + Order(4, "Atlas", "Hybrid", 303, "2015-01-23", "Jones S", "San Mateo", "CA", 20151), + Order(7, "Next", "MTB", 356, "2015-01-04", "Jane D", "Daly City", "CA", 20151), + Order(10, "Next", "YFlikr", 187, "2015-01-09", "John D", "Fremont", "CA", 20151), + Order(11, "Swift", "YFlikr", 187, "2015-01-23", "John D", "Hayward", "CA", 20151), + Order(2, "Next", "Hybrid", 324, "2015-02-03", "Jane D", "Daly City", "CA", 20152), + Order(5, "Next", "Street", 187, "2015-02-08", "John D", "Fremont", "CA", 20152), + Order(6, "Atlas", "Street", 154, "2015-02-09", "John D", "Pacifica", "CA", 20152), + Order(8, "Swift", "Hybrid", 485, "2015-02-19", "John S", "Redwood City", "CA", 20152), + Order(9, "Atlas", "Split", 303, "2015-02-28", "Jones S", "San Mateo", "CA", 20152)) + + val orderUpdates = Seq( + Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151)) + + orders.toDF.registerTempTable("orders1") + orderUpdates.toDF.registerTempTable("orderupdates1") + + sql( + """CREATE TABLE orders( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql( + """CREATE TABLE orderupdates( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("INSERT INTO TABLE orders PARTITION(state, month) SELECT * FROM orders1") + sql("INSERT INTO TABLE orderupdates PARTITION(state, month) SELECT * FROM orderupdates1") + + checkAnswer( + sql( + """ + |select orders.state, orders.month + |from orders + |join ( + | select distinct orders.state,orders.month + | from orders + | join orderupdates + | on orderupdates.id = orders.id) ao + | on ao.state = orders.state and ao.month = orders.month + """.stripMargin), + (1 to 6).map(_ => Row("CA", 20151))) + } + + test("SPARK-5371: union with null and sum") { + val df = Seq((1, 1)).toDF("c1", "c2") + df.registerTempTable("table1") + + val query = sql( + """ + |SELECT + | MIN(c1), + | MIN(c2) + |FROM ( + | SELECT + | SUM(c1) c1, + | NULL c2 + | FROM table1 + | UNION ALL + | SELECT + | NULL c1, + | SUM(c2) c2 + | FROM table1 + |) a + """.stripMargin) + checkAnswer(query, Row(1, 1) :: Nil) + } test("explode nested Field") { - Seq(NestedArray1(NestedArray2(Seq(1,2,3)))).toDF.registerTempTable("nestedArray") + Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.registerTempTable("nestedArray") checkAnswer( sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), Row(1) :: Row(2) :: Row(3) :: Nil) @@ -122,6 +239,35 @@ class SQLQuerySuite extends QueryTest { setConf("spark.sql.hive.convertCTAS", originalConf) } + test("SQL Dialect Switching") { + assert(getSQLDialect().getClass === classOf[HiveQLDialect]) + setConf("spark.sql.dialect", classOf[MyDialect].getCanonicalName()) + assert(getSQLDialect().getClass === classOf[MyDialect]) + assert(sql("SELECT 1").collect() === Array(Row(1))) + + // set the dialect back to the DefaultSQLDialect + sql("SET spark.sql.dialect=sql") + assert(getSQLDialect().getClass === classOf[DefaultDialect]) + sql("SET spark.sql.dialect=hiveql") + assert(getSQLDialect().getClass === classOf[HiveQLDialect]) + + // set invalid dialect + sql("SET spark.sql.dialect.abc=MyTestClass") + sql("SET spark.sql.dialect=abc") + intercept[Exception] { + sql("SELECT 1") + } + // test if the dialect set back to HiveQLDialect + getSQLDialect().getClass === classOf[HiveQLDialect] + + sql("SET spark.sql.dialect=MyTestClass") + intercept[DialectException] { + sql("SELECT 1") + } + // test if the dialect set back to HiveQLDialect + assert(getSQLDialect().getClass === classOf[HiveQLDialect]) + } + test("CTAS with serde") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect() sql( @@ -398,7 +544,7 @@ class SQLQuerySuite extends QueryTest { } test("resolve udtf with single alias") { - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") @@ -411,7 +557,7 @@ class SQLQuerySuite extends QueryTest { // is not in a valid state (cannot be executed). Because of this bug, the analysis rule of // PreInsertionCasts will actually start to work before ImplicitGenerate and then // generates an invalid query plan. - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val originalConf = getConf("spark.sql.hive.convertCTAS", "false") setConf("spark.sql.hive.convertCTAS", "false") @@ -433,4 +579,176 @@ class SQLQuerySuite extends QueryTest { dropTempTable("data") setConf("spark.sql.hive.convertCTAS", originalConf) } + + test("sanity test for SPARK-6618") { + (1 to 100).par.map { i => + val tableName = s"SPARK_6618_table_$i" + sql(s"CREATE TABLE $tableName (col1 string)") + catalog.lookupRelation(Seq(tableName)) + table(tableName) + tables() + sql(s"DROP TABLE $tableName") + } + } + + test("SPARK-5203 union with different decimal precision") { + Seq.empty[(Decimal, Decimal)] + .toDF("d1", "d2") + .select($"d1".cast(DecimalType(10, 15)).as("d")) + .registerTempTable("dn") + + sql("select d from dn union all select d * 2 from dn") + .queryExecution.analyzed + } + + test("test script transform") { + val data = (1 to 100000).map { i => (i, i, i) } + data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + assert(100000 === + sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") + .queryExecution.toRdd.count()) + } + + test("window function: udaf with aggregate expressin") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product), sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 11), + ("a", 6, 11), + ("b", 7, 15), + ("b", 8, 15), + ("c", 9, 19), + ("c", 10, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product) - 1, sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 4, 11), + ("a", 5, 11), + ("b", 6, 15), + ("b", 7, 15), + ("c", 8, 19), + ("c", 9, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/11), + ("a", 6, 6d/11), + ("b", 7, 7d/15), + ("b", 8, 8d/15), + ("c", 10, 10d/19), + ("c", 9, 9d/19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/9), + ("a", 6, 6d/9), + ("b", 7, 7d/13), + ("b", 8, 8d/13), + ("c", 10, 10d/17), + ("c", 9, 9d/17) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("window function: partition and order expressions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 51), + (2, "a", 6, 51), + (3, "b", 7, 51), + (4, "b", 8, 51), + (5, "c", 9, 51), + (6, "c", 10, 51) + ).map(i => Row(i._1, i._2, i._3, i._4))) + + checkAnswer( + sql( + """ + |select month, area, product, sum(product) + |over (partition by month % 2 order by 10 - product) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 21), + (2, "a", 6, 24), + (3, "b", 7, 16), + (4, "b", 8, 18), + (5, "c", 9, 9), + (6, "c", 10, 10) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("window function: expressions in arguments of a window functions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().registerTempTable("windowData") + + checkAnswer( + sql( + """ + |select month, area, month % 2, + |lag(product, 1 + 1, product) over (partition by month % 2 order by area) + |from windowData + """.stripMargin), + Seq( + (1, "a", 1, 5), + (2, "a", 0, 6), + (3, "b", 1, 7), + (4, "b", 0, 8), + (5, "c", 1, 5), + (6, "c", 0, 6) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index d891c4e8903d9..d5dd0bf58e702 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -26,8 +25,10 @@ import org.apache.spark.sql.{QueryTest, SQLConf, SaveMode} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.sql.json.JSONRelation import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.SaveMode @@ -292,7 +293,7 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { Seq(Row(1, "str1")) ) - table("test_parquet_ctas").queryExecution.analyzed match { + table("test_parquet_ctas").queryExecution.optimizedPlan match { case LogicalRelation(p: ParquetRelation2) => // OK case _ => fail( @@ -365,6 +366,139 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase { sql("DROP TABLE IF EXISTS test_insert_parquet") } + + test("SPARK-6450 regression test") { + sql( + """CREATE TABLE IF NOT EXISTS ms_convert (key INT) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + // This shouldn't throw AnalysisException + val analyzed = sql( + """SELECT key FROM ms_convert + |UNION ALL + |SELECT key FROM ms_convert + """.stripMargin).queryExecution.analyzed + + assertResult(2) { + analyzed.collect { + case r @ LogicalRelation(_: ParquetRelation2) => r + }.size + } + + sql("DROP TABLE ms_convert") + } + + test("Caching converted data source Parquet Relations") { + def checkCached(tableIdentifer: catalog.QualifiedTableName): Unit = { + // Converted test_parquet should be cached. + catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) match { + case null => fail("Converted test_parquet should be cached in the cache.") + case logical @ LogicalRelation(parquetRelation: ParquetRelation2) => // OK + case other => + fail( + "The cached test_parquet should be a Parquet Relation. " + + s"However, $other is returned form the cache.") + } + } + + sql("DROP TABLE IF EXISTS test_insert_parquet") + sql("DROP TABLE IF EXISTS test_parquet_partitioned_cache_test") + + sql( + """ + |create table test_insert_parquet + |( + | intField INT, + | stringField STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + var tableIdentifer = catalog.QualifiedTableName("default", "test_insert_parquet") + + // First, make sure the converted test_parquet is not cached. + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + // Table lookup will make the table cached. + table("test_insert_parquet") + checkCached(tableIdentifer) + // For insert into non-partitioned table, we will do the conversion, + // so the converted test_insert_parquet should be cached. + invalidateTable("test_insert_parquet") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + sql( + """ + |INSERT INTO TABLE test_insert_parquet + |select a, b from jt + """.stripMargin) + checkCached(tableIdentifer) + // Make sure we can read the data. + checkAnswer( + sql("select * from test_insert_parquet"), + sql("select a, b from jt").collect()) + // Invalidate the cache. + invalidateTable("test_insert_parquet") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + + // Create a partitioned table. + sql( + """ + |create table test_parquet_partitioned_cache_test + |( + | intField INT, + | stringField STRING + |) + |PARTITIONED BY (date string) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin) + + tableIdentifer = catalog.QualifiedTableName("default", "test_parquet_partitioned_cache_test") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + sql( + """ + |INSERT INTO TABLE test_parquet_partitioned_cache_test + |PARTITION (date='2015-04-01') + |select a, b from jt + """.stripMargin) + // Right now, insert into a partitioned Parquet is not supported in data source Parquet. + // So, we expect it is not cached. + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + sql( + """ + |INSERT INTO TABLE test_parquet_partitioned_cache_test + |PARTITION (date='2015-04-02') + |select a, b from jt + """.stripMargin) + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + + // Make sure we can cache the partitioned table. + table("test_parquet_partitioned_cache_test") + checkCached(tableIdentifer) + // Make sure we can read the data. + checkAnswer( + sql("select STRINGField, date, intField from test_parquet_partitioned_cache_test"), + sql( + """ + |select b, '2015-04-01', a FROM jt + |UNION ALL + |select b, '2015-04-02', a FROM jt + """.stripMargin).collect()) + + invalidateTable("test_parquet_partitioned_cache_test") + assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifer) === null) + + sql("DROP TABLE test_insert_parquet") + sql("DROP TABLE test_parquet_partitioned_cache_test") + } } class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase { @@ -553,6 +687,22 @@ class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { sql("DROP TABLE alwaysNullable") } + + test("Aggregation attribute names can't contain special chars \" ,;{}()\\n\\t=\"") { + val tempDir = Utils.createTempDir() + val filePath = new File(tempDir, "testParquet").getCanonicalPath + val filePath2 = new File(tempDir, "testParquet2").getCanonicalPath + + val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str") + val df2 = df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").max("y.int") + intercept[RuntimeException](df2.saveAsParquetFile(filePath)) + + val df3 = df2.toDF("str", "max_int") + df3.saveAsParquetFile(filePath2) + val df4 = parquetFile(filePath2) + checkAnswer(df4, Row("1", 1) :: Row("2", 2) :: Row("3", 3) :: Nil) + assert(df4.columns === Array("str", "max_int")) + } } class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { @@ -736,7 +886,11 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll test(s"SPARK-5775 read struct from $table") { checkAnswer( - sql(s"SELECT p, structField.intStructField, structField.stringStructField FROM $table WHERE p = 1"), + sql( + s""" + |SELECT p, structField.intStructField, structField.stringStructField + |FROM $table WHERE p = 1 + """.stripMargin), (1 to 10).map(i => Row(1, i, f"${i}_string"))) } diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 30646ddbc29d8..33e96eaabfbf6 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -34,14 +34,14 @@ import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, ObjectInspector, PrimitiveObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.types.{UTF8String, Decimal, DecimalType} private[hive] case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { @@ -135,7 +135,7 @@ private[hive] object HiveShim { PrimitiveCategory.VOID, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) @@ -210,7 +210,7 @@ private[hive] object HiveShim { def getDataLocationPath(p: Partition) = p.getPartitionPath - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) def compatibilityBlackList = Seq( "decimal_.*", @@ -244,6 +244,12 @@ private[hive] object HiveShim { } } + def getConvertedOI( + inputOI: ObjectInspector, + outputOI: ObjectInspector): ObjectInspector = { + ObjectInspectorConverters.getConvertedOI(inputOI, outputOI, true) + } + def prepareWritable(w: Writable): Writable = { w } diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index f9fcbdae15745..dbc5e029e2047 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -17,36 +17,40 @@ package org.apache.spark.sql.hive -import java.util.{ArrayList => JArrayList} -import java.util.Properties import java.rmi.server.UID +import java.util.{Properties, ArrayList => JArrayList} +import java.io.{OutputStream, InputStream} import scala.collection.JavaConversions._ import scala.language.implicitConversions +import scala.reflect.ClassTag +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.io.Input +import com.esotericsoftware.kryo.io.Output import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{NullWritable, Writable} -import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfo, TypeInfoFactory} +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} +import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging -import org.apache.spark.sql.types.{Decimal, DecimalType} - +import org.apache.spark.sql.types.{Decimal, DecimalType, UTF8String} +import org.apache.spark.util.Utils._ /** * This class provides the UDF creation and also the UDF instance serialization and @@ -62,43 +66,34 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) // for Serialization def this() = this(null) - import java.io.{OutputStream, InputStream} - import com.esotericsoftware.kryo.Kryo - import org.apache.spark.util.Utils._ - import org.apache.hadoop.hive.ql.exec.Utilities - import org.apache.hadoop.hive.ql.exec.UDF - @transient - private val methodDeSerialize = { - val method = classOf[Utilities].getDeclaredMethod( - "deserializeObjectByKryo", - classOf[Kryo], - classOf[InputStream], - classOf[Class[_]]) - method.setAccessible(true) - - method + def deserializeObjectByKryo[T: ClassTag]( + kryo: Kryo, + in: InputStream, + clazz: Class[_]): T = { + val inp = new Input(in) + val t: T = kryo.readObject(inp,clazz).asInstanceOf[T] + inp.close() + t } @transient - private val methodSerialize = { - val method = classOf[Utilities].getDeclaredMethod( - "serializeObjectByKryo", - classOf[Kryo], - classOf[Object], - classOf[OutputStream]) - method.setAccessible(true) - - method + def serializeObjectByKryo( + kryo: Kryo, + plan: Object, + out: OutputStream ) { + val output: Output = new Output(out) + kryo.writeObject(output, plan) + output.close() } def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { - methodDeSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), is, clazz) + deserializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), is, clazz) .asInstanceOf[UDFType] } def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { - methodSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), function, out) + serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out) } private var instance: AnyRef = null @@ -223,7 +218,7 @@ private[hive] object HiveShim { TypeInfoFactory.voidTypeInfo, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) @@ -400,7 +395,11 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } - + + def getConvertedOI(inputOI: ObjectInspector, outputOI: ObjectInspector): ObjectInspector = { + ObjectInspectorConverters.getConvertedOI(inputOI, outputOI) + } + /* * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that * is needed to initialize before serialization. diff --git a/streaming/pom.xml b/streaming/pom.xml index 23a8358d45c2a..5ca55a4f680bb 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -97,34 +97,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - test-jar-on-test-compile - test-compile - - test-jar - - - - - org.apache.maven.plugins maven-shade-plugin diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java new file mode 100644 index 0000000000000..8c0fdfa9c7478 --- /dev/null +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java @@ -0,0 +1,60 @@ +/* + * 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.streaming.util; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +/** + * This abstract class represents a write ahead log (aka journal) that is used by Spark Streaming + * to save the received data (by receivers) and associated metadata to a reliable storage, so that + * they can be recovered after driver failures. See the Spark documentation for more information + * on how to plug in your own custom implementation of a write ahead log. + */ +@org.apache.spark.annotation.DeveloperApi +public abstract class WriteAheadLog { + /** + * Write the record to the log and return a record handle, which contains all the information + * necessary to read back the written record. The time is used to the index the record, + * such that it can be cleaned later. Note that implementations of this abstract class must + * ensure that the written data is durable and readable (using the record handle) by the + * time this function returns. + */ + abstract public WriteAheadLogRecordHandle write(ByteBuffer record, long time); + + /** + * Read a written record based on the given record handle. + */ + abstract public ByteBuffer read(WriteAheadLogRecordHandle handle); + + /** + * Read and return an iterator of all the records that have been written but not yet cleaned up. + */ + abstract public Iterator readAll(); + + /** + * Clean all the records that are older than the threshold time. It can wait for + * the completion of the deletion. + */ + abstract public void clean(long threshTime, boolean waitForCompletion); + + /** + * Close this log and release any resources. + */ + abstract public void close(); +} diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java new file mode 100644 index 0000000000000..02324189b7822 --- /dev/null +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java @@ -0,0 +1,30 @@ +/* + * 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.streaming.util; + +/** + * This abstract class represents a handle that refers to a record written in a + * {@link org.apache.spark.streaming.util.WriteAheadLog WriteAheadLog}. + * It must contain all the information necessary for the record to be read and returned by + * an implemenation of the WriteAheadLog class. + * + * @see org.apache.spark.streaming.util.WriteAheadLog + */ +@org.apache.spark.annotation.DeveloperApi +public abstract class WriteAheadLogRecordHandle implements java.io.Serializable { +} 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 db64e11e16304..7bfae253c3a0c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkException, SparkConf, Logging} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.MetadataCleaner +import org.apache.spark.util.{MetadataCleaner, Utils} import org.apache.spark.streaming.scheduler.JobGenerator @@ -67,17 +67,18 @@ object Checkpoint extends Logging { val REGEX = (PREFIX + """([\d]+)([\w\.]*)""").r /** Get the checkpoint file for the given checkpoint time */ - def checkpointFile(checkpointDir: String, checkpointTime: Time) = { + def checkpointFile(checkpointDir: String, checkpointTime: Time): Path = { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds) } /** Get the checkpoint backup file for the given checkpoint time */ - def checkpointBackupFile(checkpointDir: String, checkpointTime: Time) = { + def checkpointBackupFile(checkpointDir: String, checkpointTime: Time): Path = { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } /** Get checkpoint files present in the give directory, ordered by oldest-first */ - def getCheckpointFiles(checkpointDir: String, fs: FileSystem): Seq[Path] = { + def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { + def sortFunc(path1: Path, path2: Path): Boolean = { val (time1, bk1) = path1.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } val (time2, bk2) = path2.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } @@ -85,6 +86,7 @@ object Checkpoint extends Logging { } val path = new Path(checkpointDir) + val fs = fsOption.getOrElse(path.getFileSystem(new Configuration())) if (fs.exists(path)) { val statuses = fs.listStatus(path) if (statuses != null) { @@ -139,8 +141,11 @@ class CheckpointWriter( // Write checkpoint to temp file fs.delete(tempFile, true) // just in case it exists val fos = fs.create(tempFile) - fos.write(bytes) - fos.close() + Utils.tryWithSafeFinally { + fos.write(bytes) + } { + fos.close() + } // If the checkpoint file exists, back it up // If the backup exists as well, just delete it, otherwise rename will fail @@ -157,7 +162,7 @@ class CheckpointWriter( } // Delete old checkpoint files - val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs) + val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach(file => { logInfo("Deleting " + file) @@ -187,9 +192,11 @@ class CheckpointWriter( val bos = new ByteArrayOutputStream() val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) - oos.writeObject(checkpoint) - oos.close() - bos.close() + Utils.tryWithSafeFinally { + oos.writeObject(checkpoint) + } { + oos.close() + } try { executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) @@ -229,13 +236,24 @@ class CheckpointWriter( private[streaming] object CheckpointReader extends Logging { - def read(checkpointDir: String, conf: SparkConf, hadoopConf: Configuration): Option[Checkpoint] = - { + /** + * Read checkpoint files present in the given checkpoint directory. If there are no checkpoint + * files, then return None, else try to return the latest valid checkpoint object. If no + * checkpoint files could be read correctly, then return None (if ignoreReadError = true), + * or throw exception (if ignoreReadError = false). + */ + def read( + checkpointDir: String, + conf: SparkConf, + hadoopConf: Configuration, + ignoreReadError: Boolean = false): Option[Checkpoint] = { val checkpointPath = new Path(checkpointDir) - def fs = checkpointPath.getFileSystem(hadoopConf) + + // TODO(rxin): Why is this a def?! + def fs: FileSystem = checkpointPath.getFileSystem(hadoopConf) // Try to find the checkpoint files - val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse + val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).reverse if (checkpointFiles.isEmpty) { return None } @@ -246,18 +264,24 @@ object CheckpointReader extends Logging { checkpointFiles.foreach(file => { logInfo("Attempting to load checkpoint from file " + file) try { - val fis = fs.open(file) - // ObjectInputStream uses the last defined user-defined class loader in the stack - // to find classes, which maybe the wrong class loader. Hence, a inherited version - // of ObjectInputStream is used to explicitly use the current thread's default class - // loader to find and load classes. This is a well know Java issue and has popped up - // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val zis = compressionCodec.compressedInputStream(fis) - val ois = new ObjectInputStreamWithLoader(zis, - Thread.currentThread().getContextClassLoader) - val cp = ois.readObject.asInstanceOf[Checkpoint] - ois.close() - fs.close() + var ois: ObjectInputStreamWithLoader = null + var cp: Checkpoint = null + Utils.tryWithSafeFinally { + val fis = fs.open(file) + // ObjectInputStream uses the last defined user-defined class loader in the stack + // to find classes, which maybe the wrong class loader. Hence, a inherited version + // of ObjectInputStream is used to explicitly use the current thread's default class + // loader to find and load classes. This is a well know Java issue and has popped up + // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) + val zis = compressionCodec.compressedInputStream(fis) + ois = new ObjectInputStreamWithLoader(zis, + Thread.currentThread().getContextClassLoader) + cp = ois.readObject.asInstanceOf[Checkpoint] + } { + if (ois != null) { + ois.close() + } + } cp.validate() logInfo("Checkpoint successfully loaded from file " + file) logInfo("Checkpoint was generated at time " + cp.checkpointTime) @@ -269,7 +293,10 @@ object CheckpointReader extends Logging { }) // If none of checkpoint files could be read, then throw exception - throw new SparkException("Failed to read checkpoint from directory " + checkpointPath) + if (!ignoreReadError) { + throw new SparkException(s"Failed to read checkpoint from directory $checkpointPath") + } + None } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 0e285d6088ec1..175140481e5ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -100,11 +100,11 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } } - def getInputStreams() = this.synchronized { inputStreams.toArray } + def getInputStreams(): Array[InputDStream[_]] = this.synchronized { inputStreams.toArray } - def getOutputStreams() = this.synchronized { outputStreams.toArray } + def getOutputStreams(): Array[DStream[_]] = this.synchronized { outputStreams.toArray } - def getReceiverInputStreams() = this.synchronized { + def getReceiverInputStreams(): Array[ReceiverInputDStream[_]] = this.synchronized { inputStreams.filter(_.isInstanceOf[ReceiverInputDStream[_]]) .map(_.asInstanceOf[ReceiverInputDStream[_]]) .toArray diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index a0d8fb5ab93ec..3249bb348981f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -55,7 +55,6 @@ case class Duration (private val millis: Long) { def div(that: Duration): Double = this / that - def isMultipleOf(that: Duration): Boolean = (this.millis % that.millis == 0) @@ -71,7 +70,7 @@ case class Duration (private val millis: Long) { def milliseconds: Long = millis - def prettyPrint = Utils.msDurationToString(millis) + def prettyPrint: String = Utils.msDurationToString(millis) } @@ -80,7 +79,7 @@ case class Duration (private val millis: Long) { * a given number of milliseconds. */ object Milliseconds { - def apply(milliseconds: Long) = new Duration(milliseconds) + def apply(milliseconds: Long): Duration = new Duration(milliseconds) } /** @@ -88,7 +87,7 @@ object Milliseconds { * a given number of seconds. */ object Seconds { - def apply(seconds: Long) = new Duration(seconds * 1000) + def apply(seconds: Long): Duration = new Duration(seconds * 1000) } /** @@ -96,7 +95,7 @@ object Seconds { * a given number of minutes. */ object Minutes { - def apply(minutes: Long) = new Duration(minutes * 60000) + def apply(minutes: Long): Duration = new Duration(minutes * 60000) } // Java-friendlier versions of the objects above. @@ -107,16 +106,16 @@ object Durations { /** * @return [[org.apache.spark.streaming.Duration]] representing given number of milliseconds. */ - def milliseconds(milliseconds: Long) = Milliseconds(milliseconds) + def milliseconds(milliseconds: Long): Duration = Milliseconds(milliseconds) /** * @return [[org.apache.spark.streaming.Duration]] representing given number of seconds. */ - def seconds(seconds: Long) = Seconds(seconds) + def seconds(seconds: Long): Duration = Seconds(seconds) /** * @return [[org.apache.spark.streaming.Duration]] representing given number of minutes. */ - def minutes(minutes: Long) = Minutes(minutes) + def minutes(minutes: Long): Duration = Minutes(minutes) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index ad4f3fdd14ad6..3f5be785e1b1a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -39,18 +39,18 @@ class Interval(val beginTime: Time, val endTime: Time) { this.endTime < that.endTime } - def <= (that: Interval) = (this < that || this == that) + def <= (that: Interval): Boolean = (this < that || this == that) - def > (that: Interval) = !(this <= that) + def > (that: Interval): Boolean = !(this <= that) - def >= (that: Interval) = !(this < that) + def >= (that: Interval): Boolean = !(this < that) - override def toString = "[" + beginTime + ", " + endTime + "]" + override def toString: String = "[" + beginTime + ", " + endTime + "]" } private[streaming] object Interval { - def currentInterval(duration: Duration): Interval = { + def currentInterval(duration: Duration): Interval = { val time = new Time(System.currentTimeMillis) val intervalBegin = time.floor(duration) new Interval(intervalBegin, intervalBegin + duration) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 543224d4b07bc..b1ad0d42ec8a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -107,6 +107,19 @@ class StreamingContext private[streaming] ( */ def this(path: String) = this(path, new Configuration) + /** + * Recreate a StreamingContext from a checkpoint file using an existing SparkContext. + * @param path Path to the directory that was specified as the checkpoint directory + * @param sparkContext Existing SparkContext + */ + def this(path: String, sparkContext: SparkContext) = { + this( + sparkContext, + CheckpointReader.read(path, sparkContext.conf, sparkContext.hadoopConfiguration).get, + null) + } + + if (sc_ == null && cp_ == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") @@ -115,10 +128,12 @@ class StreamingContext private[streaming] ( private[streaming] val isCheckpointPresent = (cp_ != null) private[streaming] val sc: SparkContext = { - if (isCheckpointPresent) { + if (sc_ != null) { + sc_ + } else if (isCheckpointPresent) { new SparkContext(cp_.createSparkConf()) } else { - sc_ + throw new SparkException("Cannot create StreamingContext without a SparkContext") } } @@ -129,7 +144,7 @@ class StreamingContext private[streaming] ( private[streaming] val conf = sc.conf - private[streaming] val env = SparkEnv.get + private[streaming] val env = sc.env private[streaming] val graph: DStreamGraph = { if (isCheckpointPresent) { @@ -144,7 +159,7 @@ class StreamingContext private[streaming] ( } } - private val nextReceiverInputStreamId = new AtomicInteger(0) + private val nextInputStreamId = new AtomicInteger(0) private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { @@ -174,7 +189,9 @@ class StreamingContext private[streaming] ( /** Register streaming source to metrics system */ private val streamingSource = new StreamingSource(this) - SparkEnv.get.metricsSystem.registerSource(streamingSource) + assert(env != null) + assert(env.metricsSystem != null) + env.metricsSystem.registerSource(streamingSource) /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { @@ -188,7 +205,7 @@ class StreamingContext private[streaming] ( /** * Return the associated Spark context */ - def sparkContext = sc + def sparkContext: SparkContext = sc /** * Set each DStreams in this context to remember RDDs it generated in the last given duration. @@ -224,7 +241,7 @@ class StreamingContext private[streaming] ( if (isCheckpointPresent) cp_ else null } - private[streaming] def getNewReceiverStreamId() = nextReceiverInputStreamId.getAndIncrement() + private[streaming] def getNewInputStreamId() = nextInputStreamId.getAndIncrement() /** * Create an input stream with any arbitrary user implemented receiver. @@ -510,6 +527,7 @@ class StreamingContext private[streaming] ( validate() sparkContext.setCallSite(DStream.getCreationSite()) scheduler.start() + uiTab.foreach(_.attach()) state = Started } @@ -596,7 +614,8 @@ object StreamingContext extends Logging { @deprecated("Replaced by implicit functions in the DStream companion object. This is " + "kept here only for backward compatibility.", "1.3.0") def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) + : PairDStreamFunctions[K, V] = { DStream.toPairDStreamFunctions(stream)(kt, vt, ord) } @@ -620,19 +639,59 @@ object StreamingContext extends Logging { hadoopConf: Configuration = new Configuration(), createOnError: Boolean = false ): StreamingContext = { - val checkpointOption = try { - CheckpointReader.read(checkpointPath, new SparkConf(), hadoopConf) - } catch { - case e: Exception => - if (createOnError) { - None - } else { - throw e - } - } + val checkpointOption = CheckpointReader.read( + checkpointPath, new SparkConf(), hadoopConf, createOnError) checkpointOption.map(new StreamingContext(null, _, null)).getOrElse(creatingFunc()) } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc` on the provided `sparkContext`. Note + * that the SparkConf configuration in the checkpoint data will not be restored as the + * SparkContext has already been created. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new StreamingContext using the given SparkContext + * @param sparkContext SparkContext using which the StreamingContext will be created + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: SparkContext => StreamingContext, + sparkContext: SparkContext + ): StreamingContext = { + getOrCreate(checkpointPath, creatingFunc, sparkContext, createOnError = false) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the StreamingContext + * will be created by called the provided `creatingFunc` on the provided `sparkContext`. Note + * that the SparkConf configuration in the checkpoint data will not be restored as the + * SparkContext has already been created. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new StreamingContext using the given SparkContext + * @param sparkContext SparkContext using which the StreamingContext will be created + * @param createOnError Whether to create a new StreamingContext if there is an + * error in reading checkpoint data. By default, an exception will be + * thrown on error. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: SparkContext => StreamingContext, + sparkContext: SparkContext, + createOnError: Boolean + ): StreamingContext = { + val checkpointOption = CheckpointReader.read( + checkpointPath, sparkContext.conf, sparkContext.hadoopConfiguration, createOnError) + checkpointOption.map(new StreamingContext(sparkContext, _, null)) + .getOrElse(creatingFunc(sparkContext)) + } + /** * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. 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 2eabdd9387913..808dcc174cf9a 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 @@ -169,7 +169,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala new JavaDStream(dstream.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -179,7 +179,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.call(x).asScala + def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala def cm: ClassTag[(K2, V2)] = fakeClassTag new JavaPairDStream(dstream.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -190,7 +190,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of the RDD. */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[U] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaDStream(dstream.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -201,7 +203,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) : JavaPairDStream[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + def fn: (Iterator[T]) => Iterator[(K2, V2)] = { + (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + } new JavaPairDStream(dstream.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -415,8 +419,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T implicit val cmv2: ClassTag[V2] = fakeClassTag implicit val cmw: ClassTag[W] = fakeClassTag - def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] = + def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] = { transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd + } dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _)) } 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 f94f2d0e8bd31..93baad19e3ee1 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 @@ -526,7 +526,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = { import scala.collection.JavaConverters._ - def fn = (x: V) => f.apply(x).asScala + def fn: (V) => Iterable[U] = (x: V) => f.apply(x).asScala implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.flatMapValues(fn) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index e3db01c1e12c6..572d7d8e8753d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -32,13 +32,14 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.function.{Function0 => JFunction0} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.scheduler.StreamingListener -import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.dstream.{PluggableInputDStream, ReceiverInputDStream, DStream} +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver +import org.apache.hadoop.conf.Configuration /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main @@ -192,7 +193,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) : JavaReceiverInputDStream[T] = { - def fn = (x: InputStream) => converter.call(x).toIterator + def fn: (InputStream) => Iterator[T] = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.socketStream(hostname, port, fn, storageLevel) @@ -313,7 +314,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cmk: ClassTag[K] = ClassTag(kClass) implicit val cmv: ClassTag[V] = ClassTag(vClass) implicit val cmf: ClassTag[F] = ClassTag(fClass) - def fn = (x: Path) => filter.call(x).booleanValue() + def fn: (Path) => Boolean = (x: Path) => filter.call(x).booleanValue() ssc.fileStream[K, V, F](directory, fn, newFilesOnly) } @@ -344,7 +345,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cmk: ClassTag[K] = ClassTag(kClass) implicit val cmv: ClassTag[V] = ClassTag(vClass) implicit val cmf: ClassTag[F] = ClassTag(fClass) - def fn = (x: Path) => filter.call(x).booleanValue() + def fn: (Path) => Boolean = (x: Path) => filter.call(x).booleanValue() ssc.fileStream[K, V, F](directory, fn, newFilesOnly, conf) } @@ -625,7 +626,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean): Unit = ssc.stop(stopSparkContext) /** * Stop the execution of the streams. @@ -633,7 +634,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @param stopGracefully Stop gracefully by waiting for the processing of all * received data to be completed */ - def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { + def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = { ssc.stop(stopSparkContext, stopGracefully) } @@ -655,6 +656,7 @@ object JavaStreamingContext { * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext */ + @deprecated("use getOrCreate without JavaStreamingContextFactor", "1.4.0") def getOrCreate( checkpointPath: String, factory: JavaStreamingContextFactory @@ -676,6 +678,7 @@ object JavaStreamingContext { * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible * file system */ + @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") def getOrCreate( checkpointPath: String, hadoopConf: Configuration, @@ -700,6 +703,7 @@ object JavaStreamingContext { * @param createOnError Whether to create a new JavaStreamingContext if there is an * error in reading checkpoint data. */ + @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") def getOrCreate( checkpointPath: String, hadoopConf: Configuration, @@ -712,6 +716,117 @@ object JavaStreamingContext { new JavaStreamingContext(ssc) } + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext] + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext], + hadoopConf: Configuration + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }, hadoopConf) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible + * file system + * @param createOnError Whether to create a new JavaStreamingContext if there is an + * error in reading checkpoint data. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction0[JavaStreamingContext], + hadoopConf: Configuration, + createOnError: Boolean + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, () => { + creatingFunc.call().ssc + }, hadoopConf, createOnError) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param sparkContext SparkContext using which the StreamingContext will be created + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction[JavaSparkContext, JavaStreamingContext], + sparkContext: JavaSparkContext + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, (sparkContext: SparkContext) => { + creatingFunc.call(new JavaSparkContext(sparkContext)).ssc + }, sparkContext.sc) + new JavaStreamingContext(ssc) + } + + /** + * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + * recreated from the checkpoint data. If the data does not exist, then the provided factory + * will be used to create a JavaStreamingContext. + * + * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program + * @param creatingFunc Function to create a new JavaStreamingContext + * @param sparkContext SparkContext using which the StreamingContext will be created + * @param createOnError Whether to create a new JavaStreamingContext if there is an + * error in reading checkpoint data. + */ + def getOrCreate( + checkpointPath: String, + creatingFunc: JFunction[JavaSparkContext, JavaStreamingContext], + sparkContext: JavaSparkContext, + createOnError: Boolean + ): JavaStreamingContext = { + val ssc = StreamingContext.getOrCreate(checkpointPath, (sparkContext: SparkContext) => { + creatingFunc.call(new JavaSparkContext(sparkContext)).ssc + }, sparkContext.sc, createOnError) + new JavaStreamingContext(ssc) + } + /** * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index 7053f47ec69a2..4c28654ef6413 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -176,11 +176,11 @@ private[python] abstract class PythonDStream( val func = new TransformFunction(pfunc) - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration - val asJavaDStream = JavaDStream.fromDStream(this) + val asJavaDStream: JavaDStream[Array[Byte]] = JavaDStream.fromDStream(this) } /** @@ -212,7 +212,7 @@ private[python] class PythonTransformed2DStream( val func = new TransformFunction(pfunc) - override def dependencies = List(parent, parent2) + override def dependencies: List[DStream[_]] = List(parent, parent2) override def slideDuration: Duration = parent.slideDuration @@ -223,7 +223,7 @@ private[python] class PythonTransformed2DStream( func(Some(rdd1), Some(rdd2), validTime) } - val asJavaDStream = JavaDStream.fromDStream(this) + val asJavaDStream: JavaDStream[Array[Byte]] = JavaDStream.fromDStream(this) } /** @@ -260,12 +260,15 @@ private[python] class PythonReducedWindowedDStream( extends PythonDStream(parent, preduceFunc) { super.persist(StorageLevel.MEMORY_ONLY) - override val mustCheckpoint = true - val invReduceFunc = new TransformFunction(pinvReduceFunc) + override val mustCheckpoint: Boolean = true + + val invReduceFunc: TransformFunction = new TransformFunction(pinvReduceFunc) def windowDuration: Duration = _windowDuration + override def slideDuration: Duration = _slideDuration + override def parentRememberDuration: Duration = rememberDuration + windowDuration override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { 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 b874f561c12eb..f1f8a70655996 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 @@ -104,7 +104,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def parentRememberDuration = rememberDuration /** Return the StreamingContext associated with this DStream */ - def context = ssc + def context: StreamingContext = ssc /* Set the creation call site */ private[streaming] val creationSite = DStream.getCreationSite() @@ -553,7 +553,8 @@ abstract class DStream[T: ClassTag] ( // because the DStream is reachable from the outer object here, and because // DStreams can't be serialized with closures, we can't proactively check // it for serializability and so we pass the optional false to SparkContext.clean - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) + val cleanedF = context.sparkContext.clean(transformFunc, false) + transform((r: RDD[T], t: Time) => cleanedF(r)) } /** @@ -619,14 +620,16 @@ abstract class DStream[T: ClassTag] ( * operator, so this DStream will be registered as an output stream and there materialized. */ def print(num: Int) { - def foreachFunc = (rdd: RDD[T], time: Time) => { - val firstNum = rdd.take(num + 1) - println ("-------------------------------------------") - println ("Time: " + time) - println ("-------------------------------------------") - firstNum.take(num).foreach(println) - if (firstNum.size > num) println("...") - println() + def foreachFunc: (RDD[T], Time) => Unit = { + (rdd: RDD[T], time: Time) => { + val firstNum = rdd.take(num + 1) + println("-------------------------------------------") + println("Time: " + time) + println("-------------------------------------------") + firstNum.take(num).foreach(println) + if (firstNum.length > num) println("...") + println() + } } new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() } @@ -837,7 +840,7 @@ object DStream { /** Filtering function that excludes non-user classes for a streaming application */ def streamingExclustionFunction(className: String): Boolean = { - def doesMatch(r: Regex) = r.findFirstIn(className).isDefined + def doesMatch(r: Regex): Boolean = r.findFirstIn(className).isDefined val isSparkClass = doesMatch(SPARK_CLASS_REGEX) val isSparkExampleClass = doesMatch(SPARK_EXAMPLES_CLASS_REGEX) val isSparkStreamingTestClass = doesMatch(SPARK_STREAMING_TESTCLASS_REGEX) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 0dc72790fbdbd..39fd21342813e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -114,7 +114,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } } - override def toString() = { + override def toString: String = { "[\n" + currentCheckpointFiles.size + " checkpoint files \n" + currentCheckpointFiles.mkString("\n") + "\n]" } 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 22de8c02e63c8..eca69f00188e4 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 @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkConf, SerializableWritable} import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.streaming._ import org.apache.spark.util.{TimeStampedHashMap, Utils} @@ -63,7 +63,7 @@ import org.apache.spark.util.{TimeStampedHashMap, Utils} * the streaming app. * - If a file is to be visible in the directory listings, it must be visible within a certain * duration of the mod time of the file. This duration is the "remember window", which is set to - * 1 minute (see `FileInputDStream.MIN_REMEMBER_DURATION`). Otherwise, the file will never be + * 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be * selected as the mod time will be less than the ignore threshold when it becomes visible. * - Once a file is visible, the mod time cannot change. If it does due to appends, then the * processing semantics are undefined. @@ -80,6 +80,15 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private val serializableConfOpt = conf.map(new SerializableWritable(_)) + /** + * Minimum duration of remembering the information of selected files. Defaults to 60 seconds. + * + * Files with mod times older than this "window" of remembering will be ignored. So if new + * files are visible within this window, then the file will get selected in the next batch. + */ + private val minRememberDurationS = + Seconds(ssc.conf.getTimeAsSeconds("spark.streaming.minRememberDuration", "60s")) + // This is a def so that it works during checkpoint recovery: private def clock = ssc.scheduler.clock @@ -95,7 +104,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( * This would allow us to filter away not-too-old files which have already been recently * selected and processed. */ - private val numBatchesToRemember = FileInputDStream.calculateNumBatchesToRemember(slideDuration) + private val numBatchesToRemember = FileInputDStream + .calculateNumBatchesToRemember(slideDuration, minRememberDurationS) private val durationToRemember = slideDuration * numBatchesToRemember remember(durationToRemember) @@ -298,7 +308,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] + private def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] override def update(time: Time) { hadoopFiles.clear() @@ -320,7 +330,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( } } - override def toString() = { + override def toString: String = { "[\n" + hadoopFiles.size + " file sets\n" + hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } @@ -330,20 +340,14 @@ class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( private[streaming] object FileInputDStream { - /** - * Minimum duration of remembering the information of selected files. Files with mod times - * older than this "window" of remembering will be ignored. So if new files are visible - * within this window, then the file will get selected in the next batch. - */ - private val MIN_REMEMBER_DURATION = Minutes(1) - def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") /** * Calculate the number of last batches to remember, such that all the files selected in - * at least last MIN_REMEMBER_DURATION duration can be remembered. + * at least last minRememberDurationS duration can be remembered. */ - def calculateNumBatchesToRemember(batchDuration: Duration): Int = { - math.ceil(MIN_REMEMBER_DURATION.milliseconds.toDouble / batchDuration.milliseconds).toInt + def calculateNumBatchesToRemember(batchDuration: Duration, + minRememberDurationS: Duration): Int = { + math.ceil(minRememberDurationS.milliseconds.toDouble / batchDuration.milliseconds).toInt } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index c81534ae584ea..fcd5216f101af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -27,7 +27,7 @@ class FilteredDStream[T: ClassTag]( filterFunc: T => Boolean ) extends DStream[T](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 658623455498c..9d09a3baf37ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -28,7 +28,7 @@ class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( flatMapValueFunc: V => TraversableOnce[U] ) extends DStream[(K, U)](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index c7bb2833eabb8..475ea2d2d4f38 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -27,7 +27,7 @@ class FlatMappedDStream[T: ClassTag, U: ClassTag]( flatMapFunc: T => Traversable[U] ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 1361c30395b57..685a32e1d280d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -28,7 +28,7 @@ class ForEachDStream[T: ClassTag] ( foreachFunc: (RDD[T], Time) => Unit ) extends DStream[Unit](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index a9bb51f054048..dbb295fe54f71 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -25,7 +25,7 @@ private[streaming] class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index aa1993f0580a8..e4ad4b509d8d8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -41,6 +41,9 @@ abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext) ssc.graph.addInputStream(this) + /** This is an unique identifier for the input stream. */ + val id = ssc.getNewInputStreamId() + /** * Checks whether the 'time' is valid wrt slideDuration for generating RDD. * Additionally it also ensures valid times are in strictly increasing order. @@ -61,7 +64,7 @@ abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext) } } - override def dependencies = List() + override def dependencies: List[DStream[_]] = List() override def slideDuration: Duration = { if (ssc == null) throw new Exception("ssc is null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 3d8ee29df1e82..5994bc1e23f2b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -28,7 +28,7 @@ class MapPartitionedDStream[T: ClassTag, U: ClassTag]( preservePartitioning: Boolean ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 7aea1f945d9db..954d2eb4a7b00 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -28,7 +28,7 @@ class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( mapValueFunc: V => U ) extends DStream[(K, U)](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index 02704a8d1c2e0..fa14b2e897c3e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -27,7 +27,7 @@ class MappedDStream[T: ClassTag, U: ClassTag] ( mapFunc: T => U ) extends DStream[U](parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 8be04314c4285..15d9710d37cd4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -20,11 +20,11 @@ package org.apache.spark.streaming.dstream import scala.reflect.ClassTag import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.BlockId import org.apache.spark.streaming._ import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD -import org.apache.spark.streaming.receiver.{Receiver, WriteAheadLogBasedStoreResult} -import org.apache.spark.streaming.scheduler.ReceivedBlockInfo +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.util.WriteAheadLogUtils /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -39,9 +39,6 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - /** This is an unique identifier for the receiver input stream. */ - val id = ssc.getNewReceiverStreamId() - /** * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation @@ -67,27 +64,30 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont } else { // Otherwise, ask the tracker for all the blocks that have been allocated to this stream // for this batch - val blockInfos = - ssc.scheduler.receiverTracker.getBlocksOfBatch(validTime).get(id).getOrElse(Seq.empty) - val blockStoreResults = blockInfos.map { _.blockStoreResult } - val blockIds = blockStoreResults.map { _.blockId.asInstanceOf[BlockId] }.toArray + val receiverTracker = ssc.scheduler.receiverTracker + val blockInfos = receiverTracker.getBlocksOfBatch(validTime).getOrElse(id, Seq.empty) + val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray - // Check whether all the results are of the same type - val resultTypes = blockStoreResults.map { _.getClass }.distinct - if (resultTypes.size > 1) { - logWarning("Multiple result types in block information, WAL information will be ignored.") - } + // Are WAL record handles present with all the blocks + val areWALRecordHandlesPresent = blockInfos.forall { _.walRecordHandleOption.nonEmpty } - // If all the results are of type WriteAheadLogBasedStoreResult, then create - // WriteAheadLogBackedBlockRDD else create simple BlockRDD. - if (resultTypes.size == 1 && resultTypes.head == classOf[WriteAheadLogBasedStoreResult]) { - val logSegments = blockStoreResults.map { - _.asInstanceOf[WriteAheadLogBasedStoreResult].segment - }.toArray - // Since storeInBlockManager = false, the storage level does not matter. - new WriteAheadLogBackedBlockRDD[T](ssc.sparkContext, - blockIds, logSegments, storeInBlockManager = false, StorageLevel.MEMORY_ONLY_SER) + if (areWALRecordHandlesPresent) { + // If all the blocks have WAL record handle, then create a WALBackedBlockRDD + val isBlockIdValid = blockInfos.map { _.isBlockIdValid() }.toArray + val walRecordHandles = blockInfos.map { _.walRecordHandleOption.get }.toArray + new WriteAheadLogBackedBlockRDD[T]( + ssc.sparkContext, blockIds, walRecordHandles, isBlockIdValid) } else { + // Else, create a BlockRDD. However, if there are some blocks with WAL info but not others + // then that is unexpected and log a warning accordingly. + if (blockInfos.find(_.walRecordHandleOption.nonEmpty).nonEmpty) { + if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { + logError("Some blocks do not have Write Ahead Log information; " + + "this is unexpected and data may not be recoverable after driver failures") + } else { + logWarning("Some blocks have Write Ahead Log information; this is unexpected") + } + } new BlockRDD[T](ssc.sc, blockIds) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index c0a5af0b65cc3..1385ccbf56ee5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -52,7 +52,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( // Reduce each batch of data using reduceByKey which will be further reduced by window // by ReducedWindowedDStream - val reducedStream = parent.reduceByKey(reduceFunc, partitioner) + private val reducedStream = parent.reduceByKey(reduceFunc, partitioner) // Persist RDDs to memory by default as these RDDs are going to be reused. super.persist(StorageLevel.MEMORY_ONLY_SER) @@ -60,7 +60,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( def windowDuration: Duration = _windowDuration - override def dependencies = List(reducedStream) + override def dependencies: List[DStream[_]] = List(reducedStream) override def slideDuration: Duration = _slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 880a89bc36895..7757ccac09a58 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -33,7 +33,7 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( mapSideCombine: Boolean = true ) extends DStream[(K,C)] (parent.ssc) { - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index ebb04dd35b9a2..de8718d0a80fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -36,7 +36,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( super.persist(StorageLevel.MEMORY_ONLY_SER) - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = parent.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 71b61856e23c0..5d46ca0715ffd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -32,7 +32,7 @@ class TransformedDStream[U: ClassTag] ( require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") - override def dependencies = parents.toList + override def dependencies: List[DStream[_]] = parents.toList override def slideDuration: Duration = parents.head.slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index abbc40befa95b..9405dbaa12329 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -33,17 +33,17 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") - override def dependencies = parents.toList + override def dependencies: List[DStream[_]] = parents.toList override def slideDuration: Duration = parents.head.slideDuration override def compute(validTime: Time): Option[RDD[T]] = { val rdds = new ArrayBuffer[RDD[T]]() - parents.map(_.getOrCompute(validTime)).foreach(_ match { + parents.map(_.getOrCompute(validTime)).foreach { case Some(rdd) => rdds += rdd case None => throw new Exception("Could not generate RDD from a parent for unifying at time " + validTime) - }) + } if (rdds.size > 0) { Some(new UnionRDD(ssc.sc, rdds)) } else { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 775b6bfd065c0..899865a906c27 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -46,7 +46,7 @@ class WindowedDStream[T: ClassTag]( def windowDuration: Duration = _windowDuration - override def dependencies = List(parent) + override def dependencies: List[DStream[_]] = List(parent) override def slideDuration: Duration = _slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index dd1e96334952f..ffce6a4c3c74c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -16,41 +16,59 @@ */ package org.apache.spark.streaming.rdd +import java.io.File +import java.nio.ByteBuffer +import java.util.UUID + import scala.reflect.ClassTag +import scala.util.control.NonFatal -import org.apache.hadoop.conf.Configuration +import org.apache.commons.io.FileUtils import org.apache.spark._ import org.apache.spark.rdd.BlockRDD import org.apache.spark.storage.{BlockId, StorageLevel} -import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader} +import org.apache.spark.streaming.util._ /** * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. * It contains information about the id of the blocks having this partition's data and - * the segment of the write ahead log that backs the partition. + * the corresponding record handle in the write ahead log that backs the partition. * @param index index of the partition * @param blockId id of the block having the partition data - * @param segment segment of the write ahead log having the partition data + * @param isBlockIdValid Whether the block Ids are valid (i.e., the blocks are present in the Spark + * executors). If not, then block lookups by the block ids will be skipped. + * By default, this is an empty array signifying true for all the blocks. + * @param walRecordHandle Handle of the record in a write ahead log having the partition data */ private[streaming] class WriteAheadLogBackedBlockRDDPartition( val index: Int, val blockId: BlockId, - val segment: WriteAheadLogFileSegment) - extends Partition + val isBlockIdValid: Boolean, + val walRecordHandle: WriteAheadLogRecordHandle + ) extends Partition /** * This class represents a special case of the BlockRDD where the data blocks in - * the block manager are also backed by segments in write ahead logs. For reading + * the block manager are also backed by data in write ahead logs. For reading * the data, this RDD first looks up the blocks by their ids in the block manager. - * If it does not find them, it looks up the corresponding file segment. + * If it does not find them, it looks up the WAL using the corresponding record handle. + * The lookup of the blocks from the block manager can be skipped by setting the corresponding + * element in isBlockIdValid to false. This is a performance optimization which does not affect + * correctness, and it can be used in situations where it is known that the block + * does not exist in the Spark executors (e.g. after a failed driver is restarted). + * * * @param sc SparkContext * @param blockIds Ids of the blocks that contains this RDD's data - * @param segments Segments in write ahead logs that contain this RDD's data - * @param storeInBlockManager Whether to store in the block manager after reading from the segment + * @param walRecordHandles Record handles in write ahead logs that contain this RDD's data + * @param isBlockIdValid Whether the block Ids are valid (i.e., the blocks are present in the Spark + * executors). If not, then block lookups by the block ids will be skipped. + * By default, this is an empty array signifying true for all the blocks. + * @param storeInBlockManager Whether to store a block in the block manager + * after reading it from the WAL * @param storageLevel storage level to store when storing in block manager * (applicable when storeInBlockManager = true) */ @@ -58,30 +76,39 @@ private[streaming] class WriteAheadLogBackedBlockRDD[T: ClassTag]( @transient sc: SparkContext, @transient blockIds: Array[BlockId], - @transient segments: Array[WriteAheadLogFileSegment], - storeInBlockManager: Boolean, - storageLevel: StorageLevel) + @transient walRecordHandles: Array[WriteAheadLogRecordHandle], + @transient isBlockIdValid: Array[Boolean] = Array.empty, + storeInBlockManager: Boolean = false, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER) extends BlockRDD[T](sc, blockIds) { require( - blockIds.length == segments.length, - s"Number of block ids (${blockIds.length}) must be " + - s"the same as number of segments (${segments.length}})!") + blockIds.length == walRecordHandles.length, + s"Number of block Ids (${blockIds.length}) must be " + + s" same as number of WAL record handles (${walRecordHandles.length}})") + + require( + isBlockIdValid.isEmpty || isBlockIdValid.length == blockIds.length, + s"Number of elements in isBlockIdValid (${isBlockIdValid.length}) must be " + + s" same as number of block Ids (${blockIds.length})") // Hadoop configuration is not serializable, so broadcast it as a serializable. @transient private val hadoopConfig = sc.hadoopConfiguration private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig) + override def isValid(): Boolean = true + override def getPartitions: Array[Partition] = { assertValid() - Array.tabulate(blockIds.size) { i => - new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) + Array.tabulate(blockIds.length) { i => + val isValid = if (isBlockIdValid.length == 0) true else isBlockIdValid(i) + new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), isValid, walRecordHandles(i)) } } /** * Gets the partition data by getting the corresponding block from the block manager. - * If the block does not exist, then the data is read from the corresponding segment + * If the block does not exist, then the data is read from the corresponding record * in write ahead log files. */ override def compute(split: Partition, context: TaskContext): Iterator[T] = { @@ -90,35 +117,87 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( val blockManager = SparkEnv.get.blockManager val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] val blockId = partition.blockId - blockManager.get(blockId) match { - case Some(block) => // Data is in Block Manager - val iterator = block.data.asInstanceOf[Iterator[T]] - logDebug(s"Read partition data of $this from block manager, block $blockId") - iterator - case None => // Data not found in Block Manager, grab it from write ahead log file - val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf) - val dataRead = reader.read(partition.segment) - reader.close() - logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}") - if (storeInBlockManager) { - blockManager.putBytes(blockId, dataRead, storageLevel) - logDebug(s"Stored partition data of $this into block manager with level $storageLevel") - dataRead.rewind() + + def getBlockFromBlockManager(): Option[Iterator[T]] = { + blockManager.get(blockId).map(_.data.asInstanceOf[Iterator[T]]) + } + + def getBlockFromWriteAheadLog(): Iterator[T] = { + var dataRead: ByteBuffer = null + var writeAheadLog: WriteAheadLog = null + try { + // The WriteAheadLogUtils.createLog*** method needs a directory to create a + // WriteAheadLog object as the default FileBasedWriteAheadLog needs a directory for + // writing log data. However, the directory is not needed if data needs to be read, hence + // a dummy path is provided to satisfy the method parameter requirements. + // FileBasedWriteAheadLog will not create any file or directory at that path. + // FileBasedWriteAheadLog will not create any file or directory at that path. Also, + // this dummy directory should not already exist otherwise the WAL will try to recover + // past events from the directory and throw errors. + val nonExistentDirectory = new File( + System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath + writeAheadLog = WriteAheadLogUtils.createLogForReceiver( + SparkEnv.get.conf, nonExistentDirectory, hadoopConf) + dataRead = writeAheadLog.read(partition.walRecordHandle) + } catch { + case NonFatal(e) => + throw new SparkException( + s"Could not read data from write ahead log record ${partition.walRecordHandle}", e) + } finally { + if (writeAheadLog != null) { + writeAheadLog.close() + writeAheadLog = null } - blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + } + if (dataRead == null) { + throw new SparkException( + s"Could not read data from write ahead log record ${partition.walRecordHandle}, " + + s"read returned null") + } + logInfo(s"Read partition data of $this from write ahead log, record handle " + + partition.walRecordHandle) + if (storeInBlockManager) { + blockManager.putBytes(blockId, dataRead, storageLevel) + logDebug(s"Stored partition data of $this into block manager with level $storageLevel") + dataRead.rewind() + } + blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + } + + if (partition.isBlockIdValid) { + getBlockFromBlockManager().getOrElse { getBlockFromWriteAheadLog() } + } else { + getBlockFromWriteAheadLog() } } /** * Get the preferred location of the partition. This returns the locations of the block - * if it is present in the block manager, else it returns the location of the - * corresponding segment in HDFS. + * if it is present in the block manager, else if FileBasedWriteAheadLogSegment is used, + * it returns the location of the corresponding file segment in HDFS . */ override def getPreferredLocations(split: Partition): Seq[String] = { val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] - val blockLocations = getBlockIdLocations().get(partition.blockId) - def segmentLocations = HdfsUtils.getFileSegmentLocations( - partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig) - blockLocations.getOrElse(segmentLocations) + val blockLocations = if (partition.isBlockIdValid) { + getBlockIdLocations().get(partition.blockId) + } else { + None + } + + blockLocations.getOrElse { + partition.walRecordHandle match { + case fileSegment: FileBasedWriteAheadLogSegment => + try { + HdfsUtils.getFileSegmentLocations( + fileSegment.path, fileSegment.offset, fileSegment.length, hadoopConfig) + } catch { + case NonFatal(e) => + logError("Error getting WAL file segment locations", e) + Seq.empty + } + case _ => + Seq.empty + } + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index a7d63bd4f2dbf..cd309788a7717 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.receiver +import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration._ @@ -25,10 +26,10 @@ import scala.reflect.ClassTag import akka.actor._ import akka.actor.SupervisorStrategy.{Escalate, Restart} + import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.StorageLevel -import java.nio.ByteBuffer import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -149,13 +150,13 @@ private[streaming] class ActorReceiver[T: ClassTag]( class Supervisor extends Actor { override val supervisorStrategy = receiverSupervisorStrategy - val worker = context.actorOf(props, name) + private val worker = context.actorOf(props, name) logInfo("Started receiver worker at:" + worker.path) - val n: AtomicInteger = new AtomicInteger(0) - val hiccups: AtomicInteger = new AtomicInteger(0) + private val n: AtomicInteger = new AtomicInteger(0) + private val hiccups: AtomicInteger = new AtomicInteger(0) - def receive = { + override def receive: PartialFunction[Any, Unit] = { case IteratorData(iterator) => logDebug("received iterator") @@ -189,13 +190,12 @@ private[streaming] class ActorReceiver[T: ClassTag]( } } - def onStart() = { + def onStart(): Unit = { supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) - } - def onStop() = { + def onStop(): Unit = { supervisor ! PoisonPill } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index ee5e639b26d91..4bebcc5aa7ca0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.SystemClock +import org.apache.spark.util.{SystemClock, Utils} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -79,9 +79,9 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms") private val blockIntervalTimer = - new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } @@ -120,19 +120,33 @@ private[streaming] class BlockGenerator( * `BlockGeneratorListener.onAddData` callback will be called. All received data items * will be periodically pushed into BlockManager. */ - def addDataWithCallback(data: Any, metadata: Any) = synchronized { + def addDataWithCallback(data: Any, metadata: Any): Unit = synchronized { waitToPush() currentBuffer += data listener.onAddData(data, metadata) } + /** + * Push multiple data items into the buffer. After buffering the data, the + * `BlockGeneratorListener.onAddData` callback will be called. All received data items + * will be periodically pushed into BlockManager. Note that all the data items is guaranteed + * to be present in a single block. + */ + def addMultipleDataWithCallback(dataIterator: Iterator[Any], metadata: Any): Unit = synchronized { + dataIterator.foreach { data => + waitToPush() + currentBuffer += data + } + listener.onAddData(dataIterator, metadata) + } + /** Change the buffer to which single records are added to. */ private def updateCurrentBuffer(time: Long): Unit = synchronized { try { val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(receiverId, time - blockInterval) + val blockId = StreamBlockId(receiverId, time - blockIntervalMs) val newBlock = new Block(blockId, newBlockBuffer) listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala index e4f6ba626ebbf..97db9ded83367 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.receiver import org.apache.spark.{Logging, SparkConf} -import java.util.concurrent.TimeUnit._ +import com.google.common.util.concurrent.{RateLimiter=>GuavaRateLimiter} /** Provides waitToPush() method to limit the rate at which receivers consume data. * @@ -33,37 +33,12 @@ import java.util.concurrent.TimeUnit._ */ private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { - private var lastSyncTime = System.nanoTime - private var messagesWrittenSinceSync = 0L private val desiredRate = conf.getInt("spark.streaming.receiver.maxRate", 0) - private val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS) + private lazy val rateLimiter = GuavaRateLimiter.create(desiredRate) def waitToPush() { - if( desiredRate <= 0 ) { - return - } - val now = System.nanoTime - val elapsedNanosecs = math.max(now - lastSyncTime, 1) - val rate = messagesWrittenSinceSync.toDouble * 1000000000 / elapsedNanosecs - if (rate < desiredRate) { - // It's okay to write; just update some variables and return - messagesWrittenSinceSync += 1 - if (now > lastSyncTime + SYNC_INTERVAL) { - // Sync interval has passed; let's resync - lastSyncTime = now - messagesWrittenSinceSync = 1 - } - } else { - // Calculate how much time we should sleep to bring ourselves to the desired rate. - val targetTimeInMillis = messagesWrittenSinceSync * 1000 / desiredRate - val elapsedTimeInMillis = elapsedNanosecs / 1000000 - val sleepTimeInMillis = targetTimeInMillis - elapsedTimeInMillis - if (sleepTimeInMillis > 0) { - logTrace("Natural rate is " + rate + " per second but desired rate is " + - desiredRate + ", sleeping for " + sleepTimeInMillis + " ms to compensate.") - Thread.sleep(sleepTimeInMillis) - } - waitToPush() + if (desiredRate > 0) { + rateLimiter.acquire() } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index dcdc27d29c270..651b534ac1900 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -17,18 +17,18 @@ package org.apache.spark.streaming.receiver -import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future} import scala.language.{existentials, postfixOps} -import WriteAheadLogBasedBlockHandler._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage._ -import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogManager} -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler._ +import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} +import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} +import org.apache.spark.{Logging, SparkConf, SparkException} /** Trait that represents the metadata related to storage of blocks */ private[streaming] trait ReceivedBlockStoreResult { @@ -96,7 +96,7 @@ private[streaming] class BlockManagerBasedBlockHandler( */ private[streaming] case class WriteAheadLogBasedStoreResult( blockId: StreamBlockId, - segment: WriteAheadLogFileSegment + walRecordHandle: WriteAheadLogRecordHandle ) extends ReceivedBlockStoreResult @@ -116,10 +116,6 @@ private[streaming] class WriteAheadLogBasedBlockHandler( private val blockStoreTimeout = conf.getInt( "spark.streaming.receiver.blockStoreTimeout", 30).seconds - private val rollingInterval = conf.getInt( - "spark.streaming.receiver.writeAheadLog.rollingInterval", 60) - private val maxFailures = conf.getInt( - "spark.streaming.receiver.writeAheadLog.maxFailures", 3) private val effectiveStorageLevel = { if (storageLevel.deserialized) { @@ -139,18 +135,14 @@ private[streaming] class WriteAheadLogBasedBlockHandler( s"$effectiveStorageLevel when write ahead log is enabled") } - // Manages rolling log files - private val logManager = new WriteAheadLogManager( - checkpointDirToLogDir(checkpointDir, streamId), - hadoopConf, rollingInterval, maxFailures, - callerName = this.getClass.getSimpleName, - clock = clock - ) + // Write ahead log manages + private val writeAheadLog = WriteAheadLogUtils.createLogForReceiver( + conf, checkpointDirToLogDir(checkpointDir, streamId), hadoopConf) // For processing futures used in parallel block storing into block manager and write ahead log // # threads = 2, so that both writing to BM and WAL can proceed in parallel implicit private val executionContext = ExecutionContext.fromExecutorService( - Utils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) + ThreadUtils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) /** * This implementation stores the block into the block manager as well as a write ahead log. @@ -183,21 +175,22 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Store the block in write ahead log val storeInWriteAheadLogFuture = Future { - logManager.writeToLog(serializedBlock) + writeAheadLog.write(serializedBlock, clock.getTimeMillis()) } - // Combine the futures, wait for both to complete, and return the write ahead log segment + // Combine the futures, wait for both to complete, and return the write ahead log record handle val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2) - val segment = Await.result(combinedFuture, blockStoreTimeout) - WriteAheadLogBasedStoreResult(blockId, segment) + val walRecordHandle = Await.result(combinedFuture, blockStoreTimeout) + WriteAheadLogBasedStoreResult(blockId, walRecordHandle) } def cleanupOldBlocks(threshTime: Long) { - logManager.cleanupOldLogs(threshTime, waitForCompletion = false) + writeAheadLog.clean(threshTime, false) } def stop() { - logManager.stop() + writeAheadLog.close() + executionContext.shutdown() } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 5acf8a9a811ee..5b5a3fe648602 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -245,7 +245,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable * Get the unique identifier the receiver input stream that this * receiver is associated with. */ - def streamId = id + def streamId: Int = id /* * ================= diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 1f0244c251eba..4943f29395d12 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -162,13 +162,13 @@ private[streaming] abstract class ReceiverSupervisor( } /** Check if receiver has been marked for stopping */ - def isReceiverStarted() = { + def isReceiverStarted(): Boolean = { logDebug("state = " + receiverState) receiverState == Started } /** Check if receiver has been marked for stopping */ - def isReceiverStopped() = { + def isReceiverStopped(): Boolean = { logDebug("state = " + receiverState) receiverState == Stopped } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 7d29ed88cfcb4..92938379b9c17 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -21,18 +21,17 @@ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await -import akka.actor.{Actor, Props} -import akka.pattern.ask import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.Time import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.streaming.util.WriteAheadLogUtils +import org.apache.spark.util.{RpcUtils, Utils} +import org.apache.spark.{Logging, SparkEnv, SparkException} /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -48,7 +47,7 @@ private[streaming] class ReceiverSupervisorImpl( ) extends ReceiverSupervisor(receiver, env.conf) with Logging { private val receivedBlockHandler: ReceivedBlockHandler = { - if (env.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { + if (WriteAheadLogUtils.enableReceiverLog(env.conf)) { if (checkpointDirOption.isEmpty) { throw new SparkException( "Cannot enable receiver write-ahead log without checkpoint directory set. " + @@ -63,37 +62,23 @@ private[streaming] class ReceiverSupervisorImpl( } - /** Remote Akka actor for the ReceiverTracker */ - private val trackerActor = { - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = AkkaUtils.address( - AkkaUtils.protocol(env.actorSystem), - SparkEnv.driverActorSystemName, - ip, - port, - "ReceiverTracker") - env.actorSystem.actorSelection(url) - } + /** Remote RpcEndpointRef for the ReceiverTracker */ + private val trackerEndpoint = RpcUtils.makeDriverRef("ReceiverTracker", env.conf, env.rpcEnv) - /** Timeout for Akka actor messages */ - private val askTimeout = AkkaUtils.askTimeout(env.conf) + /** RpcEndpointRef for receiving messages from the ReceiverTracker in the driver */ + private val endpoint = env.rpcEnv.setupEndpoint( + "Receiver-" + streamId + "-" + System.currentTimeMillis(), new ThreadSafeRpcEndpoint { + override val rpcEnv: RpcEnv = env.rpcEnv - /** Akka actor for receiving messages from the ReceiverTracker in the driver */ - private val actor = env.actorSystem.actorOf( - Props(new Actor { - - override def receive() = { + override def receive: PartialFunction[Any, Unit] = { case StopReceiver => logInfo("Received stop signal") - stop("Stopped by driver", None) + ReceiverSupervisorImpl.this.stop("Stopped by driver", None) case CleanupOldBlocks(threshTime) => logDebug("Received delete old batch signal") cleanupOldBlocks(threshTime) } - - def ref = self - }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) + }) /** Unique block ids if one wants to add blocks directly */ private val newBlockId = new AtomicLong(System.currentTimeMillis()) @@ -161,16 +146,15 @@ private[streaming] class ReceiverSupervisorImpl( val blockStoreResult = receivedBlockHandler.storeBlock(blockId, receivedBlock) logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms") - val blockInfo = ReceivedBlockInfo(streamId, numRecords, blockStoreResult) - val future = trackerActor.ask(AddBlock(blockInfo))(askTimeout) - Await.result(future, askTimeout) + val blockInfo = ReceivedBlockInfo(streamId, numRecords, metadataOption, blockStoreResult) + trackerEndpoint.askWithRetry[Boolean](AddBlock(blockInfo)) logDebug(s"Reported block $blockId") } /** Report error to the receiver tracker */ def reportError(message: String, error: Throwable) { val errorString = Option(error).map(Throwables.getStackTraceAsString).getOrElse("") - trackerActor ! ReportError(streamId, message, errorString) + trackerEndpoint.send(ReportError(streamId, message, errorString)) logWarning("Reported error " + message + " - " + error) } @@ -180,22 +164,19 @@ private[streaming] class ReceiverSupervisorImpl( override protected def onStop(message: String, error: Option[Throwable]) { blockGenerator.stop() - env.actorSystem.stop(actor) + env.rpcEnv.stop(endpoint) } override protected def onReceiverStart() { val msg = RegisterReceiver( - streamId, receiver.getClass.getSimpleName, Utils.localHostName(), actor) - val future = trackerActor.ask(msg)(askTimeout) - Await.result(future, askTimeout) + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), endpoint) + trackerEndpoint.askWithRetry[Boolean](msg) } override protected def onReceiverStop(message: String, error: Option[Throwable]) { logInfo("Deregistering receiver " + streamId) val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") - val future = trackerActor.ask( - DeregisterReceiver(streamId, message, errorString))(askTimeout) - Await.result(future, askTimeout) + trackerEndpoint.askWithRetry[Boolean](DeregisterReceiver(streamId, message, errorString)) logInfo("Stopped receiver " + streamId) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 92dc113f397ca..5b9bfbf9b01e3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -24,6 +24,7 @@ import org.apache.spark.streaming.Time * :: DeveloperApi :: * Class having information on completed batches. * @param batchTime Time of the batch + * @param streamIdToNumRecords A map of input stream id to record number * @param submissionTime Clock time of when jobs of this batch was submitted to * the streaming scheduler queue * @param processingStartTime Clock time of when the first job of this batch started processing @@ -32,7 +33,7 @@ import org.apache.spark.streaming.Time @DeveloperApi case class BatchInfo( batchTime: Time, - receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], + streamIdToNumRecords: Map[Int, Long], submissionTime: Long, processingStartTime: Option[Long], processingEndTime: Option[Long] @@ -58,4 +59,9 @@ case class BatchInfo( */ def totalDelay: Option[Long] = schedulingDelay.zip(processingDelay) .map(x => x._1 + x._2).headOption + + /** + * The number of recorders received by the receivers in this batch. + */ + def numRecords: Long = streamIdToNumRecords.values.sum } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala new file mode 100644 index 0000000000000..a72efccf2f994 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import scala.collection.mutable + +import org.apache.spark.Logging +import org.apache.spark.streaming.{Time, StreamingContext} + +/** To track the information of input stream at specified batch time. */ +private[streaming] case class InputInfo(inputStreamId: Int, numRecords: Long) + +/** + * This class manages all the input streams as well as their input data statistics. The information + * will be exposed through StreamingListener for monitoring. + */ +private[streaming] class InputInfoTracker(ssc: StreamingContext) extends Logging { + + // Map to track all the InputInfo related to specific batch time and input stream. + private val batchTimeToInputInfos = new mutable.HashMap[Time, mutable.HashMap[Int, InputInfo]] + + /** Report the input information with batch time to the tracker */ + def reportInfo(batchTime: Time, inputInfo: InputInfo): Unit = synchronized { + val inputInfos = batchTimeToInputInfos.getOrElseUpdate(batchTime, + new mutable.HashMap[Int, InputInfo]()) + + if (inputInfos.contains(inputInfo.inputStreamId)) { + throw new IllegalStateException(s"Input stream ${inputInfo.inputStreamId}} for batch" + + s"$batchTime is already added into InputInfoTracker, this is a illegal state") + } + inputInfos += ((inputInfo.inputStreamId, inputInfo)) + } + + /** Get the all the input stream's information of specified batch time */ + def getInfo(batchTime: Time): Map[Int, InputInfo] = synchronized { + val inputInfos = batchTimeToInputInfos.get(batchTime) + // Convert mutable HashMap to immutable Map for the caller + inputInfos.map(_.toMap).getOrElse(Map[Int, InputInfo]()) + } + + /** Cleanup the tracked input information older than threshold batch time */ + def cleanup(batchThreshTime: Time): Unit = synchronized { + val timesToCleanup = batchTimeToInputInfos.keys.filter(_ < batchThreshTime) + logInfo(s"remove old batch metadata: ${timesToCleanup.mkString(" ")}") + batchTimeToInputInfos --= timesToCleanup + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala index 7e0f6b2cdfc08..3c481bf3491f9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -25,16 +25,50 @@ import scala.util.Try */ private[streaming] class Job(val time: Time, func: () => _) { - var id: String = _ - var result: Try[_] = null + private var _id: String = _ + private var _outputOpId: Int = _ + private var isSet = false + private var _result: Try[_] = null def run() { - result = Try(func()) + _result = Try(func()) } - def setId(number: Int) { - id = "streaming job " + time + "." + number + def result: Try[_] = { + if (_result == null) { + throw new IllegalStateException("Cannot access result before job finishes") + } + _result } - override def toString = id + /** + * @return the global unique id of this Job. + */ + def id: String = { + if (!isSet) { + throw new IllegalStateException("Cannot access id before calling setId") + } + _id + } + + /** + * @return the output op id of this Job. Each Job has a unique output op id in the same JobSet. + */ + def outputOpId: Int = { + if (!isSet) { + throw new IllegalStateException("Cannot access number before calling setId") + } + _outputOpId + } + + def setOutputOpId(outputOpId: Int) { + if (isSet) { + throw new IllegalStateException("Cannot call setOutputOpId more than once") + } + isSet = true + _id = s"streaming job $time.$outputOpId" + _outputOpId = outputOpId + } + + override def toString: String = id } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 59488dfb0f8c6..9f93d6cbc3c20 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -19,12 +19,10 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Success, Try} -import akka.actor.{ActorRef, Props, Actor} - import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock} +import org.apache.spark.util.{Clock, EventLoop, ManualClock} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -58,7 +56,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, - longTime => eventActor ! GenerateJobs(new Time(longTime)), "JobGenerator") + longTime => eventLoop.post(GenerateJobs(new Time(longTime))), "JobGenerator") // This is marked lazy so that this is initialized after checkpoint duration has been set // in the context and the generator has been started. @@ -70,22 +68,26 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { null } - // eventActor is created when generator starts. + // eventLoop is created when generator starts. // This not being null means the scheduler has been started and not stopped - private var eventActor: ActorRef = null + private var eventLoop: EventLoop[JobGeneratorEvent] = null // last batch whose completion,checkpointing and metadata cleanup has been completed private var lastProcessedBatch: Time = null /** Start generation of jobs */ def start(): Unit = synchronized { - if (eventActor != null) return // generator has already been started + if (eventLoop != null) return // generator has already been started + + eventLoop = new EventLoop[JobGeneratorEvent]("JobGenerator") { + override protected def onReceive(event: JobGeneratorEvent): Unit = processEvent(event) - eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - def receive = { - case event: JobGeneratorEvent => processEvent(event) + override protected def onError(e: Throwable): Unit = { + jobScheduler.reportError("Error in job generator", e) } - }), "JobGenerator") + } + eventLoop.start() + if (ssc.isCheckpointPresent) { restart() } else { @@ -99,22 +101,20 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { * checkpoints written. */ def stop(processReceivedData: Boolean): Unit = synchronized { - if (eventActor == null) return // generator has already been stopped + if (eventLoop == null) return // generator has already been stopped if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = conf.getLong( - "spark.streaming.gracefulStopTimeout", - 10 * ssc.graph.batchDuration.milliseconds - ) + val stopTimeoutMs = conf.getTimeAsMs( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently - def hasTimedOut = { - val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout + def hasTimedOut: Boolean = { + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs if (timedOut) { - logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } timedOut } @@ -133,7 +133,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Stopped generation timer") // Wait for the jobs to complete and checkpoints to be written - def haveAllBatchesBeenProcessed = { + def haveAllBatchesBeenProcessed: Boolean = { lastProcessedBatch != null && lastProcessedBatch.milliseconds == stopTime } logInfo("Waiting for jobs to be processed and checkpoints to be written") @@ -148,9 +148,9 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { graph.stop() } - // Stop the actor and checkpoint writer + // Stop the event loop and checkpoint writer if (shouldCheckpoint) checkpointWriter.stop() - ssc.env.actorSystem.stop(eventActor) + eventLoop.stop() logInfo("Stopped JobGenerator") } @@ -158,7 +158,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { * Callback called when a batch has been completely processed. */ def onBatchCompletion(time: Time) { - eventActor ! ClearMetadata(time) + eventLoop.post(ClearMetadata(time)) } /** @@ -166,7 +166,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { */ def onCheckpointCompletion(time: Time, clearCheckpointDataLater: Boolean) { if (clearCheckpointDataLater) { - eventActor ! ClearCheckpointData(time) + eventLoop.post(ClearCheckpointData(time)) } } @@ -243,13 +243,13 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { graph.generateJobs(time) // generate jobs using allocated block } match { case Success(jobs) => - val receivedBlockInfos = - jobScheduler.receiverTracker.getBlocksOfBatch(time).mapValues { _.toArray } - jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfos)) + val streamIdToInputInfos = jobScheduler.inputInfoTracker.getInfo(time) + val streamIdToNumRecords = streamIdToInputInfos.mapValues(_.numRecords) + jobScheduler.submitJobSet(JobSet(time, jobs, streamIdToNumRecords)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } - eventActor ! DoCheckpoint(time, clearCheckpointDataLater = false) + eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false)) } /** Clear DStream metadata for the given `time`. */ @@ -259,13 +259,14 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // If checkpointing is enabled, then checkpoint, // else mark batch to be fully processed if (shouldCheckpoint) { - eventActor ! DoCheckpoint(time, clearCheckpointDataLater = true) + eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = true)) } else { // If checkpointing is not enabled, then delete metadata information about // received blocks (block data not saved in any case). Otherwise, wait for // checkpointing of this batch to complete. val maxRememberDuration = graph.getMaxInputStreamRememberDuration() jobScheduler.receiverTracker.cleanupOldBlocksAndBatches(time - maxRememberDuration) + jobScheduler.inputInfoTracker.cleanup(time - maxRememberDuration) markBatchFullyProcessed(time) } } @@ -278,6 +279,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // been saved to checkpoints, so its safe to delete block metadata and data WAL files val maxRememberDuration = graph.getMaxInputStreamRememberDuration() jobScheduler.receiverTracker.cleanupOldBlocksAndBatches(time - maxRememberDuration) + jobScheduler.inputInfoTracker.cleanup(time - maxRememberDuration) markBatchFullyProcessed(time) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 60bc099b27a4c..1d1ddaaccf217 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -17,13 +17,15 @@ package org.apache.spark.streaming.scheduler -import scala.util.{Failure, Success, Try} -import scala.collection.JavaConversions._ import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors} -import akka.actor.{ActorRef, Actor, Props} -import org.apache.spark.{SparkException, Logging, SparkEnv} + +import scala.collection.JavaConversions._ +import scala.util.{Failure, Success} + +import org.apache.spark.Logging import org.apache.spark.rdd.PairRDDFunctions import org.apache.spark.streaming._ +import org.apache.spark.util.EventLoop private[scheduler] sealed trait JobSchedulerEvent @@ -46,30 +48,34 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { val listenerBus = new StreamingListenerBus() // These two are created only when scheduler starts. - // eventActor not being null means the scheduler has been started and not stopped + // eventLoop not being null means the scheduler has been started and not stopped var receiverTracker: ReceiverTracker = null - private var eventActor: ActorRef = null + // A tracker to track all the input stream information as well as processed record number + var inputInfoTracker: InputInfoTracker = null + private var eventLoop: EventLoop[JobSchedulerEvent] = null def start(): Unit = synchronized { - if (eventActor != null) return // scheduler has already been started + if (eventLoop != null) return // scheduler has already been started logDebug("Starting JobScheduler") - eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { - def receive = { - case event: JobSchedulerEvent => processEvent(event) - } - }), "JobScheduler") + eventLoop = new EventLoop[JobSchedulerEvent]("JobScheduler") { + override protected def onReceive(event: JobSchedulerEvent): Unit = processEvent(event) + + override protected def onError(e: Throwable): Unit = reportError("Error in job scheduler", e) + } + eventLoop.start() listenerBus.start(ssc.sparkContext) receiverTracker = new ReceiverTracker(ssc) + inputInfoTracker = new InputInfoTracker(ssc) receiverTracker.start() jobGenerator.start() logInfo("Started JobScheduler") } def stop(processAllReceivedData: Boolean): Unit = synchronized { - if (eventActor == null) return // scheduler has already been stopped + if (eventLoop == null) return // scheduler has already been stopped logDebug("Stopping JobScheduler") // First, stop receiving @@ -96,8 +102,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // Stop everything else listenerBus.stop() - ssc.env.actorSystem.stop(eventActor) - eventActor = null + eventLoop.stop() + eventLoop = null logInfo("Stopped JobScheduler") } @@ -105,6 +111,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { if (jobSet.jobs.isEmpty) { logInfo("No jobs added for time " + jobSet.time) } else { + listenerBus.post(StreamingListenerBatchSubmitted(jobSet.toBatchInfo)) jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) logInfo("Added jobs for time " + jobSet.time) @@ -116,7 +123,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } def reportError(msg: String, e: Throwable) { - eventActor ! ErrorReported(msg, e) + eventLoop.post(ErrorReported(msg, e)) } private def processEvent(event: JobSchedulerEvent) { @@ -134,10 +141,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private def handleJobStart(job: Job) { val jobSet = jobSets.get(job.time) - if (!jobSet.hasStarted) { + val isFirstJobOfJobSet = !jobSet.hasStarted + jobSet.handleJobStart(job) + if (isFirstJobOfJobSet) { + // "StreamingListenerBatchStarted" should be posted after calling "handleJobStart" to get the + // correct "jobSet.processingStartTime". listenerBus.post(StreamingListenerBatchStarted(jobSet.toBatchInfo)) } - jobSet.handleJobStart(job) logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) } @@ -166,16 +176,28 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { ssc.waiter.notifyError(e) } - private class JobHandler(job: Job) extends Runnable { + private class JobHandler(job: Job) extends Runnable with Logging { def run() { - eventActor ! JobStarted(job) - // Disable checks for existing output directories in jobs launched by the streaming scheduler, - // since we may need to write output to an existing directory during checkpoint recovery; - // see SPARK-4835 for more details. - PairRDDFunctions.disableOutputSpecValidation.withValue(true) { - job.run() + ssc.sc.setLocalProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, job.time.milliseconds.toString) + ssc.sc.setLocalProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, job.outputOpId.toString) + try { + eventLoop.post(JobStarted(job)) + // Disable checks for existing output directories in jobs launched by the streaming + // scheduler, since we may need to write output to an existing directory during checkpoint + // recovery; see SPARK-4835 for more details. + PairRDDFunctions.disableOutputSpecValidation.withValue(true) { + job.run() + } + eventLoop.post(JobCompleted(job)) + } finally { + ssc.sc.setLocalProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, null) + ssc.sc.setLocalProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, null) } - eventActor ! JobCompleted(job) } } } + +private[streaming] object JobScheduler { + val BATCH_TIME_PROPERTY_KEY = "spark.streaming.internal.batchTime" + val OUTPUT_OP_ID_PROPERTY_KEY = "spark.streaming.internal.outputOpId" +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index 8c15a75b1b0e0..e6be63b2ddbdc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -28,15 +28,14 @@ private[streaming] case class JobSet( time: Time, jobs: Seq[Job], - receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty - ) { + streamIdToNumRecords: Map[Int, Long] = Map.empty) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted private var processingStartTime = -1L // when the first job of this jobset started processing private var processingEndTime = -1L // when the last job of this jobset finished processing - jobs.zipWithIndex.foreach { case (job, i) => job.setId(i) } + jobs.zipWithIndex.foreach { case (job, i) => job.setOutputOpId(i) } incompleteJobs ++= jobs def handleJobStart(job: Job) { @@ -48,24 +47,24 @@ case class JobSet( if (hasCompleted) processingEndTime = System.currentTimeMillis() } - def hasStarted = processingStartTime > 0 + def hasStarted: Boolean = processingStartTime > 0 - def hasCompleted = incompleteJobs.isEmpty + def hasCompleted: Boolean = incompleteJobs.isEmpty // Time taken to process all the jobs from the time they started processing // (i.e. not including the time they wait in the streaming scheduler queue) - def processingDelay = processingEndTime - processingStartTime + def processingDelay: Long = processingEndTime - processingStartTime // Time taken to process all the jobs from the time they were submitted // (i.e. including the time they wait in the streaming scheduler queue) - def totalDelay = { + def totalDelay: Long = { processingEndTime - time.milliseconds } def toBatchInfo: BatchInfo = { new BatchInfo( time, - receivedBlockInfo, + streamIdToNumRecords, submissionTime, if (processingStartTime >= 0 ) Some(processingStartTime) else None, if (processingEndTime >= 0 ) Some(processingEndTime) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala index 94beb590f52d6..dc11e84f29965 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala @@ -17,12 +17,38 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.receiver.ReceivedBlockStoreResult +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.receiver.{ReceivedBlockStoreResult, WriteAheadLogBasedStoreResult} +import org.apache.spark.streaming.util.WriteAheadLogRecordHandle /** Information about blocks received by the receiver */ private[streaming] case class ReceivedBlockInfo( streamId: Int, numRecords: Long, + metadataOption: Option[Any], blockStoreResult: ReceivedBlockStoreResult - ) + ) { + + @volatile private var _isBlockIdValid = true + + def blockId: StreamBlockId = blockStoreResult.blockId + + def walRecordHandleOption: Option[WriteAheadLogRecordHandle] = { + blockStoreResult match { + case walStoreResult: WriteAheadLogBasedStoreResult => Some(walStoreResult.walRecordHandle) + case _ => None + } + } + + /** Is the block ID valid, that is, is the block present in the Spark executors. */ + def isBlockIdValid(): Boolean = _isBlockIdValid + + /** + * Set the block ID as invalid. This is useful when it is known that the block is not present + * in the Spark executors. + */ + def setBlockIdInvalid(): Unit = { + _isBlockIdValid = false + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 200cf4ef4b0f1..a9f4147a5f020 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -25,10 +25,10 @@ import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkException, Logging, SparkConf} import org.apache.spark.streaming.Time -import org.apache.spark.streaming.util.WriteAheadLogManager +import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.{Logging, SparkConf, SparkException} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ private[streaming] sealed trait ReceivedBlockTrackerLogEvent @@ -45,7 +45,7 @@ private[streaming] case class BatchCleanupEvent(times: Seq[Time]) private[streaming] case class AllocatedBlocks(streamIdToAllocatedBlocks: Map[Int, Seq[ReceivedBlockInfo]]) { def getBlocksOfStream(streamId: Int): Seq[ReceivedBlockInfo] = { - streamIdToAllocatedBlocks.get(streamId).getOrElse(Seq.empty) + streamIdToAllocatedBlocks.getOrElse(streamId, Seq.empty) } } @@ -63,6 +63,7 @@ private[streaming] class ReceivedBlockTracker( hadoopConf: Configuration, streamIds: Seq[Int], clock: Clock, + recoverFromWriteAheadLog: Boolean, checkpointDirOption: Option[String]) extends Logging { @@ -70,12 +71,14 @@ private[streaming] class ReceivedBlockTracker( private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue] private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] - private val logManagerOption = createLogManager() + private val writeAheadLogOption = createWriteAheadLog() private var lastAllocatedBatchTime: Time = null // Recover block information from write ahead logs - recoverFromWriteAheadLogs() + if (recoverFromWriteAheadLog) { + recoverPastEvents() + } /** Add received block. This event will get written to the write ahead log (if enabled). */ def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { @@ -155,22 +158,23 @@ private[streaming] class ReceivedBlockTracker( logInfo("Deleting batches " + timesToCleanup) writeToLog(BatchCleanupEvent(timesToCleanup)) timeToAllocatedBlocks --= timesToCleanup - logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds, waitForCompletion)) + writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) } /** Stop the block tracker. */ def stop() { - logManagerOption.foreach { _.stop() } + writeAheadLogOption.foreach { _.close() } } /** * Recover all the tracker actions from the write ahead logs to recover the state (unallocated * and allocated block info) prior to failure. */ - private def recoverFromWriteAheadLogs(): Unit = synchronized { + private def recoverPastEvents(): Unit = synchronized { // Insert the recovered block information def insertAddedBlock(receivedBlockInfo: ReceivedBlockInfo) { logTrace(s"Recovery: Inserting added block $receivedBlockInfo") + receivedBlockInfo.setBlockIdInvalid() getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo } @@ -190,9 +194,10 @@ private[streaming] class ReceivedBlockTracker( timeToAllocatedBlocks --= batchTimes } - logManagerOption.foreach { logManager => + writeAheadLogOption.foreach { writeAheadLog => logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") - logManager.readFromLog().foreach { byteBuffer => + import scala.collection.JavaConversions._ + writeAheadLog.readAll().foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { case BlockAdditionEvent(receivedBlockInfo) => @@ -208,10 +213,10 @@ private[streaming] class ReceivedBlockTracker( /** Write an update to the tracker to the write ahead log */ private def writeToLog(record: ReceivedBlockTrackerLogEvent) { - if (isLogManagerEnabled) { + if (isWriteAheadLogEnabled) { logDebug(s"Writing to log $record") - logManagerOption.foreach { logManager => - logManager.writeToLog(ByteBuffer.wrap(Utils.serialize(record))) + writeAheadLogOption.foreach { logManager => + logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) } } } @@ -222,28 +227,15 @@ private[streaming] class ReceivedBlockTracker( } /** Optionally create the write ahead log manager only if the feature is enabled */ - private def createLogManager(): Option[WriteAheadLogManager] = { - if (conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { - if (checkpointDirOption.isEmpty) { - throw new SparkException( - "Cannot enable receiver write-ahead log without checkpoint directory set. " + - "Please use streamingContext.checkpoint() to set the checkpoint directory. " + - "See documentation for more details.") - } + private def createWriteAheadLog(): Option[WriteAheadLog] = { + checkpointDirOption.map { checkpointDir => val logDir = ReceivedBlockTracker.checkpointDirToLogDir(checkpointDirOption.get) - val rollingIntervalSecs = conf.getInt( - "spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", 60) - val logManager = new WriteAheadLogManager(logDir, hadoopConf, - rollingIntervalSecs = rollingIntervalSecs, clock = clock, - callerName = "ReceivedBlockHandlerMaster") - Some(logManager) - } else { - None + WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) } } - /** Check if the log manager is enabled. This is only used for testing purposes. */ - private[streaming] def isLogManagerEnabled: Boolean = logManagerOption.nonEmpty + /** Check if the write ahead log is enabled. This is only used for testing purposes. */ + private[streaming] def isWriteAheadLogEnabled: Boolean = writeAheadLogOption.nonEmpty } private[streaming] object ReceivedBlockTracker { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala index d7e39c528c519..52f08b9c9de68 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.scheduler -import akka.actor.ActorRef import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rpc.RpcEndpointRef /** * :: DeveloperApi :: @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi case class ReceiverInfo( streamId: Int, name: String, - private[streaming] val actor: ActorRef, + private[streaming] val endpoint: RpcEndpointRef, active: Boolean, location: String, lastErrorMessage: String = "", diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index b36aeb341d25e..3c341390eda39 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -17,13 +17,12 @@ package org.apache.spark.streaming.scheduler - import scala.collection.mutable.{HashMap, SynchronizedMap} import scala.language.existentials -import akka.actor._ - +import org.apache.spark.streaming.util.WriteAheadLogUtils import org.apache.spark.{Logging, SerializableWritable, SparkEnv, SparkException} +import org.apache.spark.rpc._ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{CleanupOldBlocks, Receiver, ReceiverSupervisorImpl, StopReceiver} @@ -36,7 +35,7 @@ private[streaming] case class RegisterReceiver( streamId: Int, typ: String, host: String, - receiverActor: ActorRef + receiverEndpoint: RpcEndpointRef ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) extends ReceiverTrackerMessage @@ -63,37 +62,38 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false ssc.sparkContext.hadoopConfiguration, receiverInputStreamIds, ssc.scheduler.clock, + ssc.isCheckpointPresent, Option(ssc.checkpointDir) ) private val listenerBus = ssc.scheduler.listenerBus - // actor is created when generator starts. + // endpoint is created when generator starts. // This not being null means the tracker has been started and not stopped - private var actor: ActorRef = null + private var endpoint: RpcEndpointRef = null - /** Start the actor and receiver execution thread. */ - def start() = synchronized { - if (actor != null) { + /** Start the endpoint and receiver execution thread. */ + def start(): Unit = synchronized { + if (endpoint != null) { throw new SparkException("ReceiverTracker already started") } if (!receiverInputStreams.isEmpty) { - actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), - "ReceiverTracker") + endpoint = ssc.env.rpcEnv.setupEndpoint( + "ReceiverTracker", new ReceiverTrackerEndpoint(ssc.env.rpcEnv)) if (!skipReceiverLaunch) receiverExecutor.start() logInfo("ReceiverTracker started") } } /** Stop the receiver execution thread. */ - def stop(graceful: Boolean) = synchronized { - if (!receiverInputStreams.isEmpty && actor != null) { + def stop(graceful: Boolean): Unit = synchronized { + if (!receiverInputStreams.isEmpty && endpoint != null) { // First, stop the receivers if (!skipReceiverLaunch) receiverExecutor.stop(graceful) - // Finally, stop the actor - ssc.env.actorSystem.stop(actor) - actor = null + // Finally, stop the endpoint + ssc.env.rpcEnv.stop(endpoint) + endpoint = null receivedBlockTracker.stop() logInfo("ReceiverTracker stopped") } @@ -127,10 +127,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) // Signal the receivers to delete old block data - if (ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { + if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { logInfo(s"Cleanup old received batch data: $cleanupThreshTime") - receiverInfo.values.flatMap { info => Option(info.actor) } - .foreach { _ ! CleanupOldBlocks(cleanupThreshTime) } + receiverInfo.values.flatMap { info => Option(info.endpoint) } + .foreach { _.send(CleanupOldBlocks(cleanupThreshTime)) } } } @@ -139,23 +139,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false streamId: Int, typ: String, host: String, - receiverActor: ActorRef, - sender: ActorRef + receiverEndpoint: RpcEndpointRef, + senderAddress: RpcAddress ) { if (!receiverInputStreamIds.contains(streamId)) { throw new SparkException("Register received for unexpected id " + streamId) } receiverInfo(streamId) = ReceiverInfo( - streamId, s"${typ}-${streamId}", receiverActor, true, host) + streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) - logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) + logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) } /** Deregister a receiver */ private def deregisterReceiver(streamId: Int, message: String, error: String) { val newReceiverInfo = receiverInfo.get(streamId) match { case Some(oldInfo) => - oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) + oldInfo.copy(endpoint = null, active = false, lastErrorMessage = message, lastError = error) case None => logWarning("No prior receiver info") ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) @@ -199,19 +199,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receivedBlockTracker.hasUnallocatedReceivedBlocks } - /** Actor to receive messages from the receivers. */ - private class ReceiverTrackerActor extends Actor { - def receive = { - case RegisterReceiver(streamId, typ, host, receiverActor) => - registerReceiver(streamId, typ, host, receiverActor, sender) - sender ! true - case AddBlock(receivedBlockInfo) => - sender ! addBlock(receivedBlockInfo) + /** RpcEndpoint to receive messages from the receivers. */ + private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { + + override def receive: PartialFunction[Any, Unit] = { case ReportError(streamId, message, error) => reportError(streamId, message, error) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RegisterReceiver(streamId, typ, host, receiverEndpoint) => + registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) + context.reply(true) + case AddBlock(receivedBlockInfo) => + context.reply(addBlock(receivedBlockInfo)) case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) - sender ! true + context.reply(true) } } @@ -244,16 +248,15 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (graceful) { val pollTime = 100 - def done = { receiverInfo.isEmpty && !running } logInfo("Waiting for receiver job to terminate gracefully") - while(!done) { + while (receiverInfo.nonEmpty || running) { Thread.sleep(pollTime) } logInfo("Waited for receiver job to terminate gracefully") } // Check if all the receivers have been deregistered or not - if (!receiverInfo.isEmpty) { + if (receiverInfo.nonEmpty) { logWarning("Not all of the receivers have deregistered, " + receiverInfo) } else { logInfo("All of the receivers have deregistered successfully") @@ -315,8 +318,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Stops the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.flatMap { info => Option(info.actor)} - .foreach { _ ! StopReceiver } + receiverInfo.values.flatMap { info => Option(info.endpoint)} + .foreach { _.send(StopReceiver) } logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala new file mode 100644 index 0000000000000..e219e27785533 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -0,0 +1,116 @@ +/* + * 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.streaming.ui + +import scala.xml.Node + +import org.apache.spark.ui.UIUtils + +private[ui] abstract class BatchTableBase(tableId: String) { + + protected def columns: Seq[Node] = { + + + + + } + + protected def baseRow(batch: BatchUIData): Seq[Node] = { + val batchTime = batch.batchTime.milliseconds + val formattedBatchTime = UIUtils.formatDate(batch.batchTime.milliseconds) + val eventCount = batch.numRecords + val schedulingDelay = batch.schedulingDelay + val formattedSchedulingDelay = schedulingDelay.map(UIUtils.formatDuration).getOrElse("-") + val processingTime = batch.processingDelay + val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-") + + + + + + } + + private def batchTable: Seq[Node] = { +
    {info.userName} + {jobLink} + {info.groupId}{formatDate(info.startTimestamp)}{if(info.finishTimestamp > 0) formatDate(info.finishTimestamp)}{formatDurationOption(Some(info.totalTime))}{info.statement}{info.state}
    {errorSummary}{details}
    {session.userName} {session.ip} {session.sessionId} {formatDate(session.startTimestamp)} {if(session.finishTimestamp > 0) formatDate(session.finishTimestamp)} {formatDurationOption(Some(session.totalTime))} {session.totalExecution.toString}
    {d}
    {info.userName} + {jobLink} + {info.groupId}{formatDate(info.startTimestamp)}{formatDate(info.finishTimestamp)}{formatDurationOption(Some(info.totalTime))}{info.statement}{info.state}
    {errorSummary}{details}
    {d}
    Batch TimeInput SizeScheduling DelayProcessing Time + + {formattedBatchTime} + + {eventCount.toString} events + {formattedSchedulingDelay} + + {formattedProcessingTime} +
    + + {columns} + + + {renderRows} + +
    + } + + def toNodeSeq: Seq[Node] = { + batchTable + } + + /** + * Return HTML for all rows of this table. + */ + protected def renderRows: Seq[Node] +} + +private[ui] class ActiveBatchTable( + runningBatches: Seq[BatchUIData], + waitingBatches: Seq[BatchUIData]) extends BatchTableBase("active-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Status + + override protected def renderRows: Seq[Node] = { + // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display + // waiting batches before running batches + waitingBatches.flatMap(batch => {waitingBatchRow(batch)}) ++ + runningBatches.flatMap(batch => {runningBatchRow(batch)}) + } + + private def runningBatchRow(batch: BatchUIData): Seq[Node] = { + baseRow(batch) ++ processing + } + + private def waitingBatchRow(batch: BatchUIData): Seq[Node] = { + baseRow(batch) ++ queued + } +} + +private[ui] class CompletedBatchTable(batches: Seq[BatchUIData]) + extends BatchTableBase("completed-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Total Delay + + override protected def renderRows: Seq[Node] = { + batches.flatMap(batch => {completedBatchRow(batch)}) + } + + private def completedBatchRow(batch: BatchUIData): Seq[Node] = { + val totalDelay = batch.totalDelay + val formattedTotalDelay = totalDelay.map(UIUtils.formatDuration).getOrElse("-") + baseRow(batch) ++ + + {formattedTotalDelay} + + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala new file mode 100644 index 0000000000000..2da9a29e2529e --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -0,0 +1,264 @@ +/* + * 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.streaming.ui + +import javax.servlet.http.HttpServletRequest + +import scala.xml.{NodeSeq, Node} + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.streaming.Time +import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} +import org.apache.spark.ui.jobs.UIData.JobUIData + + +private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { + private val streamingListener = parent.listener + private val sparkListener = parent.ssc.sc.jobProgressListener + + private def columns: Seq[Node] = { + Output Op Id + Description + Duration + Job Id + Duration + Stages: Succeeded/Total + Tasks (for all stages): Succeeded/Total + Error + } + + /** + * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into + * one cell, we use "rowspan" for the first row of a output op. + */ + def generateJobRow( + outputOpId: OutputOpId, + formattedOutputOpDuration: String, + numSparkJobRowsInOutputOp: Int, + isFirstRow: Boolean, + sparkJob: JobUIData): Seq[Node] = { + val lastStageInfo = Option(sparkJob.stageIds) + .filter(_.nonEmpty) + .flatMap { ids => sparkListener.stageIdToInfo.get(ids.max) } + val lastStageData = lastStageInfo.flatMap { s => + sparkListener.stageIdToData.get((s.stageId, s.attemptId)) + } + + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + val duration: Option[Long] = { + sparkJob.submissionTime.map { start => + val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val lastFailureReason = + sparkJob.stageIds.sorted.reverse.flatMap(sparkListener.stageIdToInfo.get). + dropWhile(_.failureReason == None).take(1). // get the first info that contains failure + flatMap(info => info.failureReason).headOption.getOrElse("") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("-") + val detailUrl = s"${UIUtils.prependBaseUri(parent.basePath)}/jobs/job?id=${sparkJob.jobId}" + + // In the first row, output op id and its information needs to be shown. In other rows, these + // cells will be taken up due to "rowspan". + // scalastyle:off + val prefixCells = + if (isFirstRow) { + {outputOpId.toString} + + + {lastStageDescription} + {lastStageName} + + {formattedOutputOpDuration} + } else { + Nil + } + // scalastyle:on + + + {prefixCells} + + + {sparkJob.jobId}{sparkJob.jobGroup.map(id => s"($id)").getOrElse("")} + + + + {formattedDuration} + + + {sparkJob.completedStageIndices.size}/{sparkJob.stageIds.size - sparkJob.numSkippedStages} + {if (sparkJob.numFailedStages > 0) s"(${sparkJob.numFailedStages} failed)"} + {if (sparkJob.numSkippedStages > 0) s"(${sparkJob.numSkippedStages} skipped)"} + + + { + UIUtils.makeProgressBar( + started = sparkJob.numActiveTasks, + completed = sparkJob.numCompletedTasks, + failed = sparkJob.numFailedTasks, + skipped = sparkJob.numSkippedTasks, + total = sparkJob.numTasks - sparkJob.numSkippedTasks) + } + + {failureReasonCell(lastFailureReason)} + + } + + private def generateOutputOpIdRow( + outputOpId: OutputOpId, sparkJobs: Seq[JobUIData]): Seq[Node] = { + val sparkjobDurations = sparkJobs.map(sparkJob => { + sparkJob.submissionTime.map { start => + val end = sparkJob.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + }) + val formattedOutputOpDuration = + if (sparkjobDurations.exists(_ == None)) { + // If any job does not finish, set "formattedOutputOpDuration" to "-" + "-" + } else { + UIUtils.formatDuration(sparkjobDurations.flatMap(x => x).sum) + } + generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, true, sparkJobs.head) ++ + sparkJobs.tail.map { sparkJob => + generateJobRow(outputOpId, formattedOutputOpDuration, sparkJobs.size, false, sparkJob) + }.flatMap(x => x) + } + + private def failureReasonCell(failureReason: String): Seq[Node] = { + val isMultiline = failureReason.indexOf('\n') >= 0 + // Display the first line by default + val failureReasonSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + failureReason.substring(0, failureReason.indexOf('\n')) + } else { + failureReason + }) + val details = if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + "" + } + {failureReasonSummary}{details} + } + + private def getJobData(sparkJobId: SparkJobId): Option[JobUIData] = { + sparkListener.activeJobs.get(sparkJobId).orElse { + sparkListener.completedJobs.find(_.jobId == sparkJobId).orElse { + sparkListener.failedJobs.find(_.jobId == sparkJobId) + } + } + } + + /** + * Generate the job table for the batch. + */ + private def generateJobTable(batchUIData: BatchUIData): Seq[Node] = { + val outputOpIdToSparkJobIds = batchUIData.outputOpIdSparkJobIdPairs.groupBy(_.outputOpId).toSeq. + sortBy(_._1). // sorted by OutputOpId + map { case (outputOpId, outputOpIdAndSparkJobIds) => + // sort SparkJobIds for each OutputOpId + (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).sorted) + } + sparkListener.synchronized { + val outputOpIdWithJobs: Seq[(OutputOpId, Seq[JobUIData])] = + outputOpIdToSparkJobIds.map { case (outputOpId, sparkJobIds) => + // Filter out spark Job ids that don't exist in sparkListener + (outputOpId, sparkJobIds.flatMap(getJobData)) + } + + + + {columns} + + + { + outputOpIdWithJobs.map { + case (outputOpId, jobs) => generateOutputOpIdRow(outputOpId, jobs) + } + } + +
    + } + } + + def render(request: HttpServletRequest): Seq[Node] = { + val batchTime = Option(request.getParameter("id")).map(id => Time(id.toLong)).getOrElse { + throw new IllegalArgumentException(s"Missing id parameter") + } + val formattedBatchTime = UIUtils.formatDate(batchTime.milliseconds) + + val batchUIData = streamingListener.getBatchUIData(batchTime).getOrElse { + throw new IllegalArgumentException(s"Batch $formattedBatchTime does not exist") + } + + val formattedSchedulingDelay = + batchUIData.schedulingDelay.map(UIUtils.formatDuration).getOrElse("-") + val formattedProcessingTime = + batchUIData.processingDelay.map(UIUtils.formatDuration).getOrElse("-") + val formattedTotalDelay = batchUIData.totalDelay.map(UIUtils.formatDuration).getOrElse("-") + + val summary: NodeSeq = +
    +
      +
    • + Batch Duration: + {UIUtils.formatDuration(streamingListener.batchDuration)} +
    • +
    • + Input data size: + {batchUIData.numRecords} records +
    • +
    • + Scheduling delay: + {formattedSchedulingDelay} +
    • +
    • + Processing time: + {formattedProcessingTime} +
    • +
    • + Total delay: + {formattedTotalDelay} +
    • +
    +
    + + val jobTable = + if (batchUIData.outputOpIdSparkJobIdPairs.isEmpty) { +
    Cannot find any job for Batch {formattedBatchTime}.
    + } else { + generateJobTable(batchUIData) + } + + val content = summary ++ jobTable + + UIUtils.headerSparkPage(s"Details of batch at $formattedBatchTime", content, parent) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala new file mode 100644 index 0000000000000..99e10d2b0be12 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala @@ -0,0 +1,75 @@ +/* + * 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.streaming.ui + +import org.apache.spark.streaming.Time +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.streaming.ui.StreamingJobProgressListener._ + +private[ui] case class OutputOpIdAndSparkJobId(outputOpId: OutputOpId, sparkJobId: SparkJobId) + +private[ui] case class BatchUIData( + val batchTime: Time, + val receiverNumRecords: Map[Int, Long], + val submissionTime: Long, + val processingStartTime: Option[Long], + val processingEndTime: Option[Long], + var outputOpIdSparkJobIdPairs: Seq[OutputOpIdAndSparkJobId] = Seq.empty) { + + /** + * Time taken for the first job of this batch to start processing from the time this batch + * was submitted to the streaming scheduler. Essentially, it is + * `processingStartTime` - `submissionTime`. + */ + def schedulingDelay: Option[Long] = processingStartTime.map(_ - submissionTime) + + /** + * Time taken for the all jobs of this batch to finish processing from the time they started + * processing. Essentially, it is `processingEndTime` - `processingStartTime`. + */ + def processingDelay: Option[Long] = { + for (start <- processingStartTime; + end <- processingEndTime) + yield end - start + } + + /** + * Time taken for all the jobs of this batch to finish processing from the time they + * were submitted. Essentially, it is `processingDelay` + `schedulingDelay`. + */ + def totalDelay: Option[Long] = processingEndTime.map(_ - submissionTime) + + /** + * The number of recorders received by the receivers in this batch. + */ + def numRecords: Long = receiverNumRecords.map(_._2).sum +} + +private[ui] object BatchUIData { + + def apply(batchInfo: BatchInfo): BatchUIData = { + new BatchUIData( + batchInfo.batchTime, + batchInfo.streamIdToNumRecords, + batchInfo.submissionTime, + batchInfo.processingStartTime, + batchInfo.processingEndTime + ) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 5ee53a5c5f561..24cbb2bf9d8fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -17,28 +17,58 @@ package org.apache.spark.streaming.ui +import java.util.LinkedHashMap +import java.util.{Map => JMap} +import java.util.Properties + +import scala.collection.mutable.{ArrayBuffer, Queue, HashMap, SynchronizedBuffer} + +import org.apache.spark.scheduler._ import org.apache.spark.streaming.{Time, StreamingContext} import org.apache.spark.streaming.scheduler._ -import scala.collection.mutable.{Queue, HashMap} import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted -import org.apache.spark.streaming.scheduler.BatchInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) - extends StreamingListener { + extends StreamingListener with SparkListener { - private val waitingBatchInfos = new HashMap[Time, BatchInfo] - private val runningBatchInfos = new HashMap[Time, BatchInfo] - private val completedaBatchInfos = new Queue[BatchInfo] - private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + private val waitingBatchUIData = new HashMap[Time, BatchUIData] + private val runningBatchUIData = new HashMap[Time, BatchUIData] + private val completedBatchUIData = new Queue[BatchUIData] + private val batchUIDataLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L private var totalReceivedRecords = 0L private var totalProcessedRecords = 0L private val receiverInfos = new HashMap[Int, ReceiverInfo] + // Because onJobStart and onBatchXXX messages are processed in different threads, + // we may not be able to get the corresponding BatchUIData when receiving onJobStart. So here we + // cannot use a map of (Time, BatchUIData). + private[ui] val batchTimeToOutputOpIdSparkJobIdPair = + new LinkedHashMap[Time, SynchronizedBuffer[OutputOpIdAndSparkJobId]] { + override def removeEldestEntry( + p1: JMap.Entry[Time, SynchronizedBuffer[OutputOpIdAndSparkJobId]]): Boolean = { + // If a lot of "onBatchCompleted"s happen before "onJobStart" (image if + // SparkContext.listenerBus is very slow), "batchTimeToOutputOpIdToSparkJobIds" + // may add some information for a removed batch when processing "onJobStart". It will be a + // memory leak. + // + // To avoid the memory leak, we control the size of "batchTimeToOutputOpIdToSparkJobIds" and + // evict the eldest one. + // + // Note: if "onJobStart" happens before "onBatchSubmitted", the size of + // "batchTimeToOutputOpIdToSparkJobIds" may be greater than the number of the retained + // batches temporarily, so here we use "10" to handle such case. This is not a perfect + // solution, but at least it can handle most of cases. + size() > + waitingBatchUIData.size + runningBatchUIData.size + completedBatchUIData.size + 10 + } + } + + val batchDuration = ssc.graph.batchDuration.milliseconds override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { @@ -59,35 +89,64 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } - override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { - runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + synchronized { + waitingBatchUIData(batchSubmitted.batchInfo.batchTime) = + BatchUIData(batchSubmitted.batchInfo) + } + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = synchronized { + val batchUIData = BatchUIData(batchStarted.batchInfo) + runningBatchUIData(batchStarted.batchInfo.batchTime) = BatchUIData(batchStarted.batchInfo) + waitingBatchUIData.remove(batchStarted.batchInfo.batchTime) + + totalReceivedRecords += batchUIData.numRecords } - override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { - runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo - waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + synchronized { + waitingBatchUIData.remove(batchCompleted.batchInfo.batchTime) + runningBatchUIData.remove(batchCompleted.batchInfo.batchTime) + val batchUIData = BatchUIData(batchCompleted.batchInfo) + completedBatchUIData.enqueue(batchUIData) + if (completedBatchUIData.size > batchUIDataLimit) { + val removedBatch = completedBatchUIData.dequeue() + batchTimeToOutputOpIdSparkJobIdPair.remove(removedBatch.batchTime) + } + totalCompletedBatches += 1L - batchStarted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => - totalReceivedRecords += infos.map(_.numRecords).sum + totalProcessedRecords += batchUIData.numRecords } } - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { - waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) - runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) - completedaBatchInfos.enqueue(batchCompleted.batchInfo) - if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() - totalCompletedBatches += 1L - - batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => - totalProcessedRecords += infos.map(_.numRecords).sum + override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { + getBatchTimeAndOutputOpId(jobStart.properties).foreach { case (batchTime, outputOpId) => + var outputOpIdToSparkJobIds = batchTimeToOutputOpIdSparkJobIdPair.get(batchTime) + if (outputOpIdToSparkJobIds == null) { + outputOpIdToSparkJobIds = + new ArrayBuffer[OutputOpIdAndSparkJobId]() + with SynchronizedBuffer[OutputOpIdAndSparkJobId] + batchTimeToOutputOpIdSparkJobIdPair.put(batchTime, outputOpIdToSparkJobIds) + } + outputOpIdToSparkJobIds += OutputOpIdAndSparkJobId(outputOpId, jobStart.jobId) } } - def numReceivers = synchronized { - ssc.graph.getReceiverInputStreams().size + private def getBatchTimeAndOutputOpId(properties: Properties): Option[(Time, Int)] = { + val batchTime = properties.getProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY) + if (batchTime == null) { + // Not submitted from JobScheduler + None + } else { + val outputOpId = properties.getProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY) + assert(outputOpId != null) + Some(Time(batchTime.toLong) -> outputOpId.toInt) + } } + def numReceivers: Int = ssc.graph.getReceiverInputStreams().size + def numTotalCompletedBatches: Long = synchronized { totalCompletedBatches } @@ -101,19 +160,19 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def numUnprocessedBatches: Long = synchronized { - waitingBatchInfos.size + runningBatchInfos.size + waitingBatchUIData.size + runningBatchUIData.size } - def waitingBatches: Seq[BatchInfo] = synchronized { - waitingBatchInfos.values.toSeq + def waitingBatches: Seq[BatchUIData] = synchronized { + waitingBatchUIData.values.toSeq } - def runningBatches: Seq[BatchInfo] = synchronized { - runningBatchInfos.values.toSeq + def runningBatches: Seq[BatchUIData] = synchronized { + runningBatchUIData.values.toSeq } - def retainedCompletedBatches: Seq[BatchInfo] = synchronized { - completedaBatchInfos.toSeq + def retainedCompletedBatches: Seq[BatchUIData] = synchronized { + completedBatchUIData.toSeq } def processingDelayDistribution: Option[Distribution] = synchronized { @@ -129,50 +188,70 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { - val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) - val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) - (0 until numReceivers).map { receiverId => - val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => - batchInfo.get(receiverId).getOrElse(Array.empty) - } - val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map { blockInfo => - // calculate records per second for each batch - blockInfo.map(_.numRecords).sum.toDouble * 1000 / batchDuration - } - val distributionOption = Distribution(recordsOfParticularReceiver) - (receiverId, distributionOption) + val latestBatchInfos = retainedBatches.reverse.take(batchUIDataLimit) + val latestReceiverNumRecords = latestBatchInfos.map(_.receiverNumRecords) + val streamIds = ssc.graph.getInputStreams().map(_.id) + streamIds.map { id => + val recordsOfParticularReceiver = + latestReceiverNumRecords.map(v => v.getOrElse(id, 0L).toDouble * 1000 / batchDuration) + val distribution = Distribution(recordsOfParticularReceiver) + (id, distribution) }.toMap } - def lastReceivedBatchRecords: Map[Int, Long] = { - val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) - lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => - (0 until numReceivers).map { receiverId => - (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) + def lastReceivedBatchRecords: Map[Int, Long] = synchronized { + val lastReceiverNumRecords = lastReceivedBatch.map(_.receiverNumRecords) + val streamIds = ssc.graph.getInputStreams().map(_.id) + lastReceiverNumRecords.map { receiverNumRecords => + streamIds.map { id => + (id, receiverNumRecords.getOrElse(id, 0L)) }.toMap }.getOrElse { - (0 until numReceivers).map(receiverId => (receiverId, 0L)).toMap + streamIds.map(id => (id, 0L)).toMap } } - def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = synchronized { receiverInfos.get(receiverId) } - def lastCompletedBatch: Option[BatchInfo] = { - completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + def receiverIds(): Iterable[Int] = synchronized { + receiverInfos.keys + } + + def lastCompletedBatch: Option[BatchUIData] = synchronized { + completedBatchUIData.sortBy(_.batchTime)(Time.ordering).lastOption } - def lastReceivedBatch: Option[BatchInfo] = { + def lastReceivedBatch: Option[BatchUIData] = synchronized { retainedBatches.lastOption } - private def retainedBatches: Seq[BatchInfo] = synchronized { - (waitingBatchInfos.values.toSeq ++ - runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + private def retainedBatches: Seq[BatchUIData] = { + (waitingBatchUIData.values.toSeq ++ + runningBatchUIData.values.toSeq ++ completedBatchUIData).sortBy(_.batchTime)(Time.ordering) } - private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { - Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + private def extractDistribution(getMetric: BatchUIData => Option[Long]): Option[Distribution] = { + Distribution(completedBatchUIData.flatMap(getMetric(_)).map(_.toDouble)) } + + def getBatchUIData(batchTime: Time): Option[BatchUIData] = synchronized { + val batchUIData = waitingBatchUIData.get(batchTime).orElse { + runningBatchUIData.get(batchTime).orElse { + completedBatchUIData.find(batch => batch.batchTime == batchTime) + } + } + batchUIData.foreach { _batchUIData => + val outputOpIdToSparkJobIds = + Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime)).getOrElse(Seq.empty) + _batchUIData.outputOpIdSparkJobIdPairs = outputOpIdToSparkJobIds + } + batchUIData + } +} + +private[streaming] object StreamingJobProgressListener { + type SparkJobId = Int + type OutputOpId = Int } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index bfe8086fcf8fe..db37ae815bdf5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -37,20 +37,23 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = + val content = listener.synchronized { generateBasicStats() ++

    ++

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateReceiverStats() ++ - generateBatchStatsTable() + generateBatchStatsTable() ++ + generateBatchListTables() + } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime + // scalastyle:off
    • - Started at: {startTime.toString} + Started at: {UIUtils.formatDate(startTime)}
    • Time since start: {formatDurationVerbose(timeSinceStart)} @@ -62,18 +65,19 @@ private[ui] class StreamingPage(parent: StreamingTab) Batch interval: {formatDurationVerbose(listener.batchDuration)}
    • - Processed batches: {listener.numTotalCompletedBatches} + Completed batches: {listener.numTotalCompletedBatches}
    • - Waiting batches: {listener.numUnprocessedBatches} + Active batches: {listener.numUnprocessedBatches}
    • - Received records: {listener.numTotalReceivedRecords} + Received events: {listener.numTotalReceivedRecords}
    • - Processed records: {listener.numTotalProcessedRecords} + Processed events: {listener.numTotalProcessedRecords}
    + // scalastyle:on } /** Generate stats of data received by the receivers in the streaming program */ @@ -85,13 +89,13 @@ private[ui] class StreamingPage(parent: StreamingTab) "Receiver", "Status", "Location", - "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", - "Minimum rate\n[records/sec]", - "Median rate\n[records/sec]", - "Maximum rate\n[records/sec]", + "Events in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[events/sec]", + "Median rate\n[events/sec]", + "Maximum rate\n[events/sec]", "Last Error" ) - val dataRows = (0 until listener.numReceivers).map { receiverId => + val dataRows = listener.receiverIds().map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) val receiverName = receiverInfo.map(_.name).getOrElse(s"Receiver-$receiverId") val receiverActive = receiverInfo.map { info => @@ -110,7 +114,7 @@ private[ui] class StreamingPage(parent: StreamingTab) }.getOrElse(emptyCell) Seq(receiverName, receiverActive, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats ++ Seq(receiverLastError) - } + }.toSeq Some(listingTable(headerRow, dataRows)) } else { None @@ -189,5 +193,26 @@ private[ui] class StreamingPage(parent: StreamingTab) } UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } + + private def generateBatchListTables(): Seq[Node] = { + val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse + val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse + val completedBatches = listener.retainedCompletedBatches. + sortBy(_.batchTime.milliseconds).reverse + + val activeBatchesContent = { +

    Active Batches ({runningBatches.size + waitingBatches.size})

    ++ + new ActiveBatchTable(runningBatches, waitingBatches).toNodeSeq + } + + val completedBatchesContent = { +

    + Completed Batches (last {completedBatches.size} out of {listener.numTotalCompletedBatches}) +

    ++ + new CompletedBatchTable(completedBatches).toNodeSeq + } + + activeBatchesContent ++ completedBatchesContent + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 9a860ea4a6c68..f307b54bb9630 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -27,15 +27,20 @@ import StreamingTab._ * Spark Web UI tab that shows statistics of a streaming job. * This assumes the given SparkContext has enabled its SparkUI. */ -private[spark] class StreamingTab(ssc: StreamingContext) +private[spark] class StreamingTab(val ssc: StreamingContext) extends SparkUITab(getSparkUI(ssc), "streaming") with Logging { val parent = getSparkUI(ssc) val listener = ssc.progressListener ssc.addStreamingListener(listener) + ssc.sc.addSparkListener(listener) attachPage(new StreamingPage(this)) - parent.attachTab(this) + attachPage(new BatchPage(this)) + + def attach() { + getSparkUI(ssc).attachTab(this) + } def detach() { getSparkUI(ssc).detachTab(this) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala similarity index 78% rename from streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index 6bdfe45dc7f83..9985fedc35141 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer +import java.util.{Iterator => JIterator} import scala.collection.mutable.ArrayBuffer import scala.concurrent.{Await, ExecutionContext, Future} @@ -24,9 +25,9 @@ import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.Logging -import org.apache.spark.util.{Clock, SystemClock, Utils} -import WriteAheadLogManager._ + +import org.apache.spark.util.ThreadUtils +import org.apache.spark.{Logging, SparkConf} /** * This class manages write ahead log files. @@ -34,37 +35,32 @@ import WriteAheadLogManager._ * - Recovers the log files and the reads the recovered records upon failures. * - Cleans up old log files. * - * Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write - * and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read. + * Uses [[org.apache.spark.streaming.util.FileBasedWriteAheadLogWriter]] to write + * and [[org.apache.spark.streaming.util.FileBasedWriteAheadLogReader]] to read. * * @param logDirectory Directory when rotating log files will be created. * @param hadoopConf Hadoop configuration for reading/writing log files. - * @param rollingIntervalSecs The interval in seconds with which logs will be rolled over. - * Default is one minute. - * @param maxFailures Max number of failures that is tolerated for every attempt to write to log. - * Default is three. - * @param callerName Optional name of the class who is using this manager. - * @param clock Optional clock that is used to check for rotation interval. */ -private[streaming] class WriteAheadLogManager( +private[streaming] class FileBasedWriteAheadLog( + conf: SparkConf, logDirectory: String, hadoopConf: Configuration, - rollingIntervalSecs: Int = 60, - maxFailures: Int = 3, - callerName: String = "", - clock: Clock = new SystemClock - ) extends Logging { + rollingIntervalSecs: Int, + maxFailures: Int + ) extends WriteAheadLog with Logging { + + import FileBasedWriteAheadLog._ private val pastLogs = new ArrayBuffer[LogInfo] - private val callerNameTag = - if (callerName.nonEmpty) s" for $callerName" else "" + private val callerNameTag = getCallerName.map(c => s" for $c").getOrElse("") + private val threadpoolName = s"WriteAheadLogManager $callerNameTag" implicit private val executionContext = ExecutionContext.fromExecutorService( - Utils.newDaemonFixedThreadPool(1, threadpoolName)) + ThreadUtils.newDaemonSingleThreadExecutor(threadpoolName)) override protected val logName = s"WriteAheadLogManager $callerNameTag" private var currentLogPath: Option[String] = None - private var currentLogWriter: WriteAheadLogWriter = null + private var currentLogWriter: FileBasedWriteAheadLogWriter = null private var currentLogWriterStartTime: Long = -1L private var currentLogWriterStopTime: Long = -1L @@ -75,14 +71,14 @@ private[streaming] class WriteAheadLogManager( * ByteBuffer to HDFS. When this method returns, the data is guaranteed to have been flushed * to HDFS, and will be available for readers to read. */ - def writeToLog(byteBuffer: ByteBuffer): WriteAheadLogFileSegment = synchronized { - var fileSegment: WriteAheadLogFileSegment = null + def write(byteBuffer: ByteBuffer, time: Long): FileBasedWriteAheadLogSegment = synchronized { + var fileSegment: FileBasedWriteAheadLogSegment = null var failures = 0 var lastException: Exception = null var succeeded = false while (!succeeded && failures < maxFailures) { try { - fileSegment = getLogWriter(clock.getTimeMillis()).write(byteBuffer) + fileSegment = getLogWriter(time).write(byteBuffer) succeeded = true } catch { case ex: Exception => @@ -99,6 +95,19 @@ private[streaming] class WriteAheadLogManager( fileSegment } + def read(segment: WriteAheadLogRecordHandle): ByteBuffer = { + val fileSegment = segment.asInstanceOf[FileBasedWriteAheadLogSegment] + var reader: FileBasedWriteAheadLogRandomReader = null + var byteBuffer: ByteBuffer = null + try { + reader = new FileBasedWriteAheadLogRandomReader(fileSegment.path, hadoopConf) + byteBuffer = reader.read(fileSegment) + } finally { + reader.close() + } + byteBuffer + } + /** * Read all the existing logs from the log directory. * @@ -108,12 +117,14 @@ private[streaming] class WriteAheadLogManager( * the latest the records. This does not deal with currently active log files, and * hence the implementation is kept simple. */ - def readFromLog(): Iterator[ByteBuffer] = synchronized { + def readAll(): JIterator[ByteBuffer] = synchronized { + import scala.collection.JavaConversions._ val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) + logFilesToRead.iterator.map { file => logDebug(s"Creating log reader with $file") - new WriteAheadLogReader(file, hadoopConf) + new FileBasedWriteAheadLogReader(file, hadoopConf) } flatMap { x => x } } @@ -129,7 +140,7 @@ private[streaming] class WriteAheadLogManager( * deleted. This should be set to true only for testing. Else the files will be deleted * asynchronously. */ - def cleanupOldLogs(threshTime: Long, waitForCompletion: Boolean): Unit = { + def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") @@ -160,7 +171,7 @@ private[streaming] class WriteAheadLogManager( /** Stop the manager, close any open log writer */ - def stop(): Unit = synchronized { + def close(): Unit = synchronized { if (currentLogWriter != null) { currentLogWriter.close() } @@ -169,7 +180,7 @@ private[streaming] class WriteAheadLogManager( } /** Get the current log writer while taking care of rotation */ - private def getLogWriter(currentTime: Long): WriteAheadLogWriter = synchronized { + private def getLogWriter(currentTime: Long): FileBasedWriteAheadLogWriter = synchronized { if (currentLogWriter == null || currentTime > currentLogWriterStopTime) { resetWriter() currentLogPath.foreach { @@ -180,7 +191,7 @@ private[streaming] class WriteAheadLogManager( val newLogPath = new Path(logDirectory, timeToLogFile(currentLogWriterStartTime, currentLogWriterStopTime)) currentLogPath = Some(newLogPath.toString) - currentLogWriter = new WriteAheadLogWriter(currentLogPath.get, hadoopConf) + currentLogWriter = new FileBasedWriteAheadLogWriter(currentLogPath.get, hadoopConf) } currentLogWriter } @@ -207,7 +218,7 @@ private[streaming] class WriteAheadLogManager( } } -private[util] object WriteAheadLogManager { +private[streaming] object FileBasedWriteAheadLog { case class LogInfo(startTime: Long, endTime: Long, path: String) @@ -217,6 +228,11 @@ private[util] object WriteAheadLogManager { s"log-$startTime-$stopTime" } + def getCallerName(): Option[String] = { + val stackTraceClasses = Thread.currentThread.getStackTrace().map(_.getClassName) + stackTraceClasses.find(!_.contains("WriteAheadLog")).flatMap(_.split(".").lastOption) + } + /** Convert a sequence of files to a sequence of sorted LogInfo objects */ def logFilesTologInfo(files: Seq[Path]): Seq[LogInfo] = { files.flatMap { file => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala similarity index 83% rename from streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala index 003989092a42a..f7168229ec15a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala @@ -23,16 +23,16 @@ import org.apache.hadoop.conf.Configuration /** * A random access reader for reading write ahead log files written using - * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. Given the file segment info, - * this reads the record (bytebuffer) from the log file. + * [[org.apache.spark.streaming.util.FileBasedWriteAheadLogWriter]]. Given the file segment info, + * this reads the record (ByteBuffer) from the log file. */ -private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configuration) +private[streaming] class FileBasedWriteAheadLogRandomReader(path: String, conf: Configuration) extends Closeable { private val instream = HdfsUtils.getInputStream(path, conf) private var closed = false - def read(segment: WriteAheadLogFileSegment): ByteBuffer = synchronized { + def read(segment: FileBasedWriteAheadLogSegment): ByteBuffer = synchronized { assertOpen() instream.seek(segment.offset) val nextLength = instream.readInt() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala similarity index 93% rename from streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala index 2afc0d1551acf..c3bb59f3fef94 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala @@ -24,11 +24,11 @@ import org.apache.spark.Logging /** * A reader for reading write ahead log files written using - * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. This reads + * [[org.apache.spark.streaming.util.FileBasedWriteAheadLogWriter]]. This reads * the records (bytebuffers) in the log file sequentially and return them as an * iterator of bytebuffers. */ -private[streaming] class WriteAheadLogReader(path: String, conf: Configuration) +private[streaming] class FileBasedWriteAheadLogReader(path: String, conf: Configuration) extends Iterator[ByteBuffer] with Closeable with Logging { private val instream = HdfsUtils.getInputStream(path, conf) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogSegment.scala similarity index 86% rename from streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogSegment.scala index 1005a2c8ec303..2e1f1528fad20 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogSegment.scala @@ -17,4 +17,5 @@ package org.apache.spark.streaming.util /** Class for representing a segment of data in a write ahead log file */ -private[streaming] case class WriteAheadLogFileSegment (path: String, offset: Long, length: Int) +private[streaming] case class FileBasedWriteAheadLogSegment(path: String, offset: Long, length: Int) + extends WriteAheadLogRecordHandle diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala similarity index 88% rename from streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala index 679f6a6dfd7c1..e146bec32a456 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala @@ -17,18 +17,17 @@ package org.apache.spark.streaming.util import java.io._ -import java.net.URI import java.nio.ByteBuffer import scala.util.Try import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem} +import org.apache.hadoop.fs.FSDataOutputStream /** * A writer for writing byte-buffers to a write ahead log file. */ -private[streaming] class WriteAheadLogWriter(path: String, hadoopConf: Configuration) +private[streaming] class FileBasedWriteAheadLogWriter(path: String, hadoopConf: Configuration) extends Closeable { private lazy val stream = HdfsUtils.getOutputStream(path, hadoopConf) @@ -43,11 +42,11 @@ private[streaming] class WriteAheadLogWriter(path: String, hadoopConf: Configura private var closed = false /** Write the bytebuffer to the log file */ - def write(data: ByteBuffer): WriteAheadLogFileSegment = synchronized { + def write(data: ByteBuffer): FileBasedWriteAheadLogSegment = synchronized { assertOpen() data.rewind() // Rewind to ensure all data in the buffer is retrieved val lengthToWrite = data.remaining() - val segment = new WriteAheadLogFileSegment(path, nextOffset, lengthToWrite) + val segment = new FileBasedWriteAheadLogSegment(path, nextOffset, lengthToWrite) stream.writeInt(lengthToWrite) if (data.hasArray) { stream.write(data.array()) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 858ba3c9eb4e5..f60688f173c44 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -27,7 +27,7 @@ private[streaming] object HdfsUtils { // If the file exists and we have append support, append instead of creating a new file val stream: FSDataOutputStream = { if (dfs.isFile(dfsPath)) { - if (conf.getBoolean("hdfs.append.support", false)) { + if (conf.getBoolean("hdfs.append.support", false) || dfs.isInstanceOf[RawLocalFileSystem]) { dfs.append(dfsPath) } else { throw new IllegalStateException("File exists and there is no append support!") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index a73d6f3bf0661..4d968f8bfa7a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -18,9 +18,7 @@ package org.apache.spark.streaming.util import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.util.collection.OpenHashMap -import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { @@ -71,7 +69,7 @@ object RawTextHelper { var count = 0 while(data.hasNext) { - value = data.next + value = data.next() if (value != null) { count += 1 if (len == 0) { @@ -108,9 +106,13 @@ object RawTextHelper { } } - def add(v1: Long, v2: Long) = (v1 + v2) + def add(v1: Long, v2: Long): Long = { + v1 + v2 + } - def subtract(v1: Long, v2: Long) = (v1 - v2) + def subtract(v1: Long, v2: Long): Long = { + v1 - v2 + } - def max(v1: Long, v2: Long) = math.max(v1, v2) + def max(v1: Long, v2: Long): Long = math.max(v1, v2) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index a7850812bd612..ca2f319f174a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -72,7 +72,8 @@ object RawTextSender extends Logging { } catch { case e: IOException => logError("Client disconnected") - socket.close() + } finally { + socket.close() } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala new file mode 100644 index 0000000000000..7f6ff12c58d47 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -0,0 +1,129 @@ +/* + * 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.streaming.util + +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf, SparkException} + +/** A helper class with utility functions related to the WriteAheadLog interface */ +private[streaming] object WriteAheadLogUtils extends Logging { + val RECEIVER_WAL_ENABLE_CONF_KEY = "spark.streaming.receiver.writeAheadLog.enable" + val RECEIVER_WAL_CLASS_CONF_KEY = "spark.streaming.receiver.writeAheadLog.class" + val RECEIVER_WAL_ROLLING_INTERVAL_CONF_KEY = + "spark.streaming.receiver.writeAheadLog.rollingIntervalSecs" + val RECEIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.receiver.writeAheadLog.maxFailures" + + val DRIVER_WAL_CLASS_CONF_KEY = "spark.streaming.driver.writeAheadLog.class" + val DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY = + "spark.streaming.driver.writeAheadLog.rollingIntervalSecs" + val DRIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.driver.writeAheadLog.maxFailures" + + val DEFAULT_ROLLING_INTERVAL_SECS = 60 + val DEFAULT_MAX_FAILURES = 3 + + def enableReceiverLog(conf: SparkConf): Boolean = { + conf.getBoolean(RECEIVER_WAL_ENABLE_CONF_KEY, false) + } + + def getRollingIntervalSecs(conf: SparkConf, isDriver: Boolean): Int = { + if (isDriver) { + conf.getInt(DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY, DEFAULT_ROLLING_INTERVAL_SECS) + } else { + conf.getInt(RECEIVER_WAL_ROLLING_INTERVAL_CONF_KEY, DEFAULT_ROLLING_INTERVAL_SECS) + } + } + + def getMaxFailures(conf: SparkConf, isDriver: Boolean): Int = { + if (isDriver) { + conf.getInt(DRIVER_WAL_MAX_FAILURES_CONF_KEY, DEFAULT_MAX_FAILURES) + } else { + conf.getInt(RECEIVER_WAL_MAX_FAILURES_CONF_KEY, DEFAULT_MAX_FAILURES) + } + } + + /** + * Create a WriteAheadLog for the driver. If configured with custom WAL class, it will try + * to create instance of that class, otherwise it will create the default FileBasedWriteAheadLog. + */ + def createLogForDriver( + sparkConf: SparkConf, + fileWalLogDirectory: String, + fileWalHadoopConf: Configuration + ): WriteAheadLog = { + createLog(true, sparkConf, fileWalLogDirectory, fileWalHadoopConf) + } + + /** + * Create a WriteAheadLog for the receiver. If configured with custom WAL class, it will try + * to create instance of that class, otherwise it will create the default FileBasedWriteAheadLog. + */ + def createLogForReceiver( + sparkConf: SparkConf, + fileWalLogDirectory: String, + fileWalHadoopConf: Configuration + ): WriteAheadLog = { + createLog(false, sparkConf, fileWalLogDirectory, fileWalHadoopConf) + } + + /** + * Create a WriteAheadLog based on the value of the given config key. The config key is used + * to get the class name from the SparkConf. If the class is configured, it will try to + * create instance of that class by first trying `new CustomWAL(sparkConf, logDir)` then trying + * `new CustomWAL(sparkConf)`. If either fails, it will fail. If no class is configured, then + * it will create the default FileBasedWriteAheadLog. + */ + private def createLog( + isDriver: Boolean, + sparkConf: SparkConf, + fileWalLogDirectory: String, + fileWalHadoopConf: Configuration + ): WriteAheadLog = { + + val classNameOption = if (isDriver) { + sparkConf.getOption(DRIVER_WAL_CLASS_CONF_KEY) + } else { + sparkConf.getOption(RECEIVER_WAL_CLASS_CONF_KEY) + } + classNameOption.map { className => + try { + instantiateClass( + Utils.classForName(className).asInstanceOf[Class[_ <: WriteAheadLog]], sparkConf) + } catch { + case NonFatal(e) => + throw new SparkException(s"Could not create a write ahead log of class $className", e) + } + }.getOrElse { + new FileBasedWriteAheadLog(sparkConf, fileWalLogDirectory, fileWalHadoopConf, + getRollingIntervalSecs(sparkConf, isDriver), getMaxFailures(sparkConf, isDriver)) + } + } + + /** Instantiate the class, either using single arg constructor or zero arg constructor */ + private def instantiateClass(cls: Class[_ <: WriteAheadLog], conf: SparkConf): WriteAheadLog = { + try { + cls.getConstructor(classOf[SparkConf]).newInstance(conf) + } catch { + case nsme: NoSuchMethodException => + cls.getConstructor().newInstance() + } + } +} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 90340753a4eed..b1adf881dd0f5 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -21,11 +21,13 @@ import java.lang.Iterable; import java.nio.charset.Charset; import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; + import scala.Tuple2; import org.junit.Assert; @@ -45,6 +47,7 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.*; import org.apache.spark.util.Utils; +import org.apache.spark.SparkConf; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -929,7 +932,7 @@ public void testPairMap() { // Maps pair -> pair of different type public Tuple2 call(Tuple2 in) throws Exception { return in.swap(); } - }); + }); JavaTestUtils.attachTestOutputStream(reversed); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -987,12 +990,12 @@ public void testPairMap2() { // Maps pair -> single JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaDStream reversed = pairStream.map( - new Function, Integer>() { - @Override - public Integer call(Tuple2 in) throws Exception { - return in._2(); - } - }); + new Function, Integer>() { + @Override + public Integer call(Tuple2 in) throws Exception { + return in._2(); + } + }); JavaTestUtils.attachTestOutputStream(reversed); List> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1123,7 +1126,7 @@ public void testCombineByKey() { JavaPairDStream combined = pairStream.combineByKey( new Function() { - @Override + @Override public Integer call(Integer i) throws Exception { return i; } @@ -1144,14 +1147,14 @@ public void testCountByValue() { Arrays.asList("hello")); List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2("hello", 1L), - new Tuple2("world", 1L)), - Arrays.asList( - new Tuple2("hello", 1L), - new Tuple2("moon", 1L)), - Arrays.asList( - new Tuple2("hello", 1L))); + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("world", 1L)), + Arrays.asList( + new Tuple2("hello", 1L), + new Tuple2("moon", 1L)), + Arrays.asList( + new Tuple2("hello", 1L))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream counted = stream.countByValue(); @@ -1249,17 +1252,17 @@ public void testUpdateStateByKey() { JavaPairDStream updated = pairStream.updateStateByKey( new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out = out + state.get(); - } - for (Integer v: values) { - out = out + v; + @Override + public Optional call(List values, Optional state) { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); } - return Optional.of(out); - } }); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1292,17 +1295,17 @@ public void testUpdateStateByKeyWithInitial() { JavaPairDStream updated = pairStream.updateStateByKey( new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out = out + state.get(); - } - for (Integer v: values) { - out = out + v; + @Override + public Optional call(List values, Optional state) { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); } - return Optional.of(out); - } }, new HashPartitioner(1), initialRDD); JavaTestUtils.attachTestOutputStream(updated); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1328,7 +1331,7 @@ public void testReduceByKeyAndWindowWithInverse() { JavaPairDStream reduceWindowed = pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), - new Duration(2000), new Duration(1000)); + new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reduceWindowed); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1707,6 +1710,74 @@ public Integer call(String s) throws Exception { Utils.deleteRecursively(tempDir); } + @SuppressWarnings("unchecked") + @Test + public void testContextGetOrCreate() throws InterruptedException { + + final SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("newContext", "true"); + + File emptyDir = Files.createTempDir(); + emptyDir.deleteOnExit(); + StreamingContextSuite contextSuite = new StreamingContextSuite(); + String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint(); + String checkpointDir = contextSuite.createValidCheckpoint(); + + // Function to create JavaStreamingContext without any output operations + // (used to detect the new context) + final AtomicBoolean newContextCreated = new AtomicBoolean(false); + Function0 creatingFunc = new Function0() { + public JavaStreamingContext call() { + newContextCreated.set(true); + return new JavaStreamingContext(conf, Seconds.apply(1)); + } + }; + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc, + new org.apache.hadoop.conf.Configuration(), true); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, + new org.apache.hadoop.conf.Configuration()); + Assert.assertTrue("old context not recovered", !newContextCreated.get()); + ssc.stop(); + + // Function to create JavaStreamingContext using existing JavaSparkContext + // without any output operations (used to detect the new context) + Function creatingFunc2 = + new Function() { + public JavaStreamingContext call(JavaSparkContext context) { + newContextCreated.set(true); + return new JavaStreamingContext(context, Seconds.apply(1)); + } + }; + + JavaSparkContext sc = new JavaSparkContext(conf); + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc2, sc); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(false); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc2, sc, true); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(false); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc2, sc); + Assert.assertTrue("old context not recovered", !newContextCreated.get()); + ssc.stop(); + } /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD @SuppressWarnings("unchecked") diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java new file mode 100644 index 0000000000000..50e8f9fc159c8 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java @@ -0,0 +1,129 @@ +/* + * 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.streaming; + +import java.util.ArrayList; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.Transformer; +import org.apache.spark.SparkConf; +import org.apache.spark.streaming.util.WriteAheadLog; +import org.apache.spark.streaming.util.WriteAheadLogRecordHandle; +import org.apache.spark.streaming.util.WriteAheadLogUtils; + +import org.junit.Test; +import org.junit.Assert; + +class JavaWriteAheadLogSuiteHandle extends WriteAheadLogRecordHandle { + int index = -1; + public JavaWriteAheadLogSuiteHandle(int idx) { + index = idx; + } +} + +public class JavaWriteAheadLogSuite extends WriteAheadLog { + + class Record { + long time; + int index; + ByteBuffer buffer; + + public Record(long tym, int idx, ByteBuffer buf) { + index = idx; + time = tym; + buffer = buf; + } + } + private int index = -1; + private ArrayList records = new ArrayList(); + + + // Methods for WriteAheadLog + @Override + public WriteAheadLogRecordHandle write(java.nio.ByteBuffer record, long time) { + index += 1; + records.add(new org.apache.spark.streaming.JavaWriteAheadLogSuite.Record(time, index, record)); + return new JavaWriteAheadLogSuiteHandle(index); + } + + @Override + public java.nio.ByteBuffer read(WriteAheadLogRecordHandle handle) { + if (handle instanceof JavaWriteAheadLogSuiteHandle) { + int reqdIndex = ((JavaWriteAheadLogSuiteHandle) handle).index; + for (Record record: records) { + if (record.index == reqdIndex) { + return record.buffer; + } + } + } + return null; + } + + @Override + public java.util.Iterator readAll() { + Collection buffers = CollectionUtils.collect(records, new Transformer() { + @Override + public Object transform(Object input) { + return ((Record) input).buffer; + } + }); + return buffers.iterator(); + } + + @Override + public void clean(long threshTime, boolean waitForCompletion) { + for (int i = 0; i < records.size(); i++) { + if (records.get(i).time < threshTime) { + records.remove(i); + i--; + } + } + } + + @Override + public void close() { + records.clear(); + } + + @Test + public void testCustomWAL() { + SparkConf conf = new SparkConf(); + conf.set("spark.streaming.driver.writeAheadLog.class", JavaWriteAheadLogSuite.class.getName()); + WriteAheadLog wal = WriteAheadLogUtils.createLogForDriver(conf, null, null); + + String data1 = "data1"; + WriteAheadLogRecordHandle handle = wal.write(ByteBuffer.wrap(data1.getBytes()), 1234); + Assert.assertTrue(handle instanceof JavaWriteAheadLogSuiteHandle); + Assert.assertTrue(new String(wal.read(handle).array()).equals(data1)); + + wal.write(ByteBuffer.wrap("data2".getBytes()), 1235); + wal.write(ByteBuffer.wrap("data3".getBytes()), 1236); + wal.write(ByteBuffer.wrap("data4".getBytes()), 1237); + wal.clean(1236, false); + + java.util.Iterator dataIterator = wal.readAll(); + ArrayList readData = new ArrayList(); + while (dataIterator.hasNext()) { + readData.add(new String(dataIterator.next().array())); + } + Assert.assertTrue(readData.equals(Arrays.asList("data3", "data4"))); + } +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index cf191715d29d6..87bc20f79c3cd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -171,7 +171,9 @@ class BasicOperationsSuite extends TestSuiteBase { test("flatMapValues") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)), + (s: DStream[String]) => { + s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)) + }, Seq( Seq(("a", 2), ("a", 12), ("b", 1), ("b", 11)), Seq(("", 2), ("", 12)), Seq() ), true ) @@ -474,7 +476,7 @@ class BasicOperationsSuite extends TestSuiteBase { stream.foreachRDD(_ => {}) // Dummy output stream ssc.start() Thread.sleep(2000) - def getInputFromSlice(fromMillis: Long, toMillis: Long) = { + def getInputFromSlice(fromMillis: Long, toMillis: Long): Set[Int] = { stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 91a2b2bba461d..6b0a3f91d4d06 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -43,7 +43,7 @@ class CheckpointSuite extends TestSuiteBase { var ssc: StreamingContext = null - override def batchDuration = Milliseconds(500) + override def batchDuration: Duration = Milliseconds(500) override def beforeFunction() { super.beforeFunction() @@ -72,7 +72,7 @@ class CheckpointSuite extends TestSuiteBase { val input = (1 to 10).map(_ => Seq("a")).toSeq val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Int], state: Option[Int]) => { - Some((values.sum + state.getOrElse(0))) + Some(values.sum + state.getOrElse(0)) } st.map(x => (x, 1)) .updateStateByKey(updateFunc) @@ -199,7 +199,12 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ), (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _), - Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), Seq() ), 3 ) } @@ -212,7 +217,8 @@ class CheckpointSuite extends TestSuiteBase { val n = 10 val w = 4 val input = (1 to n).map(_ => Seq("a")).toSeq - val output = Seq(Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4))) + val output = Seq( + Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4))) val operation = (st: DStream[String]) => { st.map(x => (x, 1)) .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration) @@ -236,7 +242,13 @@ class CheckpointSuite extends TestSuiteBase { classOf[TextOutputFormat[Text, IntWritable]]) output }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -259,7 +271,13 @@ class CheckpointSuite extends TestSuiteBase { classOf[NewTextOutputFormat[Text, IntWritable]]) output }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -298,7 +316,13 @@ class CheckpointSuite extends TestSuiteBase { output } }, - Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + Seq( + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq(), + Seq(("a", 2), ("b", 1)), + Seq(("", 2)), + Seq()), 3 ) } finally { @@ -406,9 +430,8 @@ class CheckpointSuite extends TestSuiteBase { assert(recordedFiles(ssc) === Seq(1, 2, 3) && batchCounter.getNumStartedBatches === 3) } // Wait for a checkpoint to be written - val fs = new Path(checkpointDir).getFileSystem(ssc.sc.hadoopConfiguration) eventually(eventuallyTimeout) { - assert(Checkpoint.getCheckpointFiles(checkpointDir, fs).size === 6) + assert(Checkpoint.getCheckpointFiles(checkpointDir).size === 6) } ssc.stop() // Check that we shut down while the third batch was being processed @@ -533,7 +556,8 @@ class CheckpointSuite extends TestSuiteBase { * Advances the manual clock on the streaming scheduler by given number of batches. * It also waits for the expected amount of time for each batch. */ - def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = { + def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = + { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] logInfo("Manual clock before advancing = " + clock.getTimeMillis()) for (i <- 1 to numBatches.toInt) { @@ -543,7 +567,7 @@ class CheckpointSuite extends TestSuiteBase { logInfo("Manual clock after advancing = " + clock.getTimeMillis()) Thread.sleep(batchDuration.milliseconds) - val outputStream = ssc.graph.getOutputStreams.filter { dstream => + val outputStream = ssc.graph.getOutputStreams().filter { dstream => dstream.isInstanceOf[TestOutputStreamWithPartitions[V]] }.head.asInstanceOf[TestOutputStreamWithPartitions[V]] outputStream.output.map(_.flatten) @@ -552,4 +576,4 @@ class CheckpointSuite extends TestSuiteBase { private object CheckpointSuite extends Serializable { var batchThreeShouldBlockIndefinitely: Boolean = true -} \ No newline at end of file +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 26435d8515815..0c4c06534a693 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -29,9 +29,9 @@ class FailureSuite extends TestSuiteBase with Logging { val directory = Utils.createTempDir() val numBatches = 30 - override def batchDuration = Milliseconds(1000) + override def batchDuration: Duration = Milliseconds(1000) - override def useManualClock = false + override def useManualClock: Boolean = false override def afterFunction() { Utils.deleteRecursively(directory) 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 7ed6320a3d0bc..60745325029f6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -18,75 +18,83 @@ package org.apache.spark.streaming import java.io.{File, BufferedWriter, OutputStreamWriter} -import java.net.{SocketException, ServerSocket} +import java.net.{Socket, SocketException, ServerSocket} import java.nio.charset.Charset -import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} +import java.util.concurrent.{CountDownLatch, Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} import scala.language.postfixOps import com.google.common.io.Files +import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListener} import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.rdd.RDD -import org.apache.hadoop.io.{Text, LongWritable} -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat -import org.apache.hadoop.fs.Path class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { test("socket input stream") { - // Start the server - val testServer = new TestServer() - testServer.start() - - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.socketTextStream( - "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(networkStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) - outputStream.register() - ssc.start() + withTestServer(new TestServer()) { testServer => + // Start the server + testServer.start() - // Feed data to the server to send to the network receiver - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = Seq(1, 2, 3, 4, 5) - val expectedOutput = input.map(_.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - testServer.send(input(i).toString + "\n") - Thread.sleep(500) - clock.advance(batchDuration.milliseconds) - } - Thread.sleep(1000) - logInfo("Stopping server") - testServer.stop() - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received was as expected - logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) - logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output.size = " + expectedOutput.size) - logInfo("expected output") - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val input = Seq(1, 2, 3, 4, 5) + // Use "batchCount" to make sure we check the result after all batches finish + val batchCounter = new BatchCounter(ssc) + val networkStream = ssc.socketTextStream( + "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(networkStream, outputBuffer) + outputStream.register() + ssc.start() - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val expectedOutput = input.map(_.toString) + for (i <- 0 until input.size) { + testServer.send(input(i).toString + "\n") + Thread.sleep(500) + clock.advance(batchDuration.milliseconds) + } + // Make sure we finish all batches before "stop" + if (!batchCounter.waitUntilBatchesCompleted(input.size, 30000)) { + fail("Timeout: cannot finish all batches in 30 seconds") + } + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + ssc.stop() + + // Verify whether data received was as expected + logInfo("--------------------------------") + logInfo("output.size = " + outputBuffer.size) + logInfo("output") + outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output.size = " + expectedOutput.size) + logInfo("expected output") + expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + // (whether the elements were received one in each interval is not verified) + val output: ArrayBuffer[String] = outputBuffer.flatMap(x => x) + assert(output.size === expectedOutput.size) + for (i <- 0 until output.size) { + assert(output(i) === expectedOutput(i)) + } + } } } @@ -164,7 +172,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] val outputStream = new TestOutputStream(countStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) + def output: ArrayBuffer[Long] = outputBuffer.flatMap(x => x) outputStream.register() ssc.start() @@ -196,7 +204,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val queueStream = ssc.queueStream(queue, oneAtATime = true) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output = outputBuffer.filter(_.size > 0) + def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) outputStream.register() ssc.start() @@ -204,7 +212,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq("1", "2", "3", "4", "5") val expectedOutput = input.map(Seq(_)) - //Thread.sleep(1000) + val inputIterator = input.toIterator for (i <- 0 until input.size) { // Enqueue more than 1 item per tick but they should dequeue one at a time @@ -239,7 +247,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val queueStream = ssc.queueStream(queue, oneAtATime = false) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output = outputBuffer.filter(_.size > 0) + def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) outputStream.register() ssc.start() @@ -278,6 +286,30 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } + test("test track the number of input stream") { + val ssc = new StreamingContext(conf, batchDuration) + + class TestInputDStream extends InputDStream[String](ssc) { + def start() { } + def stop() { } + def compute(validTime: Time): Option[RDD[String]] = None + } + + class TestReceiverInputDStream extends ReceiverInputDStream[String](ssc) { + def getReceiver: Receiver[String] = null + } + + // Register input streams + val receiverInputStreams = Array(new TestReceiverInputDStream, new TestReceiverInputDStream) + val inputStreams = Array(new TestInputDStream, new TestInputDStream, new TestInputDStream) + + assert(ssc.graph.getInputStreams().length == receiverInputStreams.length + inputStreams.length) + assert(ssc.graph.getReceiverInputStreams().length == receiverInputStreams.length) + assert(ssc.graph.getReceiverInputStreams() === receiverInputStreams) + assert(ssc.graph.getInputStreams().map(_.id) === Array.tabulate(5)(i => i)) + assert(receiverInputStreams.map(_.id) === Array(0, 1)) + } + def testFileStream(newFilesOnly: Boolean) { val testDir: File = null try { @@ -343,30 +375,45 @@ class TestServer(portToBind: Int = 0) extends Logging { val serverSocket = new ServerSocket(portToBind) + private val startLatch = new CountDownLatch(1) + val servingThread = new Thread() { override def run() { try { while(true) { logInfo("Accepting connections on port " + port) val clientSocket = serverSocket.accept() - logInfo("New connection") - try { - clientSocket.setTcpNoDelay(true) - val outputStream = new BufferedWriter(new OutputStreamWriter(clientSocket.getOutputStream)) - - while(clientSocket.isConnected) { - val msg = queue.poll(100, TimeUnit.MILLISECONDS) - if (msg != null) { - outputStream.write(msg) - outputStream.flush() - logInfo("Message '" + msg + "' sent") + if (startLatch.getCount == 1) { + // The first connection is a test connection to implement "waitForStart", so skip it + // and send a signal + if (!clientSocket.isClosed) { + clientSocket.close() + } + startLatch.countDown() + } else { + // Real connections + logInfo("New connection") + try { + clientSocket.setTcpNoDelay(true) + val outputStream = new BufferedWriter( + new OutputStreamWriter(clientSocket.getOutputStream)) + + while (clientSocket.isConnected) { + val msg = queue.poll(100, TimeUnit.MILLISECONDS) + if (msg != null) { + outputStream.write(msg) + outputStream.flush() + logInfo("Message '" + msg + "' sent") + } + } + } catch { + case e: SocketException => logError("TestServer error", e) + } finally { + logInfo("Connection closed") + if (!clientSocket.isClosed) { + clientSocket.close() } } - } catch { - case e: SocketException => logError("TestServer error", e) - } finally { - logInfo("Connection closed") - if (!clientSocket.isClosed) clientSocket.close() } } } catch { @@ -378,13 +425,35 @@ class TestServer(portToBind: Int = 0) extends Logging { } } - def start() { servingThread.start() } + def start(): Unit = { + servingThread.start() + if (!waitForStart(10000)) { + stop() + throw new AssertionError("Timeout: TestServer cannot start in 10 seconds") + } + } + + /** + * Wait until the server starts. Return true if the server starts in "millis" milliseconds. + * Otherwise, return false to indicate it's timeout. + */ + private def waitForStart(millis: Long): Boolean = { + // We will create a test connection to the server so that we can make sure it has started. + val socket = new Socket("localhost", port) + try { + startLatch.await(millis, TimeUnit.MILLISECONDS) + } finally { + if (!socket.isClosed) { + socket.close() + } + } + } def send(msg: String) { queue.put(msg) } def stop() { servingThread.interrupt() } - def port = serverSocket.getLocalPort + def port: Int = serverSocket.getLocalPort } /** This is a receiver to test multiple threads inserting data using block generator */ 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 18a477f92094d..23804237bda80 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -24,26 +24,26 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps -import akka.actor.{ActorSystem, Props} import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util._ -import org.apache.spark.util.{AkkaUtils, ManualClock, Utils} +import org.apache.spark.util.{ManualClock, Utils} import WriteAheadLogBasedBlockHandler._ import WriteAheadLogSuite._ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { - val conf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + val conf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1") val hadoopConf = new Configuration() val storageLevel = StorageLevel.MEMORY_ONLY_SER val streamId = 1 @@ -54,22 +54,19 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche val manualClock = new ManualClock val blockManagerSize = 10000000 - var actorSystem: ActorSystem = null + var rpcEnv: RpcEnv = null var blockManagerMaster: BlockManagerMaster = null var blockManager: BlockManager = null var tempDirectory: File = null before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "test", "localhost", 0, conf = conf, securityManager = securityMgr) - this.actorSystem = actorSystem - conf.set("spark.driver.port", boundPort.toString) + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) + conf.set("spark.driver.port", rpcEnv.address.port.toString) - blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) + blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", + new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) - blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, + blockManager = new BlockManager("bm", rpcEnv, blockManagerMaster, serializer, blockManagerSize, conf, mapOutputTracker, shuffleManager, new NioBlockTransferService(conf, securityMgr), securityMgr, 0) blockManager.initialize("app-id") @@ -87,9 +84,9 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche blockManagerMaster.stop() blockManagerMaster = null } - actorSystem.shutdown() - actorSystem.awaitTermination() - actorSystem = null + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null Utils.deleteRecursively(tempDirectory) } @@ -99,7 +96,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data @@ -123,7 +120,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) }.toList storedData shouldEqual data @@ -133,10 +130,13 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche "Unexpected store result type" ) // Verify the data in write ahead log files is correct - val fileSegments = storeResults.map { _.asInstanceOf[WriteAheadLogBasedStoreResult].segment} - val loggedData = fileSegments.flatMap { segment => - val reader = new WriteAheadLogRandomReader(segment.path, hadoopConf) - val bytes = reader.read(segment) + val walSegments = storeResults.map { result => + result.asInstanceOf[WriteAheadLogBasedStoreResult].walRecordHandle + } + val loggedData = walSegments.flatMap { walSegment => + val fileSegment = walSegment.asInstanceOf[FileBasedWriteAheadLogSegment] + val reader = new FileBasedWriteAheadLogRandomReader(fileSegment.path, hadoopConf) + val bytes = reader.read(fileSegment) reader.close() blockManager.dataDeserialize(generateBlockId(), bytes).toList } @@ -151,13 +151,13 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche } } - test("WriteAheadLogBasedBlockHandler - cleanup old blocks") { + test("WriteAheadLogBasedBlockHandler - clean old blocks") { withWriteAheadLogBasedBlockHandler { handler => val blocks = Seq.tabulate(10) { i => IteratorBlock(Iterator(1 to i)) } storeBlocks(handler, blocks) val preCleanupLogFiles = getWriteAheadLogFiles() - preCleanupLogFiles.size should be > 1 + require(preCleanupLogFiles.size > 1) // this depends on the number of blocks inserted using generateAndStoreData() manualClock.getTimeMillis() shouldEqual 5000L @@ -221,6 +221,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche /** Instantiate a WriteAheadLogBasedBlockHandler and run a code with it */ private def withWriteAheadLogBasedBlockHandler(body: WriteAheadLogBasedBlockHandler => Unit) { + require(WriteAheadLogUtils.getRollingIntervalSecs(conf, isDriver = false) === 1) val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, 1, storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock) try { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 42fad769f0c1a..b1af8d5eaacfb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.util.WriteAheadLogReader +import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadLogReader} import org.apache.spark.streaming.util.WriteAheadLogSuite._ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} @@ -59,7 +59,7 @@ class ReceivedBlockTrackerSuite test("block addition, and block to batch allocation") { val receivedBlockTracker = createTracker(setCheckpointDir = false) - receivedBlockTracker.isLogManagerEnabled should be (false) // should be disable by default + receivedBlockTracker.isWriteAheadLogEnabled should be (false) // should be disable by default receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty val blockInfos = generateBlockInfos() @@ -67,15 +67,20 @@ class ReceivedBlockTrackerSuite // Verify added blocks are unallocated blocks receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos + receivedBlockTracker.hasUnallocatedReceivedBlocks should be (true) + // Allocate the blocks to a batch and verify that all of them have been allocated receivedBlockTracker.allocateBlocksToBatch(1) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos + receivedBlockTracker.getBlocksOfBatch(1) shouldEqual Map(streamId -> blockInfos) receivedBlockTracker.getUnallocatedBlocks(streamId) shouldBe empty + receivedBlockTracker.hasUnallocatedReceivedBlocks should be (false) // Allocate no blocks to another batch receivedBlockTracker.allocateBlocksToBatch(2) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty + receivedBlockTracker.getBlocksOfBatch(2) shouldEqual Map(streamId -> Seq.empty) // Verify that older batches have no operation on batch allocation, // will return the same blocks as previously allocated. @@ -88,7 +93,7 @@ class ReceivedBlockTrackerSuite receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos } - test("block addition, block to batch allocation and cleanup with write ahead log") { + test("recovery and cleanup with write ahead logs") { val manualClock = new ManualClock // Set the time increment level to twice the rotation interval so that every increment creates // a new log file @@ -113,11 +118,13 @@ class ReceivedBlockTrackerSuite logInfo(s"\n\n=====================\n$message\n$fileContents\n=====================\n") } + // Set WAL configuration + conf.set("spark.streaming.driver.writeAheadLog.rollingIntervalSecs", "1") + require(WriteAheadLogUtils.getRollingIntervalSecs(conf, isDriver = true) === 1) + // Start tracker and add blocks - conf.set("spark.streaming.receiver.writeAheadLog.enable", "true") - conf.set("spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", "1") val tracker1 = createTracker(clock = manualClock) - tracker1.isLogManagerEnabled should be (true) + tracker1.isWriteAheadLogEnabled should be (true) val blockInfos1 = addBlockInfos(tracker1) tracker1.getUnallocatedBlocks(streamId).toList shouldEqual blockInfos1 @@ -127,15 +134,27 @@ class ReceivedBlockTrackerSuite getWrittenLogData() shouldEqual expectedWrittenData1 getWriteAheadLogFiles() should have size 1 - // Restart tracker and verify recovered list of unallocated blocks incrementTime() - val tracker2 = createTracker(clock = manualClock) - tracker2.getUnallocatedBlocks(streamId).toList shouldEqual blockInfos1 + + // Recovery without recovery from WAL and verify list of unallocated blocks is empty + val tracker1_ = createTracker(clock = manualClock, recoverFromWriteAheadLog = false) + tracker1_.getUnallocatedBlocks(streamId) shouldBe empty + tracker1_.hasUnallocatedReceivedBlocks should be (false) + + // Restart tracker and verify recovered list of unallocated blocks + val tracker2 = createTracker(clock = manualClock, recoverFromWriteAheadLog = true) + val unallocatedBlocks = tracker2.getUnallocatedBlocks(streamId).toList + unallocatedBlocks shouldEqual blockInfos1 + unallocatedBlocks.foreach { block => + block.isBlockIdValid() should be (false) + } + // Allocate blocks to batch and verify whether the unallocated blocks got allocated val batchTime1 = manualClock.getTimeMillis() tracker2.allocateBlocksToBatch(batchTime1) tracker2.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 + tracker2.getBlocksOfBatch(batchTime1) shouldEqual Map(streamId -> blockInfos1) // Add more blocks and allocate to another batch incrementTime() @@ -153,7 +172,7 @@ class ReceivedBlockTrackerSuite // Restart tracker and verify recovered state incrementTime() - val tracker3 = createTracker(clock = manualClock) + val tracker3 = createTracker(clock = manualClock, recoverFromWriteAheadLog = true) tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 tracker3.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 tracker3.getUnallocatedBlocks(streamId) shouldBe empty @@ -171,51 +190,41 @@ class ReceivedBlockTrackerSuite eventually(timeout(10 seconds), interval(10 millisecond)) { getWriteAheadLogFiles() should not contain oldestLogFile } - printLogFiles("After cleanup") + printLogFiles("After clean") // Restart tracker and verify recovered state, specifically whether info about the first // batch has been removed, but not the second batch incrementTime() - val tracker4 = createTracker(clock = manualClock) + val tracker4 = createTracker(clock = manualClock, recoverFromWriteAheadLog = true) tracker4.getUnallocatedBlocks(streamId) shouldBe empty tracker4.getBlocksOfBatchAndStream(batchTime1, streamId) shouldBe empty // should be cleaned tracker4.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 } - test("enabling write ahead log but not setting checkpoint dir") { - conf.set("spark.streaming.receiver.writeAheadLog.enable", "true") - intercept[SparkException] { - createTracker(setCheckpointDir = false) - } - } - - test("setting checkpoint dir but not enabling write ahead log") { - // When WAL config is not set, log manager should not be enabled - val tracker1 = createTracker(setCheckpointDir = true) - tracker1.isLogManagerEnabled should be (false) - - // When WAL is explicitly disabled, log manager should not be enabled - conf.set("spark.streaming.receiver.writeAheadLog.enable", "false") - val tracker2 = createTracker(setCheckpointDir = true) - tracker2.isLogManagerEnabled should be(false) + test("disable write ahead log when checkpoint directory is not set") { + // When checkpoint is disabled, then the write ahead log is disabled + val tracker1 = createTracker(setCheckpointDir = false) + tracker1.isWriteAheadLogEnabled should be (false) } /** * Create tracker object with the optional provided clock. Use fake clock if you - * want to control time by manually incrementing it to test log cleanup. + * want to control time by manually incrementing it to test log clean. */ def createTracker( setCheckpointDir: Boolean = true, + recoverFromWriteAheadLog: Boolean = false, clock: Clock = new SystemClock): ReceivedBlockTracker = { val cpDirOption = if (setCheckpointDir) Some(checkpointDirectory.toString) else None - val tracker = new ReceivedBlockTracker(conf, hadoopConf, Seq(streamId), clock, cpDirOption) + val tracker = new ReceivedBlockTracker( + conf, hadoopConf, Seq(streamId), clock, recoverFromWriteAheadLog, cpDirOption) allReceivedBlockTrackers += tracker tracker } /** Generate blocks infos using random ids */ def generateBlockInfos(): Seq[ReceivedBlockInfo] = { - List.fill(5)(ReceivedBlockInfo(streamId, 0, + List.fill(5)(ReceivedBlockInfo(streamId, 0, None, BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt))))) } @@ -228,9 +237,10 @@ class ReceivedBlockTrackerSuite * Get all the data written in the given write ahead log files. By default, it will read all * files in the test log directory. */ - def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles): Seq[ReceivedBlockTrackerLogEvent] = { + def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles) + : Seq[ReceivedBlockTrackerLogEvent] = { logFiles.flatMap { - file => new WriteAheadLogReader(file, hadoopConf).toSeq + file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq }.map { byteBuffer => Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) }.toList @@ -244,11 +254,12 @@ class ReceivedBlockTrackerSuite } /** Create batch allocation object from the given info */ - def createBatchAllocation(time: Long, blockInfos: Seq[ReceivedBlockInfo]): BatchAllocationEvent = { + def createBatchAllocation(time: Long, blockInfos: Seq[ReceivedBlockInfo]) + : BatchAllocationEvent = { BatchAllocationEvent(time, AllocatedBlocks(Map((streamId -> blockInfos)))) } - /** Create batch cleanup object from the given info */ + /** Create batch clean object from the given info */ def createBatchCleanup(time: Long, moreTimes: Long*): BatchCleanupEvent = { BatchCleanupEvent((Seq(time) ++ moreTimes).map(Time.apply)) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index aa20ad0b5374e..5d7127627eea5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,11 +131,11 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockIntervalMs = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 - val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) val generatedData = new ArrayBuffer[Int] // Generate blocks @@ -155,17 +155,17 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { assert(recordedData.toSet === generatedData.toSet) } - test("block generator throttling") { + ignore("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 100 - val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + val blockIntervalMs = 100 + val maxRate = 1001 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 - val waitTime = expectedBlocks * blockInterval + val waitTime = expectedBlocks * blockIntervalMs val expectedMessages = maxRate * waitTime / 1000 - val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000 val generatedData = new ArrayBuffer[Int] // Generate blocks @@ -176,7 +176,6 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { blockGenerator.addData(count) generatedData += count count += 1 - Thread.sleep(1) } blockGenerator.stop() @@ -185,25 +184,31 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received") assert(recordedData.toSet === generatedData.toSet, "Received data not same") - // recordedData size should be close to the expected rate - val minExpectedMessages = expectedMessages - 3 - val maxExpectedMessages = expectedMessages + 1 + // recordedData size should be close to the expected rate; use an error margin proportional to + // the value, so that rate changes don't cause a brittle test + val minExpectedMessages = expectedMessages - 0.05 * expectedMessages + val maxExpectedMessages = expectedMessages + 0.05 * expectedMessages val numMessages = recordedData.size assert( numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages, s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages" ) - val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 3 - val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 1 + // XXX Checking every block would require an even distribution of messages across blocks, + // which throttling code does not control. Therefore, test against the average. + val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 0.05 * expectedMessagesPerBlock + val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 0.05 * expectedMessagesPerBlock val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",") + + // the first and last block may be incomplete, so we slice them out + val validBlocks = recordedBlocks.drop(1).dropRight(1) + val averageBlockSize = validBlocks.map(block => block.size).sum / validBlocks.size + assert( - // the first and last block may be incomplete, so we slice them out - recordedBlocks.drop(1).dropRight(1).forall { block => - block.size >= minExpectedMessagesPerBlock && block.size <= maxExpectedMessagesPerBlock - }, + averageBlockSize >= minExpectedMessagesPerBlock && + averageBlockSize <= maxExpectedMessagesPerBlock, s"# records in received blocks = [$receivedBlockSizes], not between " + - s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock" + s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock, on average" ) } @@ -220,7 +225,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { .setAppName(framework) .set("spark.ui.enabled", "true") .set("spark.streaming.receiver.writeAheadLog.enable", "true") - .set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + .set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1") val batchDuration = Milliseconds(500) val tempDirectory = Utils.createTempDir() val logDirectory1 = new File(checkpointDirToLogDir(tempDirectory.getAbsolutePath, 0)) @@ -251,8 +256,8 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { } withStreamingContext(new StreamingContext(sparkConf, batchDuration)) { ssc => - val receiver1 = ssc.sparkContext.clean(new FakeReceiver(sendData = true)) - val receiver2 = ssc.sparkContext.clean(new FakeReceiver(sendData = true)) + val receiver1 = new FakeReceiver(sendData = true) + val receiver2 = new FakeReceiver(sendData = true) val receiverStream1 = ssc.receiverStream(receiver1) val receiverStream2 = ssc.receiverStream(receiver2) receiverStream1.register() @@ -308,7 +313,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { val errors = new ArrayBuffer[Throwable] /** Check if all data structures are clean */ - def isAllEmpty = { + def isAllEmpty: Boolean = { singles.isEmpty && byteBuffers.isEmpty && iterators.isEmpty && arrayBuffers.isEmpty && errors.isEmpty } @@ -320,24 +325,21 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { def pushBytes( bytes: ByteBuffer, optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { byteBuffers += bytes } def pushIterator( iterator: Iterator[_], optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { iterators += iterator } def pushArrayBuffer( arrayBuffer: ArrayBuffer[_], optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] - ) { + optionalBlockId: Option[StreamBlockId]) { arrayBuffers += arrayBuffer } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 2e5005ef6ff14..5207b7109e69b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -17,8 +17,10 @@ package org.apache.spark.streaming +import java.io.File import java.util.concurrent.atomic.AtomicInteger +import org.apache.commons.io.FileUtils import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ @@ -73,9 +75,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -85,24 +87,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") + assert( + Utils.timeStringAsSeconds(cp.sparkConfPairs + .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { @@ -176,7 +180,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600") + conf.set("spark.cleaner.ttl", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -207,13 +211,13 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000") + conf.set("spark.streaming.gracefulStopTimeout", "20000s") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 SlowTestReceiver.receivedAllRecords = false - //Create test receiver that sleeps in onStop() + // Create test receiver that sleeps in onStop() val totalNumRecords = 15 val recordsPerSecond = 1 val input = ssc.receiverStream(new SlowTestReceiver(totalNumRecords, recordsPerSecond)) @@ -328,6 +332,139 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } + test("getOrCreate") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + + // Function to create StreamingContext that has a config to identify it to be new context + var newContextCreated = false + def creatingFunction(): StreamingContext = { + newContextCreated = true + new StreamingContext(conf, batchDuration) + } + + // Call ssc.stop after a body of code + def testGetOrCreate(body: => Unit): Unit = { + newContextCreated = false + try { + body + } finally { + if (ssc != null) { + ssc.stop() + } + ssc = null + } + } + + val emptyPath = Utils.createTempDir().getAbsolutePath() + + // getOrCreate should create new context with empty path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(emptyPath, creatingFunction _) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + } + + val corrutedCheckpointPath = createCorruptedCheckpoint() + + // getOrCreate should throw exception with fake checkpoint file and createOnError = false + intercept[Exception] { + ssc = StreamingContext.getOrCreate(corrutedCheckpointPath, creatingFunction _) + } + + // getOrCreate should throw exception with fake checkpoint file + intercept[Exception] { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, createOnError = false) + } + + // getOrCreate should create new context with fake checkpoint file and createOnError = true + testGetOrCreate { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + } + + val checkpointPath = createValidCheckpoint() + + // getOrCreate should recover context with checkpoint path, and recover old configuration + testGetOrCreate { + ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _) + assert(ssc != null, "no context created") + assert(!newContextCreated, "old context not recovered") + assert(ssc.conf.get("someKey") === "someValue") + } + } + + test("getOrCreate with existing SparkContext") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + sc = new SparkContext(conf) + + // Function to create StreamingContext that has a config to identify it to be new context + var newContextCreated = false + def creatingFunction(sparkContext: SparkContext): StreamingContext = { + newContextCreated = true + new StreamingContext(sparkContext, batchDuration) + } + + // Call ssc.stop(stopSparkContext = false) after a body of cody + def testGetOrCreate(body: => Unit): Unit = { + newContextCreated = false + try { + body + } finally { + if (ssc != null) { + ssc.stop(stopSparkContext = false) + } + ssc = null + } + } + + val emptyPath = Utils.createTempDir().getAbsolutePath() + + // getOrCreate should create new context with empty path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(emptyPath, creatingFunction _, sc, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + } + + val corrutedCheckpointPath = createCorruptedCheckpoint() + + // getOrCreate should throw exception with fake checkpoint file and createOnError = false + intercept[Exception] { + ssc = StreamingContext.getOrCreate(corrutedCheckpointPath, creatingFunction _, sc) + } + + // getOrCreate should throw exception with fake checkpoint file + intercept[Exception] { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, sc, createOnError = false) + } + + // getOrCreate should create new context with fake checkpoint file and createOnError = true + testGetOrCreate { + ssc = StreamingContext.getOrCreate( + corrutedCheckpointPath, creatingFunction _, sc, createOnError = true) + assert(ssc != null, "no context created") + assert(newContextCreated, "new context not created") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + } + + val checkpointPath = createValidCheckpoint() + + // StreamingContext.getOrCreate should recover context with checkpoint path + testGetOrCreate { + ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _, sc) + assert(ssc != null, "no context created") + assert(!newContextCreated, "old context not recovered") + assert(ssc.sparkContext === sc, "new StreamingContext does not use existing SparkContext") + assert(!ssc.conf.contains("someKey"), + "recovered StreamingContext unexpectedly has old config") + } + } + test("DStream and generated RDD creation sites") { testPackage.test() } @@ -337,6 +474,30 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val inputStream = new TestInputStream(s, input, 1) inputStream } + + def createValidCheckpoint(): String = { + val testDirectory = Utils.createTempDir().getAbsolutePath() + val checkpointDirectory = Utils.createTempDir().getAbsolutePath() + val conf = new SparkConf().setMaster(master).setAppName(appName) + conf.set("someKey", "someValue") + ssc = new StreamingContext(conf, batchDuration) + ssc.checkpoint(checkpointDirectory) + ssc.textFileStream(testDirectory).foreachRDD { rdd => rdd.count() } + ssc.start() + eventually(timeout(10000 millis)) { + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + } + ssc.stop() + checkpointDirectory + } + + def createCorruptedCheckpoint(): String = { + val checkpointDirectory = Utils.createTempDir().getAbsolutePath() + val fakeCheckpointFile = Checkpoint.checkpointFile(checkpointDirectory, Time(1000)) + FileUtils.write(new File(fakeCheckpointFile.toString()), "blablabla") + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).nonEmpty) + checkpointDirectory + } } class TestException(msg: String) extends Exception(msg) @@ -361,7 +522,7 @@ class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging } def onStop() { - // no cleanup to be done, the receiving thread should stop on it own + // no clean to be done, the receiving thread should stop on it own } } @@ -370,7 +531,8 @@ object TestReceiver { } /** Custom receiver for testing whether a slow receiver can be shutdown gracefully or not */ -class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { +class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) + extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { var receivingThreadOption: Option[Thread] = None @@ -393,7 +555,7 @@ class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) extends Receive def onStop() { // Simulate slow receiver by waiting for all records to be produced while(!SlowTestReceiver.receivedAllRecords) Thread.sleep(100) - // no cleanup to be done, the receiving thread should stop on it own + // no clean to be done, the receiving thread should stop on it own } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index f52562b0a0f73..312cce408cfe7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global @@ -38,18 +38,56 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { // To make sure that the processing start and end times in collected // information are different for successive batches - override def batchDuration = Milliseconds(100) - override def actuallyWait = true + override def batchDuration: Duration = Milliseconds(100) + override def actuallyWait: Boolean = true test("batch info reporting") { val ssc = setupStreams(input, operation) val collector = new BatchInfoCollector ssc.addStreamingListener(collector) runStreams(ssc, input.size, input.size) - val batchInfos = collector.batchInfos - batchInfos should have size 4 - batchInfos.foreach(info => { + // SPARK-6766: batch info should be submitted + val batchInfosSubmitted = collector.batchInfosSubmitted + batchInfosSubmitted should have size 4 + + batchInfosSubmitted.foreach(info => { + info.schedulingDelay should be (None) + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + batchInfosSubmitted.foreach { info => + info.numRecords should be (1L) + info.streamIdToNumRecords should be (Map(0 -> 1L)) + } + + isInIncreasingOrder(batchInfosSubmitted.map(_.submissionTime)) should be (true) + + // SPARK-6766: processingStartTime of batch info should not be None when starting + val batchInfosStarted = collector.batchInfosStarted + batchInfosStarted should have size 4 + + batchInfosStarted.foreach(info => { + info.schedulingDelay should not be None + info.schedulingDelay.get should be >= 0L + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + batchInfosStarted.foreach { info => + info.numRecords should be (1L) + info.streamIdToNumRecords should be (Map(0 -> 1L)) + } + + isInIncreasingOrder(batchInfosStarted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosStarted.map(_.processingStartTime.get)) should be (true) + + // test onBatchCompleted + val batchInfosCompleted = collector.batchInfosCompleted + batchInfosCompleted should have size 4 + + batchInfosCompleted.foreach(info => { info.schedulingDelay should not be None info.processingDelay should not be None info.totalDelay should not be None @@ -58,9 +96,14 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { info.totalDelay.get should be >= 0L }) - isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) + batchInfosCompleted.foreach { info => + info.numRecords should be (1L) + info.streamIdToNumRecords should be (Map(0 -> 1L)) + } + + isInIncreasingOrder(batchInfosCompleted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingEndTime.get)) should be (true) } test("receiver info reporting") { @@ -99,17 +142,29 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { /** Listener that collects information on processed batches */ class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] + val batchInfosCompleted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] + val batchInfosStarted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] + val batchInfosSubmitted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { + batchInfosSubmitted += batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { + batchInfosStarted += batchStarted.batchInfo + } + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo + batchInfosCompleted += batchCompleted.batchInfo } } /** Listener that collects information on processed batches */ class ReceiverInfoCollector extends StreamingListener { - val startedReceiverStreamIds = new ArrayBuffer[Int] - val stoppedReceiverStreamIds = new ArrayBuffer[Int]() - val receiverErrors = new ArrayBuffer[(Int, String, String)]() + val startedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] + val stoppedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] + val receiverErrors = + new ArrayBuffer[(Int, String, String)] with SynchronizedBuffer[(Int, String, String)] override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { startedReceiverStreamIds += receiverStarted.receiverInfo.streamId diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 3565d621e8a6c..4d0cd7516f42e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -29,10 +29,10 @@ import org.scalatest.time.{Span, Seconds => ScalaTestSeconds} import org.scalatest.concurrent.Eventually.timeout import org.scalatest.concurrent.PatienceConfiguration -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} -import org.apache.spark.streaming.scheduler.{StreamingListenerBatchStarted, StreamingListenerBatchCompleted, StreamingListener} import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ManualClock, Utils} /** @@ -53,8 +53,13 @@ class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], val selectedInput = if (index < input.size) input(index) else Seq[T]() // lets us test cases where RDDs are not created - if (selectedInput == null) + if (selectedInput == null) { return None + } + + // Report the input data's information to InputInfoTracker for testing + val inputInfo = InputInfo(id, selectedInput.length.toLong) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) val rdd = ssc.sc.makeRDD(selectedInput, numPartitions) logInfo("Created RDD " + rdd.id + " with " + selectedInput) @@ -104,7 +109,9 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], output.clear() } - def toTestOutputStream = new TestOutputStream[T](this.parent, this.output.map(_.flatten)) + def toTestOutputStream: TestOutputStream[T] = { + new TestOutputStream[T](this.parent, this.output.map(_.flatten)) + } } /** @@ -139,6 +146,40 @@ class BatchCounter(ssc: StreamingContext) { def getNumStartedBatches: Int = this.synchronized { numStartedBatches } + + /** + * Wait until `expectedNumCompletedBatches` batches are completed, or timeout. Return true if + * `expectedNumCompletedBatches` batches are completed. Otherwise, return false to indicate it's + * timeout. + * + * @param expectedNumCompletedBatches the `expectedNumCompletedBatches` batches to wait + * @param timeout the maximum time to wait in milliseconds. + */ + def waitUntilBatchesCompleted(expectedNumCompletedBatches: Int, timeout: Long): Boolean = + waitUntilConditionBecomeTrue(numCompletedBatches >= expectedNumCompletedBatches, timeout) + + /** + * Wait until `expectedNumStartedBatches` batches are completed, or timeout. Return true if + * `expectedNumStartedBatches` batches are completed. Otherwise, return false to indicate it's + * timeout. + * + * @param expectedNumStartedBatches the `expectedNumStartedBatches` batches to wait + * @param timeout the maximum time to wait in milliseconds. + */ + def waitUntilBatchesStarted(expectedNumStartedBatches: Int, timeout: Long): Boolean = + waitUntilConditionBecomeTrue(numStartedBatches >= expectedNumStartedBatches, timeout) + + private def waitUntilConditionBecomeTrue(condition: => Boolean, timeout: Long): Boolean = { + synchronized { + var now = System.currentTimeMillis() + val timeoutTick = now + timeout + while (!condition && timeoutTick > now) { + wait(timeoutTick - now) + now = System.currentTimeMillis() + } + condition + } + } } /** @@ -148,34 +189,34 @@ class BatchCounter(ssc: StreamingContext) { trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Name of the framework for Spark context - def framework = this.getClass.getSimpleName + def framework: String = this.getClass.getSimpleName // Master for Spark context - def master = "local[2]" + def master: String = "local[2]" // Batch duration - def batchDuration = Seconds(1) + def batchDuration: Duration = Seconds(1) // Directory where the checkpoint data will be saved - lazy val checkpointDir = { + lazy val checkpointDir: String = { val dir = Utils.createTempDir() logDebug(s"checkpointDir: $dir") dir.toString } // Number of partitions of the input parallel collections created for testing - def numInputPartitions = 2 + def numInputPartitions: Int = 2 // Maximum time to wait before the test times out - def maxWaitTimeMillis = 10000 + def maxWaitTimeMillis: Int = 10000 // Whether to use manual clock or not - def useManualClock = true + def useManualClock: Boolean = true // Whether to actually wait in real time before changing manual clock - def actuallyWait = false + def actuallyWait: Boolean = false - //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things. + // A SparkConf to use in tests. Can be modified before calling setupStreams to configure things. val conf = new SparkConf() .setMaster(master) .setAppName(framework) @@ -346,7 +387,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Wait until expected number of output items have been generated val startTime = System.currentTimeMillis() - while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + while (output.size < numExpectedOutput && + System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput) ssc.awaitTerminationOrTimeout(50) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 87a0395efbf2a..8de43baabc21d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming +import scala.collection.mutable.Queue + import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ @@ -32,7 +34,8 @@ import org.apache.spark._ /** * Selenium tests for the Spark Web UI. */ -class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { +class UISeleniumSuite + extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll with TestSuiteBase { implicit var webDriver: WebDriver = _ @@ -59,8 +62,28 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before ssc } + private def setupStreams(ssc: StreamingContext): Unit = { + val rdds = Queue(ssc.sc.parallelize(1 to 4, 4)) + val inputStream = ssc.queueStream(rdds) + inputStream.foreachRDD { rdd => + rdd.foreach(_ => {}) + rdd.foreach(_ => {}) + } + inputStream.foreachRDD { rdd => + rdd.foreach(_ => {}) + try { + rdd.foreach(_ => throw new RuntimeException("Oops")) + } catch { + case e: SparkException if e.getMessage.contains("Oops") => + } + } + } + test("attaching and detaching a Streaming tab") { withStreamingContext(newSparkStreamingContext()) { ssc => + setupStreams(ssc) + ssc.start() + val sparkUI = ssc.sparkContext.ui.get eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -74,6 +97,74 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq statisticText should contain("Network receivers:") statisticText should contain("Batch interval:") + + val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq + h4Text.exists(_.matches("Active Batches \\(\\d+\\)")) should be (true) + h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true) + + findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status") + } + findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay") + } + + val batchLinks = + findAll(cssSelector("""#completed-batches-table a""")).flatMap(_.attribute("href")).toSeq + batchLinks.size should be >= 1 + + // Check a normal batch page + go to (batchLinks.last) // Last should be the first batch, so it will have some jobs + val summaryText = findAll(cssSelector("li strong")).map(_.text).toSeq + summaryText should contain ("Batch Duration:") + summaryText should contain ("Input data size:") + summaryText should contain ("Scheduling delay:") + summaryText should contain ("Processing time:") + summaryText should contain ("Total delay:") + + findAll(cssSelector("""#batch-job-table th""")).map(_.text).toSeq should be { + List("Output Op Id", "Description", "Duration", "Job Id", "Duration", + "Stages: Succeeded/Total", "Tasks (for all stages): Succeeded/Total", "Error") + } + + // Check we have 2 output op ids + val outputOpIds = findAll(cssSelector(".output-op-id-cell")).toSeq + outputOpIds.map(_.attribute("rowspan")) should be (List(Some("2"), Some("2"))) + outputOpIds.map(_.text) should be (List("0", "1")) + + // Check job ids + val jobIdCells = findAll(cssSelector( """#batch-job-table a""")).toSeq + jobIdCells.map(_.text) should be (List("0", "1", "2", "3")) + + val jobLinks = jobIdCells.flatMap(_.attribute("href")) + jobLinks.size should be (4) + + // Check stage progress + findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toSeq should be + (List("1/1", "1/1", "1/1", "0/1 (1 failed)")) + + // Check job progress + findAll(cssSelector(""".progress-cell""")).map(_.text).toSeq should be + (List("1/1", "1/1", "1/1", "0/1 (1 failed)")) + + // Check stacktrace + val errorCells = findAll(cssSelector(""".stacktrace-details""")).map(_.text).toSeq + errorCells should have size 1 + errorCells(0) should include("java.lang.RuntimeException: Oops") + + // Check the job link in the batch page is right + go to (jobLinks(0)) + val jobDetails = findAll(cssSelector("li strong")).map(_.text).toSeq + jobDetails should contain("Status:") + jobDetails should contain("Completed Stages:") + + // Check a batch page without id + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/") + webDriver.getPageSource should include ("Missing id parameter") + + // Check a non-exist batch + go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/?id=12345") + webDriver.getPageSource should include ("does not exist") } ssc.stop(false) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index a5d2bb2fde16c..c39ad05f41520 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.storage.StorageLevel class WindowOperationsSuite extends TestSuiteBase { - override def maxWaitTimeMillis = 20000 // large window tests can sometimes take longer + override def maxWaitTimeMillis: Int = 20000 // large window tests can sometimes take longer - override def batchDuration = Seconds(1) // making sure its visible in this class + override def batchDuration: Duration = Seconds(1) // making sure its visible in this class val largerSlideInput = Seq( Seq(("a", 1)), diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index 7a6a2f3e577dd..6859b65c7165f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -21,17 +21,20 @@ import java.io.File import scala.util.Random import org.apache.hadoop.conf.Configuration -import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} -import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} +import org.apache.spark.streaming.util.{FileBasedWriteAheadLogSegment, FileBasedWriteAheadLogWriter} import org.apache.spark.util.Utils +import org.apache.spark.{SparkConf, SparkContext, SparkException} + +class WriteAheadLogBackedBlockRDDSuite + extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { -class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) + val hadoopConf = new Configuration() var sparkContext: SparkContext = null @@ -57,24 +60,35 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w System.clearProperty("spark.driver.port") } - test("Read data available in block manager and write ahead log") { - testRDD(5, 5) + test("Read data available in both block manager and write ahead log") { + testRDD(numPartitions = 5, numPartitionsInBM = 5, numPartitionsInWAL = 5) } test("Read data available only in block manager, not in write ahead log") { - testRDD(5, 0) + testRDD(numPartitions = 5, numPartitionsInBM = 5, numPartitionsInWAL = 0) } test("Read data available only in write ahead log, not in block manager") { - testRDD(0, 5) + testRDD(numPartitions = 5, numPartitionsInBM = 0, numPartitionsInWAL = 5) } - test("Read data available only in write ahead log, and test storing in block manager") { - testRDD(0, 5, testStoreInBM = true) + test("Read data with partially available in block manager, and rest in write ahead log") { + testRDD(numPartitions = 5, numPartitionsInBM = 3, numPartitionsInWAL = 2) } - test("Read data with partially available in block manager, and rest in write ahead log") { - testRDD(3, 2) + test("Test isBlockValid skips block fetching from BlockManager") { + testRDD( + numPartitions = 5, numPartitionsInBM = 5, numPartitionsInWAL = 0, testIsBlockValid = true) + } + + test("Test whether RDD is valid after removing blocks from block manager") { + testRDD( + numPartitions = 5, numPartitionsInBM = 5, numPartitionsInWAL = 5, testBlockRemove = true) + } + + test("Test storing of blocks recovered from write ahead log back into block manager") { + testRDD( + numPartitions = 5, numPartitionsInBM = 0, numPartitionsInWAL = 5, testStoreInBM = true) } /** @@ -82,23 +96,54 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w * and the rest to a write ahead log, and then reading reading it all back using the RDD. * It can also test if the partitions that were read from the log were again stored in * block manager. - * @param numPartitionsInBM Number of partitions to write to the Block Manager - * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log - * @param testStoreInBM Test whether blocks read from log are stored back into block manager + * + * + * + * @param numPartitions Number of partitions in RDD + * @param numPartitionsInBM Number of partitions to write to the BlockManager. + * Partitions 0 to (numPartitionsInBM-1) will be written to BlockManager + * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log. + * Partitions (numPartitions - 1 - numPartitionsInWAL) to + * (numPartitions - 1) will be written to WAL + * @param testIsBlockValid Test whether setting isBlockValid to false skips block fetching + * @param testBlockRemove Test whether calling rdd.removeBlock() makes the RDD still usable with + * reads falling back to the WAL + * @param testStoreInBM Test whether blocks read from log are stored back into block manager + * + * Example with numPartitions = 5, numPartitionsInBM = 3, and numPartitionsInWAL = 4 + * + * numPartitionsInBM = 3 + * |------------------| + * | | + * 0 1 2 3 4 + * | | + * |-------------------------| + * numPartitionsInWAL = 4 */ - private def testRDD(numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { - val numBlocks = numPartitionsInBM + numPartitionsInWAL - val data = Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50)) + private def testRDD( + numPartitions: Int, + numPartitionsInBM: Int, + numPartitionsInWAL: Int, + testIsBlockValid: Boolean = false, + testBlockRemove: Boolean = false, + testStoreInBM: Boolean = false + ) { + require(numPartitionsInBM <= numPartitions, + "Can't put more partitions in BlockManager than that in RDD") + require(numPartitionsInWAL <= numPartitions, + "Can't put more partitions in write ahead log than that in RDD") + val data = Seq.fill(numPartitions, 10)(scala.util.Random.nextString(50)) // Put the necessary blocks in the block manager - val blockIds = Array.fill(numBlocks)(StreamBlockId(Random.nextInt(), Random.nextInt())) + val blockIds = Array.fill(numPartitions)(StreamBlockId(Random.nextInt(), Random.nextInt())) data.zip(blockIds).take(numPartitionsInBM).foreach { case(block, blockId) => blockManager.putIterator(blockId, block.iterator, StorageLevel.MEMORY_ONLY_SER) } - // Generate write ahead log segments - val segments = generateFakeSegments(numPartitionsInBM) ++ - writeLogSegments(data.takeRight(numPartitionsInWAL), blockIds.takeRight(numPartitionsInWAL)) + // Generate write ahead log record handles + val recordHandles = generateFakeRecordHandles(numPartitions - numPartitionsInWAL) ++ + generateWALRecordHandles(data.takeRight(numPartitionsInWAL), + blockIds.takeRight(numPartitionsInWAL)) // Make sure that the left `numPartitionsInBM` blocks are in block manager, and others are not require( @@ -106,30 +151,53 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w "Expected blocks not in BlockManager" ) require( - blockIds.takeRight(numPartitionsInWAL).forall(blockManager.get(_).isEmpty), + blockIds.takeRight(numPartitions - numPartitionsInBM).forall(blockManager.get(_).isEmpty), "Unexpected blocks in BlockManager" ) - // Make sure that the right `numPartitionsInWAL` blocks are in write ahead logs, and other are not + // Make sure that the right `numPartitionsInWAL` blocks are in WALs, and other are not require( - segments.takeRight(numPartitionsInWAL).forall(s => + recordHandles.takeRight(numPartitionsInWAL).forall(s => new File(s.path.stripPrefix("file://")).exists()), "Expected blocks not in write ahead log" ) require( - segments.take(numPartitionsInBM).forall(s => + recordHandles.take(numPartitions - numPartitionsInWAL).forall(s => !new File(s.path.stripPrefix("file://")).exists()), "Unexpected blocks in write ahead log" ) // Create the RDD and verify whether the returned data is correct val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, blockIds.toArray, - segments.toArray, storeInBlockManager = false, StorageLevel.MEMORY_ONLY) + recordHandles.toArray, storeInBlockManager = false) assert(rdd.collect() === data.flatten) + // Verify that the block fetching is skipped when isBlockValid is set to false. + // This is done by using a RDD whose data is only in memory but is set to skip block fetching + // Using that RDD will throw exception, as it skips block fetching even if the blocks are in + // in BlockManager. + if (testIsBlockValid) { + require(numPartitionsInBM === numPartitions, "All partitions must be in BlockManager") + require(numPartitionsInWAL === 0, "No partitions must be in WAL") + val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, blockIds.toArray, + recordHandles.toArray, isBlockIdValid = Array.fill(blockIds.length)(false)) + intercept[SparkException] { + rdd2.collect() + } + } + + // Verify that the RDD is not invalid after the blocks are removed and can still read data + // from write ahead log + if (testBlockRemove) { + require(numPartitions === numPartitionsInWAL, "All partitions must be in WAL for this test") + require(numPartitionsInBM > 0, "Some partitions must be in BlockManager for this test") + rdd.removeBlocks() + assert(rdd.collect() === data.flatten) + } + if (testStoreInBM) { val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, blockIds.toArray, - segments.toArray, storeInBlockManager = true, StorageLevel.MEMORY_ONLY) + recordHandles.toArray, storeInBlockManager = true, storageLevel = StorageLevel.MEMORY_ONLY) assert(rdd2.collect() === data.flatten) assert( blockIds.forall(blockManager.get(_).nonEmpty), @@ -138,12 +206,12 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w } } - private def writeLogSegments( + private def generateWALRecordHandles( blockData: Seq[Seq[String]], blockIds: Seq[BlockId] - ): Seq[WriteAheadLogFileSegment] = { + ): Seq[FileBasedWriteAheadLogSegment] = { require(blockData.size === blockIds.size) - val writer = new WriteAheadLogWriter(new File(dir, "logFile").toString, hadoopConf) + val writer = new FileBasedWriteAheadLogWriter(new File(dir, "logFile").toString, hadoopConf) val segments = blockData.zip(blockIds).map { case (data, id) => writer.write(blockManager.dataSerialize(id, data.iterator)) } @@ -151,7 +219,7 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll w segments } - private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = { - Array.fill(count)(new WriteAheadLogFileSegment("random", 0l, 0)) + private def generateFakeRecordHandles(count: Int): Seq[FileBasedWriteAheadLogSegment] = { + Array.fill(count)(new FileBasedWriteAheadLogSegment("random", 0L, 0)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala new file mode 100644 index 0000000000000..5478b41845943 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala @@ -0,0 +1,79 @@ +/* + * 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.streaming.scheduler + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Time, Duration, StreamingContext} + +class InputInfoTrackerSuite extends FunSuite with BeforeAndAfter { + + private var ssc: StreamingContext = _ + + before { + val conf = new SparkConf().setMaster("local[2]").setAppName("DirectStreamTacker") + if (ssc == null) { + ssc = new StreamingContext(conf, Duration(1000)) + } + } + + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + } + + test("test report and get InputInfo from InputInfoTracker") { + val inputInfoTracker = new InputInfoTracker(ssc) + + val streamId1 = 0 + val streamId2 = 1 + val time = Time(0L) + val inputInfo1 = InputInfo(streamId1, 100L) + val inputInfo2 = InputInfo(streamId2, 300L) + inputInfoTracker.reportInfo(time, inputInfo1) + inputInfoTracker.reportInfo(time, inputInfo2) + + val batchTimeToInputInfos = inputInfoTracker.getInfo(time) + assert(batchTimeToInputInfos.size == 2) + assert(batchTimeToInputInfos.keys === Set(streamId1, streamId2)) + assert(batchTimeToInputInfos(streamId1) === inputInfo1) + assert(batchTimeToInputInfos(streamId2) === inputInfo2) + assert(inputInfoTracker.getInfo(time)(streamId1) === inputInfo1) + } + + test("test cleanup InputInfo from InputInfoTracker") { + val inputInfoTracker = new InputInfoTracker(ssc) + + val streamId1 = 0 + val inputInfo1 = InputInfo(streamId1, 100L) + val inputInfo2 = InputInfo(streamId1, 300L) + inputInfoTracker.reportInfo(Time(0), inputInfo1) + inputInfoTracker.reportInfo(Time(1), inputInfo2) + + inputInfoTracker.cleanup(Time(0)) + assert(inputInfoTracker.getInfo(Time(0))(streamId1) === inputInfo1) + assert(inputInfoTracker.getInfo(Time(1))(streamId1) === inputInfo2) + + inputInfoTracker.cleanup(Time(1)) + assert(inputInfoTracker.getInfo(Time(0)).get(streamId1) === None) + assert(inputInfoTracker.getInfo(Time(1))(streamId1) === inputInfo2) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala index 4150b60635ed6..7865b06c2e3c2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -90,7 +90,7 @@ class JobGeneratorSuite extends TestSuiteBase { val receiverTracker = ssc.scheduler.receiverTracker // Get the blocks belonging to a batch - def getBlocksOfBatch(batchTime: Long) = { + def getBlocksOfBatch(batchTime: Long): Seq[ReceivedBlockInfo] = { receiverTracker.getBlocksOfBatchAndStream(Time(batchTime), inputStream.id) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala new file mode 100644 index 0000000000000..e874536e63518 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -0,0 +1,206 @@ +/* + * 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.streaming.ui + +import java.util.Properties + +import org.scalatest.Matchers + +import org.apache.spark.scheduler.SparkListenerJobStart +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.{Duration, Time, Milliseconds, TestSuiteBase} + +class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { + + val input = (1 to 4).map(Seq(_)).toSeq + val operation = (d: DStream[Int]) => d.map(x => x) + + private def createJobStart( + batchTime: Time, outputOpId: Int, jobId: Int): SparkListenerJobStart = { + val properties = new Properties() + properties.setProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, batchTime.milliseconds.toString) + properties.setProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, outputOpId.toString) + SparkListenerJobStart(jobId = jobId, + 0L, // unused + Nil, // unused + properties) + } + + override def batchDuration: Duration = Milliseconds(100) + + test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + + "onReceiverStarted, onReceiverError, onReceiverStopped") { + val ssc = setupStreams(input, operation) + val listener = new StreamingJobProgressListener(ssc) + + val streamIdToNumRecords = Map(0 -> 300L, 1 -> 300L) + + // onBatchSubmitted + val batchInfoSubmitted = BatchInfo(Time(1000), streamIdToNumRecords, 1000, None, None) + listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) + listener.waitingBatches should be (List(BatchUIData(batchInfoSubmitted))) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (0) + + // onBatchStarted + val batchInfoStarted = BatchInfo(Time(1000), streamIdToNumRecords, 1000, Some(2000), None) + listener.onBatchStarted(StreamingListenerBatchStarted(batchInfoStarted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (List(BatchUIData(batchInfoStarted))) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (600) + + // onJobStart + val jobStart1 = createJobStart(Time(1000), outputOpId = 0, jobId = 0) + listener.onJobStart(jobStart1) + + val jobStart2 = createJobStart(Time(1000), outputOpId = 0, jobId = 1) + listener.onJobStart(jobStart2) + + val jobStart3 = createJobStart(Time(1000), outputOpId = 1, jobId = 0) + listener.onJobStart(jobStart3) + + val jobStart4 = createJobStart(Time(1000), outputOpId = 1, jobId = 1) + listener.onJobStart(jobStart4) + + val batchUIData = listener.getBatchUIData(Time(1000)) + batchUIData should not be None + batchUIData.get.batchTime should be (batchInfoStarted.batchTime) + batchUIData.get.schedulingDelay should be (batchInfoStarted.schedulingDelay) + batchUIData.get.processingDelay should be (batchInfoStarted.processingDelay) + batchUIData.get.totalDelay should be (batchInfoStarted.totalDelay) + batchUIData.get.receiverNumRecords should be (Map(0 -> 300L, 1 -> 300L)) + batchUIData.get.numRecords should be(600) + batchUIData.get.outputOpIdSparkJobIdPairs should be + Seq(OutputOpIdAndSparkJobId(0, 0), + OutputOpIdAndSparkJobId(0, 1), + OutputOpIdAndSparkJobId(1, 0), + OutputOpIdAndSparkJobId(1, 1)) + + // onBatchCompleted + val batchInfoCompleted = BatchInfo(Time(1000), streamIdToNumRecords, 1000, Some(2000), None) + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (List(BatchUIData(batchInfoCompleted))) + listener.lastCompletedBatch should be (Some(BatchUIData(batchInfoCompleted))) + listener.numUnprocessedBatches should be (0) + listener.numTotalCompletedBatches should be (1) + listener.numTotalProcessedRecords should be (600) + listener.numTotalReceivedRecords should be (600) + + // onReceiverStarted + val receiverInfoStarted = ReceiverInfo(0, "test", null, true, "localhost") + listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (None) + + // onReceiverError + val receiverInfoError = ReceiverInfo(1, "test", null, true, "localhost") + listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (None) + + // onReceiverStopped + val receiverInfoStopped = ReceiverInfo(2, "test", null, true, "localhost") + listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (Some(receiverInfoStopped)) + listener.receiverInfo(3) should be (None) + } + + test("Remove the old completed batches when exceeding the limit") { + val ssc = setupStreams(input, operation) + val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + val listener = new StreamingJobProgressListener(ssc) + + val streamIdToNumRecords = Map(0 -> 300L, 1 -> 300L) + + val batchInfoCompleted = BatchInfo(Time(1000), streamIdToNumRecords, 1000, Some(2000), None) + + for(_ <- 0 until (limit + 10)) { + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + } + + listener.retainedCompletedBatches.size should be (limit) + listener.numTotalCompletedBatches should be(limit + 10) + } + + test("out-of-order onJobStart and onBatchXXX") { + val ssc = setupStreams(input, operation) + val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + val listener = new StreamingJobProgressListener(ssc) + + // fulfill completedBatchInfos + for(i <- 0 until limit) { + val batchInfoCompleted = + BatchInfo(Time(1000 + i * 100), Map.empty, 1000 + i * 100, Some(2000 + i * 100), None) + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + val jobStart = createJobStart(Time(1000 + i * 100), outputOpId = 0, jobId = 1) + listener.onJobStart(jobStart) + } + + // onJobStart happens before onBatchSubmitted + val jobStart = createJobStart(Time(1000 + limit * 100), outputOpId = 0, jobId = 0) + listener.onJobStart(jobStart) + + val batchInfoSubmitted = + BatchInfo(Time(1000 + limit * 100), Map.empty, (1000 + limit * 100), None, None) + listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) + + // We still can see the info retrieved from onJobStart + val batchUIData = listener.getBatchUIData(Time(1000 + limit * 100)) + batchUIData should not be None + batchUIData.get.batchTime should be (batchInfoSubmitted.batchTime) + batchUIData.get.schedulingDelay should be (batchInfoSubmitted.schedulingDelay) + batchUIData.get.processingDelay should be (batchInfoSubmitted.processingDelay) + batchUIData.get.totalDelay should be (batchInfoSubmitted.totalDelay) + batchUIData.get.receiverNumRecords should be (Map.empty) + batchUIData.get.numRecords should be (0) + batchUIData.get.outputOpIdSparkJobIdPairs should be (Seq(OutputOpIdAndSparkJobId(0, 0))) + + // A lot of "onBatchCompleted"s happen before "onJobStart" + for(i <- limit + 1 to limit * 2) { + val batchInfoCompleted = + BatchInfo(Time(1000 + i * 100), Map.empty, 1000 + i * 100, Some(2000 + i * 100), None) + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + } + + for(i <- limit + 1 to limit * 2) { + val jobStart = createJobStart(Time(1000 + i * 100), outputOpId = 0, jobId = 1) + listener.onJobStart(jobStart) + } + + // We should not leak memory + listener.batchTimeToOutputOpIdSparkJobIdPair.size() should be <= + (listener.waitingBatches.size + listener.runningBatches.size + + listener.retainedCompletedBatches.size + 10) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8335659667f22..79098bcf4861c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -18,33 +18,38 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer +import java.util import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} +import scala.reflect.ClassTag -import WriteAheadLogSuite._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.util.{ManualClock, Utils} -import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Eventually._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.{SparkConf, SparkException} class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { + import WriteAheadLogSuite._ + val hadoopConf = new Configuration() var tempDir: File = null var testDir: String = null var testFile: String = null - var manager: WriteAheadLogManager = null + var writeAheadLog: FileBasedWriteAheadLog = null before { tempDir = Utils.createTempDir() testDir = tempDir.toString testFile = new File(tempDir, "testFile").toString - if (manager != null) { - manager.stop() - manager = null + if (writeAheadLog != null) { + writeAheadLog.close() + writeAheadLog = null } } @@ -52,16 +57,60 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { Utils.deleteRecursively(tempDir) } - test("WriteAheadLogWriter - writing data") { + test("WriteAheadLogUtils - log selection and creation") { + val logDir = Utils.createTempDir().getAbsolutePath() + + def assertDriverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + val emptyConf = new SparkConf() // no log configuration + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](conf1) + assertReceiverLogClass[FileBasedWriteAheadLog](conf1) + + // Verify setting receiver WAL class + val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + + // Verify setting receiver WAL class with 1-arg constructor + val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog1].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + + // Verify failure setting receiver WAL class with 2-arg constructor + intercept[SparkException] { + val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog2].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + } + } + + test("FileBasedWriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) val writtenData = readDataManually(segments) assert(writtenData === dataToWrite) } - test("WriteAheadLogWriter - syncing of data by writing and reading immediately") { + test("FileBasedWriteAheadLogWriter - syncing of data by writing and reading immediately") { val dataToWrite = generateRandomData() - val writer = new WriteAheadLogWriter(testFile, hadoopConf) + val writer = new FileBasedWriteAheadLogWriter(testFile, hadoopConf) dataToWrite.foreach { data => val segment = writer.write(stringToByteBuffer(data)) val dataRead = readDataManually(Seq(segment)).head @@ -70,10 +119,10 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { writer.close() } - test("WriteAheadLogReader - sequentially reading data") { + test("FileBasedWriteAheadLogReader - sequentially reading data") { val writtenData = generateRandomData() writeDataManually(writtenData, testFile) - val reader = new WriteAheadLogReader(testFile, hadoopConf) + val reader = new FileBasedWriteAheadLogReader(testFile, hadoopConf) val readData = reader.toSeq.map(byteBufferToString) assert(readData === writtenData) assert(reader.hasNext === false) @@ -83,14 +132,14 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { reader.close() } - test("WriteAheadLogReader - sequentially reading data written with writer") { + test("FileBasedWriteAheadLogReader - sequentially reading data written with writer") { val dataToWrite = generateRandomData() writeDataUsingWriter(testFile, dataToWrite) val readData = readDataUsingReader(testFile) assert(readData === dataToWrite) } - test("WriteAheadLogReader - reading data written with writer after corrupted write") { + test("FileBasedWriteAheadLogReader - reading data written with writer after corrupted write") { // Write data manually for testing the sequential reader val dataToWrite = generateRandomData() writeDataUsingWriter(testFile, dataToWrite) @@ -113,38 +162,38 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { assert(readDataUsingReader(testFile) === (dataToWrite.dropRight(1))) } - test("WriteAheadLogRandomReader - reading data using random reader") { + test("FileBasedWriteAheadLogRandomReader - reading data using random reader") { // Write data manually for testing the random reader val writtenData = generateRandomData() val segments = writeDataManually(writtenData, testFile) // Get a random order of these segments and read them back val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10).flatten - val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + val reader = new FileBasedWriteAheadLogRandomReader(testFile, hadoopConf) writtenDataAndSegments.foreach { case (data, segment) => assert(data === byteBufferToString(reader.read(segment))) } reader.close() } - test("WriteAheadLogRandomReader - reading data using random reader written with writer") { + test("FileBasedWriteAheadLogRandomReader- reading data using random reader written with writer") { // Write data using writer for testing the random reader val data = generateRandomData() val segments = writeDataUsingWriter(testFile, data) // Read a random sequence of segments and verify read data val dataAndSegments = data.zip(segments).toSeq.permutations.take(10).flatten - val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + val reader = new FileBasedWriteAheadLogRandomReader(testFile, hadoopConf) dataAndSegments.foreach { case (data, segment) => assert(data === byteBufferToString(reader.read(segment))) } reader.close() } - test("WriteAheadLogManager - write rotating logs") { - // Write data using manager + test("FileBasedWriteAheadLog - write rotating logs") { + // Write data with rotation using WriteAheadLog class val dataToWrite = generateRandomData() - writeDataUsingManager(testDir, dataToWrite) + writeDataUsingWriteAheadLog(testDir, dataToWrite) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) @@ -153,8 +202,8 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { assert(writtenData === dataToWrite) } - test("WriteAheadLogManager - read rotating logs") { - // Write data manually for testing reading through manager + test("FileBasedWriteAheadLog - read rotating logs") { + // Write data manually for testing reading through WriteAheadLog val writtenData = (1 to 10).map { i => val data = generateRandomData() val file = testDir + s"/log-$i-$i" @@ -167,25 +216,25 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { assert(fileSystem.exists(logDirectoryPath) === true) // Read data using manager and verify - val readData = readDataUsingManager(testDir) + val readData = readDataUsingWriteAheadLog(testDir) assert(readData === writtenData) } - test("WriteAheadLogManager - recover past logs when creating new manager") { + test("FileBasedWriteAheadLog - recover past logs when creating new manager") { // Write data with manager, recover with new manager and verify val dataToWrite = generateRandomData() - writeDataUsingManager(testDir, dataToWrite) + writeDataUsingWriteAheadLog(testDir, dataToWrite) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - val readData = readDataUsingManager(testDir) + val readData = readDataUsingWriteAheadLog(testDir) assert(dataToWrite === readData) } - test("WriteAheadLogManager - cleanup old logs") { + test("FileBasedWriteAheadLog - clean old logs") { logCleanUpTest(waitForCompletion = false) } - test("WriteAheadLogManager - cleanup old logs synchronously") { + test("FileBasedWriteAheadLog - clean old logs synchronously") { logCleanUpTest(waitForCompletion = true) } @@ -193,11 +242,11 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() - manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) + writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - manager.cleanupOldLogs(manualClock.getTimeMillis() / 2, waitForCompletion) + writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion) if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) @@ -208,11 +257,11 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } } - test("WriteAheadLogManager - handling file errors while reading rotating logs") { + test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") { // Generate a set of log files val manualClock = new ManualClock val dataToWrite1 = generateRandomData() - writeDataUsingManager(testDir, dataToWrite1, manualClock) + writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock) val logFiles1 = getLogFilesInDirectory(testDir) assert(logFiles1.size > 1) @@ -220,12 +269,12 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() manualClock.advance(100000) - writeDataUsingManager(testDir, dataToWrite2, manualClock) + writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) // Read the files and verify that all the written data can be read - val readData1 = readDataUsingManager(testDir) + val readData1 = readDataUsingWriteAheadLog(testDir) assert(readData1 === (dataToWrite1 ++ dataToWrite2)) // Corrupt the first set of files so that they are basically unreadable @@ -236,25 +285,51 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } // Verify that the corrupted files do not prevent reading of the second set of data - val readData = readDataUsingManager(testDir) + val readData = readDataUsingWriteAheadLog(testDir) assert(readData === dataToWrite2) } + + test("FileBasedWriteAheadLog - do not create directories or files unless write") { + val nonexistentTempPath = File.createTempFile("test", "") + nonexistentTempPath.delete() + assert(!nonexistentTempPath.exists()) + + val writtenSegment = writeDataManually(generateRandomData(), testFile) + val wal = new FileBasedWriteAheadLog( + new SparkConf(), tempDir.getAbsolutePath, new Configuration(), 1, 1) + assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") + wal.read(writtenSegment.head) + assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") + } } object WriteAheadLogSuite { + class MockWriteAheadLog0() extends WriteAheadLog { + override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } + override def read(handle: WriteAheadLogRecordHandle): ByteBuffer = { null } + override def readAll(): util.Iterator[ByteBuffer] = { null } + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { } + override def close(): Unit = { } + } + + class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() + + class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() + + private val hadoopConf = new Configuration() /** Write data to a file directly and return an array of the file segments written. */ - def writeDataManually(data: Seq[String], file: String): Seq[WriteAheadLogFileSegment] = { - val segments = new ArrayBuffer[WriteAheadLogFileSegment]() + def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { + val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() val writer = HdfsUtils.getOutputStream(file, hadoopConf) data.foreach { item => val offset = writer.getPos val bytes = Utils.serialize(item) writer.writeInt(bytes.size) writer.write(bytes) - segments += WriteAheadLogFileSegment(file, offset, bytes.size) + segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) } writer.close() segments @@ -263,8 +338,11 @@ object WriteAheadLogSuite { /** * Write data to a file using the writer class and return an array of the file segments written. */ - def writeDataUsingWriter(filePath: String, data: Seq[String]): Seq[WriteAheadLogFileSegment] = { - val writer = new WriteAheadLogWriter(filePath, hadoopConf) + def writeDataUsingWriter( + filePath: String, + data: Seq[String] + ): Seq[FileBasedWriteAheadLogSegment] = { + val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) val segments = data.map { item => writer.write(item) } @@ -272,27 +350,27 @@ object WriteAheadLogSuite { segments } - /** Write data to rotating files in log directory using the manager class. */ - def writeDataUsingManager( + /** Write data to rotating files in log directory using the WriteAheadLog class. */ + def writeDataUsingWriteAheadLog( logDirectory: String, data: Seq[String], manualClock: ManualClock = new ManualClock, - stopManager: Boolean = true - ): WriteAheadLogManager = { + closeLog: Boolean = true + ): FileBasedWriteAheadLog = { if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) - val manager = new WriteAheadLogManager(logDirectory, hadoopConf, - rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = manualClock) + val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1) + // Ensure that 500 does not get sorted after 2000, so put a high base value. data.foreach { item => manualClock.advance(500) - manager.writeToLog(item) + wal.write(item, manualClock.getTimeMillis()) } - if (stopManager) manager.stop() - manager + if (closeLog) wal.close() + wal } - /** Read data from a segments of a log file directly and return the list of byte buffers.*/ - def readDataManually(segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { + /** Read data from a segments of a log file directly and return the list of byte buffers. */ + def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = { segments.map { segment => val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) try { @@ -331,18 +409,18 @@ object WriteAheadLogSuite { /** Read all the data from a log file using reader class and return the list of byte buffers. */ def readDataUsingReader(file: String): Seq[String] = { - val reader = new WriteAheadLogReader(file, hadoopConf) + val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) val readData = reader.toList.map(byteBufferToString) reader.close() readData } - /** Read all the data in the log file in a directory using the manager class. */ - def readDataUsingManager(logDirectory: String): Seq[String] = { - val manager = new WriteAheadLogManager(logDirectory, hadoopConf, - callerName = "WriteAheadLogSuite") - val data = manager.readFromLog().map(byteBufferToString).toSeq - manager.stop() + /** Read all the data in the log file in a directory using the WriteAheadLog class. */ + def readDataUsingWriteAheadLog(logDirectory: String): Seq[String] = { + import scala.collection.JavaConversions._ + val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1) + val data = wal.readAll().map(byteBufferToString).toSeq + wal.close() data } diff --git a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala index d0bf328f2b74d..d66750463033a 100644 --- a/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streamingtest/ImplicitSuite.scala @@ -25,7 +25,8 @@ package org.apache.spark.streamingtest */ class ImplicitSuite { - // We only want to test if `implict` works well with the compiler, so we don't need a real DStream. + // We only want to test if `implicit` works well with the compiler, + // so we don't need a real DStream. def mockDStream[T]: org.apache.spark.streaming.dstream.DStream[T] = null def testToPairDStreamFunctions(): Unit = { diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 8d0f09933c8d3..583823c90c5c6 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -17,7 +17,7 @@ package org.apache.spark.tools -import java.lang.reflect.Method +import java.lang.reflect.{Type, Method} import scala.collection.mutable.ArrayBuffer import scala.language.existentials @@ -302,7 +302,7 @@ object JavaAPICompletenessChecker { private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") - def toComparisionKey(method: Method) = + def toComparisionKey(method: Method): (Class[_], String, Type) = (method.getReturnType, method.getName, method.getGenericReturnType) val interfaces = method.getDeclaringClass.getInterfaces.filter { i => excludedInterfaces.contains(i.getName) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 15ee95070a3d3..baa97616eaff3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Writes simulated shuffle output from several threads and records the observed throughput. */ object StoragePerfTester { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { /** Total amount of data to generate. Distributed evenly amongst maps and reduce splits. */ val dataSizeMb = Utils.memoryStringToMb(sys.env.getOrElse("OUTPUT_DATA", "1g")) @@ -46,7 +46,8 @@ object StoragePerfTester { val totalRecords = dataSizeMb * 1000 val recordsPerMap = totalRecords / numMaps - val writeData = "1" * recordLength + val writeKey = "1" * (recordLength / 2) + val writeValue = "1" * (recordLength / 2) val executor = Executors.newFixedThreadPool(numMaps) val conf = new SparkConf() @@ -58,12 +59,12 @@ object StoragePerfTester { val sc = new SparkContext("local[4]", "Write Tester", conf) val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] - def writeOutputBytes(mapId: Int, total: AtomicLong) = { - val shuffle = hashShuffleManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, + def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { + val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { - writers(i % numOutputSplits).write(writeData) + writers(i % numOutputSplits).write(writeKey, writeValue) } writers.map { w => w.commitAndClose() @@ -78,7 +79,7 @@ object StoragePerfTester { val totalBytes = new AtomicLong() for (task <- 1 to numMaps) { executor.submit(new Runnable() { - override def run() = { + override def run(): Unit = { try { writeOutputBytes(task, totalBytes) latch.countDown() diff --git a/unsafe/pom.xml b/unsafe/pom.xml new file mode 100644 index 0000000000000..5b0733206b2bc --- /dev/null +++ b/unsafe/pom.xml @@ -0,0 +1,93 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.10 + 1.4.0-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-unsafe_2.10 + jar + Spark Project Unsafe + http://spark.apache.org/ + + unsafe + + + + + + + com.google.code.findbugs + jsr305 + + + + + org.slf4j + slf4j-api + provided + + + + + junit + junit + test + + + com.novocode + junit-interface + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + net.alchim31.maven + scala-maven-plugin + + + + -XDignore.symbol.file + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + -XDignore.symbol.file + + + + + + + diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java b/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java new file mode 100644 index 0000000000000..24b2892098059 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java @@ -0,0 +1,162 @@ +/* + * 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.unsafe; + +import java.lang.reflect.Field; + +import sun.misc.Unsafe; + +public final class PlatformDependent { + + /** + * Facade in front of {@link sun.misc.Unsafe}, used to avoid directly exposing Unsafe outside of + * this package. This also lets us aovid accidental use of deprecated methods or methods that + * aren't present in Java 6. + */ + public static final class UNSAFE { + + private UNSAFE() { } + + public static int getInt(Object object, long offset) { + return _UNSAFE.getInt(object, offset); + } + + public static void putInt(Object object, long offset, int value) { + _UNSAFE.putInt(object, offset, value); + } + + public static boolean getBoolean(Object object, long offset) { + return _UNSAFE.getBoolean(object, offset); + } + + public static void putBoolean(Object object, long offset, boolean value) { + _UNSAFE.putBoolean(object, offset, value); + } + + public static byte getByte(Object object, long offset) { + return _UNSAFE.getByte(object, offset); + } + + public static void putByte(Object object, long offset, byte value) { + _UNSAFE.putByte(object, offset, value); + } + + public static short getShort(Object object, long offset) { + return _UNSAFE.getShort(object, offset); + } + + public static void putShort(Object object, long offset, short value) { + _UNSAFE.putShort(object, offset, value); + } + + public static long getLong(Object object, long offset) { + return _UNSAFE.getLong(object, offset); + } + + public static void putLong(Object object, long offset, long value) { + _UNSAFE.putLong(object, offset, value); + } + + public static float getFloat(Object object, long offset) { + return _UNSAFE.getFloat(object, offset); + } + + public static void putFloat(Object object, long offset, float value) { + _UNSAFE.putFloat(object, offset, value); + } + + public static double getDouble(Object object, long offset) { + return _UNSAFE.getDouble(object, offset); + } + + public static void putDouble(Object object, long offset, double value) { + _UNSAFE.putDouble(object, offset, value); + } + + public static long allocateMemory(long size) { + return _UNSAFE.allocateMemory(size); + } + + public static void freeMemory(long address) { + _UNSAFE.freeMemory(address); + } + + } + + private static final Unsafe _UNSAFE; + + public static final int BYTE_ARRAY_OFFSET; + + public static final int INT_ARRAY_OFFSET; + + public static final int LONG_ARRAY_OFFSET; + + public static final int DOUBLE_ARRAY_OFFSET; + + /** + * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to + * allow safepoint polling during a large copy. + */ + private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L; + + static { + sun.misc.Unsafe unsafe; + try { + Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe"); + unsafeField.setAccessible(true); + unsafe = (sun.misc.Unsafe) unsafeField.get(null); + } catch (Throwable cause) { + unsafe = null; + } + _UNSAFE = unsafe; + + if (_UNSAFE != null) { + BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); + INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); + LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class); + DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); + } else { + BYTE_ARRAY_OFFSET = 0; + INT_ARRAY_OFFSET = 0; + LONG_ARRAY_OFFSET = 0; + DOUBLE_ARRAY_OFFSET = 0; + } + } + + static public void copyMemory( + Object src, + long srcOffset, + Object dst, + long dstOffset, + long length) { + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + srcOffset += size; + dstOffset += size; + } + } + + /** + * Raises an exception bypassing compiler checks for checked exceptions. + */ + public static void throwException(Throwable t) { + _UNSAFE.throwException(t); + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java new file mode 100644 index 0000000000000..53eadf96a6b52 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -0,0 +1,56 @@ +/* + * 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.unsafe.array; + +import org.apache.spark.unsafe.PlatformDependent; + +public class ByteArrayMethods { + + private ByteArrayMethods() { + // Private constructor, since this class only contains static methods. + } + + public static int roundNumberOfBytesToNearestWord(int numBytes) { + int remainder = numBytes & 0x07; // This is equivalent to `numBytes % 8` + if (remainder == 0) { + return numBytes; + } else { + return numBytes + (8 - remainder); + } + } + + /** + * Optimized byte array equality check for 8-byte-word-aligned byte arrays. + * @return true if the arrays are equal, false otherwise + */ + public static boolean wordAlignedArrayEquals( + Object leftBaseObject, + long leftBaseOffset, + Object rightBaseObject, + long rightBaseOffset, + long arrayLengthInBytes) { + for (int i = 0; i < arrayLengthInBytes; i += 8) { + final long left = + PlatformDependent.UNSAFE.getLong(leftBaseObject, leftBaseOffset + i); + final long right = + PlatformDependent.UNSAFE.getLong(rightBaseObject, rightBaseOffset + i); + if (left != right) return false; + } + return true; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java new file mode 100644 index 0000000000000..18d1f0d2d7eb2 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -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. + */ + +package org.apache.spark.unsafe.array; + +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.memory.MemoryBlock; + +/** + * An array of long values. Compared with native JVM arrays, this: + *
      + *
    • supports using both in-heap and off-heap memory
    • + *
    • has no bound checking, and thus can crash the JVM process when assert is turned off
    • + *
    + */ +public final class LongArray { + + // This is a long so that we perform long multiplications when computing offsets. + private static final long WIDTH = 8; + + private final MemoryBlock memory; + private final Object baseObj; + private final long baseOffset; + + private final long length; + + public LongArray(MemoryBlock memory) { + assert memory.size() % WIDTH == 0 : "Memory not aligned (" + memory.size() + ")"; + assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size > 4 billion elements"; + this.memory = memory; + this.baseObj = memory.getBaseObject(); + this.baseOffset = memory.getBaseOffset(); + this.length = memory.size() / WIDTH; + } + + public MemoryBlock memoryBlock() { + return memory; + } + + /** + * Returns the number of elements this array can hold. + */ + public long size() { + return length; + } + + /** + * Sets the value at position {@code index}. + */ + public void set(int index, long value) { + assert index >= 0 : "index (" + index + ") should >= 0"; + assert index < length : "index (" + index + ") should < length (" + length + ")"; + PlatformDependent.UNSAFE.putLong(baseObj, baseOffset + index * WIDTH, value); + } + + /** + * Returns the value at position {@code index}. + */ + public long get(int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + assert index < length : "index (" + index + ") should < length (" + length + ")"; + return PlatformDependent.UNSAFE.getLong(baseObj, baseOffset + index * WIDTH); + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java new file mode 100644 index 0000000000000..f72e07fce92fd --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSet.java @@ -0,0 +1,105 @@ +/* + * 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.unsafe.bitset; + +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; + +/** + * A fixed size uncompressed bit set backed by a {@link LongArray}. + * + * Each bit occupies exactly one bit of storage. + */ +public final class BitSet { + + /** A long array for the bits. */ + private final LongArray words; + + /** Length of the long array. */ + private final int numWords; + + private final Object baseObject; + private final long baseOffset; + + /** + * Creates a new {@link BitSet} using the specified memory block. Size of the memory block must be + * multiple of 8 bytes (i.e. 64 bits). + */ + public BitSet(MemoryBlock memory) { + words = new LongArray(memory); + assert (words.size() <= Integer.MAX_VALUE); + numWords = (int) words.size(); + baseObject = words.memoryBlock().getBaseObject(); + baseOffset = words.memoryBlock().getBaseOffset(); + } + + public MemoryBlock memoryBlock() { + return words.memoryBlock(); + } + + /** + * Returns the number of bits in this {@code BitSet}. + */ + public long capacity() { + return numWords * 64; + } + + /** + * Sets the bit at the specified index to {@code true}. + */ + public void set(int index) { + assert index < numWords * 64 : "index (" + index + ") should < length (" + numWords * 64 + ")"; + BitSetMethods.set(baseObject, baseOffset, index); + } + + /** + * Sets the bit at the specified index to {@code false}. + */ + public void unset(int index) { + assert index < numWords * 64 : "index (" + index + ") should < length (" + numWords * 64 + ")"; + BitSetMethods.unset(baseObject, baseOffset, index); + } + + /** + * Returns {@code true} if the bit is set at the specified index. + */ + public boolean isSet(int index) { + assert index < numWords * 64 : "index (" + index + ") should < length (" + numWords * 64 + ")"; + return BitSetMethods.isSet(baseObject, baseOffset, index); + } + + /** + * Returns the index of the first bit that is set to true that occurs on or after the + * specified starting index. If no such bit exists then {@code -1} is returned. + *

    + * To iterate over the true bits in a BitSet, use the following loop: + *

    +   * 
    +   *  for (long i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i + 1)) {
    +   *    // operate on index i here
    +   *  }
    +   * 
    +   * 
    + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next set bit, or -1 if there is no such bit + */ + public int nextSetBit(int fromIndex) { + return BitSetMethods.nextSetBit(baseObject, baseOffset, fromIndex, numWords); + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java new file mode 100644 index 0000000000000..f30626d8f4317 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -0,0 +1,129 @@ +/* + * 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.unsafe.bitset; + +import org.apache.spark.unsafe.PlatformDependent; + +/** + * Methods for working with fixed-size uncompressed bitsets. + * + * We assume that the bitset data is word-aligned (that is, a multiple of 8 bytes in length). + * + * Each bit occupies exactly one bit of storage. + */ +public final class BitSetMethods { + + private static final long WORD_SIZE = 8; + + private BitSetMethods() { + // Make the default constructor private, since this only holds static methods. + } + + /** + * Sets the bit at the specified index to {@code true}. + */ + public static void set(Object baseObject, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset); + PlatformDependent.UNSAFE.putLong(baseObject, wordOffset, word | mask); + } + + /** + * Sets the bit at the specified index to {@code false}. + */ + public static void unset(Object baseObject, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset); + PlatformDependent.UNSAFE.putLong(baseObject, wordOffset, word & ~mask); + } + + /** + * Returns {@code true} if the bit is set at the specified index. + */ + public static boolean isSet(Object baseObject, long baseOffset, int index) { + assert index >= 0 : "index (" + index + ") should >= 0"; + final long mask = 1L << (index & 0x3f); // mod 64 and shift + final long wordOffset = baseOffset + (index >> 6) * WORD_SIZE; + final long word = PlatformDependent.UNSAFE.getLong(baseObject, wordOffset); + return (word & mask) != 0; + } + + /** + * Returns {@code true} if any bit is set. + */ + public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidthInBytes) { + for (int i = 0; i <= bitSetWidthInBytes; i++) { + if (PlatformDependent.UNSAFE.getByte(baseObject, baseOffset + i) != 0) { + return true; + } + } + return false; + } + + /** + * Returns the index of the first bit that is set to true that occurs on or after the + * specified starting index. If no such bit exists then {@code -1} is returned. + *

    + * To iterate over the true bits in a BitSet, use the following loop: + *

    +   * 
    +   *  for (long i = bs.nextSetBit(0, sizeInWords); i >= 0; i = bs.nextSetBit(i + 1, sizeInWords)) {
    +   *    // operate on index i here
    +   *  }
    +   * 
    +   * 
    + * + * @param fromIndex the index to start checking from (inclusive) + * @param bitsetSizeInWords the size of the bitset, measured in 8-byte words + * @return the index of the next set bit, or -1 if there is no such bit + */ + public static int nextSetBit( + Object baseObject, + long baseOffset, + int fromIndex, + int bitsetSizeInWords) { + int wi = fromIndex >> 6; + if (wi >= bitsetSizeInWords) { + return -1; + } + + // Try to find the next set bit in the current word + final int subIndex = fromIndex & 0x3f; + long word = + PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + wi * WORD_SIZE) >> subIndex; + if (word != 0) { + return (wi << 6) + subIndex + java.lang.Long.numberOfTrailingZeros(word); + } + + // Find the next set bit in the rest of the words + wi += 1; + while (wi < bitsetSizeInWords) { + word = PlatformDependent.UNSAFE.getLong(baseObject, baseOffset + wi * WORD_SIZE); + if (word != 0) { + return (wi << 6) + java.lang.Long.numberOfTrailingZeros(word); + } + wi += 1; + } + + return -1; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java new file mode 100644 index 0000000000000..85cd02469adb7 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -0,0 +1,96 @@ +/* + * 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.unsafe.hash; + +import org.apache.spark.unsafe.PlatformDependent; + +/** + * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. + */ +public final class Murmur3_x86_32 { + private static final int C1 = 0xcc9e2d51; + private static final int C2 = 0x1b873593; + + private final int seed; + + public Murmur3_x86_32(int seed) { + this.seed = seed; + } + + @Override + public String toString() { + return "Murmur3_32(seed=" + seed + ")"; + } + + public int hashInt(int input) { + int k1 = mixK1(input); + int h1 = mixH1(seed, k1); + + return fmix(h1, 4); + } + + public int hashUnsafeWords(Object baseObject, long baseOffset, int lengthInBytes) { + // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. + assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; + int h1 = seed; + for (int offset = 0; offset < lengthInBytes; offset += 4) { + int halfWord = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset + offset); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); + } + + public int hashLong(long input) { + int low = (int) input; + int high = (int) (input >>> 32); + + int k1 = mixK1(low); + int h1 = mixH1(seed, k1); + + k1 = mixK1(high); + h1 = mixH1(h1, k1); + + return fmix(h1, 8); + } + + private static int mixK1(int k1) { + k1 *= C1; + k1 = Integer.rotateLeft(k1, 15); + k1 *= C2; + return k1; + } + + private static int mixH1(int h1, int k1) { + h1 ^= k1; + h1 = Integer.rotateLeft(h1, 13); + h1 = h1 * 5 + 0xe6546b64; + return h1; + } + + // Finalization mix - force all bits of a hash block to avalanche + private static int fmix(int h1, int length) { + h1 ^= length; + h1 ^= h1 >>> 16; + h1 *= 0x85ebca6b; + h1 ^= h1 >>> 13; + h1 *= 0xc2b2ae35; + h1 ^= h1 >>> 16; + return h1; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java new file mode 100644 index 0000000000000..19d6a169fd2ad --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -0,0 +1,548 @@ +/* + * 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.unsafe.map; + +import java.lang.Override; +import java.lang.UnsupportedOperationException; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +import org.apache.spark.unsafe.*; +import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.bitset.BitSet; +import org.apache.spark.unsafe.hash.Murmur3_x86_32; +import org.apache.spark.unsafe.memory.*; + +/** + * An append-only hash map where keys and values are contiguous regions of bytes. + *

    + * This is backed by a power-of-2-sized hash table, using quadratic probing with triangular numbers, + * which is guaranteed to exhaust the space. + *

    + * The map can support up to 2^31 keys because we use 32 bit MurmurHash. If the key cardinality is + * higher than this, you should probably be using sorting instead of hashing for better cache + * locality. + *

    + * This class is not thread safe. + */ +public final class BytesToBytesMap { + + private static final Murmur3_x86_32 HASHER = new Murmur3_x86_32(0); + + private static final HashMapGrowthStrategy growthStrategy = HashMapGrowthStrategy.DOUBLING; + + private final TaskMemoryManager memoryManager; + + /** + * A linked list for tracking all allocated data pages so that we can free all of our memory. + */ + private final List dataPages = new LinkedList(); + + /** + * The data page that will be used to store keys and values for new hashtable entries. When this + * page becomes full, a new page will be allocated and this pointer will change to point to that + * new page. + */ + private MemoryBlock currentDataPage = null; + + /** + * Offset into `currentDataPage` that points to the location where new data can be inserted into + * the page. + */ + private long pageCursor = 0; + + /** + * The size of the data pages that hold key and value data. Map entries cannot span multiple + * pages, so this limits the maximum entry size. + */ + private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes + + // This choice of page table size and page size means that we can address up to 500 gigabytes + // of memory. + + /** + * A single array to store the key and value. + * + * Position {@code 2 * i} in the array is used to track a pointer to the key at index {@code i}, + * while position {@code 2 * i + 1} in the array holds key's full 32-bit hashcode. + */ + private LongArray longArray; + // TODO: we're wasting 32 bits of space here; we can probably store fewer bits of the hashcode + // and exploit word-alignment to use fewer bits to hold the address. This might let us store + // only one long per map entry, increasing the chance that this array will fit in cache at the + // expense of maybe performing more lookups if we have hash collisions. Say that we stored only + // 27 bits of the hashcode and 37 bits of the address. 37 bits is enough to address 1 terabyte + // of RAM given word-alignment. If we use 13 bits of this for our page table, that gives us a + // maximum page size of 2^24 * 8 = ~134 megabytes per page. This change will require us to store + // full base addresses in the page table for off-heap mode so that we can reconstruct the full + // absolute memory addresses. + + /** + * A {@link BitSet} used to track location of the map where the key is set. + * Size of the bitset should be half of the size of the long array. + */ + private BitSet bitset; + + private final double loadFactor; + + /** + * Number of keys defined in the map. + */ + private int size; + + /** + * The map will be expanded once the number of keys exceeds this threshold. + */ + private int growthThreshold; + + /** + * Mask for truncating hashcodes so that they do not exceed the long array's size. + * This is a strength reduction optimization; we're essentially performing a modulus operation, + * but doing so with a bitmask because this is a power-of-2-sized hash map. + */ + private int mask; + + /** + * Return value of {@link BytesToBytesMap#lookup(Object, long, int)}. + */ + private final Location loc; + + private final boolean enablePerfMetrics; + + private long timeSpentResizingNs = 0; + + private long numProbes = 0; + + private long numKeyLookups = 0; + + private long numHashCollisions = 0; + + public BytesToBytesMap( + TaskMemoryManager memoryManager, + int initialCapacity, + double loadFactor, + boolean enablePerfMetrics) { + this.memoryManager = memoryManager; + this.loadFactor = loadFactor; + this.loc = new Location(); + this.enablePerfMetrics = enablePerfMetrics; + allocate(initialCapacity); + } + + public BytesToBytesMap(TaskMemoryManager memoryManager, int initialCapacity) { + this(memoryManager, initialCapacity, 0.70, false); + } + + public BytesToBytesMap( + TaskMemoryManager memoryManager, + int initialCapacity, + boolean enablePerfMetrics) { + this(memoryManager, initialCapacity, 0.70, enablePerfMetrics); + } + + /** + * Returns the number of keys defined in the map. + */ + public int size() { return size; } + + /** + * Returns an iterator for iterating over the entries of this map. + * + * For efficiency, all calls to `next()` will return the same {@link Location} object. + * + * If any other lookups or operations are performed on this map while iterating over it, including + * `lookup()`, the behavior of the returned iterator is undefined. + */ + public Iterator iterator() { + return new Iterator() { + + private int nextPos = bitset.nextSetBit(0); + + @Override + public boolean hasNext() { + return nextPos != -1; + } + + @Override + public Location next() { + final int pos = nextPos; + nextPos = bitset.nextSetBit(nextPos + 1); + return loc.with(pos, 0, true); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Looks up a key, and return a {@link Location} handle that can be used to test existence + * and read/write values. + * + * This function always return the same {@link Location} instance to avoid object allocation. + */ + public Location lookup( + Object keyBaseObject, + long keyBaseOffset, + int keyRowLengthBytes) { + if (enablePerfMetrics) { + numKeyLookups++; + } + final int hashcode = HASHER.hashUnsafeWords(keyBaseObject, keyBaseOffset, keyRowLengthBytes); + int pos = hashcode & mask; + int step = 1; + while (true) { + if (enablePerfMetrics) { + numProbes++; + } + if (!bitset.isSet(pos)) { + // This is a new key. + return loc.with(pos, hashcode, false); + } else { + long stored = longArray.get(pos * 2 + 1); + if ((int) (stored) == hashcode) { + // Full hash code matches. Let's compare the keys for equality. + loc.with(pos, hashcode, true); + if (loc.getKeyLength() == keyRowLengthBytes) { + final MemoryLocation keyAddress = loc.getKeyAddress(); + final Object storedKeyBaseObject = keyAddress.getBaseObject(); + final long storedKeyBaseOffset = keyAddress.getBaseOffset(); + final boolean areEqual = ByteArrayMethods.wordAlignedArrayEquals( + keyBaseObject, + keyBaseOffset, + storedKeyBaseObject, + storedKeyBaseOffset, + keyRowLengthBytes + ); + if (areEqual) { + return loc; + } else { + if (enablePerfMetrics) { + numHashCollisions++; + } + } + } + } + } + pos = (pos + step) & mask; + step++; + } + } + + /** + * Handle returned by {@link BytesToBytesMap#lookup(Object, long, int)} function. + */ + public final class Location { + /** An index into the hash map's Long array */ + private int pos; + /** True if this location points to a position where a key is defined, false otherwise */ + private boolean isDefined; + /** + * The hashcode of the most recent key passed to + * {@link BytesToBytesMap#lookup(Object, long, int)}. Caching this hashcode here allows us to + * avoid re-hashing the key when storing a value for that key. + */ + private int keyHashcode; + private final MemoryLocation keyMemoryLocation = new MemoryLocation(); + private final MemoryLocation valueMemoryLocation = new MemoryLocation(); + private int keyLength; + private int valueLength; + + private void updateAddressesAndSizes(long fullKeyAddress) { + final Object page = memoryManager.getPage(fullKeyAddress); + final long keyOffsetInPage = memoryManager.getOffsetInPage(fullKeyAddress); + long position = keyOffsetInPage; + keyLength = (int) PlatformDependent.UNSAFE.getLong(page, position); + position += 8; // word used to store the key size + keyMemoryLocation.setObjAndOffset(page, position); + position += keyLength; + valueLength = (int) PlatformDependent.UNSAFE.getLong(page, position); + position += 8; // word used to store the key size + valueMemoryLocation.setObjAndOffset(page, position); + } + + Location with(int pos, int keyHashcode, boolean isDefined) { + this.pos = pos; + this.isDefined = isDefined; + this.keyHashcode = keyHashcode; + if (isDefined) { + final long fullKeyAddress = longArray.get(pos * 2); + updateAddressesAndSizes(fullKeyAddress); + } + return this; + } + + /** + * Returns true if the key is defined at this position, and false otherwise. + */ + public boolean isDefined() { + return isDefined; + } + + /** + * Returns the address of the key defined at this position. + * This points to the first byte of the key data. + * Unspecified behavior if the key is not defined. + * For efficiency reasons, calls to this method always returns the same MemoryLocation object. + */ + public MemoryLocation getKeyAddress() { + assert (isDefined); + return keyMemoryLocation; + } + + /** + * Returns the length of the key defined at this position. + * Unspecified behavior if the key is not defined. + */ + public int getKeyLength() { + assert (isDefined); + return keyLength; + } + + /** + * Returns the address of the value defined at this position. + * This points to the first byte of the value data. + * Unspecified behavior if the key is not defined. + * For efficiency reasons, calls to this method always returns the same MemoryLocation object. + */ + public MemoryLocation getValueAddress() { + assert (isDefined); + return valueMemoryLocation; + } + + /** + * Returns the length of the value defined at this position. + * Unspecified behavior if the key is not defined. + */ + public int getValueLength() { + assert (isDefined); + return valueLength; + } + + /** + * Store a new key and value. This method may only be called once for a given key; if you want + * to update the value associated with a key, then you can directly manipulate the bytes stored + * at the value address. + *

    + * It is only valid to call this method immediately after calling `lookup()` using the same key. + *

    + * After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length` + * will return information on the data stored by this `putNewKey` call. + *

    + * As an example usage, here's the proper way to store a new key: + *

    + *

    +     *   Location loc = map.lookup(keyBaseObject, keyBaseOffset, keyLengthInBytes);
    +     *   if (!loc.isDefined()) {
    +     *     loc.putNewKey(keyBaseObject, keyBaseOffset, keyLengthInBytes, ...)
    +     *   }
    +     * 
    + *

    + * Unspecified behavior if the key is not defined. + */ + public void putNewKey( + Object keyBaseObject, + long keyBaseOffset, + int keyLengthBytes, + Object valueBaseObject, + long valueBaseOffset, + int valueLengthBytes) { + assert (!isDefined) : "Can only set value once for a key"; + isDefined = true; + assert (keyLengthBytes % 8 == 0); + assert (valueLengthBytes % 8 == 0); + // Here, we'll copy the data into our data pages. Because we only store a relative offset from + // the key address instead of storing the absolute address of the value, the key and value + // must be stored in the same memory page. + // (8 byte key length) (key) (8 byte value length) (value) + final long requiredSize = 8 + keyLengthBytes + 8 + valueLengthBytes; + assert(requiredSize <= PAGE_SIZE_BYTES); + size++; + bitset.set(pos); + + // If there's not enough space in the current page, allocate a new page: + if (currentDataPage == null || PAGE_SIZE_BYTES - pageCursor < requiredSize) { + MemoryBlock newPage = memoryManager.allocatePage(PAGE_SIZE_BYTES); + dataPages.add(newPage); + pageCursor = 0; + currentDataPage = newPage; + } + + // Compute all of our offsets up-front: + final Object pageBaseObject = currentDataPage.getBaseObject(); + final long pageBaseOffset = currentDataPage.getBaseOffset(); + final long keySizeOffsetInPage = pageBaseOffset + pageCursor; + pageCursor += 8; // word used to store the key size + final long keyDataOffsetInPage = pageBaseOffset + pageCursor; + pageCursor += keyLengthBytes; + final long valueSizeOffsetInPage = pageBaseOffset + pageCursor; + pageCursor += 8; // word used to store the value size + final long valueDataOffsetInPage = pageBaseOffset + pageCursor; + pageCursor += valueLengthBytes; + + // Copy the key + PlatformDependent.UNSAFE.putLong(pageBaseObject, keySizeOffsetInPage, keyLengthBytes); + PlatformDependent.copyMemory( + keyBaseObject, keyBaseOffset, pageBaseObject, keyDataOffsetInPage, keyLengthBytes); + // Copy the value + PlatformDependent.UNSAFE.putLong(pageBaseObject, valueSizeOffsetInPage, valueLengthBytes); + PlatformDependent.copyMemory( + valueBaseObject, valueBaseOffset, pageBaseObject, valueDataOffsetInPage, valueLengthBytes); + + final long storedKeyAddress = memoryManager.encodePageNumberAndOffset( + currentDataPage, keySizeOffsetInPage); + longArray.set(pos * 2, storedKeyAddress); + longArray.set(pos * 2 + 1, keyHashcode); + updateAddressesAndSizes(storedKeyAddress); + isDefined = true; + if (size > growthThreshold) { + growAndRehash(); + } + } + } + + /** + * Allocate new data structures for this map. When calling this outside of the constructor, + * make sure to keep references to the old data structures so that you can free them. + * + * @param capacity the new map capacity + */ + private void allocate(int capacity) { + capacity = Math.max((int) Math.min(Integer.MAX_VALUE, nextPowerOf2(capacity)), 64); + longArray = new LongArray(memoryManager.allocate(capacity * 8 * 2)); + bitset = new BitSet(MemoryBlock.fromLongArray(new long[capacity / 64])); + + this.growthThreshold = (int) (capacity * loadFactor); + this.mask = capacity - 1; + } + + /** + * Free all allocated memory associated with this map, including the storage for keys and values + * as well as the hash map array itself. + * + * This method is idempotent. + */ + public void free() { + if (longArray != null) { + memoryManager.free(longArray.memoryBlock()); + longArray = null; + } + if (bitset != null) { + // The bitset's heap memory isn't managed by a memory manager, so no need to free it here. + bitset = null; + } + Iterator dataPagesIterator = dataPages.iterator(); + while (dataPagesIterator.hasNext()) { + memoryManager.freePage(dataPagesIterator.next()); + dataPagesIterator.remove(); + } + assert(dataPages.isEmpty()); + } + + /** Returns the total amount of memory, in bytes, consumed by this map's managed structures. */ + public long getTotalMemoryConsumption() { + return ( + dataPages.size() * PAGE_SIZE_BYTES + + bitset.memoryBlock().size() + + longArray.memoryBlock().size()); + } + + /** + * Returns the total amount of time spent resizing this map (in nanoseconds). + */ + public long getTimeSpentResizingNs() { + if (!enablePerfMetrics) { + throw new IllegalStateException(); + } + return timeSpentResizingNs; + } + + + /** + * Returns the average number of probes per key lookup. + */ + public double getAverageProbesPerLookup() { + if (!enablePerfMetrics) { + throw new IllegalStateException(); + } + return (1.0 * numProbes) / numKeyLookups; + } + + public long getNumHashCollisions() { + if (!enablePerfMetrics) { + throw new IllegalStateException(); + } + return numHashCollisions; + } + + /** + * Grows the size of the hash table and re-hash everything. + */ + private void growAndRehash() { + long resizeStartTime = -1; + if (enablePerfMetrics) { + resizeStartTime = System.nanoTime(); + } + // Store references to the old data structures to be used when we re-hash + final LongArray oldLongArray = longArray; + final BitSet oldBitSet = bitset; + final int oldCapacity = (int) oldBitSet.capacity(); + + // Allocate the new data structures + allocate(Math.min(Integer.MAX_VALUE, growthStrategy.nextCapacity(oldCapacity))); + + // Re-mask (we don't recompute the hashcode because we stored all 32 bits of it) + for (int pos = oldBitSet.nextSetBit(0); pos >= 0; pos = oldBitSet.nextSetBit(pos + 1)) { + final long keyPointer = oldLongArray.get(pos * 2); + final int hashcode = (int) oldLongArray.get(pos * 2 + 1); + int newPos = hashcode & mask; + int step = 1; + boolean keepGoing = true; + + // No need to check for equality here when we insert so this has one less if branch than + // the similar code path in addWithoutResize. + while (keepGoing) { + if (!bitset.isSet(newPos)) { + bitset.set(newPos); + longArray.set(newPos * 2, keyPointer); + longArray.set(newPos * 2 + 1, hashcode); + keepGoing = false; + } else { + newPos = (newPos + step) & mask; + step++; + } + } + } + + // Deallocate the old data structures. + memoryManager.free(oldLongArray.memoryBlock()); + if (enablePerfMetrics) { + timeSpentResizingNs += System.nanoTime() - resizeStartTime; + } + } + + /** Returns the next number greater or equal num that is power of 2. */ + private static long nextPowerOf2(long num) { + final long highBit = Long.highestOneBit(num); + return (highBit == num) ? num : highBit << 1; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java b/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java new file mode 100644 index 0000000000000..7c321baffe82d --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java @@ -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. + */ + +package org.apache.spark.unsafe.map; + +/** + * Interface that defines how we can grow the size of a hash map when it is over a threshold. + */ +public interface HashMapGrowthStrategy { + + int nextCapacity(int currentCapacity); + + /** + * Double the size of the hash map every time. + */ + HashMapGrowthStrategy DOUBLING = new Doubling(); + + class Doubling implements HashMapGrowthStrategy { + @Override + public int nextCapacity(int currentCapacity) { + return currentCapacity * 2; + } + } + +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java new file mode 100644 index 0000000000000..62c29c8cc1e4d --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/ExecutorMemoryManager.java @@ -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.unsafe.memory; + +/** + * Manages memory for an executor. Individual operators / tasks allocate memory through + * {@link TaskMemoryManager} objects, which obtain their memory from ExecutorMemoryManager. + */ +public class ExecutorMemoryManager { + + /** + * Allocator, exposed for enabling untracked allocations of temporary data structures. + */ + public final MemoryAllocator allocator; + + /** + * Tracks whether memory will be allocated on the JVM heap or off-heap using sun.misc.Unsafe. + */ + final boolean inHeap; + + /** + * Construct a new ExecutorMemoryManager. + * + * @param allocator the allocator that will be used + */ + public ExecutorMemoryManager(MemoryAllocator allocator) { + this.inHeap = allocator instanceof HeapMemoryAllocator; + this.allocator = allocator; + } + + /** + * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed + * to be zeroed out (call `zero()` on the result if this is necessary). + */ + MemoryBlock allocate(long size) throws OutOfMemoryError { + return allocator.allocate(size); + } + + void free(MemoryBlock memory) { + allocator.free(memory); + } + +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java new file mode 100644 index 0000000000000..bbe83d36cf36b --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -0,0 +1,35 @@ +/* + * 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.unsafe.memory; + +/** + * A simple {@link MemoryAllocator} that can allocate up to 16GB using a JVM long primitive array. + */ +public class HeapMemoryAllocator implements MemoryAllocator { + + @Override + public MemoryBlock allocate(long size) throws OutOfMemoryError { + long[] array = new long[(int) (size / 8)]; + return MemoryBlock.fromLongArray(array); + } + + @Override + public void free(MemoryBlock memory) { + // Do nothing + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java new file mode 100644 index 0000000000000..5192f68c862cf --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java @@ -0,0 +1,33 @@ +/* + * 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.unsafe.memory; + +public interface MemoryAllocator { + + /** + * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed + * to be zeroed out (call `zero()` on the result if this is necessary). + */ + MemoryBlock allocate(long size) throws OutOfMemoryError; + + void free(MemoryBlock memory); + + MemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); + + MemoryAllocator HEAP = new HeapMemoryAllocator(); +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java new file mode 100644 index 0000000000000..3dc82d8c2eb39 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -0,0 +1,55 @@ +/* + * 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.unsafe.memory; + +import javax.annotation.Nullable; + +import org.apache.spark.unsafe.PlatformDependent; + +/** + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. + */ +public class MemoryBlock extends MemoryLocation { + + private final long length; + + /** + * Optional page number; used when this MemoryBlock represents a page allocated by a + * MemoryManager. This is package-private and is modified by MemoryManager. + */ + int pageNumber = -1; + + MemoryBlock(@Nullable Object obj, long offset, long length) { + super(obj, offset); + this.length = length; + } + + /** + * Returns the size of the memory block. + */ + public long size() { + return length; + } + + /** + * Creates a memory block pointing to the memory used by the long array. + */ + public static MemoryBlock fromLongArray(final long[] array) { + return new MemoryBlock(array, PlatformDependent.LONG_ARRAY_OFFSET, array.length * 8); + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java new file mode 100644 index 0000000000000..74ebc87dc978c --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java @@ -0,0 +1,54 @@ +/* + * 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.unsafe.memory; + +import javax.annotation.Nullable; + +/** + * A memory location. Tracked either by a memory address (with off-heap allocation), + * or by an offset from a JVM object (in-heap allocation). + */ +public class MemoryLocation { + + @Nullable + Object obj; + + long offset; + + public MemoryLocation(@Nullable Object obj, long offset) { + this.obj = obj; + this.offset = offset; + } + + public MemoryLocation() { + this(null, 0); + } + + public void setObjAndOffset(Object newObj, long newOffset) { + this.obj = newObj; + this.offset = newOffset; + } + + public final Object getBaseObject() { + return obj; + } + + public final long getBaseOffset() { + return offset; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java new file mode 100644 index 0000000000000..9224988e6ad69 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/TaskMemoryManager.java @@ -0,0 +1,237 @@ +/* + * 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.unsafe.memory; + +import java.util.*; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages the memory allocated by an individual task. + *

    + * Most of the complexity in this class deals with encoding of off-heap addresses into 64-bit longs. + * In off-heap mode, memory can be directly addressed with 64-bit longs. In on-heap mode, memory is + * addressed by the combination of a base Object reference and a 64-bit offset within that object. + * This is a problem when we want to store pointers to data structures inside of other structures, + * such as record pointers inside hashmaps or sorting buffers. Even if we decided to use 128 bits + * to address memory, we can't just store the address of the base object since it's not guaranteed + * to remain stable as the heap gets reorganized due to GC. + *

    + * Instead, we use the following approach to encode record pointers in 64-bit longs: for off-heap + * mode, just store the raw address, and for on-heap mode use the upper 13 bits of the address to + * store a "page number" and the lower 51 bits to store an offset within this page. These page + * numbers are used to index into a "page table" array inside of the MemoryManager in order to + * retrieve the base object. + *

    + * This allows us to address 8192 pages. In on-heap mode, the maximum page size is limited by the + * maximum size of a long[] array, allowing us to address 8192 * 2^32 * 8 bytes, which is + * approximately 35 terabytes of memory. + */ +public final class TaskMemoryManager { + + private final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class); + + /** + * The number of entries in the page table. + */ + private static final int PAGE_TABLE_SIZE = 1 << 13; + + /** 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 + * representation and the baseObject+offset representation which we use to support both in- and + * off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`. + * When using an in-heap allocator, the entries in this map will point to pages' base objects. + * Entries are added to this map as new data pages are allocated. + */ + private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; + + /** + * Bitmap for tracking free pages. + */ + private final BitSet allocatedPages = new BitSet(PAGE_TABLE_SIZE); + + /** + * Tracks memory allocated with {@link TaskMemoryManager#allocate(long)}, used to detect / clean + * up leaked memory. + */ + private final HashSet allocatedNonPageMemory = new HashSet(); + + private final ExecutorMemoryManager executorMemoryManager; + + /** + * Tracks whether we're in-heap or off-heap. For off-heap, we short-circuit most of these methods + * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, + * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. + */ + private final boolean inHeap; + + /** + * Construct a new MemoryManager. + */ + public TaskMemoryManager(ExecutorMemoryManager executorMemoryManager) { + this.inHeap = executorMemoryManager.inHeap; + this.executorMemoryManager = executorMemoryManager; + } + + /** + * Allocate a block of memory that will be tracked in the MemoryManager's page table; this is + * intended for allocating large blocks of memory that will be shared between operators. + */ + public MemoryBlock allocatePage(long size) { + if (logger.isTraceEnabled()) { + logger.trace("Allocating {} byte page", size); + } + if (size >= (1L << 51)) { + throw new IllegalArgumentException("Cannot allocate a page with more than 2^51 bytes"); + } + + final int pageNumber; + synchronized (this) { + pageNumber = allocatedPages.nextClearBit(0); + if (pageNumber >= PAGE_TABLE_SIZE) { + throw new IllegalStateException( + "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); + } + allocatedPages.set(pageNumber); + } + final MemoryBlock page = executorMemoryManager.allocate(size); + page.pageNumber = pageNumber; + pageTable[pageNumber] = page; + if (logger.isDebugEnabled()) { + logger.debug("Allocate page number {} ({} bytes)", pageNumber, size); + } + return page; + } + + /** + * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage(long)}. + */ + public void freePage(MemoryBlock page) { + if (logger.isTraceEnabled()) { + logger.trace("Freeing page number {} ({} bytes)", page.pageNumber, page.size()); + } + assert (page.pageNumber != -1) : + "Called freePage() on memory that wasn't allocated with allocatePage()"; + executorMemoryManager.free(page); + synchronized (this) { + allocatedPages.clear(page.pageNumber); + } + pageTable[page.pageNumber] = null; + if (logger.isDebugEnabled()) { + logger.debug("Freed page number {} ({} bytes)", page.pageNumber, page.size()); + } + } + + /** + * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed + * to be zeroed out (call `zero()` on the result if this is necessary). This method is intended + * to be used for allocating operators' internal data structures. For data pages that you want to + * exchange between operators, consider using {@link TaskMemoryManager#allocatePage(long)}, since + * that will enable intra-memory pointers (see + * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} and this class's + * top-level Javadoc for more details). + */ + public MemoryBlock allocate(long size) throws OutOfMemoryError { + final MemoryBlock memory = executorMemoryManager.allocate(size); + allocatedNonPageMemory.add(memory); + return memory; + } + + /** + * Free memory allocated by {@link TaskMemoryManager#allocate(long)}. + */ + public void free(MemoryBlock memory) { + assert (memory.pageNumber == -1) : "Should call freePage() for pages, not free()"; + executorMemoryManager.free(memory); + final boolean wasAlreadyRemoved = !allocatedNonPageMemory.remove(memory); + assert (!wasAlreadyRemoved) : "Called free() on memory that was already freed!"; + } + + /** + * Given a memory page and offset within that page, encode this address into a 64-bit long. + * This address will remain valid as long as the corresponding page has not been freed. + */ + public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { + if (inHeap) { + assert (page.pageNumber != -1) : "encodePageNumberAndOffset called with invalid page"; + return (((long) page.pageNumber) << 51) | (offsetInPage & MASK_LONG_LOWER_51_BITS); + } else { + return offsetInPage; + } + } + + /** + * Get the page associated with an address encoded by + * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} + */ + public Object getPage(long pagePlusOffsetAddress) { + if (inHeap) { + final int pageNumber = (int) ((pagePlusOffsetAddress & MASK_LONG_UPPER_13_BITS) >>> 51); + assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); + final Object page = pageTable[pageNumber].getBaseObject(); + assert (page != null); + return page; + } else { + return null; + } + } + + /** + * Get the offset associated with an address encoded by + * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} + */ + public long getOffsetInPage(long pagePlusOffsetAddress) { + if (inHeap) { + return (pagePlusOffsetAddress & MASK_LONG_LOWER_51_BITS); + } else { + return pagePlusOffsetAddress; + } + } + + /** + * Clean up all allocated memory and pages. Returns the number of bytes freed. A non-zero return + * value can be used to detect memory leaks. + */ + public long cleanUpAllAllocatedMemory() { + long freedBytes = 0; + for (MemoryBlock page : pageTable) { + if (page != null) { + freedBytes += page.size(); + freePage(page); + } + } + final Iterator iter = allocatedNonPageMemory.iterator(); + while (iter.hasNext()) { + final MemoryBlock memory = iter.next(); + freedBytes += memory.size(); + // We don't call free() here because that calls Set.remove, which would lead to a + // ConcurrentModificationException here. + executorMemoryManager.free(memory); + iter.remove(); + } + return freedBytes; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java new file mode 100644 index 0000000000000..15898771fef25 --- /dev/null +++ b/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -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. + */ + +package org.apache.spark.unsafe.memory; + +import org.apache.spark.unsafe.PlatformDependent; + +/** + * A simple {@link MemoryAllocator} that uses {@code Unsafe} to allocate off-heap memory. + */ +public class UnsafeMemoryAllocator implements MemoryAllocator { + + @Override + public MemoryBlock allocate(long size) throws OutOfMemoryError { + long address = PlatformDependent.UNSAFE.allocateMemory(size); + return new MemoryBlock(null, address, size); + } + + @Override + public void free(MemoryBlock memory) { + assert (memory.obj == null) : + "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; + PlatformDependent.UNSAFE.freeMemory(memory.offset); + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java new file mode 100644 index 0000000000000..5974cf91ff993 --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -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.unsafe.array; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.unsafe.memory.MemoryBlock; + +public class LongArraySuite { + + @Test + public void basicTest() { + long[] bytes = new long[2]; + LongArray arr = new LongArray(MemoryBlock.fromLongArray(bytes)); + arr.set(0, 1L); + arr.set(1, 2L); + arr.set(1, 3L); + Assert.assertEquals(2, arr.size()); + Assert.assertEquals(1L, arr.get(0)); + Assert.assertEquals(3L, arr.get(1)); + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java new file mode 100644 index 0000000000000..e3a824e29b768 --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/bitset/BitSetSuite.java @@ -0,0 +1,82 @@ +/* + * 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.unsafe.bitset; + +import junit.framework.Assert; +import org.apache.spark.unsafe.bitset.BitSet; +import org.junit.Test; + +import org.apache.spark.unsafe.memory.MemoryBlock; + +public class BitSetSuite { + + private static BitSet createBitSet(int capacity) { + assert capacity % 64 == 0; + return new BitSet(MemoryBlock.fromLongArray(new long[capacity / 64])); + } + + @Test + public void basicOps() { + BitSet bs = createBitSet(64); + Assert.assertEquals(64, bs.capacity()); + + // Make sure the bit set starts empty. + for (int i = 0; i < bs.capacity(); i++) { + Assert.assertFalse(bs.isSet(i)); + } + + // Set every bit and check it. + for (int i = 0; i < bs.capacity(); i++) { + bs.set(i); + Assert.assertTrue(bs.isSet(i)); + } + + // Unset every bit and check it. + for (int i = 0; i < bs.capacity(); i++) { + Assert.assertTrue(bs.isSet(i)); + bs.unset(i); + Assert.assertFalse(bs.isSet(i)); + } + } + + @Test + public void traversal() { + BitSet bs = createBitSet(256); + + Assert.assertEquals(-1, bs.nextSetBit(0)); + Assert.assertEquals(-1, bs.nextSetBit(10)); + Assert.assertEquals(-1, bs.nextSetBit(64)); + + bs.set(10); + Assert.assertEquals(10, bs.nextSetBit(0)); + Assert.assertEquals(10, bs.nextSetBit(1)); + Assert.assertEquals(10, bs.nextSetBit(10)); + Assert.assertEquals(-1, bs.nextSetBit(11)); + + bs.set(11); + Assert.assertEquals(10, bs.nextSetBit(10)); + Assert.assertEquals(11, bs.nextSetBit(11)); + + // Skip a whole word and find it + bs.set(190); + Assert.assertEquals(190, bs.nextSetBit(12)); + + Assert.assertEquals(-1, bs.nextSetBit(191)); + Assert.assertEquals(-1, bs.nextSetBit(256)); + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java b/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java new file mode 100644 index 0000000000000..3b9175835229c --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java @@ -0,0 +1,119 @@ +/* + * 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.unsafe.hash; + +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +import junit.framework.Assert; +import org.apache.spark.unsafe.PlatformDependent; +import org.junit.Test; + +/** + * Test file based on Guava's Murmur3Hash32Test. + */ +public class Murmur3_x86_32Suite { + + private static final Murmur3_x86_32 hasher = new Murmur3_x86_32(0); + + @Test + public void testKnownIntegerInputs() { + Assert.assertEquals(593689054, hasher.hashInt(0)); + Assert.assertEquals(-189366624, hasher.hashInt(-42)); + Assert.assertEquals(-1134849565, hasher.hashInt(42)); + Assert.assertEquals(-1718298732, hasher.hashInt(Integer.MIN_VALUE)); + Assert.assertEquals(-1653689534, hasher.hashInt(Integer.MAX_VALUE)); + } + + @Test + public void testKnownLongInputs() { + Assert.assertEquals(1669671676, hasher.hashLong(0L)); + Assert.assertEquals(-846261623, hasher.hashLong(-42L)); + Assert.assertEquals(1871679806, hasher.hashLong(42L)); + Assert.assertEquals(1366273829, hasher.hashLong(Long.MIN_VALUE)); + Assert.assertEquals(-2106506049, hasher.hashLong(Long.MAX_VALUE)); + } + + @Test + public void randomizedStressTest() { + int size = 65536; + Random rand = new Random(); + + // A set used to track collision rate. + Set hashcodes = new HashSet(); + for (int i = 0; i < size; i++) { + int vint = rand.nextInt(); + long lint = rand.nextLong(); + Assert.assertEquals(hasher.hashInt(vint), hasher.hashInt(vint)); + Assert.assertEquals(hasher.hashLong(lint), hasher.hashLong(lint)); + + hashcodes.add(hasher.hashLong(lint)); + } + + // A very loose bound. + Assert.assertTrue(hashcodes.size() > size * 0.95); + } + + @Test + public void randomizedStressTestBytes() { + int size = 65536; + Random rand = new Random(); + + // A set used to track collision rate. + Set hashcodes = new HashSet(); + for (int i = 0; i < size; i++) { + int byteArrSize = rand.nextInt(100) * 8; + byte[] bytes = new byte[byteArrSize]; + rand.nextBytes(bytes); + + Assert.assertEquals( + hasher.hashUnsafeWords(bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize), + hasher.hashUnsafeWords(bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize)); + + hashcodes.add(hasher.hashUnsafeWords( + bytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize)); + } + + // A very loose bound. + Assert.assertTrue(hashcodes.size() > size * 0.95); + } + + @Test + public void randomizedStressTestPaddedStrings() { + int size = 64000; + // A set used to track collision rate. + Set hashcodes = new HashSet(); + for (int i = 0; i < size; i++) { + int byteArrSize = 8; + byte[] strBytes = ("" + i).getBytes(); + byte[] paddedBytes = new byte[byteArrSize]; + System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length); + + Assert.assertEquals( + hasher.hashUnsafeWords(paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize), + hasher.hashUnsafeWords(paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize)); + + hashcodes.add(hasher.hashUnsafeWords( + paddedBytes, PlatformDependent.BYTE_ARRAY_OFFSET, byteArrSize)); + } + + // A very loose bound. + Assert.assertTrue(hashcodes.size() > size * 0.95); + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java new file mode 100644 index 0000000000000..7a5c0622d1ffb --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -0,0 +1,250 @@ +/* + * 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.unsafe.map; + +import java.lang.Exception; +import java.nio.ByteBuffer; +import java.util.*; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.PlatformDependent; +import static org.apache.spark.unsafe.PlatformDependent.BYTE_ARRAY_OFFSET; +import org.apache.spark.unsafe.memory.ExecutorMemoryManager; +import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.MemoryLocation; +import org.apache.spark.unsafe.memory.TaskMemoryManager; + +public abstract class AbstractBytesToBytesMapSuite { + + private final Random rand = new Random(42); + + private TaskMemoryManager memoryManager; + + @Before + public void setup() { + memoryManager = new TaskMemoryManager(new ExecutorMemoryManager(getMemoryAllocator())); + } + + @After + public void tearDown() { + if (memoryManager != null) { + memoryManager.cleanUpAllAllocatedMemory(); + memoryManager = null; + } + } + + protected abstract MemoryAllocator getMemoryAllocator(); + + private static byte[] getByteArray(MemoryLocation loc, int size) { + final byte[] arr = new byte[size]; + PlatformDependent.copyMemory( + loc.getBaseObject(), + loc.getBaseOffset(), + arr, + BYTE_ARRAY_OFFSET, + size + ); + return arr; + } + + private byte[] getRandomByteArray(int numWords) { + Assert.assertTrue(numWords > 0); + final int lengthInBytes = numWords * 8; + final byte[] bytes = new byte[lengthInBytes]; + rand.nextBytes(bytes); + return bytes; + } + + /** + * Fast equality checking for byte arrays, since these comparisons are a bottleneck + * in our stress tests. + */ + private static boolean arrayEquals( + byte[] expected, + MemoryLocation actualAddr, + long actualLengthBytes) { + return (actualLengthBytes == expected.length) && ByteArrayMethods.wordAlignedArrayEquals( + expected, + BYTE_ARRAY_OFFSET, + actualAddr.getBaseObject(), + actualAddr.getBaseOffset(), + expected.length + ); + } + + @Test + public void emptyMap() { + BytesToBytesMap map = new BytesToBytesMap(memoryManager, 64); + try { + Assert.assertEquals(0, map.size()); + final int keyLengthInWords = 10; + final int keyLengthInBytes = keyLengthInWords * 8; + final byte[] key = getRandomByteArray(keyLengthInWords); + Assert.assertFalse(map.lookup(key, BYTE_ARRAY_OFFSET, keyLengthInBytes).isDefined()); + } finally { + map.free(); + } + } + + @Test + public void setAndRetrieveAKey() { + BytesToBytesMap map = new BytesToBytesMap(memoryManager, 64); + final int recordLengthWords = 10; + final int recordLengthBytes = recordLengthWords * 8; + final byte[] keyData = getRandomByteArray(recordLengthWords); + final byte[] valueData = getRandomByteArray(recordLengthWords); + try { + final BytesToBytesMap.Location loc = + map.lookup(keyData, BYTE_ARRAY_OFFSET, recordLengthBytes); + Assert.assertFalse(loc.isDefined()); + loc.putNewKey( + keyData, + BYTE_ARRAY_OFFSET, + recordLengthBytes, + valueData, + BYTE_ARRAY_OFFSET, + recordLengthBytes + ); + // After storing the key and value, the other location methods should return results that + // reflect the result of this store without us having to call lookup() again on the same key. + Assert.assertEquals(recordLengthBytes, loc.getKeyLength()); + Assert.assertEquals(recordLengthBytes, loc.getValueLength()); + Assert.assertArrayEquals(keyData, getByteArray(loc.getKeyAddress(), recordLengthBytes)); + Assert.assertArrayEquals(valueData, getByteArray(loc.getValueAddress(), recordLengthBytes)); + + // After calling lookup() the location should still point to the correct data. + Assert.assertTrue(map.lookup(keyData, BYTE_ARRAY_OFFSET, recordLengthBytes).isDefined()); + Assert.assertEquals(recordLengthBytes, loc.getKeyLength()); + Assert.assertEquals(recordLengthBytes, loc.getValueLength()); + Assert.assertArrayEquals(keyData, getByteArray(loc.getKeyAddress(), recordLengthBytes)); + Assert.assertArrayEquals(valueData, getByteArray(loc.getValueAddress(), recordLengthBytes)); + + try { + loc.putNewKey( + keyData, + BYTE_ARRAY_OFFSET, + recordLengthBytes, + valueData, + BYTE_ARRAY_OFFSET, + recordLengthBytes + ); + Assert.fail("Should not be able to set a new value for a key"); + } catch (AssertionError e) { + // Expected exception; do nothing. + } + } finally { + map.free(); + } + } + + @Test + public void iteratorTest() throws Exception { + final int size = 128; + BytesToBytesMap map = new BytesToBytesMap(memoryManager, size / 2); + try { + for (long i = 0; i < size; i++) { + final long[] value = new long[] { i }; + final BytesToBytesMap.Location loc = + map.lookup(value, PlatformDependent.LONG_ARRAY_OFFSET, 8); + Assert.assertFalse(loc.isDefined()); + loc.putNewKey( + value, + PlatformDependent.LONG_ARRAY_OFFSET, + 8, + value, + PlatformDependent.LONG_ARRAY_OFFSET, + 8 + ); + } + final java.util.BitSet valuesSeen = new java.util.BitSet(size); + final Iterator iter = map.iterator(); + while (iter.hasNext()) { + final BytesToBytesMap.Location loc = iter.next(); + Assert.assertTrue(loc.isDefined()); + final MemoryLocation keyAddress = loc.getKeyAddress(); + final MemoryLocation valueAddress = loc.getValueAddress(); + final long key = PlatformDependent.UNSAFE.getLong( + keyAddress.getBaseObject(), keyAddress.getBaseOffset()); + final long value = PlatformDependent.UNSAFE.getLong( + valueAddress.getBaseObject(), valueAddress.getBaseOffset()); + Assert.assertEquals(key, value); + valuesSeen.set((int) value); + } + Assert.assertEquals(size, valuesSeen.cardinality()); + } finally { + map.free(); + } + } + + @Test + public void randomizedStressTest() { + final int size = 65536; + // Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays + // into ByteBuffers in order to use them as keys here. + final Map expected = new HashMap(); + final BytesToBytesMap map = new BytesToBytesMap(memoryManager, size); + + try { + // Fill the map to 90% full so that we can trigger probing + for (int i = 0; i < size * 0.9; i++) { + final byte[] key = getRandomByteArray(rand.nextInt(256) + 1); + final byte[] value = getRandomByteArray(rand.nextInt(512) + 1); + if (!expected.containsKey(ByteBuffer.wrap(key))) { + expected.put(ByteBuffer.wrap(key), value); + final BytesToBytesMap.Location loc = map.lookup( + key, + BYTE_ARRAY_OFFSET, + key.length + ); + Assert.assertFalse(loc.isDefined()); + loc.putNewKey( + key, + BYTE_ARRAY_OFFSET, + key.length, + value, + BYTE_ARRAY_OFFSET, + value.length + ); + // After calling putNewKey, the following should be true, even before calling + // lookup(): + Assert.assertTrue(loc.isDefined()); + Assert.assertEquals(key.length, loc.getKeyLength()); + Assert.assertEquals(value.length, loc.getValueLength()); + Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), key.length)); + Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), value.length)); + } + } + + for (Map.Entry entry : expected.entrySet()) { + final byte[] key = entry.getKey().array(); + final byte[] value = entry.getValue(); + final BytesToBytesMap.Location loc = map.lookup(key, BYTE_ARRAY_OFFSET, key.length); + Assert.assertTrue(loc.isDefined()); + Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), loc.getKeyLength())); + Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), loc.getValueLength())); + } + } finally { + map.free(); + } + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOffHeapSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOffHeapSuite.java new file mode 100644 index 0000000000000..5a10de49f54fe --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOffHeapSuite.java @@ -0,0 +1,29 @@ +/* + * 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.unsafe.map; + +import org.apache.spark.unsafe.memory.MemoryAllocator; + +public class BytesToBytesMapOffHeapSuite extends AbstractBytesToBytesMapSuite { + + @Override + protected MemoryAllocator getMemoryAllocator() { + return MemoryAllocator.UNSAFE; + } + +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOnHeapSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOnHeapSuite.java new file mode 100644 index 0000000000000..12cc9b25d93b3 --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/map/BytesToBytesMapOnHeapSuite.java @@ -0,0 +1,29 @@ +/* + * 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.unsafe.map; + +import org.apache.spark.unsafe.memory.MemoryAllocator; + +public class BytesToBytesMapOnHeapSuite extends AbstractBytesToBytesMapSuite { + + @Override + protected MemoryAllocator getMemoryAllocator() { + return MemoryAllocator.HEAP; + } + +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/memory/TaskMemoryManagerSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/memory/TaskMemoryManagerSuite.java new file mode 100644 index 0000000000000..932882f1ca248 --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/memory/TaskMemoryManagerSuite.java @@ -0,0 +1,41 @@ +/* + * 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.unsafe.memory; + +import org.junit.Assert; +import org.junit.Test; + +public class TaskMemoryManagerSuite { + + @Test + public void leakedNonPageMemoryIsDetected() { + final TaskMemoryManager manager = + new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)); + manager.allocate(1024); // leak memory + Assert.assertEquals(1024, manager.cleanUpAllAllocatedMemory()); + } + + @Test + public void leakedPageMemoryIsDetected() { + final TaskMemoryManager manager = + new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)); + manager.allocatePage(4096); // leak memory + Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); + } + +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala new file mode 100644 index 0000000000000..aaae6f9734a85 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -0,0 +1,205 @@ +/* + * 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.yarn + +import java.security.PrivilegedExceptionAction +import java.util.concurrent.{Executors, TimeUnit} + +import scala.language.postfixOps + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.ThreadUtils + +/* + * The following methods are primarily meant to make sure long-running apps like Spark + * Streaming apps can run without interruption while writing to secure HDFS. The + * scheduleLoginFromKeytab method is called on the driver when the + * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC + * once 75% of the renewal interval of the original delegation tokens used for the container + * has elapsed. It then creates new delegation tokens and writes them to HDFS in a + * pre-specified location - the prefix of which is specified in the sparkConf by + * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes + * to a new file, with a monotonically increasing suffix). After this, the credentials are + * updated once 75% of the new tokens renewal interval has elapsed. + * + * On the executor side, the updateCredentialsIfRequired method is called once 80% of the + * validity of the original tokens has elapsed. At that time the executor finds the + * credentials file with the latest timestamp and checks if it has read those credentials + * before (by keeping track of the suffix of the last file it read). If a new file has + * appeared, it will read the credentials and update the currently running UGI with it. This + * process happens again once 80% of the validity of this has expired. + */ +private[yarn] class AMDelegationTokenRenewer( + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + private var lastCredentialsFileSuffix = 0 + + private val delegationTokenRenewer = + Executors.newSingleThreadScheduledExecutor( + ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) + + private val hadoopUtil = YarnSparkHadoopUtil.get + + private val daysToKeepFiles = sparkConf.getInt("spark.yarn.credentials.file.retention.days", 5) + private val numFilesToKeep = sparkConf.getInt("spark.yarn.credentials.file.retention.count", 5) + + /** + * Schedule a login from the keytab and principal set using the --principal and --keytab + * arguments to spark-submit. This login happens only when the credentials of the current user + * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from + * SparkConf to do the login. This method is a no-op in non-YARN mode. + * + */ + private[spark] def scheduleLoginFromKeytab(): Unit = { + val principal = sparkConf.get("spark.yarn.principal") + val keytab = sparkConf.get("spark.yarn.keytab") + + /** + * Schedule re-login and creation of new tokens. If tokens have already expired, this method + * will synchronously create new ones. + */ + def scheduleRenewal(runnable: Runnable): Unit = { + val credentials = UserGroupInformation.getCurrentUser.getCredentials + val renewalInterval = hadoopUtil.getTimeFromNowToRenewal(sparkConf, 0.75, credentials) + // Run now! + if (renewalInterval <= 0) { + logInfo("HDFS tokens have expired, creating new tokens now.") + runnable.run() + } else { + logInfo(s"Scheduling login from keytab in $renewalInterval millis.") + delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) + } + } + + // This thread periodically runs on the driver to update the delegation tokens on HDFS. + val driverTokenRenewerRunnable = + new Runnable { + override def run(): Unit = { + try { + writeNewTokensToHDFS(principal, keytab) + cleanupOldFiles() + } catch { + case e: Exception => + // Log the error and try to write new tokens back in an hour + logWarning("Failed to write out new credentials to HDFS, will try again in an " + + "hour! If this happens too often tasks will fail.", e) + delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) + return + } + scheduleRenewal(this) + } + } + // Schedule update of credentials. This handles the case of updating the tokens right now + // as well, since the renenwal interval will be 0, and the thread will get scheduled + // immediately. + scheduleRenewal(driverTokenRenewerRunnable) + } + + // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At + // least numFilesToKeep files are kept for safety + private def cleanupOldFiles(): Unit = { + import scala.concurrent.duration._ + try { + val remoteFs = FileSystem.get(hadoopConf) + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles days).toMillis + hadoopUtil.listFilesSorted( + remoteFs, credentialsPath.getParent, + credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + .dropRight(numFilesToKeep) + .takeWhile(_.getModificationTime < thresholdTime) + .foreach(x => remoteFs.delete(x.getPath, true)) + } catch { + // Such errors are not fatal, so don't throw. Make sure they are logged though + case e: Exception => + logWarning("Error while attempting to cleanup old tokens. If you are seeing many such " + + "warnings there may be an issue with your HDFS cluster.", e) + } + } + + private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { + // Keytab is copied by YARN to the working directory of the AM, so full path is + // not needed. + + // HACK: + // HDFS will not issue new delegation tokens, if the Credentials object + // passed in already has tokens for that FS even if the tokens are expired (it really only + // checks if there are tokens for the service, and not if they are valid). So the only real + // way to get new tokens is to make sure a different Credentials object is used each time to + // get new tokens and then the new tokens are copied over the the current user's Credentials. + // So: + // - we login as a different user and get the UGI + // - use that UGI to get the tokens (see doAs block below) + // - copy the tokens over to the current user's credentials (this will overwrite the tokens + // in the current user's Credentials object for this FS). + // The login to KDC happens each time new tokens are required, but this is rare enough to not + // have to worry about (like once every day or so). This makes this code clearer than having + // to login and then relogin every time (the HDFS API may not relogin since we don't use this + // UGI directly for HDFS communication. + logInfo(s"Attempting to login to KDC using principal: $principal") + val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + logInfo("Successfully logged into KDC.") + val tempCreds = keytabLoggedInUGI.getCredentials + val credentialsPath = new Path(sparkConf.get("spark.yarn.credentials.file")) + val dst = credentialsPath.getParent + keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { + // Get a copy of the credentials + override def run(): Void = { + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst + hadoopUtil.obtainTokensForNamenodes(nns, hadoopConf, tempCreds) + null + } + }) + // Add the temp credentials back to the original ones. + UserGroupInformation.getCurrentUser.addCredentials(tempCreds) + val remoteFs = FileSystem.get(hadoopConf) + // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM + // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file + // and update the lastCredentialsFileSuffix. + if (lastCredentialsFileSuffix == 0) { + hadoopUtil.listFilesSorted( + remoteFs, credentialsPath.getParent, + credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + .lastOption.foreach { status => + lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath) + } + } + val nextSuffix = lastCredentialsFileSuffix + 1 + val tokenPathStr = + sparkConf.get("spark.yarn.credentials.file") + + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM + nextSuffix + val tokenPath = new Path(tokenPathStr) + val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + logInfo("Writing out delegation tokens to " + tempTokenPath.toString) + val credentials = UserGroupInformation.getCurrentUser.getCredentials + credentials.writeTokenStorageFile(tempTokenPath, hadoopConf) + logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") + remoteFs.rename(tempTokenPath, tokenPath) + logInfo("Delegation token file rename complete.") + lastCredentialsFileSuffix = nextSuffix + } + + def stop(): Unit = { + delegationTokenRenewer.shutdown() + } +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 3d18690cd9cbf..29752969e6152 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -24,22 +24,19 @@ import java.lang.reflect.InvocationTargetException import java.net.{Socket, URL} import java.util.concurrent.atomic.AtomicReference -import akka.actor._ -import akka.remote._ import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.spark.rpc._ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.SparkException import org.apache.spark.deploy.{PythonRunner, SparkHadoopUtil} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.YarnSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, ChildFirstURLClassLoader, MutableURLClassLoader, - SignalLogger, Utils} +import org.apache.spark.util._ /** * Common application master functionality for Spark on Yarn. @@ -72,12 +69,14 @@ private[spark] class ApplicationMaster( @volatile private var allocator: YarnAllocator = _ // Fields used in client mode. - private var actorSystem: ActorSystem = null - private var actor: ActorRef = _ + private var rpcEnv: RpcEnv = null + private var amEndpoint: RpcEndpointRef = _ // Fields used in cluster mode. private val sparkContextRef = new AtomicReference[SparkContext](null) + private var delegationTokenRenewerOption: Option[AMDelegationTokenRenewer] = None + final def run(): Int = { try { val appAttemptId = client.getAttemptId() @@ -92,54 +91,55 @@ private[spark] class ApplicationMaster( // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) + + // Propagate the attempt if, so that in case of event logging, + // different attempt's logs gets created in different directory + System.setProperty("spark.yarn.app.attemptId", appAttemptId.getAttemptId().toString()) } logInfo("ApplicationAttemptId: " + appAttemptId) val fs = FileSystem.get(yarnConf) - val cleanupHook = new Runnable { - override def run() { - // If the SparkContext is still registered, shut it down as a best case effort in case - // users do not call sc.stop or do System.exit(). - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - } - val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - - if (!finished) { - // This happens when the user application calls System.exit(). We have the choice - // of either failing or succeeding at this point. We report success to avoid - // retrying applications that have succeeded (System.exit(0)), which means that - // applications that explicitly exit with a non-zero status will also show up as - // succeeded in the RM UI. - finish(finalStatus, - ApplicationMaster.EXIT_SUCCESS, - "Shutdown hook called before final status was reported.") - } - if (!unregistered) { - // we only want to unregister if we don't want the RM to retry - if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { - unregister(finalStatus, finalMsg) - cleanupStagingDir(fs) - } + // This shutdown hook should run *after* the SparkContext is shut down. + Utils.addShutdownHook(Utils.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1) { () => + val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) + val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + + if (!finished) { + // This happens when the user application calls System.exit(). We have the choice + // of either failing or succeeding at this point. We report success to avoid + // retrying applications that have succeeded (System.exit(0)), which means that + // applications that explicitly exit with a non-zero status will also show up as + // succeeded in the RM UI. + finish(finalStatus, + ApplicationMaster.EXIT_SUCCESS, + "Shutdown hook called before final status was reported.") + } + + if (!unregistered) { + // we only want to unregister if we don't want the RM to retry + if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { + unregister(finalStatus, finalMsg) + cleanupStagingDir(fs) } } } - // Use higher priority than FileSystem. - assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) - ShutdownHookManager - .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) - // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserApplication which does a // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) + // If the credentials file config is present, we must periodically renew tokens. So create + // a new AMDelegationTokenRenewer + if (sparkConf.contains("spark.yarn.credentials.file")) { + delegationTokenRenewerOption = Some(new AMDelegationTokenRenewer(sparkConf, yarnConf)) + // If a principal and keytab have been set, use that to create new credentials for executors + // periodically + delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) + } + if (isClusterMode) { runDriver(securityMgr) } else { @@ -162,7 +162,7 @@ private[spark] class ApplicationMaster( * status to SUCCEEDED in cluster mode to handle if the user calls System.exit * from the application code. */ - final def getDefaultFinalStatus() = { + final def getDefaultFinalStatus(): FinalApplicationStatus = { if (isClusterMode) { FinalApplicationStatus.SUCCEEDED } else { @@ -175,31 +175,36 @@ private[spark] class ApplicationMaster( * This means the ResourceManager will not retry the application attempt on your behalf if * a failure occurred. */ - final def unregister(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { - if (!unregistered) { - logInfo(s"Unregistering ApplicationMaster with $status" + - Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) - unregistered = true - client.unregister(status, Option(diagnostics).getOrElse("")) + final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = { + synchronized { + if (!unregistered) { + logInfo(s"Unregistering ApplicationMaster with $status" + + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) + unregistered = true + client.unregister(status, Option(diagnostics).getOrElse("")) + } } } - final def finish(status: FinalApplicationStatus, code: Int, msg: String = null) = synchronized { - if (!finished) { - val inShutdown = Utils.inShutdown() - logInfo(s"Final app status: ${status}, exitCode: ${code}" + - Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) - exitCode = code - finalStatus = status - finalMsg = msg - finished = true - if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) { - logDebug("shutting down reporter thread") - reporterThread.interrupt() - } - if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) { - logDebug("shutting down user thread") - userClassThread.interrupt() + final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = { + synchronized { + if (!finished) { + val inShutdown = Utils.inShutdown() + logInfo(s"Final app status: $status, exitCode: $code" + + Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) + exitCode = code + finalStatus = status + finalMsg = msg + finished = true + if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) { + logDebug("shutting down reporter thread") + reporterThread.interrupt() + } + if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) { + logDebug("shutting down user thread") + userClassThread.interrupt() + } + if (!inShutdown) delegationTokenRenewerOption.foreach(_.stop()) } } } @@ -219,9 +224,11 @@ private[spark] class ApplicationMaster( val sc = sparkContextRef.get() val appId = client.getAttemptId().getApplicationId().toString() + val attemptId = client.getAttemptId().getAttemptId().toString() val historyAddress = sparkConf.getOption("spark.yarn.historyServer.address") - .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } + .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) } + .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" } .getOrElse("") allocator = client.register(yarnConf, @@ -236,22 +243,21 @@ private[spark] class ApplicationMaster( } /** - * Create an actor that communicates with the driver. + * Create an [[RpcEndpoint]] that communicates with the driver. * * In cluster mode, the AM and the driver belong to same process - * so the AM actor need not monitor lifecycle of the driver. + * so the AMEndpoint need not monitor lifecycle of the driver. */ - private def runAMActor( + private def runAMEndpoint( host: String, port: String, isClusterMode: Boolean): Unit = { - val driverUrl = AkkaUtils.address( - AkkaUtils.protocol(actorSystem), + val driverEndpoint = rpcEnv.setupEndpointRef( SparkEnv.driverActorSystemName, - host, - port, - YarnSchedulerBackend.ACTOR_NAME) - actor = actorSystem.actorOf(Props(new AMActor(driverUrl, isClusterMode)), name = "YarnAM") + RpcAddress(host, port.toInt), + YarnSchedulerBackend.ENDPOINT_NAME) + amEndpoint = + rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode)) } private def runDriver(securityMgr: SecurityManager): Unit = { @@ -268,8 +274,8 @@ private[spark] class ApplicationMaster( ApplicationMaster.EXIT_SC_NOT_INITED, "Timed out waiting for SparkContext.") } else { - actorSystem = sc.env.actorSystem - runAMActor( + rpcEnv = sc.env.rpcEnv + runAMEndpoint( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port"), isClusterMode = true) @@ -279,8 +285,8 @@ private[spark] class ApplicationMaster( } private def runExecutorLauncher(securityMgr: SecurityManager): Unit = { - actorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf, securityManager = securityMgr)._1 + val port = sparkConf.getInt("spark.yarn.am.port", 0) + rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, port, sparkConf, securityMgr) waitForSparkDriver() addAmIpFilter() registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) @@ -295,7 +301,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) @@ -372,13 +378,7 @@ private[spark] class ApplicationMaster( private def waitForSparkContextInitialized(): SparkContext = { logInfo("Waiting for spark context initialization") sparkContextRef.synchronized { - val waitTries = sparkConf.getOption("spark.yarn.applicationMaster.waitTries") - .map(_.toLong * 10000L) - if (waitTries.isDefined) { - logWarning( - "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") - } - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { @@ -403,8 +403,8 @@ private[spark] class ApplicationMaster( // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) - val deadline = System.currentTimeMillis + totalWaitTime + val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") + val deadline = System.currentTimeMillis + totalWaitTimeMs while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { @@ -427,7 +427,7 @@ private[spark] class ApplicationMaster( sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - runAMActor(driverHost, driverPort.toString, isClusterMode = false) + runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false) } /** Add the Yarn IP filter that is required for properly securing the UI. */ @@ -439,7 +439,7 @@ private[spark] class ApplicationMaster( System.setProperty("spark.ui.filters", amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } else { - actor ! AddWebUIFilter(amFilter, params.toMap, proxyBase) + amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) } } @@ -469,6 +469,9 @@ private[spark] class ApplicationMaster( System.setProperty("spark.submit.pyFiles", PythonRunner.formatPaths(args.pyFiles).mkString(",")) } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + // TODO(davies): add R dependencies here + } val mainMethod = userClassLoader.loadClass(args.userClass) .getMethod("main", classOf[Array[String]]) @@ -501,44 +504,30 @@ private[spark] class ApplicationMaster( } /** - * An actor that communicates with the driver's scheduler backend. + * An [[RpcEndpoint]] that communicates with the driver's scheduler backend. */ - private class AMActor(driverUrl: String, isClusterMode: Boolean) extends Actor { - var driver: ActorSelection = _ - - override def preStart() = { - logInfo("Listen to driver: " + driverUrl) - driver = context.actorSelection(driverUrl) - // Send a hello message to establish the connection, after which - // we can monitor Lifecycle Events. - driver ! "Hello" - driver ! RegisterClusterManager - // In cluster mode, the AM can directly monitor the driver status instead - // of trying to deduce it from the lifecycle of the driver's actor - if (!isClusterMode) { - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - } - } + private class AMEndpoint( + override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean) + extends RpcEndpoint with Logging { - override def receive = { - case x: DisassociatedEvent => - logInfo(s"Driver terminated or disconnected! Shutting down. $x") - // In cluster mode, do not rely on the disassociated event to exit - // This avoids potentially reporting incorrect exit codes if the driver fails - if (!isClusterMode) { - finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) - } + override def onStart(): Unit = { + driver.send(RegisterClusterManager(self)) + + } + override def receive: PartialFunction[Any, Unit] = { case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") - driver ! x + driver.send(x) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RequestExecutors(requestedTotal) => Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") } - sender ! true + context.reply(true) case KillExecutors(executorIds) => logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.") @@ -546,7 +535,16 @@ private[spark] class ApplicationMaster( case Some(a) => executorIds.foreach(a.killExecutor) case None => logWarning("Container allocator is not ready to kill executors yet.") } - sender ! true + context.reply(true) + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") + // In cluster mode, do not rely on the disassociated event to exit + // This avoids potentially reporting incorrect exit codes if the driver fails + if (!isClusterMode) { + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + } } } @@ -554,8 +552,6 @@ private[spark] class ApplicationMaster( object ApplicationMaster extends Logging { - val SHUTDOWN_HOOK_PRIORITY: Int = 30 - // exit codes for different causes, no reason behind the values private val EXIT_SUCCESS = 0 private val EXIT_UNCAUGHT_EXCEPTION = 10 @@ -567,7 +563,7 @@ object ApplicationMaster extends Logging { private var master: ApplicationMaster = _ - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { SignalLogger.register(log) val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => @@ -576,11 +572,11 @@ object ApplicationMaster extends Logging { } } - private[spark] def sparkContextInitialized(sc: SparkContext) = { + private[spark] def sparkContextInitialized(sc: SparkContext): Unit = { master.sparkContextInitialized(sc) } - private[spark] def sparkContextStopped(sc: SparkContext) = { + private[spark] def sparkContextStopped(sc: SparkContext): Boolean = { master.sparkContextStopped(sc) } @@ -592,7 +588,7 @@ object ApplicationMaster extends Logging { */ object ExecutorLauncher { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { ApplicationMaster.main(args) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index e1a992af3aae7..ae6dc1094d724 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -25,6 +25,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var primaryPyFile: String = null + var primaryRFile: String = null var pyFiles: String = null var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 @@ -54,6 +55,10 @@ class ApplicationMasterArguments(val args: Array[String]) { primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--py-files") :: value :: tail => pyFiles = value args = tail @@ -79,6 +84,11 @@ class ApplicationMasterArguments(val args: Array[String]) { } } + if (primaryPyFile != null && primaryRFile != null) { + System.err.println("Cannot have primary-py-file and primary-r-file at the same time") + System.exit(-1) + } + userArgs = userArgsBuffer.readOnly } @@ -92,6 +102,7 @@ class ApplicationMasterArguments(val args: Array[String]) { | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class | --primary-py-file A main Python file + | --primary-r-file A main R file | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to | place on the PYTHONPATH for Python apps. | --args ARGS Arguments to be passed to your application's main class. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 61f8fc3f5a014..20ecaf092e3f8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,22 +17,30 @@ package org.apache.spark.deploy.yarn +import java.io.{ByteArrayInputStream, DataInputStream, File, FileOutputStream} import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction +import java.util.UUID +import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} +import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} import com.google.common.base.Objects +import com.google.common.io.Files import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.mapred.Master +import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{TokenIdentifier, Token} import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -40,10 +48,11 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} import org.apache.spark.util.Utils private[spark] class Client( @@ -61,20 +70,18 @@ private[spark] class Client( private val yarnClient = YarnClient.createYarnClient private val yarnConf = new YarnConfiguration(hadoopConf) - private val credentials = UserGroupInformation.getCurrentUser.getCredentials + private var credentials: Credentials = null private val amMemoryOverhead = args.amMemoryOverhead // MB private val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() private val isClusterMode = args.isClusterMode + private var loginFromKeytab = false + private val fireAndForget = isClusterMode && + !sparkConf.getBoolean("spark.yarn.submit.waitAppCompletion", true) - def stop(): Unit = yarnClient.stop() - /* ------------------------------------------------------------------------------------- * - | The following methods have much in common in the stable and alpha versions of Client, | - | but cannot be implemented in the parent trait due to subtle API differences across | - | hadoop versions. | - * ------------------------------------------------------------------------------------- */ + def stop(): Unit = yarnClient.stop() /** * Submit an application running our ApplicationMaster to the ResourceManager. @@ -84,6 +91,8 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { + // Setup the credentials before doing anything else, so we have don't have issues at any point. + setupCredentials() yarnClient.init(yarnConf) yarnClient.start() @@ -215,8 +224,14 @@ private[spark] class Client( // and add them as local resources to the application master. val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = getNameNodesToAccess(sparkConf) + dst - obtainTokensForNamenodes(nns, hadoopConf, credentials) + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst + YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, credentials) + // Used to keep track of URIs added to the distributed cache. If the same URI is added + // multiple times, YARN will fail to launch containers for the app with an internal + // error. + val distributedUris = new HashSet[String] + obtainTokenForHiveMetastore(hadoopConf, credentials) + obtainTokenForHBase(hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -233,6 +248,31 @@ private[spark] class Client( "for alternatives.") } + // If we passed in a keytab, make sure we copy the keytab to the staging directory on + // HDFS, and setup the relevant environment vars, so the AM can login again. + if (loginFromKeytab) { + logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + + " via the YARN Secure Distributed Cache.") + val localUri = new URI(args.keytab) + val localPath = getQualifiedLocalPath(localUri, hadoopConf) + val destinationPath = copyFileToRemote(dst, localPath, replication) + val destFs = FileSystem.get(destinationPath.toUri(), hadoopConf) + distCacheMgr.addResource( + destFs, hadoopConf, destinationPath, localResources, LocalResourceType.FILE, + sparkConf.get("spark.yarn.keytab"), statCache, appMasterOnly = true) + } + + def addDistributedUri(uri: URI): Boolean = { + val uriStr = uri.toString() + if (distributedUris.contains(uriStr)) { + logWarning(s"Resource $uri added multiple times to distributed cache.") + false + } else { + distributedUris += uriStr + true + } + } + /** * Copy the given main resource to the distributed cache if the scheme is not "local". * Otherwise, set the corresponding key in our SparkConf to handle it downstream. @@ -250,11 +290,13 @@ private[spark] class Client( if (!localPath.isEmpty()) { val localURI = new URI(localPath) if (localURI.getScheme != LOCAL_SCHEME) { - val src = getQualifiedLocalPath(localURI, hadoopConf) - val destPath = copyFileToRemote(dst, src, replication) - val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource(destFs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, destName, statCache) + if (addDistributedUri(localURI)) { + val src = getQualifiedLocalPath(localURI, hadoopConf) + val destPath = copyFileToRemote(dst, src, replication) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) + } } else if (confKey != null) { // If the resource is intended for local use only, handle this downstream // by setting the appropriate property @@ -263,6 +305,13 @@ private[spark] class Client( } } + createConfArchive().foreach { file => + require(addDistributedUri(file.toURI())) + val destPath = copyFileToRemote(dst, new Path(file.toURI()), replication) + distCacheMgr.addResource(fs, hadoopConf, destPath, localResources, LocalResourceType.ARCHIVE, + LOCALIZED_HADOOP_CONF_DIR, statCache, appMasterOnly = true) + } + /** * Do the same for any additional resources passed in through ClientArguments. * Each resource category is represented by a 3-tuple of: @@ -280,13 +329,15 @@ private[spark] class Client( flist.split(',').foreach { file => val localURI = new URI(file.trim()) if (localURI.getScheme != LOCAL_SCHEME) { - val localPath = new Path(localURI) - val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) - distCacheMgr.addResource( - fs, hadoopConf, destPath, localResources, resType, linkname, statCache) - if (addToClasspath) { - cachedSecondaryJarLinks += linkname + if (addDistributedUri(localURI)) { + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) + if (addToClasspath) { + cachedSecondaryJarLinks += linkname + } } } else if (addToClasspath) { // Resource is intended for local use only and should be added to the class path @@ -302,6 +353,81 @@ private[spark] class Client( localResources } + /** + * Create an archive with the Hadoop config files for distribution. + * + * These are only used by the AM, since executors will use the configuration object broadcast by + * the driver. The files are zipped and added to the job as an archive, so that YARN will explode + * it when distributing to the AM. This directory is then added to the classpath of the AM + * process, just to make sure that everybody is using the same default config. + * + * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR + * shows up in the classpath before YARN_CONF_DIR. + * + * Currently this makes a shallow copy of the conf directory. If there are cases where a + * Hadoop config directory contains subdirectories, this code will have to be fixed. + */ + private def createConfArchive(): Option[File] = { + val hadoopConfFiles = new HashMap[String, File]() + Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey => + sys.env.get(envKey).foreach { path => + val dir = new File(path) + if (dir.isDirectory()) { + dir.listFiles().foreach { file => + if (file.isFile && !hadoopConfFiles.contains(file.getName())) { + hadoopConfFiles(file.getName()) = file + } + } + } + } + } + + if (!hadoopConfFiles.isEmpty) { + val hadoopConfArchive = File.createTempFile(LOCALIZED_HADOOP_CONF_DIR, ".zip", + new File(Utils.getLocalDir(sparkConf))) + + val hadoopConfStream = new ZipOutputStream(new FileOutputStream(hadoopConfArchive)) + try { + hadoopConfStream.setLevel(0) + hadoopConfFiles.foreach { case (name, file) => + if (file.canRead()) { + hadoopConfStream.putNextEntry(new ZipEntry(name)) + Files.copy(file, hadoopConfStream) + hadoopConfStream.closeEntry() + } + } + } finally { + hadoopConfStream.close() + } + + Some(hadoopConfArchive) + } else { + None + } + } + + /** + * Get the renewal interval for tokens. + */ + private def getTokenRenewalInterval(stagingDirPath: Path): Long = { + // We cannot use the tokens generated above since those have renewer yarn. Trying to renew + // those will fail with an access control issue. So create new tokens with the logged in + // user as renewer. + val creds = new Credentials() + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + stagingDirPath + YarnSparkHadoopUtil.get.obtainTokensForNamenodes( + nns, hadoopConf, creds, Some(sparkConf.get("spark.yarn.principal"))) + val t = creds.getAllTokens + .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + .head + val newExpiration = t.renew(hadoopConf) + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + val interval = newExpiration - identifier.getIssueDate + logInfo(s"Renewal Interval set to $interval") + interval + } + /** * Set up the environment for launching our ApplicationMaster container. */ @@ -309,11 +435,20 @@ private[spark] class Client( logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, true, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - + if (loginFromKeytab) { + val remoteFs = FileSystem.get(hadoopConf) + val stagingDirPath = new Path(remoteFs.getHomeDirectory, stagingDir) + val credentialsFile = "credentials-" + UUID.randomUUID().toString + sparkConf.set( + "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) + logInfo(s"Credentials file set to: $credentialsFile") + val renewalInterval = getTokenRenewalInterval(stagingDirPath) + sparkConf.set("spark.yarn.token.renewal.interval", renewalInterval.toString) + } // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) @@ -378,7 +513,6 @@ private[spark] class Client( private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) : ContainerLaunchContext = { logInfo("Setting up container launch context for our AM") - val appId = newAppResponse.getApplicationId val appStagingDir = getAppStagingDir(appId) val localResources = prepareLocalResources(appStagingDir) @@ -459,6 +593,10 @@ private[spark] class Client( } javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) } + + sparkConf.getOption("spark.yarn.am.extraLibraryPath").foreach { paths => + prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(paths))) + } } // For log4j configuration to reference @@ -488,6 +626,12 @@ private[spark] class Client( } else { Nil } + val primaryRFile = + if (args.primaryRFile != null) { + Seq("--primary-r-file", args.primaryRFile) + } else { + Nil + } val amClass = if (isClusterMode) { Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName @@ -497,12 +641,15 @@ private[spark] class Client( if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs + } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) } val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++ - Seq( + Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ primaryRFile ++ + userArgs ++ Seq( "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString) @@ -540,6 +687,24 @@ private[spark] class Client( amContainer } + def setupCredentials(): Unit = { + if (args.principal != null) { + require(args.keytab != null, "Keytab must be specified when principal is specified.") + logInfo("Attempting to login to the Kerberos" + + s" using principal: ${args.principal} and keytab: ${args.keytab}") + val f = new File(args.keytab) + // Generate a file name that can be used for the keytab file, that does not conflict + // with any user file. + val keytabFileName = f.getName + "-" + UUID.randomUUID().toString + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) + loginFromKeytab = true + sparkConf.set("spark.yarn.keytab", keytabFileName) + sparkConf.set("spark.yarn.principal", args.principal) + logInfo("Successfully logged into the KDC.") + } + credentials = UserGroupInformation.getCurrentUser.getCredentials + } + /** * Report the state of an application until it has exited, either successfully or * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, @@ -559,36 +724,25 @@ private[spark] class Client( var lastState: YarnApplicationState = null while (true) { Thread.sleep(interval) - val report = getApplicationReport(appId) + val report: ApplicationReport = + try { + getApplicationReport(appId) + } catch { + case e: ApplicationNotFoundException => + logError(s"Application $appId not found.") + return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED) + } val state = report.getYarnApplicationState if (logApplicationReport) { logInfo(s"Application report for $appId (state: $state)") - val details = Seq[(String, String)]( - ("client token", getClientToken(report)), - ("diagnostics", report.getDiagnostics), - ("ApplicationMaster host", report.getHost), - ("ApplicationMaster RPC port", report.getRpcPort.toString), - ("queue", report.getQueue), - ("start time", report.getStartTime.toString), - ("final status", report.getFinalApplicationStatus.toString), - ("tracking URL", report.getTrackingUrl), - ("user", report.getUser) - ) - - // Use more loggable format if value is null or empty - val formattedDetails = details - .map { case (k, v) => - val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") - s"\n\t $k: $newValue" } - .mkString("") // If DEBUG is enabled, log report details every iteration // Otherwise, log them every time the application changes state if (log.isDebugEnabled) { - logDebug(formattedDetails) + logDebug(formatReportDetails(report)) } else if (lastState != state) { - logInfo(formattedDetails) + logInfo(formatReportDetails(report)) } } @@ -609,24 +763,57 @@ private[spark] class Client( throw new SparkException("While loop is depleted! This should never happen...") } + private def formatReportDetails(report: ApplicationReport): String = { + val details = Seq[(String, String)]( + ("client token", getClientToken(report)), + ("diagnostics", report.getDiagnostics), + ("ApplicationMaster host", report.getHost), + ("ApplicationMaster RPC port", report.getRpcPort.toString), + ("queue", report.getQueue), + ("start time", report.getStartTime.toString), + ("final status", report.getFinalApplicationStatus.toString), + ("tracking URL", report.getTrackingUrl), + ("user", report.getUser) + ) + + // Use more loggable format if value is null or empty + details.map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" + }.mkString("") + } + /** - * Submit an application to the ResourceManager and monitor its state. - * This continues until the application has exited for any reason. + * Submit an application to the ResourceManager. + * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive + * reporting the application's status until the application has exited for any reason. + * Otherwise, the client process will exit after submission. * If the application finishes with a failed, killed, or undefined status, * throw an appropriate SparkException. */ def run(): Unit = { - val (yarnApplicationState, finalApplicationStatus) = monitorApplication(submitApplication()) - if (yarnApplicationState == YarnApplicationState.FAILED || - finalApplicationStatus == FinalApplicationStatus.FAILED) { - throw new SparkException("Application finished with failed status") - } - if (yarnApplicationState == YarnApplicationState.KILLED || - finalApplicationStatus == FinalApplicationStatus.KILLED) { - throw new SparkException("Application is killed") - } - if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { - throw new SparkException("The final status of application is undefined") + val appId = submitApplication() + if (fireAndForget) { + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + logInfo(s"Application report for $appId (state: $state)") + logInfo(formatReportDetails(report)) + if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { + throw new SparkException(s"Application $appId finished with status: $state") + } + } else { + val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId) + if (yarnApplicationState == YarnApplicationState.FAILED || + finalApplicationStatus == FinalApplicationStatus.FAILED) { + throw new SparkException(s"Application $appId finished with failed status") + } + if (yarnApplicationState == YarnApplicationState.KILLED || + finalApplicationStatus == FinalApplicationStatus.KILLED) { + throw new SparkException(s"Application $appId is killed") + } + if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { + throw new SparkException(s"The final status of application $appId is undefined") + } } } } @@ -679,6 +866,9 @@ object Client extends Logging { // Distribution-defined classpath to add to processes val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH" + // Subdirectory where the user's hadoop config files will be placed. + val LOCALIZED_HADOOP_CONF_DIR = "__hadoop_conf__" + /** * Find the user-defined Spark jar if configured, or return the jar containing this * class if not. @@ -792,11 +982,19 @@ object Client extends Logging { conf: Configuration, sparkConf: SparkConf, env: HashMap[String, String], + isAM: Boolean, extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry( YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env ) + + if (isAM) { + addClasspathEntry( + YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + + LOCALIZED_HADOOP_CONF_DIR, env) + } + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { val userClassPath = if (args != null) { @@ -863,41 +1061,94 @@ object Client extends Logging { YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) /** - * Get the list of namenodes the user may access. + * Obtains token for the Hive metastore and adds them to the credentials. */ - private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "") - .split(",") - .map(_.trim()) - .filter(!_.isEmpty) - .map(new Path(_)) - .toSet - } + private def obtainTokenForHiveMetastore(conf: Configuration, credentials: Credentials) { + if (UserGroupInformation.isSecurityEnabled) { + val mirror = universe.runtimeMirror(getClass.getClassLoader) - private[yarn] def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) + try { + val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive") + val hive = hiveClass.getMethod("get").invoke(null) + + val hiveConf = hiveClass.getMethod("getConf").invoke(hive) + val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") + + val hiveConfGet = (param:String) => Option(hiveConfClass + .getMethod("get", classOf[java.lang.String]) + .invoke(hiveConf, param)) + + val metastore_uri = hiveConfGet("hive.metastore.uris") + + // Check for local metastore + if (metastore_uri != None && metastore_uri.get.toString.size > 0) { + val metastore_kerberos_principal_conf_var = mirror.classLoader + .loadClass("org.apache.hadoop.hive.conf.HiveConf$ConfVars") + .getField("METASTORE_KERBEROS_PRINCIPAL").get("varname").toString + + val principal = hiveConfGet(metastore_kerberos_principal_conf_var) + + val username = Option(UserGroupInformation.getCurrentUser().getUserName) + if (principal != None && username != None) { + val tokenStr = hiveClass.getMethod("getDelegationToken", + classOf[java.lang.String], classOf[java.lang.String]) + .invoke(hive, username.get, principal.get).asInstanceOf[java.lang.String] + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + credentials.addToken(new Text("hive.server2.delegation.token"),hive2Token) + logDebug("Added hive.Server2.delegation.token to conf.") + hiveClass.getMethod("closeCurrent").invoke(null) + } else { + logError("Username or principal == NULL") + logError(s"""username=${username.getOrElse("(NULL)")}""") + logError(s"""principal=${principal.getOrElse("(NULL)")}""") + throw new IllegalArgumentException("username and/or principal is equal to null!") + } + } else { + logDebug("HiveMetaStore configured in localmode") + } + } catch { + case e:java.lang.NoSuchMethodException => { logInfo("Hive Method not found " + e); return } + case e:java.lang.ClassNotFoundException => { logInfo("Hive Class not found " + e); return } + case e:Exception => { logError("Unexpected Exception " + e) + throw new RuntimeException("Unexpected exception", e) + } + } } - delegTokenRenewer } /** - * Obtains tokens for the namenodes passed in and adds them to the credentials. + * Obtain security token for HBase. */ - private def obtainTokensForNamenodes( - paths: Set[Path], - conf: Configuration, - creds: Credentials): Unit = { - if (UserGroupInformation.isSecurityEnabled()) { - val delegTokenRenewer = getTokenRenewer(conf) - paths.foreach { dst => - val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) + def obtainTokenForHBase(conf: Configuration, credentials: Credentials): Unit = { + if (UserGroupInformation.isSecurityEnabled) { + val mirror = universe.runtimeMirror(getClass.getClassLoader) + + try { + val confCreate = mirror.classLoader. + loadClass("org.apache.hadoop.hbase.HBaseConfiguration"). + getMethod("create", classOf[Configuration]) + val obtainToken = mirror.classLoader. + loadClass("org.apache.hadoop.hbase.security.token.TokenUtil"). + getMethod("obtainToken", classOf[Configuration]) + + logDebug("Attempting to fetch HBase security token.") + + val hbaseConf = confCreate.invoke(null, conf) + val token = obtainToken.invoke(null, hbaseConf).asInstanceOf[Token[TokenIdentifier]] + credentials.addToken(token.getService, token) + + logInfo("Added HBase security token to credentials.") + } catch { + case e:java.lang.NoSuchMethodException => + logInfo("HBase Method not found: " + e) + case e:java.lang.ClassNotFoundException => + logDebug("HBase Class not found: " + e) + case e:java.lang.NoClassDefFoundError => + logDebug("HBase Class not found: " + e) + case e:Exception => + logError("Exception when obtaining HBase security token: " + e) } } } @@ -955,8 +1206,7 @@ object Client extends Logging { if (isDriver) { conf.getBoolean("spark.driver.userClassPathFirst", false) } else { - conf.getBoolean("spark.executor.userClassPathFirst", - conf.getBoolean("spark.files.userClassPathFirst", false)) + conf.getBoolean("spark.executor.userClassPathFirst", false) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3bc7eb1abf341..5653c9f14dc6d 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -32,6 +32,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var userClass: String = null var pyFiles: String = null var primaryPyFile: String = null + var primaryRFile: String = null var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var executorMemory = 1024 // MB var executorCores = 1 @@ -41,6 +42,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var amCores: Int = 1 var appName: String = "Spark" var priority = 0 + var principal: String = null + var keytab: String = null def isClusterMode: Boolean = userClass != null private var driverMemory: Int = 512 // MB @@ -102,9 +105,13 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) * This is intended to be called only after the provided arguments have been parsed. */ private def validateArgs(): Unit = { - if (numExecutors <= 0) { + if (numExecutors < 0 || (!isDynamicAllocationEnabled && numExecutors == 0)) { throw new IllegalArgumentException( - "You must specify at least 1 executor!\n" + getUsageMessage()) + s""" + |Number of executors was $numExecutors, but must be at least 1 + |(or 0 if dynamic executor allocation is enabled). + |${getUsageMessage()} + """.stripMargin) } if (executorCores < sparkConf.getInt("spark.task.cpus", 1)) { throw new SparkException("Executor cores must not be less than " + @@ -150,6 +157,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--args" | "--arg") :: value :: tail => if (args(0) == "--args") { println("--args is deprecated. Use --arg instead.") @@ -222,12 +233,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) archives = value args = tail + case ("--principal") :: value :: tail => + principal = value + args = tail + + case ("--keytab") :: value :: tail => + keytab = value + args = tail + case Nil => case _ => throw new IllegalArgumentException(getUsageMessage(args)) } } + + if (primaryPyFile != null && primaryRFile != null) { + throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" + + " at the same time") + } } private def getUsageMessage(unknownParam: List[String] = null): String = { @@ -240,6 +264,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | mode) | --class CLASS_NAME Name of your application's main class (required) | --primary-py-file A main Python file + | --primary-r-file A main R file | --arg ARG Argument to be passed to your application's main class. | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala new file mode 100644 index 0000000000000..229c2c4d5eb36 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala @@ -0,0 +1,106 @@ +/* + * 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.yarn + +import java.util.concurrent.{Executors, TimeUnit} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.{ThreadUtils, Utils} + +import scala.util.control.NonFatal + +private[spark] class ExecutorDelegationTokenUpdater( + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + @volatile private var lastCredentialsFileSuffix = 0 + + private val credentialsFile = sparkConf.get("spark.yarn.credentials.file") + + private val delegationTokenRenewer = + Executors.newSingleThreadScheduledExecutor( + ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) + + // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. + private val executorUpdaterRunnable = + new Runnable { + override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) + } + + def updateCredentialsIfRequired(): Unit = { + try { + val credentialsFilePath = new Path(credentialsFile) + val remoteFs = FileSystem.get(hadoopConf) + SparkHadoopUtil.get.listFilesSorted( + remoteFs, credentialsFilePath.getParent, + credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) + .lastOption.foreach { credentialsStatus => + val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath) + if (suffix > lastCredentialsFileSuffix) { + logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) + val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) + lastCredentialsFileSuffix = suffix + UserGroupInformation.getCurrentUser.addCredentials(newCredentials) + logInfo("Tokens updated from credentials file.") + } else { + // Check every hour to see if new credentials arrived. + logInfo("Updated delegation tokens were expected, but the driver has not updated the " + + "tokens yet, will check again in an hour.") + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + return + } + } + val timeFromNowToRenewal = + SparkHadoopUtil.get.getTimeFromNowToRenewal( + sparkConf, 0.8, UserGroupInformation.getCurrentUser.getCredentials) + if (timeFromNowToRenewal <= 0) { + executorUpdaterRunnable.run() + } else { + logInfo(s"Scheduling token refresh from HDFS in $timeFromNowToRenewal millis.") + delegationTokenRenewer.schedule( + executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) + } + } catch { + // Since the file may get deleted while we are reading it, catch the Exception and come + // back in an hour to try again + case NonFatal(e) => + logWarning("Error while trying to update credentials, will try again in 1 hour", e) + delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) + } + } + + private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = { + val stream = remoteFs.open(tokenPath) + try { + val newCredentials = new Credentials() + newCredentials.readTokenStorageStream(stream) + newCredentials + } finally { + stream.close() + } + } + + def stop(): Unit = { + delegationTokenRenewer.shutdown() + } + +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index c1d3f7320f53c..9d04d241dae9e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -59,15 +59,15 @@ class ExecutorRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) lazy val env = prepareEnvironment(container) - def run = { + override def run(): Unit = { logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() nmClient.init(yarnConf) nmClient.start() - startContainer + startContainer() } - def startContainer = { + def startContainer(): java.util.Map[String, ByteBuffer] = { logInfo("Setting up ContainerLaunchContext") val ctx = Records.newRecord(classOf[ContainerLaunchContext]) @@ -277,7 +277,7 @@ class ExecutorRunnable( private def prepareEnvironment(container: Container): HashMap[String, String] = { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - Client.populateClasspath(null, yarnConf, sparkConf, env, extraCp) + Client.populateClasspath(null, yarnConf, sparkConf, env, false, extraCp) sparkConf.getExecutorEnv.foreach { case (key, value) => // This assumes each executor environment variable set here is a path @@ -290,10 +290,19 @@ class ExecutorRunnable( YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) } + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + // Add log urls sys.env.get("SPARK_USER").foreach { user => - val baseUrl = "http://%s/node/containerlogs/%s/%s" - .format(container.getNodeHttpAddress, ConverterUtils.toString(container.getId), user) + val containerId = ConverterUtils.toString(container.getId) + val address = container.getNodeHttpAddress + val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0" env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0" } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index c98763e15b58f..b8f42dadcb464 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -112,7 +112,7 @@ private[yarn] class YarnAllocator( SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) // For testing private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 5881dc5ffa3ad..ba91872107d0c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -24,18 +24,19 @@ import java.util.regex.Pattern import scala.collection.mutable.HashMap import scala.util.Try +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.{Master, JobConf} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records.{Priority, ApplicationAccessType} -import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.util.Utils /** @@ -43,6 +44,8 @@ import org.apache.spark.util.Utils */ class YarnSparkHadoopUtil extends SparkHadoopUtil { + private var tokenRenewer: Option[ExecutorDelegationTokenUpdater] = None + override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { dest.addCredentials(source.getCredentials()) } @@ -82,6 +85,57 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { if (credentials != null) credentials.getSecretKey(new Text(key)) else null } + /** + * Get the list of namenodes the user may access. + */ + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet + } + + def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the credentials. + */ + def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials, + renewer: Option[String] = None + ): Unit = { + if (UserGroupInformation.isSecurityEnabled()) { + val delegTokenRenewer = renewer.getOrElse(getTokenRenewer(conf)) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logInfo("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } + + private[spark] override def startExecutorDelegationTokenRenewer(sparkConf: SparkConf): Unit = { + tokenRenewer = Some(new ExecutorDelegationTokenUpdater(sparkConf, conf)) + tokenRenewer.get.updateCredentialsIfRequired() + } + + private[spark] override def stopExecutorDelegationTokenRenewer(): Unit = { + tokenRenewer.foreach(_.stop()) + } + } object YarnSparkHadoopUtil { @@ -100,6 +154,14 @@ object YarnSparkHadoopUtil { // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = Priority.newInstance(1) + def get: YarnSparkHadoopUtil = { + val yarnMode = java.lang.Boolean.valueOf( + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + if (!yarnMode) { + throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!") + } + SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] + } /** * Add a path variable to the given environment map. * If the map already contains this key, append the value to the existing value instead. @@ -212,3 +274,4 @@ object YarnSparkHadoopUtil { classPathSeparatorField.get(null).asInstanceOf[String] } } + diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 8abdc26b43806..99c05329b4d73 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -34,7 +34,7 @@ private[spark] class YarnClientSchedulerBackend( private var client: Client = null private var appId: ApplicationId = null - @volatile private var stopping: Boolean = false + private var monitorThread: Thread = null /** * Create a Yarn client to submit an application to the ResourceManager. @@ -57,7 +57,8 @@ private[spark] class YarnClientSchedulerBackend( client = new Client(args, conf) appId = client.submitApplication() waitForApplication() - asyncMonitorApplication() + monitorThread = asyncMonitorApplication() + monitorThread.start() } /** @@ -123,34 +124,22 @@ private[spark] class YarnClientSchedulerBackend( * If the application has exited for any reason, stop the SparkContext. * This assumes both `client` and `appId` have already been set. */ - private def asyncMonitorApplication(): Unit = { + private def asyncMonitorApplication(): Thread = { assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - while (!stopping) { - var state: YarnApplicationState = null - try { - val report = client.getApplicationReport(appId) - state = report.getYarnApplicationState() - } catch { - case e: ApplicationNotFoundException => - state = YarnApplicationState.KILLED - } - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.KILLED || - state == YarnApplicationState.FAILED) { - logError(s"Yarn application has already exited with state $state!") - sc.stop() - stopping = true - } - Thread.sleep(1000L) + try { + val (state, _) = client.monitorApplication(appId, logApplicationReport = false) + logError(s"Yarn application has already exited with state $state!") + sc.stop() + } catch { + case e: InterruptedException => logInfo("Interrupting monitor thread") } - Thread.currentThread().interrupt() } } t.setName("Yarn application state monitor") t.setDaemon(true) - t.start() + t } /** @@ -158,7 +147,7 @@ private[spark] class YarnClientSchedulerBackend( */ override def stop() { assert(client != null, "Attempted to stop this scheduler before starting it!") - stopping = true + monitorThread.interrupt() super.stop() client.stop() logInfo("Stopped") diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index b1de81e6a8b0f..aeb218a575455 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -39,12 +39,18 @@ private[spark] class YarnClusterSchedulerBackend( } override def applicationId(): String = - // In YARN Cluster mode, spark.yarn.app.id is expect to be set - // before user application is launched. - // So, if spark.yarn.app.id is not set, it is something wrong. + // In YARN Cluster mode, the application ID is expected to be set, so log an error if it's + // not found. sc.getConf.getOption("spark.yarn.app.id").getOrElse { logError("Application ID is not set.") super.applicationId } + override def applicationAttemptId(): Option[String] = + // In YARN Cluster mode, the attempt ID is expected to be set, so log an error if it's + // not found. + sc.getConf.getOption("spark.yarn.app.attemptId").orElse { + logError("Application attempt ID is not set.") + super.applicationAttemptId + } } diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index aab41fa49430f..6b8a5dbf6373e 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN log4j.logger.org.apache.hadoop=WARN diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 92f04b4b859b3..508819e242a26 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -20,6 +20,11 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ HashMap => MutableHashMap } +import scala.reflect.ClassTag +import scala.util.Try + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig @@ -30,11 +35,6 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ HashMap => MutableHashMap } -import scala.reflect.ClassTag -import scala.util.Try - import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils @@ -93,7 +93,7 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - Client.populateClasspath(args, conf, sparkConf, env) + Client.populateClasspath(args, conf, sparkConf, env, true) val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => @@ -104,13 +104,16 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { cp should not contain (uri.getPath()) } }) - if (classOf[Environment].getMethods().exists(_.getName == "$$")) { - cp should contain("{{PWD}}") - } else if (Utils.isWindows) { - cp should contain("%PWD%") - } else { - cp should contain(Environment.PWD.$()) - } + val pwdVar = + if (classOf[Environment].getMethods().exists(_.getName == "$$")) { + "{{PWD}}" + } else if (Utils.isWindows) { + "%PWD%" + } else { + Environment.PWD.$() + } + cp should contain(pwdVar) + cp should contain (s"$pwdVar${Path.SEPARATOR}${Client.LOCALIZED_HADOOP_CONF_DIR}") cp should not contain (Client.SPARK_JAR) cp should not contain (Client.APP_JAR) } @@ -148,57 +151,6 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { } } - test("check access nns empty") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns unset") { - val sparkConf = new SparkConf() - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access nns space") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access two nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") - val nns = Client.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) - } - - test("check token renewer") { - val hadoopConf = new Configuration() - hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") - hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - val renewer = Client.getTokenRenewer(hadoopConf) - renewer should be ("yarn/myrm:8032@SPARKTEST.COM") - } - - test("check token renewer default") { - val hadoopConf = new Configuration() - val caught = - intercept[SparkException] { - Client.getTokenRenewer(hadoopConf) - } - assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") - } - object Fixtures { val knownDefYarnAppCP: Seq[String] = @@ -232,19 +184,26 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { testCode(conf) } - def newEnv = MutableHashMap[String, String]() + def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]() - def classpath(env: MutableHashMap[String, String]) = env(Environment.CLASSPATH.name).split(":|;|") + def classpath(env: MutableHashMap[String, String]): Array[String] = + env(Environment.CLASSPATH.name).split(":|;|") - def flatten(a: Option[Seq[String]], b: Option[Seq[String]]) = (a ++ b).flatten.toArray + def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] = + (a ++ b).flatten.toArray - def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = - Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults) + def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = { + Try(clazz.getField(field)) + .map(_.get(null).asInstanceOf[A]) + .toOption + .map(mapTo) + .getOrElse(defaults) + } def getFieldValue2[A: ClassTag, A1: ClassTag, B]( clazz: Class[_], field: String, - defaults: => B)(mapTo: A => B)(mapTo1: A1 => B) : B = { + defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = { Try(clazz.getField(field)).map(_.get(null)).map { case v: A => mapTo(v) case v1: A1 => mapTo1(v1) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index c09b01bafce37..455f1019d86dd 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -79,7 +79,7 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach } class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) { - override def equals(other: Any) = false + override def equals(other: Any): Boolean = false } def createAllocator(maxExecutors: Int = 5): YarnAllocator = { @@ -118,7 +118,9 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size should be (0) + + val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size + size should be (0) } test("some containers allocated") { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 0e37276ba724b..d3c606e0ed998 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** @@ -77,6 +77,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ private var logConfDir: File = _ override def beforeAll() { @@ -85,6 +86,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit tempDir = Utils.createTempDir() logConfDir = new File(tempDir, "log4j") logConfDir.mkdir() + System.setProperty("SPARK_YARN_MODE", "true") val logConfFile = new File(logConfDir, "log4j.properties") Files.write(LOG4J_CONF, logConfFile, UTF_8) @@ -120,10 +122,14 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_HADOOP_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) } override def afterAll() { yarnCluster.stop() + System.clearProperty("SPARK_YARN_MODE") super.afterAll() } @@ -143,6 +149,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } } + // Enable this once fix SPARK-6700 test("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) @@ -257,7 +264,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit appArgs Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> tempDir.getAbsolutePath())) + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) } /** @@ -281,10 +288,10 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } -private class SaveExecutorInfo extends SparkListener { +private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor : SparkListenerExecutorAdded) { + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } @@ -292,7 +299,6 @@ private class SaveExecutorInfo extends SparkListener { private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 - var listener: SaveExecutorInfo = null def main(args: Array[String]): Unit = { if (args.length != 1) { @@ -305,10 +311,9 @@ private object YarnClusterDriver extends Logging with Matchers { System.exit(1) } - listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) - sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { @@ -322,7 +327,12 @@ private object YarnClusterDriver extends Logging with Matchers { } // verify log urls are present - listener.addedExecutorInfos.values.foreach { info => + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => assert(info.logUrlMap.nonEmpty) } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 4194f36499e66..e10b985c3c236 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import com.google.common.io.{ByteStreams, Files} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -27,7 +29,7 @@ import org.scalatest.{FunSuite, Matchers} import org.apache.hadoop.yarn.api.records.ApplicationAccessType -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.util.Utils @@ -46,7 +48,7 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { logWarning("Cannot execute bash, skipping bash tests.") } - def bashTest(name: String)(fn: => Unit) = + def bashTest(name: String)(fn: => Unit): Unit = if (hasBash) test(name)(fn) else ignore(name)(fn) bashTest("shell script escaping") { @@ -173,4 +175,62 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { YarnSparkHadoopUtil.getClassPathSeparator() should be (":") } } + + test("check access nns empty") { + val sparkConf = new SparkConf() + val util = new YarnSparkHadoopUtil + sparkConf.set("spark.yarn.access.namenodes", "") + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns unset") { + val sparkConf = new SparkConf() + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access nns space") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access two nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") + val util = new YarnSparkHadoopUtil + val nns = util.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) + } + + test("check token renewer") { + val hadoopConf = new Configuration() + hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") + hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") + val util = new YarnSparkHadoopUtil + val renewer = util.getTokenRenewer(hadoopConf) + renewer should be ("yarn/myrm:8032@SPARKTEST.COM") + } + + test("check token renewer default") { + val hadoopConf = new Configuration() + val util = new YarnSparkHadoopUtil + val caught = + intercept[SparkException] { + util.getTokenRenewer(hadoopConf) + } + assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") + } }